beam-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "ASF GitHub Bot (JIRA)" <j...@apache.org>
Subject [jira] [Work logged] (BEAM-5288) Modify Environment to support non-dockerized SDK harness deployments
Date Thu, 20 Sep 2018 02:29:01 GMT

     [ https://issues.apache.org/jira/browse/BEAM-5288?focusedWorklogId=145910&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-145910 ]

ASF GitHub Bot logged work on BEAM-5288:
----------------------------------------

                Author: ASF GitHub Bot
            Created on: 20/Sep/18 02:28
            Start Date: 20/Sep/18 02:28
    Worklog Time Spent: 10m 
      Work Description: tweise closed pull request #6373: [BEAM-5288] Enhance Environment proto to support different types of environments
URL: https://github.com/apache/beam/pull/6373
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git a/model/pipeline/src/main/proto/beam_runner_api.proto b/model/pipeline/src/main/proto/beam_runner_api.proto
index 0df8cc8e888..72e608e00d2 100644
--- a/model/pipeline/src/main/proto/beam_runner_api.proto
+++ b/model/pipeline/src/main/proto/beam_runner_api.proto
@@ -989,16 +989,40 @@ message SideInput {
   SdkFunctionSpec window_mapping_fn = 3;
 }
 
-// An environment for executing UDFs. Generally an SDK container URL, but
-// there can be many for a single SDK, for example to provide dependency
-// isolation.
+// An environment for executing UDFs. By default, an SDK container URL, but
+// can also be a process forked by a command, or an externally managed process.
 message Environment {
-
-  // (Required) The URL of a container
-  //
-  // TODO: reconcile with Fn API's DockerContainer structure by
-  // adding adequate metadata to know how to interpret the container
+  // Deprecated
   string url = 1;
+
+  // (Required) The URN of the payload
+  string urn = 2;
+
+  // (Optional) The data specifying any parameters to the URN. If
+  // the URN does not require any arguments, this may be omitted.
+  bytes payload = 3;
+}
+
+message StandardEnvironments {
+  enum Environments {
+    DOCKER = 0 [(beam_urn) = "beam:env:docker:v1"]; // A managed docker container to run user code.
+
+    PROCESS = 1 [(beam_urn) = "beam:env:process:v1"]; // A managed native process to run user code.
+
+    EXTERNAL = 2 [(beam_urn) = "beam:env:external:v1"]; // An external non managed process to run user code.
+  }
+}
+
+// The payload of a Docker image
+message DockerPayload {
+  string container_image = 1;  // implicitly linux_amd64.
+}
+
+message ProcessPayload {
+  string os = 1;  // "linux", "darwin", ..
+  string arch = 2;  // "amd64", ..
+  string command = 3; // process to execute
+  map<string, string> env = 4; // Environment variables
 }
 
 // A specification of a user defined function.
diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/Environments.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/Environments.java
index 9ed3af2f167..c75be4e9ee7 100644
--- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/Environments.java
+++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/Environments.java
@@ -21,14 +21,18 @@
 import com.google.common.base.Strings;
 import com.google.common.collect.ImmutableMap;
 import java.io.IOException;
+import java.util.Map;
 import java.util.Optional;
 import javax.annotation.Nullable;
 import org.apache.beam.model.pipeline.v1.RunnerApi.CombinePayload;
 import org.apache.beam.model.pipeline.v1.RunnerApi.Components;
+import org.apache.beam.model.pipeline.v1.RunnerApi.DockerPayload;
 import org.apache.beam.model.pipeline.v1.RunnerApi.Environment;
 import org.apache.beam.model.pipeline.v1.RunnerApi.PTransform;
 import org.apache.beam.model.pipeline.v1.RunnerApi.ParDoPayload;
+import org.apache.beam.model.pipeline.v1.RunnerApi.ProcessPayload;
 import org.apache.beam.model.pipeline.v1.RunnerApi.ReadPayload;
+import org.apache.beam.model.pipeline.v1.RunnerApi.StandardEnvironments;
 import org.apache.beam.model.pipeline.v1.RunnerApi.WindowIntoPayload;
 import org.apache.beam.vendor.protobuf.v3.com.google.protobuf.InvalidProtocolBufferException;
 
@@ -55,15 +59,39 @@
   private static final String JAVA_SDK_HARNESS_CONTAINER_URL =
       String.format("%s-docker-apache.bintray.io/beam/java", System.getenv("USER"));
   public static final Environment JAVA_SDK_HARNESS_ENVIRONMENT =
-      Environment.newBuilder().setUrl(JAVA_SDK_HARNESS_CONTAINER_URL).build();
+      createDockerEnvironment(JAVA_SDK_HARNESS_CONTAINER_URL);
 
   private Environments() {}
 
-  public static Environment createOrGetDefaultEnvironment(String url) {
+  public static Environment createOrGetDefaultDockerEnvironment(String url) {
     if (Strings.isNullOrEmpty(url)) {
       return JAVA_SDK_HARNESS_ENVIRONMENT;
     }
-    return Environment.newBuilder().setUrl(url).build();
+    return createDockerEnvironment(url);
+  }
+
+  public static Environment createDockerEnvironment(String dockerImageUrl) {
+    return Environment.newBuilder()
+        .setUrl(dockerImageUrl)
+        .setUrn(BeamUrns.getUrn(StandardEnvironments.Environments.DOCKER))
+        .setPayload(
+            DockerPayload.newBuilder().setContainerImage(dockerImageUrl).build().toByteString())
+        .build();
+  }
+
+  public static Environment createProcessEnvironment(
+      String os, String arch, String command, Map<String, String> env) {
+    return Environment.newBuilder()
+        .setUrn(BeamUrns.getUrn(StandardEnvironments.Environments.PROCESS))
+        .setPayload(
+            ProcessPayload.newBuilder()
+                .setOs(os)
+                .setArch(arch)
+                .setCommand(command)
+                .putAllEnv(env)
+                .build()
+                .toByteString())
+        .build();
   }
 
   public static Optional<Environment> getEnvironment(String ptransformId, Components components) {
diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SdkComponents.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SdkComponents.java
index c9ec5d2d60a..e733396c71b 100644
--- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SdkComponents.java
+++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SdkComponents.java
@@ -62,7 +62,7 @@ public static SdkComponents create() {
   public static SdkComponents create(PipelineOptions options) {
     SdkComponents sdkComponents = new SdkComponents();
     sdkComponents.registerEnvironment(
-        Environments.createOrGetDefaultEnvironment(
+        Environments.createOrGetDefaultDockerEnvironment(
             options.as(PortablePipelineOptions.class).getDefaultJavaEnvironmentUrl()));
     return sdkComponents;
   }
@@ -206,8 +206,7 @@ public String registerEnvironment(Environment env) {
     if (existing != null) {
       return existing;
     }
-    String url = env.getUrl();
-    String name = uniqify(url, environmentIds.values());
+    String name = uniqify(env.getUrn(), environmentIds.values());
     environmentIds.put(env, name);
     componentsBuilder.putEnvironments(name, env);
     return name;
diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/FusedPipeline.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/FusedPipeline.java
index 5b07847266c..d5b802659da 100644
--- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/FusedPipeline.java
+++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/FusedPipeline.java
@@ -107,7 +107,7 @@ static FusedPipeline of(
           String.format(
               "%s/%s",
               stage.getInputPCollection().getPCollection().getUniqueName(),
-              stage.getEnvironment().getUrl());
+              stage.getEnvironment().getUrn());
       Set<String> usedNames =
           Sets.union(topLevelTransforms.keySet(), getComponents().getTransformsMap().keySet());
       String uniqueId = SyntheticComponents.uniqueId(baseName, usedNames::contains);
diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/CoderTranslationTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/CoderTranslationTest.java
index 46b0662f5d2..86559bb8f7b 100644
--- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/CoderTranslationTest.java
+++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/CoderTranslationTest.java
@@ -33,7 +33,6 @@
 import java.util.Set;
 import org.apache.beam.model.pipeline.v1.RunnerApi;
 import org.apache.beam.model.pipeline.v1.RunnerApi.Components;
-import org.apache.beam.model.pipeline.v1.RunnerApi.Environment;
 import org.apache.beam.sdk.coders.AtomicCoder;
 import org.apache.beam.sdk.coders.AvroCoder;
 import org.apache.beam.sdk.coders.ByteArrayCoder;
@@ -135,7 +134,7 @@ public void validateCoderTranslators() {
     @Test
     public void toAndFromProto() throws Exception {
       SdkComponents sdkComponents = SdkComponents.create();
-      sdkComponents.registerEnvironment(Environment.newBuilder().setUrl("java").build());
+      sdkComponents.registerEnvironment(Environments.createDockerEnvironment("java"));
       RunnerApi.Coder coderProto = CoderTranslation.toProto(coder, sdkComponents);
 
       Components encodedComponents = sdkComponents.toComponents();
diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/CombineTranslationTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/CombineTranslationTest.java
index fafc9ac6e60..5518ae9128b 100644
--- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/CombineTranslationTest.java
+++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/CombineTranslationTest.java
@@ -26,7 +26,6 @@
 import java.util.concurrent.atomic.AtomicReference;
 import org.apache.beam.model.pipeline.v1.RunnerApi;
 import org.apache.beam.model.pipeline.v1.RunnerApi.CombinePayload;
-import org.apache.beam.model.pipeline.v1.RunnerApi.Environment;
 import org.apache.beam.sdk.Pipeline.PipelineVisitor;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.coders.CoderRegistry;
@@ -98,7 +97,7 @@ public void leaveCompositeTransform(Node node) {
       assertEquals(combineFn, combine.get().getTransform().getFn());
 
       SdkComponents sdkComponents = SdkComponents.create();
-      sdkComponents.registerEnvironment(Environment.newBuilder().setUrl("java").build());
+      sdkComponents.registerEnvironment(Environments.createDockerEnvironment("java"));
       CombinePayload combineProto = CombineTranslation.toProto(combine.get(), sdkComponents);
       RunnerApi.Components componentsProto = sdkComponents.toComponents();
 
@@ -139,7 +138,7 @@ public void leaveCompositeTransform(Node node) {
       assertEquals(combineFn, combine.get().getTransform().getFn());
 
       SdkComponents sdkComponents = SdkComponents.create();
-      sdkComponents.registerEnvironment(Environment.newBuilder().setUrl("java").build());
+      sdkComponents.registerEnvironment(Environments.createDockerEnvironment("java"));
       CombinePayload combineProto = CombineTranslation.toProto(combine.get(), sdkComponents);
       RunnerApi.Components componentsProto = sdkComponents.toComponents();
 
@@ -184,7 +183,7 @@ public void leaveCompositeTransform(Node node) {
           });
 
       SdkComponents sdkComponents = SdkComponents.create();
-      sdkComponents.registerEnvironment(Environment.newBuilder().setUrl("java").build());
+      sdkComponents.registerEnvironment(Environments.createDockerEnvironment("java"));
       CombinePayload payload = CombineTranslation.toProto(combine.get(), sdkComponents);
     }
   }
diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/CreatePCollectionViewTranslationTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/CreatePCollectionViewTranslationTest.java
index 276fbb367a4..a7e3f76217e 100644
--- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/CreatePCollectionViewTranslationTest.java
+++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/CreatePCollectionViewTranslationTest.java
@@ -21,7 +21,6 @@
 import static org.junit.Assert.assertThat;
 
 import com.google.common.collect.ImmutableList;
-import org.apache.beam.model.pipeline.v1.RunnerApi.Environment;
 import org.apache.beam.model.pipeline.v1.RunnerApi.FunctionSpec;
 import org.apache.beam.sdk.coders.StringUtf8Coder;
 import org.apache.beam.sdk.runners.AppliedPTransform;
@@ -71,7 +70,7 @@
   @Test
   public void testEncodedProto() throws Exception {
     SdkComponents components = SdkComponents.create();
-    components.registerEnvironment(Environment.newBuilder().setUrl("java").build());
+    components.registerEnvironment(Environments.createDockerEnvironment("java"));
     components.registerPCollection(testPCollection);
 
     AppliedPTransform<?, ?, ?> appliedPTransform =
@@ -96,7 +95,7 @@ public void testEncodedProto() throws Exception {
   @Test
   public void testExtractionDirectFromTransform() throws Exception {
     SdkComponents components = SdkComponents.create();
-    components.registerEnvironment(Environment.newBuilder().setUrl("java").build());
+    components.registerEnvironment(Environments.createDockerEnvironment("java"));
     components.registerPCollection(testPCollection);
 
     AppliedPTransform<?, ?, ?> appliedPTransform =
diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/EnvironmentsTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/EnvironmentsTest.java
index 3855e87ba5e..93ea0251cf1 100644
--- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/EnvironmentsTest.java
+++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/EnvironmentsTest.java
@@ -57,7 +57,7 @@
   @Test
   public void getEnvironmentUnknownFnType() throws IOException {
     SdkComponents components = SdkComponents.create();
-    components.registerEnvironment(Environment.newBuilder().setUrl("java").build());
+    components.registerEnvironment(Environments.createDockerEnvironment("java"));
     RehydratedComponents rehydratedComponents =
         RehydratedComponents.forComponents(components.toComponents());
     PTransform builder =
@@ -74,7 +74,7 @@ public void getEnvironmentUnknownFnType() throws IOException {
   @Test
   public void getEnvironmentParDo() throws IOException {
     SdkComponents components = SdkComponents.create();
-    components.registerEnvironment(Environment.newBuilder().setUrl("java").build());
+    components.registerEnvironment(Environments.createDockerEnvironment("java"));
     ParDoPayload payload =
         ParDoTranslation.translateParDo(
             ParDo.of(
@@ -107,7 +107,7 @@ public void process(ProcessContext ctxt) {}
   @Test
   public void getEnvironmentWindowIntoKnown() throws IOException {
     SdkComponents components = SdkComponents.create();
-    components.registerEnvironment(Environment.newBuilder().setUrl("java").build());
+    components.registerEnvironment(Environments.createDockerEnvironment("java"));
     WindowIntoPayload payload =
         WindowIntoPayload.newBuilder()
             .setWindowFn(
@@ -136,7 +136,7 @@ public void getEnvironmentWindowIntoKnown() throws IOException {
   @Test
   public void getEnvironmentWindowIntoCustom() throws IOException {
     SdkComponents components = SdkComponents.create();
-    components.registerEnvironment(Environment.newBuilder().setUrl("java").build());
+    components.registerEnvironment(Environments.createDockerEnvironment("java"));
     WindowIntoPayload payload =
         WindowIntoPayload.newBuilder()
             .setWindowFn(
@@ -181,7 +181,7 @@ public boolean isCompatible(WindowFn<?, ?> other) {
   @Test
   public void getEnvironmentRead() throws IOException {
     SdkComponents components = SdkComponents.create();
-    components.registerEnvironment(Environment.newBuilder().setUrl("java").build());
+    components.registerEnvironment(Environments.createDockerEnvironment("java"));
     ReadPayload payload =
         ReadTranslation.toProto(Read.from(CountingSource.unbounded()), components);
     RehydratedComponents rehydratedComponents =
@@ -206,7 +206,7 @@ public void getEnvironmentRead() throws IOException {
   @Test
   public void getEnvironmentCombine() throws IOException {
     SdkComponents components = SdkComponents.create();
-    components.registerEnvironment(Environment.newBuilder().setUrl("java").build());
+    components.registerEnvironment(Environments.createDockerEnvironment("java"));
     CombinePayload payload =
         CombinePayload.newBuilder()
             .setCombineFn(CombineTranslation.toProto(Sum.ofLongs(), components))
diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PCollectionTranslationTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PCollectionTranslationTest.java
index 0e271aa17dd..e3530797d8d 100644
--- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PCollectionTranslationTest.java
+++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PCollectionTranslationTest.java
@@ -29,7 +29,6 @@
 import java.util.Collection;
 import java.util.Collections;
 import org.apache.beam.model.pipeline.v1.RunnerApi;
-import org.apache.beam.model.pipeline.v1.RunnerApi.Environment;
 import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.coders.AtomicCoder;
 import org.apache.beam.sdk.coders.BigEndianLongCoder;
@@ -110,7 +109,7 @@
   public void testEncodeDecodeCycle() throws Exception {
     // Encode
     SdkComponents sdkComponents = SdkComponents.create();
-    sdkComponents.registerEnvironment(Environment.newBuilder().setUrl("java").build());
+    sdkComponents.registerEnvironment(Environments.createDockerEnvironment("java"));
     RunnerApi.PCollection protoCollection =
         PCollectionTranslation.toProto(testCollection, sdkComponents);
     RehydratedComponents protoComponents =
@@ -132,7 +131,7 @@ public void testEncodeDecodeCycle() throws Exception {
   @Test
   public void testEncodeDecodeFields() throws Exception {
     SdkComponents sdkComponents = SdkComponents.create();
-    sdkComponents.registerEnvironment(Environment.newBuilder().setUrl("java").build());
+    sdkComponents.registerEnvironment(Environments.createDockerEnvironment("java"));
     RunnerApi.PCollection protoCollection =
         PCollectionTranslation.toProto(testCollection, sdkComponents);
     RehydratedComponents protoComponents =
diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PCollectionViewTranslationTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PCollectionViewTranslationTest.java
index 15a88891919..2b460a36fe5 100644
--- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PCollectionViewTranslationTest.java
+++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PCollectionViewTranslationTest.java
@@ -20,7 +20,6 @@
 
 import static org.junit.Assert.assertEquals;
 
-import org.apache.beam.model.pipeline.v1.RunnerApi.Environment;
 import org.apache.beam.sdk.transforms.Materialization;
 import org.apache.beam.sdk.transforms.ViewFn;
 import org.apache.beam.sdk.transforms.windowing.GlobalWindows;
@@ -34,7 +33,7 @@
   @Test
   public void testViewFnTranslation() throws Exception {
     SdkComponents sdkComponents = SdkComponents.create();
-    sdkComponents.registerEnvironment(Environment.newBuilder().setUrl("java").build());
+    sdkComponents.registerEnvironment(Environments.createDockerEnvironment("java"));
     assertEquals(
         new TestViewFn(),
         PCollectionViewTranslation.viewFnFromProto(
@@ -44,7 +43,7 @@ public void testViewFnTranslation() throws Exception {
   @Test
   public void testWindowMappingFnTranslation() throws Exception {
     SdkComponents sdkComponents = SdkComponents.create();
-    sdkComponents.registerEnvironment(Environment.newBuilder().setUrl("java").build());
+    sdkComponents.registerEnvironment(Environments.createDockerEnvironment("java"));
     assertEquals(
         new GlobalWindows().getDefaultWindowMappingFn(),
         PCollectionViewTranslation.windowMappingFnFromProto(
diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ParDoTranslationTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ParDoTranslationTest.java
index aba0a2134f1..4290ab81251 100644
--- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ParDoTranslationTest.java
+++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ParDoTranslationTest.java
@@ -27,7 +27,6 @@
 import java.util.HashMap;
 import java.util.Map;
 import org.apache.beam.model.pipeline.v1.RunnerApi;
-import org.apache.beam.model.pipeline.v1.RunnerApi.Environment;
 import org.apache.beam.model.pipeline.v1.RunnerApi.ParDoPayload;
 import org.apache.beam.model.pipeline.v1.RunnerApi.SideInput;
 import org.apache.beam.sdk.coders.Coder;
@@ -117,7 +116,7 @@
     @Test
     public void testToProto() throws Exception {
       SdkComponents components = SdkComponents.create();
-      components.registerEnvironment(Environment.newBuilder().setUrl("java").build());
+      components.registerEnvironment(Environments.createDockerEnvironment("java"));
       ParDoPayload payload = ParDoTranslation.translateParDo(parDo, p, components);
 
       assertThat(ParDoTranslation.getDoFn(payload), equalTo(parDo.getFn()));
@@ -135,7 +134,7 @@ public void toTransformProto() throws Exception {
       PCollectionTuple output = mainInput.apply(parDo);
 
       SdkComponents sdkComponents = SdkComponents.create();
-      sdkComponents.registerEnvironment(Environment.newBuilder().setUrl("java").build());
+      sdkComponents.registerEnvironment(Environments.createDockerEnvironment("java"));
 
       // Encode
       RunnerApi.PTransform protoTransform =
@@ -222,7 +221,7 @@ public void toTransformProto() throws Exception {
     public void testStateSpecToFromProto() throws Exception {
       // Encode
       SdkComponents sdkComponents = SdkComponents.create();
-      sdkComponents.registerEnvironment(Environment.newBuilder().setUrl("java").build());
+      sdkComponents.registerEnvironment(Environments.createDockerEnvironment("java"));
       RunnerApi.StateSpec stateSpecProto =
           ParDoTranslation.translateStateSpec(stateSpec, sdkComponents);
 
diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ReadTranslationTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ReadTranslationTest.java
index 766e221c390..958c87af2de 100644
--- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ReadTranslationTest.java
+++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ReadTranslationTest.java
@@ -30,7 +30,6 @@
 import java.util.List;
 import javax.annotation.Nullable;
 import org.apache.beam.model.pipeline.v1.RunnerApi;
-import org.apache.beam.model.pipeline.v1.RunnerApi.Environment;
 import org.apache.beam.model.pipeline.v1.RunnerApi.ReadPayload;
 import org.apache.beam.sdk.coders.AtomicCoder;
 import org.apache.beam.sdk.coders.ByteArrayCoder;
@@ -73,7 +72,7 @@ public void testToFromProtoBounded() throws Exception {
     BoundedSource<?> boundedSource = (BoundedSource<?>) this.source;
     Read.Bounded<?> boundedRead = Read.from(boundedSource);
     SdkComponents components = SdkComponents.create();
-    components.registerEnvironment(Environment.newBuilder().setUrl("java").build());
+    components.registerEnvironment(Environments.createDockerEnvironment("java"));
     ReadPayload payload = ReadTranslation.toProto(boundedRead, components);
     assertThat(payload.getIsBounded(), equalTo(RunnerApi.IsBounded.Enum.BOUNDED));
     BoundedSource<?> deserializedSource = ReadTranslation.boundedSourceFromProto(payload);
@@ -86,7 +85,7 @@ public void testToFromProtoUnbounded() throws Exception {
     UnboundedSource<?, ?> unboundedSource = (UnboundedSource<?, ?>) this.source;
     Read.Unbounded<?> unboundedRead = Read.from(unboundedSource);
     SdkComponents components = SdkComponents.create();
-    components.registerEnvironment(Environment.newBuilder().setUrl("java").build());
+    components.registerEnvironment(Environments.createDockerEnvironment("java"));
     ReadPayload payload = ReadTranslation.toProto(unboundedRead, components);
     assertThat(payload.getIsBounded(), equalTo(RunnerApi.IsBounded.Enum.UNBOUNDED));
     UnboundedSource<?, ?> deserializedSource = ReadTranslation.unboundedSourceFromProto(payload);
diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/RehydratedComponentsTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/RehydratedComponentsTest.java
index 08f69e31d3c..23071056a6c 100644
--- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/RehydratedComponentsTest.java
+++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/RehydratedComponentsTest.java
@@ -45,7 +45,7 @@
   @Test
   public void testSimpleCoder() throws Exception {
     SdkComponents sdkComponents = SdkComponents.create();
-    sdkComponents.registerEnvironment(Environment.newBuilder().setUrl("java").build());
+    sdkComponents.registerEnvironment(Environments.createDockerEnvironment("java"));
     Coder<?> coder = VarIntCoder.of();
     String id = sdkComponents.registerCoder(coder);
     RehydratedComponents rehydratedComponents =
@@ -59,7 +59,7 @@ public void testSimpleCoder() throws Exception {
   @Test
   public void testCompoundCoder() throws Exception {
     SdkComponents sdkComponents = SdkComponents.create();
-    sdkComponents.registerEnvironment(Environment.newBuilder().setUrl("java").build());
+    sdkComponents.registerEnvironment(Environments.createDockerEnvironment("java"));
     Coder<?> coder = VarIntCoder.of();
     Coder<?> compoundCoder = NullableCoder.of(coder);
     String compoundCoderId = sdkComponents.registerCoder(compoundCoder);
@@ -83,7 +83,7 @@ public void testCompoundCoder() throws Exception {
   @Test
   public void testWindowingStrategy() throws Exception {
     SdkComponents sdkComponents = SdkComponents.create();
-    sdkComponents.registerEnvironment(Environment.newBuilder().setUrl("java").build());
+    sdkComponents.registerEnvironment(Environments.createDockerEnvironment("java"));
     WindowingStrategy windowingStrategy =
         WindowingStrategy.of(FixedWindows.of(Duration.millis(1)))
             .withAllowedLateness(Duration.standardSeconds(4));
@@ -101,8 +101,8 @@ public void testWindowingStrategy() throws Exception {
   @Test
   public void testEnvironment() {
     SdkComponents sdkComponents = SdkComponents.create();
-    sdkComponents.registerEnvironment(Environment.newBuilder().setUrl("java").build());
-    Environment env = Environment.newBuilder().setUrl("java_test").build();
+    sdkComponents.registerEnvironment(Environments.createDockerEnvironment("java"));
+    Environment env = Environments.createDockerEnvironment("java_test");
     String id = sdkComponents.registerEnvironment(env);
     RehydratedComponents rehydratedComponents =
         RehydratedComponents.forComponents(sdkComponents.toComponents());
diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/TestStreamTranslationTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/TestStreamTranslationTest.java
index 39a0cfb579c..73950387086 100644
--- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/TestStreamTranslationTest.java
+++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/TestStreamTranslationTest.java
@@ -24,7 +24,6 @@
 
 import com.google.common.collect.ImmutableList;
 import org.apache.beam.model.pipeline.v1.RunnerApi;
-import org.apache.beam.model.pipeline.v1.RunnerApi.Environment;
 import org.apache.beam.model.pipeline.v1.RunnerApi.TestStreamPayload;
 import org.apache.beam.sdk.coders.StringUtf8Coder;
 import org.apache.beam.sdk.coders.VarIntCoder;
@@ -68,7 +67,7 @@
   @Test
   public void testEncodedProto() throws Exception {
     SdkComponents components = SdkComponents.create();
-    components.registerEnvironment(Environment.newBuilder().setUrl("java").build());
+    components.registerEnvironment(Environments.createDockerEnvironment("java"));
     RunnerApi.TestStreamPayload payload =
         TestStreamTranslation.payloadForTestStream(testStream, components);
 
@@ -84,7 +83,7 @@ public void testRegistrarEncodedProto() throws Exception {
         AppliedPTransform.of("fakeName", PBegin.in(p).expand(), output.expand(), testStream, p);
 
     SdkComponents components = SdkComponents.create();
-    components.registerEnvironment(Environment.newBuilder().setUrl("java").build());
+    components.registerEnvironment(Environments.createDockerEnvironment("java"));
     RunnerApi.FunctionSpec spec =
         PTransformTranslation.toProto(appliedTestStream, components).getSpec();
 
diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/WindowIntoTranslationTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/WindowIntoTranslationTest.java
index 501ee304b04..0300066c8a8 100644
--- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/WindowIntoTranslationTest.java
+++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/WindowIntoTranslationTest.java
@@ -23,7 +23,6 @@
 
 import com.google.common.collect.ImmutableList;
 import java.util.concurrent.atomic.AtomicReference;
-import org.apache.beam.model.pipeline.v1.RunnerApi.Environment;
 import org.apache.beam.model.pipeline.v1.RunnerApi.WindowIntoPayload;
 import org.apache.beam.sdk.Pipeline.PipelineVisitor;
 import org.apache.beam.sdk.coders.Coder;
@@ -90,7 +89,7 @@ public void visitPrimitiveTransform(Node node) {
     checkState(assign.get() != null);
 
     SdkComponents components = SdkComponents.create();
-    components.registerEnvironment(Environment.newBuilder().setUrl("java").build());
+    components.registerEnvironment(Environments.createDockerEnvironment("java"));
     WindowIntoPayload payload =
         WindowIntoTranslation.toProto(assign.get().getTransform(), components);
 
diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/WindowingStrategyTranslationTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/WindowingStrategyTranslationTest.java
index fde368182fc..bfa25a9b7cd 100644
--- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/WindowingStrategyTranslationTest.java
+++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/WindowingStrategyTranslationTest.java
@@ -23,7 +23,6 @@
 import com.google.auto.value.AutoValue;
 import com.google.common.collect.ImmutableList;
 import org.apache.beam.model.pipeline.v1.RunnerApi;
-import org.apache.beam.model.pipeline.v1.RunnerApi.Environment;
 import org.apache.beam.sdk.transforms.windowing.AfterWatermark;
 import org.apache.beam.sdk.transforms.windowing.FixedWindows;
 import org.apache.beam.sdk.transforms.windowing.Sessions;
@@ -96,7 +95,7 @@ private static ToProtoAndBackSpec toProtoAndBackSpec(WindowingStrategy windowing
   public void testToProtoAndBack() throws Exception {
     WindowingStrategy<?, ?> windowingStrategy = toProtoAndBackSpec.getWindowingStrategy();
     SdkComponents components = SdkComponents.create();
-    components.registerEnvironment(Environment.newBuilder().setUrl("java").build());
+    components.registerEnvironment(Environments.createDockerEnvironment("java"));
     WindowingStrategy<?, ?> toProtoAndBackWindowingStrategy =
         WindowingStrategyTranslation.fromProto(
             WindowingStrategyTranslation.toMessageProto(windowingStrategy, components));
@@ -110,7 +109,7 @@ public void testToProtoAndBack() throws Exception {
   public void testToProtoAndBackWithComponents() throws Exception {
     WindowingStrategy<?, ?> windowingStrategy = toProtoAndBackSpec.getWindowingStrategy();
     SdkComponents components = SdkComponents.create();
-    components.registerEnvironment(Environment.newBuilder().setUrl("java").build());
+    components.registerEnvironment(Environments.createDockerEnvironment("java"));
     RunnerApi.WindowingStrategy proto =
         WindowingStrategyTranslation.toProto(windowingStrategy, components);
     RehydratedComponents protoComponents =
diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/WriteFilesTranslationTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/WriteFilesTranslationTest.java
index d26de9a196c..baf5bbc83f2 100644
--- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/WriteFilesTranslationTest.java
+++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/WriteFilesTranslationTest.java
@@ -25,7 +25,6 @@
 import java.util.Objects;
 import javax.annotation.Nullable;
 import org.apache.beam.model.pipeline.v1.RunnerApi;
-import org.apache.beam.model.pipeline.v1.RunnerApi.Environment;
 import org.apache.beam.sdk.io.DynamicFileDestinations;
 import org.apache.beam.sdk.io.FileBasedSink;
 import org.apache.beam.sdk.io.FileBasedSink.FilenamePolicy;
@@ -68,7 +67,7 @@
   @Test
   public void testEncodedProto() throws Exception {
     SdkComponents components = SdkComponents.create();
-    components.registerEnvironment(Environment.newBuilder().setUrl("java").build());
+    components.registerEnvironment(Environments.createDockerEnvironment("java"));
     RunnerApi.WriteFilesPayload payload =
         WriteFilesTranslation.payloadForWriteFiles(writeFiles, components);
 
diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/ExecutableStageTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/ExecutableStageTest.java
index a98799328fc..e6daf03035c 100644
--- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/ExecutableStageTest.java
+++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/ExecutableStageTest.java
@@ -40,6 +40,7 @@
 import org.apache.beam.model.pipeline.v1.RunnerApi.StateSpec;
 import org.apache.beam.model.pipeline.v1.RunnerApi.TimerSpec;
 import org.apache.beam.model.pipeline.v1.RunnerApi.WindowIntoPayload;
+import org.apache.beam.runners.core.construction.Environments;
 import org.apache.beam.runners.core.construction.PTransformTranslation;
 import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode;
 import org.junit.Test;
@@ -51,7 +52,8 @@
 public class ExecutableStageTest {
   @Test
   public void testRoundTripToFromTransform() throws Exception {
-    Environment env = Environment.newBuilder().setUrl("foo").build();
+    Environment env =
+        org.apache.beam.runners.core.construction.Environments.createDockerEnvironment("foo");
     PTransform pt =
         PTransform.newBuilder()
             .putInputs("input", "input.out")
@@ -166,7 +168,7 @@ public void testRoundTripToFromTransformFused() throws Exception {
             .putTransforms("window", windowTransform)
             .putPcollections(
                 "window.out", PCollection.newBuilder().setUniqueName("window.out").build())
-            .putEnvironments("common", Environment.newBuilder().setUrl("common").build())
+            .putEnvironments("common", Environments.createDockerEnvironment("common"))
             .build();
     QueryablePipeline p = QueryablePipeline.forPrimitivesIn(components);
 
diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/GreedyPipelineFuserTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/GreedyPipelineFuserTest.java
index c47f743013f..7e8d35b22e5 100644
--- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/GreedyPipelineFuserTest.java
+++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/GreedyPipelineFuserTest.java
@@ -37,7 +37,6 @@
 import org.apache.beam.model.pipeline.v1.RunnerApi;
 import org.apache.beam.model.pipeline.v1.RunnerApi.Coder;
 import org.apache.beam.model.pipeline.v1.RunnerApi.Components;
-import org.apache.beam.model.pipeline.v1.RunnerApi.Environment;
 import org.apache.beam.model.pipeline.v1.RunnerApi.FunctionSpec;
 import org.apache.beam.model.pipeline.v1.RunnerApi.PCollection;
 import org.apache.beam.model.pipeline.v1.RunnerApi.PTransform;
@@ -49,6 +48,7 @@
 import org.apache.beam.model.pipeline.v1.RunnerApi.TimerSpec;
 import org.apache.beam.model.pipeline.v1.RunnerApi.WindowIntoPayload;
 import org.apache.beam.model.pipeline.v1.RunnerApi.WindowingStrategy;
+import org.apache.beam.runners.core.construction.Environments;
 import org.apache.beam.runners.core.construction.PTransformTranslation;
 import org.apache.beam.runners.core.construction.graph.PipelineNode.PCollectionNode;
 import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode;
@@ -79,8 +79,8 @@ public void setup() {
                             .setUrn(PTransformTranslation.IMPULSE_TRANSFORM_URN))
                     .build())
             .putPcollections("impulse.out", pc("impulse.out"))
-            .putEnvironments("go", Environment.newBuilder().setUrl("go").build())
-            .putEnvironments("py", Environment.newBuilder().setUrl("py").build())
+            .putEnvironments("go", Environments.createDockerEnvironment("go"))
+            .putEnvironments("py", Environments.createDockerEnvironment("py"))
             .putCoders("coder", Coder.newBuilder().build())
             .putCoders("windowCoder", Coder.newBuilder().build())
             .putWindowingStrategies(
@@ -486,8 +486,8 @@ public void flattenWithHeterogenousInputsAndOutputsEntirelyMaterialized() {
                                     .toByteString()))
                     .build())
             .putPcollections("goParDo.out", pc("goParDo.out"))
-            .putEnvironments("go", Environment.newBuilder().setUrl("go").build())
-            .putEnvironments("py", Environment.newBuilder().setUrl("py").build())
+            .putEnvironments("go", Environments.createDockerEnvironment("go"))
+            .putEnvironments("py", Environments.createDockerEnvironment("py"))
             .build();
     FusedPipeline fused =
         GreedyPipelineFuser.fuse(Pipeline.newBuilder().setComponents(components).build());
@@ -645,8 +645,8 @@ public void flattenWithHeterogeneousInputsSingleEnvOutputPartiallyMaterialized()
                                     .toByteString()))
                     .build())
             .putPcollections("goParDo.out", pc("goParDo.out"))
-            .putEnvironments("go", Environment.newBuilder().setUrl("go").build())
-            .putEnvironments("py", Environment.newBuilder().setUrl("py").build())
+            .putEnvironments("go", Environments.createDockerEnvironment("go"))
+            .putEnvironments("py", Environments.createDockerEnvironment("py"))
             .build();
     FusedPipeline fused =
         GreedyPipelineFuser.fuse(Pipeline.newBuilder().setComponents(components).build());
@@ -874,7 +874,7 @@ public void sideInputRootsNewStage() {
                             .build())
                     .build())
             .putPcollections("sideParDo.out", pc("sideParDo.out"))
-            .putEnvironments("py", Environment.newBuilder().setUrl("py").build())
+            .putEnvironments("py", Environments.createDockerEnvironment("py"))
             .build();
 
     FusedPipeline fused =
@@ -956,7 +956,7 @@ public void statefulParDoRootsStage() {
             .putPcollections("parDo.out", pc("parDo.out"))
             .putTransforms("stateful", statefulTransform)
             .putPcollections("stateful.out", pc("stateful.out"))
-            .putEnvironments("common", Environment.newBuilder().setUrl("common").build())
+            .putEnvironments("common", Environments.createDockerEnvironment("common"))
             .build();
     FusedPipeline fused =
         GreedyPipelineFuser.fuse(Pipeline.newBuilder().setComponents(components).build());
@@ -1027,7 +1027,7 @@ public void parDoWithTimerRootsStage() {
             .putTransforms("timer", timerTransform)
             .putPcollections("timer.out", pc("timer.out"))
             .putPcollections("output.out", pc("output.out"))
-            .putEnvironments("common", Environment.newBuilder().setUrl("common").build())
+            .putEnvironments("common", Environments.createDockerEnvironment("common"))
             .build();
 
     FusedPipeline fused =
@@ -1236,7 +1236,7 @@ public void sanitizedTransforms() throws Exception {
                     .putWindowingStrategies(
                         "ws",
                         WindowingStrategy.newBuilder().setWindowCoderId("windowCoder").build())
-                    .putEnvironments("py", Environment.newBuilder().setUrl("py").build())
+                    .putEnvironments("py", Environments.createDockerEnvironment("py"))
                     .putPcollections(flattenOutput.getUniqueName(), flattenOutput)
                     .putTransforms(flattenTransform.getUniqueName(), flattenTransform)
                     .putPcollections(read1Output.getUniqueName(), read1Output)
diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/GreedyStageFuserTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/GreedyStageFuserTest.java
index a1175afcffb..d22fd0e9512 100644
--- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/GreedyStageFuserTest.java
+++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/GreedyStageFuserTest.java
@@ -41,6 +41,7 @@
 import org.apache.beam.model.pipeline.v1.RunnerApi.StateSpec;
 import org.apache.beam.model.pipeline.v1.RunnerApi.TimerSpec;
 import org.apache.beam.model.pipeline.v1.RunnerApi.WindowIntoPayload;
+import org.apache.beam.runners.core.construction.Environments;
 import org.apache.beam.runners.core.construction.PTransformTranslation;
 import org.apache.beam.runners.core.construction.graph.PipelineNode.PCollectionNode;
 import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode;
@@ -141,8 +142,8 @@ public void differentEnvironmentsThrows() {
                                         .toByteString()))
                         .build())
                 .putPcollections("py.out", PCollection.newBuilder().setUniqueName("py.out").build())
-                .putEnvironments("go", Environment.newBuilder().setUrl("go").build())
-                .putEnvironments("py", Environment.newBuilder().setUrl("py").build())
+                .putEnvironments("go", Environments.createDockerEnvironment("go"))
+                .putEnvironments("py", Environments.createDockerEnvironment("py"))
                 .build());
     Set<PTransformNode> differentEnvironments =
         p.getPerElementConsumers(
@@ -229,7 +230,7 @@ public void fusesCompatibleEnvironments() {
                 .putTransforms("window", windowTransform)
                 .putPcollections(
                     "window.out", PCollection.newBuilder().setUniqueName("window.out").build())
-                .putEnvironments("common", Environment.newBuilder().setUrl("common").build())
+                .putEnvironments("common", Environments.createDockerEnvironment("common"))
                 .build());
 
     ExecutableStage subgraph =
@@ -287,7 +288,7 @@ public void materializesWithStatefulConsumer() {
                 .putTransforms("stateful", statefulTransform)
                 .putPcollections(
                     "stateful.out", PCollection.newBuilder().setUniqueName("stateful.out").build())
-                .putEnvironments("common", Environment.newBuilder().setUrl("common").build())
+                .putEnvironments("common", Environments.createDockerEnvironment("common"))
                 .build());
 
     ExecutableStage subgraph =
@@ -346,7 +347,7 @@ public void materializesWithConsumerWithTimer() {
                 .putTransforms("timer", timerTransform)
                 .putPcollections(
                     "timer.out", PCollection.newBuilder().setUniqueName("timer.out").build())
-                .putEnvironments("common", Environment.newBuilder().setUrl("common").build())
+                .putEnvironments("common", Environments.createDockerEnvironment("common"))
                 .build());
 
     ExecutableStage subgraph =
@@ -432,7 +433,7 @@ public void fusesFlatten() {
                 .putTransforms("window", windowTransform)
                 .putPcollections(
                     "window.out", PCollection.newBuilder().setUniqueName("window.out").build())
-                .putEnvironments("common", Environment.newBuilder().setUrl("common").build())
+                .putEnvironments("common", Environments.createDockerEnvironment("common"))
                 .build());
 
     ExecutableStage subgraph =
@@ -512,8 +513,8 @@ public void fusesFlattenWithDifferentEnvironmentInputs() {
             .putTransforms("window", windowTransform)
             .putPcollections(
                 "window.out", PCollection.newBuilder().setUniqueName("window.out").build())
-            .putEnvironments("common", Environment.newBuilder().setUrl("common").build())
-            .putEnvironments("rare", Environment.newBuilder().setUrl("rare").build())
+            .putEnvironments("common", Environments.createDockerEnvironment("common"))
+            .putEnvironments("rare", Environments.createDockerEnvironment("rare"))
             .build();
     QueryablePipeline p = QueryablePipeline.forPrimitivesIn(components);
 
@@ -634,8 +635,8 @@ public void flattenWithHeterogeneousInputsAndOutputs() {
             .putTransforms("goWindow", goWindow)
             .putPcollections(
                 "goWindow.out", PCollection.newBuilder().setUniqueName("goWindow.out").build())
-            .putEnvironments("go", Environment.newBuilder().setUrl("go").build())
-            .putEnvironments("py", Environment.newBuilder().setUrl("py").build())
+            .putEnvironments("go", Environments.createDockerEnvironment("go"))
+            .putEnvironments("py", Environments.createDockerEnvironment("py"))
             .build();
     QueryablePipeline p = QueryablePipeline.forPrimitivesIn(components);
 
@@ -668,7 +669,7 @@ public void materializesWithDifferentEnvConsumer() {
     // Fuses into
     // (impulse.out) -> parDo -> (parDo.out)
     // (parDo.out) -> window -> window.out
-    Environment env = Environment.newBuilder().setUrl("common").build();
+    Environment env = Environments.createDockerEnvironment("common");
     PTransform parDoTransform =
         PTransform.newBuilder()
             .putInputs("input", "impulse.out")
@@ -707,7 +708,7 @@ public void materializesWithDifferentEnvConsumer() {
                         .build())
                 .putPcollections(
                     "window.out", PCollection.newBuilder().setUniqueName("window.out").build())
-                .putEnvironments("rare", Environment.newBuilder().setUrl("rare").build())
+                .putEnvironments("rare", Environments.createDockerEnvironment("rare"))
                 .putEnvironments("common", env)
                 .build());
 
@@ -735,7 +736,7 @@ public void materializesWithDifferentEnvSibling() {
     // The window can't be fused into the stage, which forces the PCollection to be materialized.
     // ParDo in this case _could_ be fused into the stage, but is not for simplicity of
     // implementation
-    Environment env = Environment.newBuilder().setUrl("common").build();
+    Environment env = Environments.createDockerEnvironment("common");
     PTransform readTransform =
         PTransform.newBuilder()
             .putInputs("input", "impulse.out")
@@ -790,7 +791,7 @@ public void materializesWithDifferentEnvSibling() {
                         .build())
                 .putPcollections(
                     "window.out", PCollection.newBuilder().setUniqueName("window.out").build())
-                .putEnvironments("rare", Environment.newBuilder().setUrl("rare").build())
+                .putEnvironments("rare", Environments.createDockerEnvironment("rare"))
                 .putEnvironments("common", env)
                 .build());
 
@@ -814,7 +815,7 @@ public void materializesWithSideInputConsumer() {
     // parDo doesn't have a per-element consumer from side_read.out, so it can't root a stage
     // which consumes from that materialized collection. Nodes with side inputs must root a stage,
     // but do not restrict fusion of consumers.
-    Environment env = Environment.newBuilder().setUrl("common").build();
+    Environment env = Environments.createDockerEnvironment("common");
     PTransform readTransform =
         PTransform.newBuilder()
             .putInputs("input", "impulse.out")
@@ -894,7 +895,7 @@ public void materializesWithSideInputConsumer() {
 
   @Test
   public void sideInputIncludedInStage() {
-    Environment env = Environment.newBuilder().setUrl("common").build();
+    Environment env = Environments.createDockerEnvironment("common");
     PTransform readTransform =
         PTransform.newBuilder()
             .setUniqueName("read")
@@ -969,7 +970,7 @@ public void executableStageProducingSideInputMaterializesIt() {
     // impulse -- ParDo(createSide)
     //         \_ ParDo(processMain) with side input from createSide
     // The ExecutableStage executing createSide must have an output.
-    Environment env = Environment.newBuilder().setUrl("common").build();
+    Environment env = Environments.createDockerEnvironment("common");
     PTransform impulse =
         PTransform.newBuilder()
             .setUniqueName("impulse")
@@ -1031,7 +1032,7 @@ public void executableStageProducingSideInputMaterializesIt() {
 
   @Test
   public void userStateIncludedInStage() {
-    Environment env = Environment.newBuilder().setUrl("common").build();
+    Environment env = Environments.createDockerEnvironment("common");
     PTransform readTransform =
         PTransform.newBuilder()
             .putInputs("input", "impulse.out")
@@ -1104,7 +1105,7 @@ public void materializesWithGroupByKeyConsumer() {
     // Fuses to
     // (impulse.out) -> read -> (read.out)
     // GBK is the responsibility of the runner, so it is not included in a stage.
-    Environment env = Environment.newBuilder().setUrl("common").build();
+    Environment env = Environments.createDockerEnvironment("common");
     PTransform readTransform =
         PTransform.newBuilder()
             .putInputs("input", "impulse.out")
diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/ImmutableExecutableStageTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/ImmutableExecutableStageTest.java
index 41b1e6c1d4a..3c7c5b20bc7 100644
--- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/ImmutableExecutableStageTest.java
+++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/ImmutableExecutableStageTest.java
@@ -33,6 +33,7 @@
 import org.apache.beam.model.pipeline.v1.RunnerApi.PCollection;
 import org.apache.beam.model.pipeline.v1.RunnerApi.PTransform;
 import org.apache.beam.model.pipeline.v1.RunnerApi.ParDoPayload;
+import org.apache.beam.runners.core.construction.Environments;
 import org.apache.beam.runners.core.construction.PTransformTranslation;
 import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode;
 import org.junit.Test;
@@ -44,7 +45,7 @@
 public class ImmutableExecutableStageTest {
   @Test
   public void ofFullComponentsOnlyHasStagePTransforms() throws Exception {
-    Environment env = Environment.newBuilder().setUrl("foo").build();
+    Environment env = Environments.createDockerEnvironment("foo");
     PTransform pt =
         PTransform.newBuilder()
             .putInputs("input", "input.out")
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/portable/BundleFactoryOutputReceiverFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/portable/BundleFactoryOutputReceiverFactoryTest.java
index dc34e0607b9..35849f07c02 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/portable/BundleFactoryOutputReceiverFactoryTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/portable/BundleFactoryOutputReceiverFactoryTest.java
@@ -31,7 +31,7 @@
 import org.apache.beam.model.pipeline.v1.RunnerApi;
 import org.apache.beam.model.pipeline.v1.RunnerApi.Components;
 import org.apache.beam.model.pipeline.v1.RunnerApi.Components.Builder;
-import org.apache.beam.model.pipeline.v1.RunnerApi.Environment;
+import org.apache.beam.runners.core.construction.Environments;
 import org.apache.beam.runners.core.construction.RehydratedComponents;
 import org.apache.beam.runners.core.construction.SdkComponents;
 import org.apache.beam.runners.core.construction.graph.PipelineNode;
@@ -76,7 +76,7 @@ public void setup() throws IOException {
     PCollection<Integer> bar = p.apply("bar", Create.of(1, 2, 3));
 
     SdkComponents sdkComponents = SdkComponents.create();
-    sdkComponents.registerEnvironment(Environment.newBuilder().setUrl("java").build());
+    sdkComponents.registerEnvironment(Environments.createDockerEnvironment("java"));
     String fooId = sdkComponents.registerPCollection(foo);
     String barId = sdkComponents.registerPCollection(bar);
     baseComponents = sdkComponents.toComponents();
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/portable/GroupByKeyOnlyEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/portable/GroupByKeyOnlyEvaluatorFactoryTest.java
index a5b191ddedc..8a19ef7f10b 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/portable/GroupByKeyOnlyEvaluatorFactoryTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/portable/GroupByKeyOnlyEvaluatorFactoryTest.java
@@ -25,12 +25,12 @@
 import com.google.common.collect.Multiset;
 import org.apache.beam.model.pipeline.v1.RunnerApi;
 import org.apache.beam.model.pipeline.v1.RunnerApi.Components;
-import org.apache.beam.model.pipeline.v1.RunnerApi.Environment;
 import org.apache.beam.model.pipeline.v1.RunnerApi.FunctionSpec;
 import org.apache.beam.model.pipeline.v1.RunnerApi.PTransform;
 import org.apache.beam.model.pipeline.v1.RunnerApi.Pipeline;
 import org.apache.beam.runners.core.KeyedWorkItem;
 import org.apache.beam.runners.core.KeyedWorkItems;
+import org.apache.beam.runners.core.construction.Environments;
 import org.apache.beam.runners.core.construction.RehydratedComponents;
 import org.apache.beam.runners.core.construction.SdkComponents;
 import org.apache.beam.runners.core.construction.graph.PipelineNode;
@@ -62,7 +62,7 @@
   public void testInMemoryEvaluator() throws Exception {
     KvCoder<String, Integer> javaCoder = KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of());
     SdkComponents sdkComponents = SdkComponents.create();
-    sdkComponents.registerEnvironment(Environment.newBuilder().setUrl("java").build());
+    sdkComponents.registerEnvironment(Environments.createDockerEnvironment("java"));
     String windowingStrategyId =
         sdkComponents.registerWindowingStrategy(WindowingStrategy.globalDefault());
     String coderId = sdkComponents.registerCoder(javaCoder);
diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/JobBundleFactoryBase.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/JobBundleFactoryBase.java
index d1717268615..551dbe82c3b 100644
--- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/JobBundleFactoryBase.java
+++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/JobBundleFactoryBase.java
@@ -142,7 +142,7 @@ abstract EnvironmentFactory getEnvironmentFactory(
     return CacheBuilder.newBuilder()
         .removalListener(
             ((RemovalNotification<Environment, WrappedSdkHarnessClient> notification) -> {
-              LOG.debug("Cleaning up for environment {}", notification.getKey().getUrl());
+              LOG.debug("Cleaning up for environment {}", notification.getKey().getUrn());
               try {
                 notification.getValue().close();
               } catch (Exception e) {
diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/environment/DockerEnvironmentFactory.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/environment/DockerEnvironmentFactory.java
index c6a0b62e4ba..6ea72383573 100644
--- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/environment/DockerEnvironmentFactory.java
+++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/environment/DockerEnvironmentFactory.java
@@ -19,13 +19,16 @@
 
 import static com.google.common.base.MoreObjects.firstNonNull;
 
+import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
 import java.nio.file.Files;
 import java.nio.file.Paths;
 import java.time.Duration;
 import java.util.List;
 import java.util.concurrent.TimeoutException;
+import org.apache.beam.model.pipeline.v1.RunnerApi;
 import org.apache.beam.model.pipeline.v1.RunnerApi.Environment;
+import org.apache.beam.runners.core.construction.BeamUrns;
 import org.apache.beam.runners.fnexecution.GrpcFnServer;
 import org.apache.beam.runners.fnexecution.artifact.ArtifactRetrievalService;
 import org.apache.beam.runners.fnexecution.control.ControlClientPool;
@@ -113,10 +116,17 @@ private DockerEnvironmentFactory(
   /** Creates a new, active {@link RemoteEnvironment} backed by a local Docker container. */
   @Override
   public RemoteEnvironment createEnvironment(Environment environment) throws Exception {
-    String workerId = idGenerator.getId();
+    Preconditions.checkState(
+        environment
+            .getUrn()
+            .equals(BeamUrns.getUrn(RunnerApi.StandardEnvironments.Environments.DOCKER)),
+        "The passed environment does not contain a DockerPayload.");
+    final RunnerApi.DockerPayload dockerPayload =
+        RunnerApi.DockerPayload.parseFrom(environment.getPayload());
+    final String workerId = idGenerator.getId();
 
     // Prepare docker invocation.
-    String containerImage = environment.getUrl();
+    String containerImage = dockerPayload.getContainerImage();
     // TODO: https://issues.apache.org/jira/browse/BEAM-4148 The default service address will not
     // work for Docker for Mac.
     String loggingEndpoint = loggingServiceServer.getApiServiceDescriptor().getUrl();
@@ -155,7 +165,7 @@ public RemoteEnvironment createEnvironment(Environment environment) throws Excep
         } catch (TimeoutException timeoutEx) {
           LOG.info(
               "Still waiting for startup of environment {} for worker id {}",
-              environment.getUrl(),
+              dockerPayload.getContainerImage(),
               workerId);
         } catch (InterruptedException interruptEx) {
           Thread.currentThread().interrupt();
diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/environment/EnvironmentFactory.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/environment/EnvironmentFactory.java
index eb62d2e9138..cb423f434f1 100644
--- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/environment/EnvironmentFactory.java
+++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/environment/EnvironmentFactory.java
@@ -25,5 +25,5 @@
 /** Creates {@link Environment environments} which communicate to an {@link SdkHarnessClient}. */
 public interface EnvironmentFactory {
   /** Creates an active {@link Environment} and returns a handle to it. */
-  RemoteEnvironment createEnvironment(RunnerApi.Environment container) throws Exception;
+  RemoteEnvironment createEnvironment(RunnerApi.Environment environment) throws Exception;
 }
diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/environment/InProcessEnvironmentFactory.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/environment/InProcessEnvironmentFactory.java
index 3499c01d881..2c018f771c2 100644
--- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/environment/InProcessEnvironmentFactory.java
+++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/environment/InProcessEnvironmentFactory.java
@@ -82,7 +82,7 @@ private InProcessEnvironmentFactory(
 
   @Override
   @SuppressWarnings("FutureReturnValueIgnored") // no need to monitor shutdown thread
-  public RemoteEnvironment createEnvironment(Environment container) throws Exception {
+  public RemoteEnvironment createEnvironment(Environment environment) throws Exception {
     ExecutorService executor = Executors.newSingleThreadExecutor();
     Future<?> fnHarness =
         executor.submit(
@@ -121,6 +121,6 @@ public RemoteEnvironment createEnvironment(Environment container) throws Excepti
 
     // TODO: find some way to populate the actual ID in FnHarness.main()
     InstructionRequestHandler handler = clientSource.take("", Duration.ofMinutes(1L));
-    return RemoteEnvironment.forHandler(container, handler);
+    return RemoteEnvironment.forHandler(environment, handler);
   }
 }
diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/environment/ProcessEnvironmentFactory.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/environment/ProcessEnvironmentFactory.java
index 7b89eccb6ee..3c109383a49 100644
--- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/environment/ProcessEnvironmentFactory.java
+++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/environment/ProcessEnvironmentFactory.java
@@ -17,11 +17,13 @@
  */
 package org.apache.beam.runners.fnexecution.environment;
 
+import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
 import java.time.Duration;
-import java.util.List;
 import java.util.concurrent.TimeoutException;
+import org.apache.beam.model.pipeline.v1.RunnerApi;
 import org.apache.beam.model.pipeline.v1.RunnerApi.Environment;
+import org.apache.beam.runners.core.construction.BeamUrns;
 import org.apache.beam.runners.fnexecution.GrpcFnServer;
 import org.apache.beam.runners.fnexecution.artifact.ArtifactRetrievalService;
 import org.apache.beam.runners.fnexecution.control.ControlClientPool;
@@ -104,16 +106,22 @@ private ProcessEnvironmentFactory(
   /** Creates a new, active {@link RemoteEnvironment} backed by a forked process. */
   @Override
   public RemoteEnvironment createEnvironment(Environment environment) throws Exception {
-    String workerId = idGenerator.getId();
+    Preconditions.checkState(
+        environment
+            .getUrn()
+            .equals(BeamUrns.getUrn(RunnerApi.StandardEnvironments.Environments.PROCESS)),
+        "The passed environment does not contain a ProcessPayload.");
+    final RunnerApi.ProcessPayload processPayload =
+        RunnerApi.ProcessPayload.parseFrom(environment.getPayload());
+    final String workerId = idGenerator.getId();
 
-    // TODO The Environment Protobuf message needs to be changed for process environment
-    String executable = environment.getUrl();
+    String executable = processPayload.getCommand();
     String loggingEndpoint = loggingServiceServer.getApiServiceDescriptor().getUrl();
     String artifactEndpoint = retrievalServiceServer.getApiServiceDescriptor().getUrl();
     String provisionEndpoint = provisioningServiceServer.getApiServiceDescriptor().getUrl();
     String controlEndpoint = controlServiceServer.getApiServiceDescriptor().getUrl();
 
-    List<String> args =
+    ImmutableList<String> args =
         ImmutableList.of(
             String.format("--id=%s", workerId),
             String.format("--logging_endpoint=%s", loggingEndpoint),
@@ -126,7 +134,7 @@ public RemoteEnvironment createEnvironment(Environment environment) throws Excep
     InstructionRequestHandler instructionHandler = null;
     try {
       ProcessManager.RunningProcess process =
-          processManager.startProcess(workerId, executable, args);
+          processManager.startProcess(workerId, executable, args, processPayload.getEnvMap());
       // Wait on a client from the gRPC server.
       while (instructionHandler == null) {
         try {
@@ -136,7 +144,7 @@ public RemoteEnvironment createEnvironment(Environment environment) throws Excep
         } catch (TimeoutException timeoutEx) {
           LOG.info(
               "Still waiting for startup of environment '{}' for worker id {}",
-              environment.getUrl(),
+              processPayload.getCommand(),
               workerId);
         } catch (InterruptedException interruptEx) {
           Thread.currentThread().interrupt();
diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/DockerJobBundleFactoryTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/DockerJobBundleFactoryTest.java
index 3ad361dd87d..6f32d58969a 100644
--- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/DockerJobBundleFactoryTest.java
+++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/DockerJobBundleFactoryTest.java
@@ -33,6 +33,7 @@
 import org.apache.beam.model.pipeline.v1.RunnerApi.PCollection;
 import org.apache.beam.model.pipeline.v1.RunnerApi.SdkFunctionSpec;
 import org.apache.beam.model.pipeline.v1.RunnerApi.WindowingStrategy;
+import org.apache.beam.runners.core.construction.Environments;
 import org.apache.beam.runners.core.construction.ModelCoders;
 import org.apache.beam.runners.core.construction.graph.ExecutableStage;
 import org.apache.beam.runners.fnexecution.GrpcFnServer;
@@ -64,7 +65,7 @@
   @Mock GrpcFnServer<ArtifactRetrievalService> retrievalServer;
   @Mock GrpcFnServer<StaticGrpcProvisionService> provisioningServer;
 
-  private final Environment environment = Environment.newBuilder().setUrl("env-url").build();
+  private final Environment environment = Environments.createDockerEnvironment("env-url");
   private final IdGenerator stageIdGenerator = IdGenerators.incrementingLongs();
   private final InstructionResponse instructionResponse =
       InstructionResponse.newBuilder().setInstructionId("instruction-id").build();
@@ -136,7 +137,7 @@ public void cachesEnvironment() throws Exception {
 
   @Test
   public void doesNotCacheDifferentEnvironments() throws Exception {
-    Environment envFoo = Environment.newBuilder().setUrl("foo-env-url").build();
+    Environment envFoo = Environments.createDockerEnvironment("foo-env-url");
     RemoteEnvironment remoteEnvFoo = mock(RemoteEnvironment.class);
     InstructionRequestHandler fooInstructionHandler = mock(InstructionRequestHandler.class);
     when(envFactory.createEnvironment(envFoo)).thenReturn(remoteEnvFoo);
diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/ProcessJobBundleFactoryTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/ProcessJobBundleFactoryTest.java
index 99f94dd0421..0c7354a4d0b 100644
--- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/ProcessJobBundleFactoryTest.java
+++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/ProcessJobBundleFactoryTest.java
@@ -33,6 +33,7 @@
 import org.apache.beam.model.pipeline.v1.RunnerApi.PCollection;
 import org.apache.beam.model.pipeline.v1.RunnerApi.SdkFunctionSpec;
 import org.apache.beam.model.pipeline.v1.RunnerApi.WindowingStrategy;
+import org.apache.beam.runners.core.construction.Environments;
 import org.apache.beam.runners.core.construction.ModelCoders;
 import org.apache.beam.runners.core.construction.graph.ExecutableStage;
 import org.apache.beam.runners.fnexecution.GrpcFnServer;
@@ -64,7 +65,7 @@
   @Mock GrpcFnServer<ArtifactRetrievalService> retrievalServer;
   @Mock GrpcFnServer<StaticGrpcProvisionService> provisioningServer;
 
-  private final Environment environment = Environment.newBuilder().setUrl("env-url").build();
+  private final Environment environment = Environments.createDockerEnvironment("env-url");
   private final IdGenerator stageIdGenerator = IdGenerators.incrementingLongs();
   private final InstructionResponse instructionResponse =
       InstructionResponse.newBuilder().setInstructionId("instruction-id").build();
@@ -136,7 +137,7 @@ public void cachesEnvironment() throws Exception {
 
   @Test
   public void doesNotCacheDifferentEnvironments() throws Exception {
-    Environment envFoo = Environment.newBuilder().setUrl("foo-env-url").build();
+    Environment envFoo = Environments.createDockerEnvironment("foo-env-url");
     RemoteEnvironment remoteEnvFoo = mock(RemoteEnvironment.class);
     InstructionRequestHandler fooInstructionHandler = mock(InstructionRequestHandler.class);
     when(envFactory.createEnvironment(envFoo)).thenReturn(remoteEnvFoo);
diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/SingleEnvironmentInstanceJobBundleFactoryTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/SingleEnvironmentInstanceJobBundleFactoryTest.java
index 81f11c97656..79cc5b74097 100644
--- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/SingleEnvironmentInstanceJobBundleFactoryTest.java
+++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/SingleEnvironmentInstanceJobBundleFactoryTest.java
@@ -36,6 +36,7 @@
 import org.apache.beam.model.fnexecution.v1.BeamFnApi.InstructionResponse;
 import org.apache.beam.model.pipeline.v1.RunnerApi.Environment;
 import org.apache.beam.model.pipeline.v1.RunnerApi.ExecutableStagePayload;
+import org.apache.beam.runners.core.construction.Environments;
 import org.apache.beam.runners.core.construction.JavaReadViaImpulse;
 import org.apache.beam.runners.core.construction.PipelineTranslation;
 import org.apache.beam.runners.core.construction.graph.ExecutableStage;
@@ -131,11 +132,11 @@ public void closeShutsDownEnvironmentsWhenSomeFail() throws Exception {
     ExecutableStagePayload basePayload =
         ExecutableStagePayload.parseFrom(firstEnvStage.toPTransform("foo").getSpec().getPayload());
 
-    Environment secondEnv = Environment.newBuilder().setUrl("second_env").build();
+    Environment secondEnv = Environments.createDockerEnvironment("second_env");
     ExecutableStage secondEnvStage =
         ExecutableStage.fromPayload(basePayload.toBuilder().setEnvironment(secondEnv).build());
 
-    Environment thirdEnv = Environment.newBuilder().setUrl("third_env").build();
+    Environment thirdEnv = Environments.createDockerEnvironment("third_env");
     ExecutableStage thirdEnvStage =
         ExecutableStage.fromPayload(basePayload.toBuilder().setEnvironment(thirdEnv).build());
 
diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/environment/DockerEnvironmentFactoryTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/environment/DockerEnvironmentFactoryTest.java
index c1d25ce4775..ba959d64374 100644
--- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/environment/DockerEnvironmentFactoryTest.java
+++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/environment/DockerEnvironmentFactoryTest.java
@@ -25,6 +25,7 @@
 
 import org.apache.beam.model.pipeline.v1.Endpoints.ApiServiceDescriptor;
 import org.apache.beam.model.pipeline.v1.RunnerApi.Environment;
+import org.apache.beam.runners.core.construction.Environments;
 import org.apache.beam.runners.fnexecution.GrpcFnServer;
 import org.apache.beam.runners.fnexecution.artifact.ArtifactRetrievalService;
 import org.apache.beam.runners.fnexecution.control.FnApiControlClientPoolService;
@@ -48,8 +49,7 @@
   private static final ApiServiceDescriptor SERVICE_DESCRIPTOR =
       ApiServiceDescriptor.newBuilder().setUrl("service-url").build();
   private static final String IMAGE_NAME = "my-image";
-  private static final Environment ENVIRONMENT =
-      Environment.newBuilder().setUrl(IMAGE_NAME).build();
+  private static final Environment ENVIRONMENT = Environments.createDockerEnvironment(IMAGE_NAME);
   private static final String CONTAINER_ID =
       "e4485f0f2b813b63470feacba5fe9cb89699878c095df4124abd320fd5401385";
 
@@ -106,11 +106,11 @@ public void destroysCorrectContainer() throws Exception {
 
   @Test
   public void createsMultipleEnvironments() throws Exception {
-    Environment fooEnv = Environment.newBuilder().setUrl("foo").build();
+    Environment fooEnv = Environments.createDockerEnvironment("foo");
     RemoteEnvironment fooHandle = factory.createEnvironment(fooEnv);
     assertThat(fooHandle.getEnvironment(), is(equalTo(fooEnv)));
 
-    Environment barEnv = Environment.newBuilder().setUrl("bar").build();
+    Environment barEnv = Environments.createDockerEnvironment("bar");
     RemoteEnvironment barHandle = factory.createEnvironment(barEnv);
     assertThat(barHandle.getEnvironment(), is(equalTo(barEnv)));
   }
diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/environment/ProcessEnvironmentFactoryTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/environment/ProcessEnvironmentFactoryTest.java
index b60dcda3b71..80e1810c2fd 100644
--- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/environment/ProcessEnvironmentFactoryTest.java
+++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/environment/ProcessEnvironmentFactoryTest.java
@@ -21,14 +21,17 @@
 import static org.hamcrest.Matchers.is;
 import static org.junit.Assert.assertThat;
 import static org.mockito.Matchers.anyList;
+import static org.mockito.Matchers.anyMap;
 import static org.mockito.Matchers.anyString;
 import static org.mockito.Matchers.eq;
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
 
 import java.io.IOException;
+import java.util.Collections;
 import org.apache.beam.model.pipeline.v1.Endpoints.ApiServiceDescriptor;
 import org.apache.beam.model.pipeline.v1.RunnerApi.Environment;
+import org.apache.beam.runners.core.construction.Environments;
 import org.apache.beam.runners.fnexecution.GrpcFnServer;
 import org.apache.beam.runners.fnexecution.artifact.ArtifactRetrievalService;
 import org.apache.beam.runners.fnexecution.control.FnApiControlClientPoolService;
@@ -52,7 +55,8 @@
   private static final ApiServiceDescriptor SERVICE_DESCRIPTOR =
       ApiServiceDescriptor.newBuilder().setUrl("service-url").build();
   private static final String COMMAND = "my-command";
-  private static final Environment ENVIRONMENT = Environment.newBuilder().setUrl(COMMAND).build();
+  private static final Environment ENVIRONMENT =
+      Environments.createProcessEnvironment("", "", COMMAND, Collections.emptyMap());
 
   private static final InspectibleIdGenerator ID_GENERATOR = new InspectibleIdGenerator();
 
@@ -70,7 +74,7 @@
   public void initMocks() throws IOException {
     MockitoAnnotations.initMocks(this);
 
-    when(processManager.startProcess(anyString(), anyString(), anyList()))
+    when(processManager.startProcess(anyString(), anyString(), anyList(), anyMap()))
         .thenReturn(Mockito.mock(ProcessManager.RunningProcess.class));
     when(controlServiceServer.getApiServiceDescriptor()).thenReturn(SERVICE_DESCRIPTOR);
     when(loggingServiceServer.getApiServiceDescriptor()).thenReturn(SERVICE_DESCRIPTOR);
@@ -92,7 +96,8 @@ public void createsCorrectEnvironment() throws Exception {
     RemoteEnvironment handle = factory.createEnvironment(ENVIRONMENT);
     assertThat(handle.getInstructionRequestHandler(), is(client));
     assertThat(handle.getEnvironment(), equalTo(ENVIRONMENT));
-    Mockito.verify(processManager).startProcess(eq(ID_GENERATOR.currentId), anyString(), anyList());
+    Mockito.verify(processManager)
+        .startProcess(eq(ID_GENERATOR.currentId), anyString(), anyList(), anyMap());
   }
 
   @Test
@@ -104,11 +109,13 @@ public void destroysCorrectContainer() throws Exception {
 
   @Test
   public void createsMultipleEnvironments() throws Exception {
-    Environment fooEnv = Environment.newBuilder().setUrl("foo").build();
+    Environment fooEnv =
+        Environments.createProcessEnvironment("", "", "foo", Collections.emptyMap());
     RemoteEnvironment fooHandle = factory.createEnvironment(fooEnv);
     assertThat(fooHandle.getEnvironment(), is(equalTo(fooEnv)));
 
-    Environment barEnv = Environment.newBuilder().setUrl("bar").build();
+    Environment barEnv =
+        Environments.createProcessEnvironment("", "", "bar", Collections.emptyMap());
     RemoteEnvironment barHandle = factory.createEnvironment(barEnv);
     assertThat(barHandle.getEnvironment(), is(equalTo(barEnv)));
   }
diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/environment/RemoteEnvironmentTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/environment/RemoteEnvironmentTest.java
index 8b190040e2f..4838f2b5392 100644
--- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/environment/RemoteEnvironmentTest.java
+++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/environment/RemoteEnvironmentTest.java
@@ -24,6 +24,7 @@
 import static org.mockito.Mockito.verify;
 
 import org.apache.beam.model.pipeline.v1.RunnerApi.Environment;
+import org.apache.beam.runners.core.construction.Environments;
 import org.apache.beam.runners.fnexecution.control.InstructionRequestHandler;
 import org.junit.Test;
 import org.junit.runner.RunWith;
@@ -63,7 +64,7 @@ public void forHandlerClosesHandlerOnClose() throws Exception {
   @Test
   public void forHandlerReturnsProvided() {
     InstructionRequestHandler handler = mock(InstructionRequestHandler.class);
-    Environment environment = Environment.newBuilder().setUrl("my_url").build();
+    Environment environment = Environments.createDockerEnvironment("my_url");
     RemoteEnvironment remoteEnvironment = RemoteEnvironment.forHandler(environment, handler);
     assertThat(remoteEnvironment.getEnvironment(), theInstance(environment));
     assertThat(remoteEnvironment.getInstructionRequestHandler(), theInstance(handler));
diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/wire/LengthPrefixUnknownCodersTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/wire/LengthPrefixUnknownCodersTest.java
index f1856b4035f..07b4b8b816e 100644
--- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/wire/LengthPrefixUnknownCodersTest.java
+++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/wire/LengthPrefixUnknownCodersTest.java
@@ -23,7 +23,7 @@
 import java.io.OutputStream;
 import java.util.Collection;
 import org.apache.beam.model.pipeline.v1.RunnerApi.Components;
-import org.apache.beam.model.pipeline.v1.RunnerApi.Environment;
+import org.apache.beam.runners.core.construction.Environments;
 import org.apache.beam.runners.core.construction.RehydratedComponents;
 import org.apache.beam.runners.core.construction.SdkComponents;
 import org.apache.beam.sdk.coders.ByteArrayCoder;
@@ -132,7 +132,7 @@ public boolean equals(Object obj) {
   @Test
   public void test() throws IOException {
     SdkComponents sdkComponents = SdkComponents.create();
-    sdkComponents.registerEnvironment(Environment.newBuilder().setUrl("java").build());
+    sdkComponents.registerEnvironment(Environments.createDockerEnvironment("java"));
     String coderId = sdkComponents.registerCoder(original);
     Components.Builder components = sdkComponents.toComponents().toBuilder();
     String updatedCoderId =
diff --git a/sdks/go/pkg/beam/core/runtime/graphx/translate.go b/sdks/go/pkg/beam/core/runtime/graphx/translate.go
index 3bc67ae7e28..d237e77a526 100644
--- a/sdks/go/pkg/beam/core/runtime/graphx/translate.go
+++ b/sdks/go/pkg/beam/core/runtime/graphx/translate.go
@@ -488,7 +488,16 @@ func boolToBounded(bounded bool) pb.IsBounded_Enum {
 func (m *marshaller) addDefaultEnv() string {
 	const id = "go"
 	if _, exists := m.environments[id]; !exists {
-		m.environments[id] = &pb.Environment{Url: m.opt.ContainerImageURL}
+		payload := &pb.DockerPayload{ContainerImage: m.opt.ContainerImageURL}
+		serialized_payload, err := proto.Marshal(payload)
+		if err != nil {
+			panic(fmt.Sprintf("Failed to serialize Environment payload %v: %v", payload, err))
+		}
+		m.environments[id] = &pb.Environment{
+		  Url: m.opt.ContainerImageURL,
+		  Urn: "beam:env:docker:v1",
+		  Payload: serialized_payload,
+		}
 	}
 	return id
 }
diff --git a/sdks/go/pkg/beam/core/runtime/pipelinex/util.go b/sdks/go/pkg/beam/core/runtime/pipelinex/util.go
index eab32be78cf..2457377e059 100644
--- a/sdks/go/pkg/beam/core/runtime/pipelinex/util.go
+++ b/sdks/go/pkg/beam/core/runtime/pipelinex/util.go
@@ -16,6 +16,7 @@
 package pipelinex
 
 import pb "github.com/apache/beam/sdks/go/pkg/beam/model/pipeline_v1"
+import "github.com/golang/protobuf/proto"
 
 // Bounded returns true iff all PCollections are bounded.
 func Bounded(p *pb.Pipeline) bool {
@@ -32,7 +33,10 @@ func Bounded(p *pb.Pipeline) bool {
 func ContainerImages(p *pb.Pipeline) []string {
 	var ret []string
 	for _, t := range p.GetComponents().GetEnvironments() {
-		ret = append(ret, t.Url)
+// TODO(angoenka) 09/14/2018 Check t.Urn before parsing the payload.
+		var payload pb.DockerPayload
+		proto.Unmarshal(t.GetPayload(), &payload)
+		ret = append(ret, payload.ContainerImage)
 	}
 	return ret
 }
diff --git a/sdks/go/pkg/beam/model/fnexecution_v1/beam_fn_api.pb.go b/sdks/go/pkg/beam/model/fnexecution_v1/beam_fn_api.pb.go
index ce2116b1158..1bcf6203deb 100644
--- a/sdks/go/pkg/beam/model/fnexecution_v1/beam_fn_api.pb.go
+++ b/sdks/go/pkg/beam/model/fnexecution_v1/beam_fn_api.pb.go
@@ -3,12 +3,14 @@
 
 package fnexecution_v1
 
-import proto "github.com/golang/protobuf/proto"
-import fmt "fmt"
-import math "math"
-import pipeline_v1 "github.com/apache/beam/sdks/go/pkg/beam/model/pipeline_v1"
-import timestamp "github.com/golang/protobuf/ptypes/timestamp"
-import wrappers "github.com/golang/protobuf/ptypes/wrappers"
+import (
+	fmt "fmt"
+	pipeline_v1 "github.com/apache/beam/sdks/go/pkg/beam/model/pipeline_v1"
+	proto "github.com/golang/protobuf/proto"
+	timestamp "github.com/golang/protobuf/ptypes/timestamp"
+	wrappers "github.com/golang/protobuf/ptypes/wrappers"
+	math "math"
+)
 
 import (
 	context "golang.org/x/net/context"
@@ -43,6 +45,7 @@ var MonitoringInfo_MonitoringInfoLabels_name = map[int32]string{
 	3: "CODER",
 	4: "ENVIRONMENT",
 }
+
 var MonitoringInfo_MonitoringInfoLabels_value = map[string]int32{
 	"TRANSFORM":          0,
 	"PCOLLECTION":        1,
@@ -54,8 +57,9 @@ var MonitoringInfo_MonitoringInfoLabels_value = map[string]int32{
 func (x MonitoringInfo_MonitoringInfoLabels) String() string {
 	return proto.EnumName(MonitoringInfo_MonitoringInfoLabels_name, int32(x))
 }
+
 func (MonitoringInfo_MonitoringInfoLabels) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_6ff4efbe085bd72e, []int{11, 0}
+	return fileDescriptor_6d954c03a4758710, []int{11, 0}
 }
 
 type LogEntry_Severity_Enum int32
@@ -91,6 +95,7 @@ var LogEntry_Severity_Enum_name = map[int32]string{
 	6: "ERROR",
 	7: "CRITICAL",
 }
+
 var LogEntry_Severity_Enum_value = map[string]int32{
 	"UNSPECIFIED": 0,
 	"TRACE":       1,
@@ -105,8 +110,9 @@ var LogEntry_Severity_Enum_value = map[string]int32{
 func (x LogEntry_Severity_Enum) String() string {
 	return proto.EnumName(LogEntry_Severity_Enum_name, int32(x))
 }
+
 func (LogEntry_Severity_Enum) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_6ff4efbe085bd72e, []int{35, 1, 0}
+	return fileDescriptor_6d954c03a4758710, []int{35, 1, 0}
 }
 
 // A representation of an input or output definition on a primitive transform.
@@ -126,7 +132,7 @@ func (m *Target) Reset()         { *m = Target{} }
 func (m *Target) String() string { return proto.CompactTextString(m) }
 func (*Target) ProtoMessage()    {}
 func (*Target) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_6ff4efbe085bd72e, []int{0}
+	return fileDescriptor_6d954c03a4758710, []int{0}
 }
 func (m *Target) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Target.Unmarshal(m, b)
@@ -134,8 +140,8 @@ func (m *Target) XXX_Unmarshal(b []byte) error {
 func (m *Target) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_Target.Marshal(b, m, deterministic)
 }
-func (dst *Target) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Target.Merge(dst, src)
+func (m *Target) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_Target.Merge(m, src)
 }
 func (m *Target) XXX_Size() int {
 	return xxx_messageInfo_Target.Size(m)
@@ -172,7 +178,7 @@ func (m *Target_List) Reset()         { *m = Target_List{} }
 func (m *Target_List) String() string { return proto.CompactTextString(m) }
 func (*Target_List) ProtoMessage()    {}
 func (*Target_List) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_6ff4efbe085bd72e, []int{0, 0}
+	return fileDescriptor_6d954c03a4758710, []int{0, 0}
 }
 func (m *Target_List) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Target_List.Unmarshal(m, b)
@@ -180,8 +186,8 @@ func (m *Target_List) XXX_Unmarshal(b []byte) error {
 func (m *Target_List) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_Target_List.Marshal(b, m, deterministic)
 }
-func (dst *Target_List) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Target_List.Merge(dst, src)
+func (m *Target_List) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_Target_List.Merge(m, src)
 }
 func (m *Target_List) XXX_Size() int {
 	return xxx_messageInfo_Target_List.Size(m)
@@ -218,7 +224,7 @@ func (m *RemoteGrpcPort) Reset()         { *m = RemoteGrpcPort{} }
 func (m *RemoteGrpcPort) String() string { return proto.CompactTextString(m) }
 func (*RemoteGrpcPort) ProtoMessage()    {}
 func (*RemoteGrpcPort) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_6ff4efbe085bd72e, []int{1}
+	return fileDescriptor_6d954c03a4758710, []int{1}
 }
 func (m *RemoteGrpcPort) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_RemoteGrpcPort.Unmarshal(m, b)
@@ -226,8 +232,8 @@ func (m *RemoteGrpcPort) XXX_Unmarshal(b []byte) error {
 func (m *RemoteGrpcPort) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_RemoteGrpcPort.Marshal(b, m, deterministic)
 }
-func (dst *RemoteGrpcPort) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_RemoteGrpcPort.Merge(dst, src)
+func (m *RemoteGrpcPort) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_RemoteGrpcPort.Merge(m, src)
 }
 func (m *RemoteGrpcPort) XXX_Size() int {
 	return xxx_messageInfo_RemoteGrpcPort.Size(m)
@@ -277,7 +283,7 @@ func (m *InstructionRequest) Reset()         { *m = InstructionRequest{} }
 func (m *InstructionRequest) String() string { return proto.CompactTextString(m) }
 func (*InstructionRequest) ProtoMessage()    {}
 func (*InstructionRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_6ff4efbe085bd72e, []int{2}
+	return fileDescriptor_6d954c03a4758710, []int{2}
 }
 func (m *InstructionRequest) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_InstructionRequest.Unmarshal(m, b)
@@ -285,8 +291,8 @@ func (m *InstructionRequest) XXX_Unmarshal(b []byte) error {
 func (m *InstructionRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_InstructionRequest.Marshal(b, m, deterministic)
 }
-func (dst *InstructionRequest) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_InstructionRequest.Merge(dst, src)
+func (m *InstructionRequest) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_InstructionRequest.Merge(m, src)
 }
 func (m *InstructionRequest) XXX_Size() int {
 	return xxx_messageInfo_InstructionRequest.Size(m)
@@ -297,6 +303,13 @@ func (m *InstructionRequest) XXX_DiscardUnknown() {
 
 var xxx_messageInfo_InstructionRequest proto.InternalMessageInfo
 
+func (m *InstructionRequest) GetInstructionId() string {
+	if m != nil {
+		return m.InstructionId
+	}
+	return ""
+}
+
 type isInstructionRequest_Request interface {
 	isInstructionRequest_Request()
 }
@@ -304,20 +317,26 @@ type isInstructionRequest_Request interface {
 type InstructionRequest_Register struct {
 	Register *RegisterRequest `protobuf:"bytes,1000,opt,name=register,proto3,oneof"`
 }
+
 type InstructionRequest_ProcessBundle struct {
 	ProcessBundle *ProcessBundleRequest `protobuf:"bytes,1001,opt,name=process_bundle,json=processBundle,proto3,oneof"`
 }
+
 type InstructionRequest_ProcessBundleProgress struct {
 	ProcessBundleProgress *ProcessBundleProgressRequest `protobuf:"bytes,1002,opt,name=process_bundle_progress,json=processBundleProgress,proto3,oneof"`
 }
+
 type InstructionRequest_ProcessBundleSplit struct {
 	ProcessBundleSplit *ProcessBundleSplitRequest `protobuf:"bytes,1003,opt,name=process_bundle_split,json=processBundleSplit,proto3,oneof"`
 }
 
-func (*InstructionRequest_Register) isInstructionRequest_Request()              {}
-func (*InstructionRequest_ProcessBundle) isInstructionRequest_Request()         {}
+func (*InstructionRequest_Register) isInstructionRequest_Request() {}
+
+func (*InstructionRequest_ProcessBundle) isInstructionRequest_Request() {}
+
 func (*InstructionRequest_ProcessBundleProgress) isInstructionRequest_Request() {}
-func (*InstructionRequest_ProcessBundleSplit) isInstructionRequest_Request()    {}
+
+func (*InstructionRequest_ProcessBundleSplit) isInstructionRequest_Request() {}
 
 func (m *InstructionRequest) GetRequest() isInstructionRequest_Request {
 	if m != nil {
@@ -326,13 +345,6 @@ func (m *InstructionRequest) GetRequest() isInstructionRequest_Request {
 	return nil
 }
 
-func (m *InstructionRequest) GetInstructionId() string {
-	if m != nil {
-		return m.InstructionId
-	}
-	return ""
-}
-
 func (m *InstructionRequest) GetRegister() *RegisterRequest {
 	if x, ok := m.GetRequest().(*InstructionRequest_Register); ok {
 		return x.Register
@@ -502,7 +514,7 @@ func (m *InstructionResponse) Reset()         { *m = InstructionResponse{} }
 func (m *InstructionResponse) String() string { return proto.CompactTextString(m) }
 func (*InstructionResponse) ProtoMessage()    {}
 func (*InstructionResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_6ff4efbe085bd72e, []int{3}
+	return fileDescriptor_6d954c03a4758710, []int{3}
 }
 func (m *InstructionResponse) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_InstructionResponse.Unmarshal(m, b)
@@ -510,8 +522,8 @@ func (m *InstructionResponse) XXX_Unmarshal(b []byte) error {
 func (m *InstructionResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_InstructionResponse.Marshal(b, m, deterministic)
 }
-func (dst *InstructionResponse) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_InstructionResponse.Merge(dst, src)
+func (m *InstructionResponse) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_InstructionResponse.Merge(m, src)
 }
 func (m *InstructionResponse) XXX_Size() int {
 	return xxx_messageInfo_InstructionResponse.Size(m)
@@ -522,6 +534,20 @@ func (m *InstructionResponse) XXX_DiscardUnknown() {
 
 var xxx_messageInfo_InstructionResponse proto.InternalMessageInfo
 
+func (m *InstructionResponse) GetInstructionId() string {
+	if m != nil {
+		return m.InstructionId
+	}
+	return ""
+}
+
+func (m *InstructionResponse) GetError() string {
+	if m != nil {
+		return m.Error
+	}
+	return ""
+}
+
 type isInstructionResponse_Response interface {
 	isInstructionResponse_Response()
 }
@@ -529,20 +555,26 @@ type isInstructionResponse_Response interface {
 type InstructionResponse_Register struct {
 	Register *RegisterResponse `protobuf:"bytes,1000,opt,name=register,proto3,oneof"`
 }
+
 type InstructionResponse_ProcessBundle struct {
 	ProcessBundle *ProcessBundleResponse `protobuf:"bytes,1001,opt,name=process_bundle,json=processBundle,proto3,oneof"`
 }
+
 type InstructionResponse_ProcessBundleProgress struct {
 	ProcessBundleProgress *ProcessBundleProgressResponse `protobuf:"bytes,1002,opt,name=process_bundle_progress,json=processBundleProgress,proto3,oneof"`
 }
+
 type InstructionResponse_ProcessBundleSplit struct {
 	ProcessBundleSplit *ProcessBundleSplitResponse `protobuf:"bytes,1003,opt,name=process_bundle_split,json=processBundleSplit,proto3,oneof"`
 }
 
-func (*InstructionResponse_Register) isInstructionResponse_Response()              {}
-func (*InstructionResponse_ProcessBundle) isInstructionResponse_Response()         {}
+func (*InstructionResponse_Register) isInstructionResponse_Response() {}
+
+func (*InstructionResponse_ProcessBundle) isInstructionResponse_Response() {}
+
 func (*InstructionResponse_ProcessBundleProgress) isInstructionResponse_Response() {}
-func (*InstructionResponse_ProcessBundleSplit) isInstructionResponse_Response()    {}
+
+func (*InstructionResponse_ProcessBundleSplit) isInstructionResponse_Response() {}
 
 func (m *InstructionResponse) GetResponse() isInstructionResponse_Response {
 	if m != nil {
@@ -551,20 +583,6 @@ func (m *InstructionResponse) GetResponse() isInstructionResponse_Response {
 	return nil
 }
 
-func (m *InstructionResponse) GetInstructionId() string {
-	if m != nil {
-		return m.InstructionId
-	}
-	return ""
-}
-
-func (m *InstructionResponse) GetError() string {
-	if m != nil {
-		return m.Error
-	}
-	return ""
-}
-
 func (m *InstructionResponse) GetRegister() *RegisterResponse {
 	if x, ok := m.GetResponse().(*InstructionResponse_Register); ok {
 		return x.Register
@@ -720,7 +738,7 @@ func (m *RegisterRequest) Reset()         { *m = RegisterRequest{} }
 func (m *RegisterRequest) String() string { return proto.CompactTextString(m) }
 func (*RegisterRequest) ProtoMessage()    {}
 func (*RegisterRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_6ff4efbe085bd72e, []int{4}
+	return fileDescriptor_6d954c03a4758710, []int{4}
 }
 func (m *RegisterRequest) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_RegisterRequest.Unmarshal(m, b)
@@ -728,8 +746,8 @@ func (m *RegisterRequest) XXX_Unmarshal(b []byte) error {
 func (m *RegisterRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_RegisterRequest.Marshal(b, m, deterministic)
 }
-func (dst *RegisterRequest) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_RegisterRequest.Merge(dst, src)
+func (m *RegisterRequest) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_RegisterRequest.Merge(m, src)
 }
 func (m *RegisterRequest) XXX_Size() int {
 	return xxx_messageInfo_RegisterRequest.Size(m)
@@ -758,7 +776,7 @@ func (m *RegisterResponse) Reset()         { *m = RegisterResponse{} }
 func (m *RegisterResponse) String() string { return proto.CompactTextString(m) }
 func (*RegisterResponse) ProtoMessage()    {}
 func (*RegisterResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_6ff4efbe085bd72e, []int{5}
+	return fileDescriptor_6d954c03a4758710, []int{5}
 }
 func (m *RegisterResponse) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_RegisterResponse.Unmarshal(m, b)
@@ -766,8 +784,8 @@ func (m *RegisterResponse) XXX_Unmarshal(b []byte) error {
 func (m *RegisterResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_RegisterResponse.Marshal(b, m, deterministic)
 }
-func (dst *RegisterResponse) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_RegisterResponse.Merge(dst, src)
+func (m *RegisterResponse) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_RegisterResponse.Merge(m, src)
 }
 func (m *RegisterResponse) XXX_Size() int {
 	return xxx_messageInfo_RegisterResponse.Size(m)
@@ -806,7 +824,7 @@ func (m *ProcessBundleDescriptor) Reset()         { *m = ProcessBundleDescriptor
 func (m *ProcessBundleDescriptor) String() string { return proto.CompactTextString(m) }
 func (*ProcessBundleDescriptor) ProtoMessage()    {}
 func (*ProcessBundleDescriptor) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_6ff4efbe085bd72e, []int{6}
+	return fileDescriptor_6d954c03a4758710, []int{6}
 }
 func (m *ProcessBundleDescriptor) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_ProcessBundleDescriptor.Unmarshal(m, b)
@@ -814,8 +832,8 @@ func (m *ProcessBundleDescriptor) XXX_Unmarshal(b []byte) error {
 func (m *ProcessBundleDescriptor) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_ProcessBundleDescriptor.Marshal(b, m, deterministic)
 }
-func (dst *ProcessBundleDescriptor) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ProcessBundleDescriptor.Merge(dst, src)
+func (m *ProcessBundleDescriptor) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_ProcessBundleDescriptor.Merge(m, src)
 }
 func (m *ProcessBundleDescriptor) XXX_Size() int {
 	return xxx_messageInfo_ProcessBundleDescriptor.Size(m)
@@ -890,17 +908,17 @@ type BundleSplit struct {
 	// Root applications that have been removed from the current bundle and
 	// have to be executed in a separate bundle (e.g. in parallel on a different
 	// worker, or after the current bundle completes, etc.)
-	ResidualRoots        []*BundleSplit_Application `protobuf:"bytes,2,rep,name=residual_roots,json=residualRoots,proto3" json:"residual_roots,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}                   `json:"-"`
-	XXX_unrecognized     []byte                     `json:"-"`
-	XXX_sizecache        int32                      `json:"-"`
+	ResidualRoots        []*BundleSplit_DelayedApplication `protobuf:"bytes,2,rep,name=residual_roots,json=residualRoots,proto3" json:"residual_roots,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}                          `json:"-"`
+	XXX_unrecognized     []byte                            `json:"-"`
+	XXX_sizecache        int32                             `json:"-"`
 }
 
 func (m *BundleSplit) Reset()         { *m = BundleSplit{} }
 func (m *BundleSplit) String() string { return proto.CompactTextString(m) }
 func (*BundleSplit) ProtoMessage()    {}
 func (*BundleSplit) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_6ff4efbe085bd72e, []int{7}
+	return fileDescriptor_6d954c03a4758710, []int{7}
 }
 func (m *BundleSplit) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_BundleSplit.Unmarshal(m, b)
@@ -908,8 +926,8 @@ func (m *BundleSplit) XXX_Unmarshal(b []byte) error {
 func (m *BundleSplit) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_BundleSplit.Marshal(b, m, deterministic)
 }
-func (dst *BundleSplit) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_BundleSplit.Merge(dst, src)
+func (m *BundleSplit) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_BundleSplit.Merge(m, src)
 }
 func (m *BundleSplit) XXX_Size() int {
 	return xxx_messageInfo_BundleSplit.Size(m)
@@ -927,7 +945,7 @@ func (m *BundleSplit) GetPrimaryRoots() []*BundleSplit_Application {
 	return nil
 }
 
-func (m *BundleSplit) GetResidualRoots() []*BundleSplit_Application {
+func (m *BundleSplit) GetResidualRoots() []*BundleSplit_DelayedApplication {
 	if m != nil {
 		return m.ResidualRoots
 	}
@@ -960,7 +978,7 @@ func (m *BundleSplit_Application) Reset()         { *m = BundleSplit_Application
 func (m *BundleSplit_Application) String() string { return proto.CompactTextString(m) }
 func (*BundleSplit_Application) ProtoMessage()    {}
 func (*BundleSplit_Application) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_6ff4efbe085bd72e, []int{7, 0}
+	return fileDescriptor_6d954c03a4758710, []int{7, 0}
 }
 func (m *BundleSplit_Application) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_BundleSplit_Application.Unmarshal(m, b)
@@ -968,8 +986,8 @@ func (m *BundleSplit_Application) XXX_Unmarshal(b []byte) error {
 func (m *BundleSplit_Application) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_BundleSplit_Application.Marshal(b, m, deterministic)
 }
-func (dst *BundleSplit_Application) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_BundleSplit_Application.Merge(dst, src)
+func (m *BundleSplit_Application) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_BundleSplit_Application.Merge(m, src)
 }
 func (m *BundleSplit_Application) XXX_Size() int {
 	return xxx_messageInfo_BundleSplit_Application.Size(m)
@@ -1015,6 +1033,55 @@ func (m *BundleSplit_Application) GetFractionOfWork() *wrappers.DoubleValue {
 	return nil
 }
 
+// An an Application should be scheduled after a delay.
+type BundleSplit_DelayedApplication struct {
+	// The delay in seconds (lower bound).
+	DelaySec float64 `protobuf:"fixed64,1,opt,name=delay_sec,json=delaySec,proto3" json:"delay_sec,omitempty"`
+	// (Required) The application that should be scheduled.
+	Application          *BundleSplit_Application `protobuf:"bytes,2,opt,name=application,proto3" json:"application,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}                 `json:"-"`
+	XXX_unrecognized     []byte                   `json:"-"`
+	XXX_sizecache        int32                    `json:"-"`
+}
+
+func (m *BundleSplit_DelayedApplication) Reset()         { *m = BundleSplit_DelayedApplication{} }
+func (m *BundleSplit_DelayedApplication) String() string { return proto.CompactTextString(m) }
+func (*BundleSplit_DelayedApplication) ProtoMessage()    {}
+func (*BundleSplit_DelayedApplication) Descriptor() ([]byte, []int) {
+	return fileDescriptor_6d954c03a4758710, []int{7, 1}
+}
+func (m *BundleSplit_DelayedApplication) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_BundleSplit_DelayedApplication.Unmarshal(m, b)
+}
+func (m *BundleSplit_DelayedApplication) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_BundleSplit_DelayedApplication.Marshal(b, m, deterministic)
+}
+func (m *BundleSplit_DelayedApplication) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_BundleSplit_DelayedApplication.Merge(m, src)
+}
+func (m *BundleSplit_DelayedApplication) XXX_Size() int {
+	return xxx_messageInfo_BundleSplit_DelayedApplication.Size(m)
+}
+func (m *BundleSplit_DelayedApplication) XXX_DiscardUnknown() {
+	xxx_messageInfo_BundleSplit_DelayedApplication.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_BundleSplit_DelayedApplication proto.InternalMessageInfo
+
+func (m *BundleSplit_DelayedApplication) GetDelaySec() float64 {
+	if m != nil {
+		return m.DelaySec
+	}
+	return 0
+}
+
+func (m *BundleSplit_DelayedApplication) GetApplication() *BundleSplit_Application {
+	if m != nil {
+		return m.Application
+	}
+	return nil
+}
+
 // A request to process a given bundle.
 // Stable
 type ProcessBundleRequest struct {
@@ -1033,7 +1100,7 @@ func (m *ProcessBundleRequest) Reset()         { *m = ProcessBundleRequest{} }
 func (m *ProcessBundleRequest) String() string { return proto.CompactTextString(m) }
 func (*ProcessBundleRequest) ProtoMessage()    {}
 func (*ProcessBundleRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_6ff4efbe085bd72e, []int{8}
+	return fileDescriptor_6d954c03a4758710, []int{8}
 }
 func (m *ProcessBundleRequest) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_ProcessBundleRequest.Unmarshal(m, b)
@@ -1041,8 +1108,8 @@ func (m *ProcessBundleRequest) XXX_Unmarshal(b []byte) error {
 func (m *ProcessBundleRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_ProcessBundleRequest.Marshal(b, m, deterministic)
 }
-func (dst *ProcessBundleRequest) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ProcessBundleRequest.Merge(dst, src)
+func (m *ProcessBundleRequest) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_ProcessBundleRequest.Merge(m, src)
 }
 func (m *ProcessBundleRequest) XXX_Size() int {
 	return xxx_messageInfo_ProcessBundleRequest.Size(m)
@@ -1084,7 +1151,7 @@ func (m *ProcessBundleResponse) Reset()         { *m = ProcessBundleResponse{} }
 func (m *ProcessBundleResponse) String() string { return proto.CompactTextString(m) }
 func (*ProcessBundleResponse) ProtoMessage()    {}
 func (*ProcessBundleResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_6ff4efbe085bd72e, []int{9}
+	return fileDescriptor_6d954c03a4758710, []int{9}
 }
 func (m *ProcessBundleResponse) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_ProcessBundleResponse.Unmarshal(m, b)
@@ -1092,8 +1159,8 @@ func (m *ProcessBundleResponse) XXX_Unmarshal(b []byte) error {
 func (m *ProcessBundleResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_ProcessBundleResponse.Marshal(b, m, deterministic)
 }
-func (dst *ProcessBundleResponse) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ProcessBundleResponse.Merge(dst, src)
+func (m *ProcessBundleResponse) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_ProcessBundleResponse.Merge(m, src)
 }
 func (m *ProcessBundleResponse) XXX_Size() int {
 	return xxx_messageInfo_ProcessBundleResponse.Size(m)
@@ -1134,7 +1201,7 @@ func (m *ProcessBundleProgressRequest) Reset()         { *m = ProcessBundleProgr
 func (m *ProcessBundleProgressRequest) String() string { return proto.CompactTextString(m) }
 func (*ProcessBundleProgressRequest) ProtoMessage()    {}
 func (*ProcessBundleProgressRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_6ff4efbe085bd72e, []int{10}
+	return fileDescriptor_6d954c03a4758710, []int{10}
 }
 func (m *ProcessBundleProgressRequest) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_ProcessBundleProgressRequest.Unmarshal(m, b)
@@ -1142,8 +1209,8 @@ func (m *ProcessBundleProgressRequest) XXX_Unmarshal(b []byte) error {
 func (m *ProcessBundleProgressRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_ProcessBundleProgressRequest.Marshal(b, m, deterministic)
 }
-func (dst *ProcessBundleProgressRequest) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ProcessBundleProgressRequest.Merge(dst, src)
+func (m *ProcessBundleProgressRequest) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_ProcessBundleProgressRequest.Merge(m, src)
 }
 func (m *ProcessBundleProgressRequest) XXX_Size() int {
 	return xxx_messageInfo_ProcessBundleProgressRequest.Size(m)
@@ -1197,7 +1264,7 @@ func (m *MonitoringInfo) Reset()         { *m = MonitoringInfo{} }
 func (m *MonitoringInfo) String() string { return proto.CompactTextString(m) }
 func (*MonitoringInfo) ProtoMessage()    {}
 func (*MonitoringInfo) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_6ff4efbe085bd72e, []int{11}
+	return fileDescriptor_6d954c03a4758710, []int{11}
 }
 func (m *MonitoringInfo) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_MonitoringInfo.Unmarshal(m, b)
@@ -1205,8 +1272,8 @@ func (m *MonitoringInfo) XXX_Unmarshal(b []byte) error {
 func (m *MonitoringInfo) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_MonitoringInfo.Marshal(b, m, deterministic)
 }
-func (dst *MonitoringInfo) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_MonitoringInfo.Merge(dst, src)
+func (m *MonitoringInfo) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_MonitoringInfo.Merge(m, src)
 }
 func (m *MonitoringInfo) XXX_Size() int {
 	return xxx_messageInfo_MonitoringInfo.Size(m)
@@ -1217,6 +1284,20 @@ func (m *MonitoringInfo) XXX_DiscardUnknown() {
 
 var xxx_messageInfo_MonitoringInfo proto.InternalMessageInfo
 
+func (m *MonitoringInfo) GetUrn() string {
+	if m != nil {
+		return m.Urn
+	}
+	return ""
+}
+
+func (m *MonitoringInfo) GetType() string {
+	if m != nil {
+		return m.Type
+	}
+	return ""
+}
+
 type isMonitoringInfo_Data interface {
 	isMonitoringInfo_Data()
 }
@@ -1224,12 +1305,14 @@ type isMonitoringInfo_Data interface {
 type MonitoringInfo_MonitoringTableData struct {
 	MonitoringTableData *MonitoringTableData `protobuf:"bytes,3,opt,name=monitoring_table_data,json=monitoringTableData,proto3,oneof"`
 }
+
 type MonitoringInfo_Metric struct {
 	Metric *Metric `protobuf:"bytes,4,opt,name=metric,proto3,oneof"`
 }
 
 func (*MonitoringInfo_MonitoringTableData) isMonitoringInfo_Data() {}
-func (*MonitoringInfo_Metric) isMonitoringInfo_Data()              {}
+
+func (*MonitoringInfo_Metric) isMonitoringInfo_Data() {}
 
 func (m *MonitoringInfo) GetData() isMonitoringInfo_Data {
 	if m != nil {
@@ -1238,20 +1321,6 @@ func (m *MonitoringInfo) GetData() isMonitoringInfo_Data {
 	return nil
 }
 
-func (m *MonitoringInfo) GetUrn() string {
-	if m != nil {
-		return m.Urn
-	}
-	return ""
-}
-
-func (m *MonitoringInfo) GetType() string {
-	if m != nil {
-		return m.Type
-	}
-	return ""
-}
-
 func (m *MonitoringInfo) GetMonitoringTableData() *MonitoringTableData {
 	if x, ok := m.GetData().(*MonitoringInfo_MonitoringTableData); ok {
 		return x.MonitoringTableData
@@ -1364,7 +1433,7 @@ func (m *Metric) Reset()         { *m = Metric{} }
 func (m *Metric) String() string { return proto.CompactTextString(m) }
 func (*Metric) ProtoMessage()    {}
 func (*Metric) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_6ff4efbe085bd72e, []int{12}
+	return fileDescriptor_6d954c03a4758710, []int{12}
 }
 func (m *Metric) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Metric.Unmarshal(m, b)
@@ -1372,8 +1441,8 @@ func (m *Metric) XXX_Unmarshal(b []byte) error {
 func (m *Metric) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_Metric.Marshal(b, m, deterministic)
 }
-func (dst *Metric) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Metric.Merge(dst, src)
+func (m *Metric) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_Metric.Merge(m, src)
 }
 func (m *Metric) XXX_Size() int {
 	return xxx_messageInfo_Metric.Size(m)
@@ -1391,16 +1460,20 @@ type isMetric_Data interface {
 type Metric_CounterData struct {
 	CounterData *CounterData `protobuf:"bytes,1,opt,name=counter_data,json=counterData,proto3,oneof"`
 }
+
 type Metric_DistributionData struct {
 	DistributionData *DistributionData `protobuf:"bytes,2,opt,name=distribution_data,json=distributionData,proto3,oneof"`
 }
+
 type Metric_ExtremaData struct {
 	ExtremaData *ExtremaData `protobuf:"bytes,3,opt,name=extrema_data,json=extremaData,proto3,oneof"`
 }
 
-func (*Metric_CounterData) isMetric_Data()      {}
+func (*Metric_CounterData) isMetric_Data() {}
+
 func (*Metric_DistributionData) isMetric_Data() {}
-func (*Metric_ExtremaData) isMetric_Data()      {}
+
+func (*Metric_ExtremaData) isMetric_Data() {}
 
 func (m *Metric) GetData() isMetric_Data {
 	if m != nil {
@@ -1541,7 +1614,7 @@ func (m *CounterData) Reset()         { *m = CounterData{} }
 func (m *CounterData) String() string { return proto.CompactTextString(m) }
 func (*CounterData) ProtoMessage()    {}
 func (*CounterData) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_6ff4efbe085bd72e, []int{13}
+	return fileDescriptor_6d954c03a4758710, []int{13}
 }
 func (m *CounterData) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_CounterData.Unmarshal(m, b)
@@ -1549,8 +1622,8 @@ func (m *CounterData) XXX_Unmarshal(b []byte) error {
 func (m *CounterData) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_CounterData.Marshal(b, m, deterministic)
 }
-func (dst *CounterData) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_CounterData.Merge(dst, src)
+func (m *CounterData) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_CounterData.Merge(m, src)
 }
 func (m *CounterData) XXX_Size() int {
 	return xxx_messageInfo_CounterData.Size(m)
@@ -1568,15 +1641,19 @@ type isCounterData_Value interface {
 type CounterData_Int64Value struct {
 	Int64Value int64 `protobuf:"varint,1,opt,name=int64_value,json=int64Value,proto3,oneof"`
 }
+
 type CounterData_DoubleValue struct {
 	DoubleValue float64 `protobuf:"fixed64,2,opt,name=double_value,json=doubleValue,proto3,oneof"`
 }
+
 type CounterData_StringValue struct {
 	StringValue string `protobuf:"bytes,3,opt,name=string_value,json=stringValue,proto3,oneof"`
 }
 
-func (*CounterData_Int64Value) isCounterData_Value()  {}
+func (*CounterData_Int64Value) isCounterData_Value() {}
+
 func (*CounterData_DoubleValue) isCounterData_Value() {}
+
 func (*CounterData_StringValue) isCounterData_Value() {}
 
 func (m *CounterData) GetValue() isCounterData_Value {
@@ -1702,7 +1779,7 @@ func (m *ExtremaData) Reset()         { *m = ExtremaData{} }
 func (m *ExtremaData) String() string { return proto.CompactTextString(m) }
 func (*ExtremaData) ProtoMessage()    {}
 func (*ExtremaData) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_6ff4efbe085bd72e, []int{14}
+	return fileDescriptor_6d954c03a4758710, []int{14}
 }
 func (m *ExtremaData) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_ExtremaData.Unmarshal(m, b)
@@ -1710,8 +1787,8 @@ func (m *ExtremaData) XXX_Unmarshal(b []byte) error {
 func (m *ExtremaData) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_ExtremaData.Marshal(b, m, deterministic)
 }
-func (dst *ExtremaData) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ExtremaData.Merge(dst, src)
+func (m *ExtremaData) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_ExtremaData.Merge(m, src)
 }
 func (m *ExtremaData) XXX_Size() int {
 	return xxx_messageInfo_ExtremaData.Size(m)
@@ -1729,11 +1806,13 @@ type isExtremaData_Extrema interface {
 type ExtremaData_IntExtremaData struct {
 	IntExtremaData *IntExtremaData `protobuf:"bytes,1,opt,name=int_extrema_data,json=intExtremaData,proto3,oneof"`
 }
+
 type ExtremaData_DoubleExtremaData struct {
 	DoubleExtremaData *DoubleExtremaData `protobuf:"bytes,2,opt,name=double_extrema_data,json=doubleExtremaData,proto3,oneof"`
 }
 
-func (*ExtremaData_IntExtremaData) isExtremaData_Extrema()    {}
+func (*ExtremaData_IntExtremaData) isExtremaData_Extrema() {}
+
 func (*ExtremaData_DoubleExtremaData) isExtremaData_Extrema() {}
 
 func (m *ExtremaData) GetExtrema() isExtremaData_Extrema {
@@ -1842,7 +1921,7 @@ func (m *IntExtremaData) Reset()         { *m = IntExtremaData{} }
 func (m *IntExtremaData) String() string { return proto.CompactTextString(m) }
 func (*IntExtremaData) ProtoMessage()    {}
 func (*IntExtremaData) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_6ff4efbe085bd72e, []int{15}
+	return fileDescriptor_6d954c03a4758710, []int{15}
 }
 func (m *IntExtremaData) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_IntExtremaData.Unmarshal(m, b)
@@ -1850,8 +1929,8 @@ func (m *IntExtremaData) XXX_Unmarshal(b []byte) error {
 func (m *IntExtremaData) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_IntExtremaData.Marshal(b, m, deterministic)
 }
-func (dst *IntExtremaData) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_IntExtremaData.Merge(dst, src)
+func (m *IntExtremaData) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_IntExtremaData.Merge(m, src)
 }
 func (m *IntExtremaData) XXX_Size() int {
 	return xxx_messageInfo_IntExtremaData.Size(m)
@@ -1880,7 +1959,7 @@ func (m *DoubleExtremaData) Reset()         { *m = DoubleExtremaData{} }
 func (m *DoubleExtremaData) String() string { return proto.CompactTextString(m) }
 func (*DoubleExtremaData) ProtoMessage()    {}
 func (*DoubleExtremaData) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_6ff4efbe085bd72e, []int{16}
+	return fileDescriptor_6d954c03a4758710, []int{16}
 }
 func (m *DoubleExtremaData) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_DoubleExtremaData.Unmarshal(m, b)
@@ -1888,8 +1967,8 @@ func (m *DoubleExtremaData) XXX_Unmarshal(b []byte) error {
 func (m *DoubleExtremaData) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_DoubleExtremaData.Marshal(b, m, deterministic)
 }
-func (dst *DoubleExtremaData) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_DoubleExtremaData.Merge(dst, src)
+func (m *DoubleExtremaData) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_DoubleExtremaData.Merge(m, src)
 }
 func (m *DoubleExtremaData) XXX_Size() int {
 	return xxx_messageInfo_DoubleExtremaData.Size(m)
@@ -1925,7 +2004,7 @@ func (m *DistributionData) Reset()         { *m = DistributionData{} }
 func (m *DistributionData) String() string { return proto.CompactTextString(m) }
 func (*DistributionData) ProtoMessage()    {}
 func (*DistributionData) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_6ff4efbe085bd72e, []int{17}
+	return fileDescriptor_6d954c03a4758710, []int{17}
 }
 func (m *DistributionData) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_DistributionData.Unmarshal(m, b)
@@ -1933,8 +2012,8 @@ func (m *DistributionData) XXX_Unmarshal(b []byte) error {
 func (m *DistributionData) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_DistributionData.Marshal(b, m, deterministic)
 }
-func (dst *DistributionData) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_DistributionData.Merge(dst, src)
+func (m *DistributionData) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_DistributionData.Merge(m, src)
 }
 func (m *DistributionData) XXX_Size() int {
 	return xxx_messageInfo_DistributionData.Size(m)
@@ -1952,11 +2031,13 @@ type isDistributionData_Distribution interface {
 type DistributionData_IntDistributionData struct {
 	IntDistributionData *IntDistributionData `protobuf:"bytes,1,opt,name=int_distribution_data,json=intDistributionData,proto3,oneof"`
 }
+
 type DistributionData_DoubleDistributionData struct {
 	DoubleDistributionData *DoubleDistributionData `protobuf:"bytes,2,opt,name=double_distribution_data,json=doubleDistributionData,proto3,oneof"`
 }
 
-func (*DistributionData_IntDistributionData) isDistributionData_Distribution()    {}
+func (*DistributionData_IntDistributionData) isDistributionData_Distribution() {}
+
 func (*DistributionData_DoubleDistributionData) isDistributionData_Distribution() {}
 
 func (m *DistributionData) GetDistribution() isDistributionData_Distribution {
@@ -2068,7 +2149,7 @@ func (m *IntDistributionData) Reset()         { *m = IntDistributionData{} }
 func (m *IntDistributionData) String() string { return proto.CompactTextString(m) }
 func (*IntDistributionData) ProtoMessage()    {}
 func (*IntDistributionData) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_6ff4efbe085bd72e, []int{18}
+	return fileDescriptor_6d954c03a4758710, []int{18}
 }
 func (m *IntDistributionData) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_IntDistributionData.Unmarshal(m, b)
@@ -2076,8 +2157,8 @@ func (m *IntDistributionData) XXX_Unmarshal(b []byte) error {
 func (m *IntDistributionData) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_IntDistributionData.Marshal(b, m, deterministic)
 }
-func (dst *IntDistributionData) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_IntDistributionData.Merge(dst, src)
+func (m *IntDistributionData) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_IntDistributionData.Merge(m, src)
 }
 func (m *IntDistributionData) XXX_Size() int {
 	return xxx_messageInfo_IntDistributionData.Size(m)
@@ -2130,7 +2211,7 @@ func (m *DoubleDistributionData) Reset()         { *m = DoubleDistributionData{}
 func (m *DoubleDistributionData) String() string { return proto.CompactTextString(m) }
 func (*DoubleDistributionData) ProtoMessage()    {}
 func (*DoubleDistributionData) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_6ff4efbe085bd72e, []int{19}
+	return fileDescriptor_6d954c03a4758710, []int{19}
 }
 func (m *DoubleDistributionData) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_DoubleDistributionData.Unmarshal(m, b)
@@ -2138,8 +2219,8 @@ func (m *DoubleDistributionData) XXX_Unmarshal(b []byte) error {
 func (m *DoubleDistributionData) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_DoubleDistributionData.Marshal(b, m, deterministic)
 }
-func (dst *DoubleDistributionData) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_DoubleDistributionData.Merge(dst, src)
+func (m *DoubleDistributionData) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_DoubleDistributionData.Merge(m, src)
 }
 func (m *DoubleDistributionData) XXX_Size() int {
 	return xxx_messageInfo_DoubleDistributionData.Size(m)
@@ -2212,7 +2293,7 @@ func (m *MonitoringTableData) Reset()         { *m = MonitoringTableData{} }
 func (m *MonitoringTableData) String() string { return proto.CompactTextString(m) }
 func (*MonitoringTableData) ProtoMessage()    {}
 func (*MonitoringTableData) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_6ff4efbe085bd72e, []int{20}
+	return fileDescriptor_6d954c03a4758710, []int{20}
 }
 func (m *MonitoringTableData) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_MonitoringTableData.Unmarshal(m, b)
@@ -2220,8 +2301,8 @@ func (m *MonitoringTableData) XXX_Unmarshal(b []byte) error {
 func (m *MonitoringTableData) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_MonitoringTableData.Marshal(b, m, deterministic)
 }
-func (dst *MonitoringTableData) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_MonitoringTableData.Merge(dst, src)
+func (m *MonitoringTableData) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_MonitoringTableData.Merge(m, src)
 }
 func (m *MonitoringTableData) XXX_Size() int {
 	return xxx_messageInfo_MonitoringTableData.Size(m)
@@ -2264,7 +2345,7 @@ func (m *MonitoringTableData_MonitoringColumnValue) Reset() {
 func (m *MonitoringTableData_MonitoringColumnValue) String() string { return proto.CompactTextString(m) }
 func (*MonitoringTableData_MonitoringColumnValue) ProtoMessage()    {}
 func (*MonitoringTableData_MonitoringColumnValue) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_6ff4efbe085bd72e, []int{20, 0}
+	return fileDescriptor_6d954c03a4758710, []int{20, 0}
 }
 func (m *MonitoringTableData_MonitoringColumnValue) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_MonitoringTableData_MonitoringColumnValue.Unmarshal(m, b)
@@ -2272,8 +2353,8 @@ func (m *MonitoringTableData_MonitoringColumnValue) XXX_Unmarshal(b []byte) erro
 func (m *MonitoringTableData_MonitoringColumnValue) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_MonitoringTableData_MonitoringColumnValue.Marshal(b, m, deterministic)
 }
-func (dst *MonitoringTableData_MonitoringColumnValue) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_MonitoringTableData_MonitoringColumnValue.Merge(dst, src)
+func (m *MonitoringTableData_MonitoringColumnValue) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_MonitoringTableData_MonitoringColumnValue.Merge(m, src)
 }
 func (m *MonitoringTableData_MonitoringColumnValue) XXX_Size() int {
 	return xxx_messageInfo_MonitoringTableData_MonitoringColumnValue.Size(m)
@@ -2291,22 +2372,28 @@ type isMonitoringTableData_MonitoringColumnValue_Value interface {
 type MonitoringTableData_MonitoringColumnValue_Int64Value struct {
 	Int64Value int64 `protobuf:"varint,1,opt,name=int64_value,json=int64Value,proto3,oneof"`
 }
+
 type MonitoringTableData_MonitoringColumnValue_DoubleValue struct {
 	DoubleValue float64 `protobuf:"fixed64,2,opt,name=double_value,json=doubleValue,proto3,oneof"`
 }
+
 type MonitoringTableData_MonitoringColumnValue_StringValue struct {
 	StringValue string `protobuf:"bytes,3,opt,name=string_value,json=stringValue,proto3,oneof"`
 }
+
 type MonitoringTableData_MonitoringColumnValue_Timestamp struct {
 	Timestamp *timestamp.Timestamp `protobuf:"bytes,4,opt,name=timestamp,proto3,oneof"`
 }
 
 func (*MonitoringTableData_MonitoringColumnValue_Int64Value) isMonitoringTableData_MonitoringColumnValue_Value() {
 }
+
 func (*MonitoringTableData_MonitoringColumnValue_DoubleValue) isMonitoringTableData_MonitoringColumnValue_Value() {
 }
+
 func (*MonitoringTableData_MonitoringColumnValue_StringValue) isMonitoringTableData_MonitoringColumnValue_Value() {
 }
+
 func (*MonitoringTableData_MonitoringColumnValue_Timestamp) isMonitoringTableData_MonitoringColumnValue_Value() {
 }
 
@@ -2454,7 +2541,7 @@ func (m *MonitoringTableData_MonitoringRow) Reset()         { *m = MonitoringTab
 func (m *MonitoringTableData_MonitoringRow) String() string { return proto.CompactTextString(m) }
 func (*MonitoringTableData_MonitoringRow) ProtoMessage()    {}
 func (*MonitoringTableData_MonitoringRow) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_6ff4efbe085bd72e, []int{20, 1}
+	return fileDescriptor_6d954c03a4758710, []int{20, 1}
 }
 func (m *MonitoringTableData_MonitoringRow) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_MonitoringTableData_MonitoringRow.Unmarshal(m, b)
@@ -2462,8 +2549,8 @@ func (m *MonitoringTableData_MonitoringRow) XXX_Unmarshal(b []byte) error {
 func (m *MonitoringTableData_MonitoringRow) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_MonitoringTableData_MonitoringRow.Marshal(b, m, deterministic)
 }
-func (dst *MonitoringTableData_MonitoringRow) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_MonitoringTableData_MonitoringRow.Merge(dst, src)
+func (m *MonitoringTableData_MonitoringRow) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_MonitoringTableData_MonitoringRow.Merge(m, src)
 }
 func (m *MonitoringTableData_MonitoringRow) XXX_Size() int {
 	return xxx_messageInfo_MonitoringTableData_MonitoringRow.Size(m)
@@ -2493,7 +2580,7 @@ func (m *Metrics) Reset()         { *m = Metrics{} }
 func (m *Metrics) String() string { return proto.CompactTextString(m) }
 func (*Metrics) ProtoMessage()    {}
 func (*Metrics) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_6ff4efbe085bd72e, []int{21}
+	return fileDescriptor_6d954c03a4758710, []int{21}
 }
 func (m *Metrics) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Metrics.Unmarshal(m, b)
@@ -2501,8 +2588,8 @@ func (m *Metrics) XXX_Unmarshal(b []byte) error {
 func (m *Metrics) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_Metrics.Marshal(b, m, deterministic)
 }
-func (dst *Metrics) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Metrics.Merge(dst, src)
+func (m *Metrics) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_Metrics.Merge(m, src)
 }
 func (m *Metrics) XXX_Size() int {
 	return xxx_messageInfo_Metrics.Size(m)
@@ -2546,7 +2633,7 @@ func (m *Metrics_PTransform) Reset()         { *m = Metrics_PTransform{} }
 func (m *Metrics_PTransform) String() string { return proto.CompactTextString(m) }
 func (*Metrics_PTransform) ProtoMessage()    {}
 func (*Metrics_PTransform) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_6ff4efbe085bd72e, []int{21, 0}
+	return fileDescriptor_6d954c03a4758710, []int{21, 0}
 }
 func (m *Metrics_PTransform) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Metrics_PTransform.Unmarshal(m, b)
@@ -2554,8 +2641,8 @@ func (m *Metrics_PTransform) XXX_Unmarshal(b []byte) error {
 func (m *Metrics_PTransform) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_Metrics_PTransform.Marshal(b, m, deterministic)
 }
-func (dst *Metrics_PTransform) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Metrics_PTransform.Merge(dst, src)
+func (m *Metrics_PTransform) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_Metrics_PTransform.Merge(m, src)
 }
 func (m *Metrics_PTransform) XXX_Size() int {
 	return xxx_messageInfo_Metrics_PTransform.Size(m)
@@ -2616,7 +2703,7 @@ func (m *Metrics_PTransform_Measured) Reset()         { *m = Metrics_PTransform_
 func (m *Metrics_PTransform_Measured) String() string { return proto.CompactTextString(m) }
 func (*Metrics_PTransform_Measured) ProtoMessage()    {}
 func (*Metrics_PTransform_Measured) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_6ff4efbe085bd72e, []int{21, 0, 0}
+	return fileDescriptor_6d954c03a4758710, []int{21, 0, 0}
 }
 func (m *Metrics_PTransform_Measured) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Metrics_PTransform_Measured.Unmarshal(m, b)
@@ -2624,8 +2711,8 @@ func (m *Metrics_PTransform_Measured) XXX_Unmarshal(b []byte) error {
 func (m *Metrics_PTransform_Measured) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_Metrics_PTransform_Measured.Marshal(b, m, deterministic)
 }
-func (dst *Metrics_PTransform_Measured) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Metrics_PTransform_Measured.Merge(dst, src)
+func (m *Metrics_PTransform_Measured) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_Metrics_PTransform_Measured.Merge(m, src)
 }
 func (m *Metrics_PTransform_Measured) XXX_Size() int {
 	return xxx_messageInfo_Metrics_PTransform_Measured.Size(m)
@@ -2670,7 +2757,7 @@ func (m *Metrics_PTransform_ProcessedElements) Reset()         { *m = Metrics_PT
 func (m *Metrics_PTransform_ProcessedElements) String() string { return proto.CompactTextString(m) }
 func (*Metrics_PTransform_ProcessedElements) ProtoMessage()    {}
 func (*Metrics_PTransform_ProcessedElements) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_6ff4efbe085bd72e, []int{21, 0, 1}
+	return fileDescriptor_6d954c03a4758710, []int{21, 0, 1}
 }
 func (m *Metrics_PTransform_ProcessedElements) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Metrics_PTransform_ProcessedElements.Unmarshal(m, b)
@@ -2678,8 +2765,8 @@ func (m *Metrics_PTransform_ProcessedElements) XXX_Unmarshal(b []byte) error {
 func (m *Metrics_PTransform_ProcessedElements) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_Metrics_PTransform_ProcessedElements.Marshal(b, m, deterministic)
 }
-func (dst *Metrics_PTransform_ProcessedElements) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Metrics_PTransform_ProcessedElements.Merge(dst, src)
+func (m *Metrics_PTransform_ProcessedElements) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_Metrics_PTransform_ProcessedElements.Merge(m, src)
 }
 func (m *Metrics_PTransform_ProcessedElements) XXX_Size() int {
 	return xxx_messageInfo_Metrics_PTransform_ProcessedElements.Size(m)
@@ -2724,7 +2811,7 @@ func (m *Metrics_PTransform_ActiveElements) Reset()         { *m = Metrics_PTran
 func (m *Metrics_PTransform_ActiveElements) String() string { return proto.CompactTextString(m) }
 func (*Metrics_PTransform_ActiveElements) ProtoMessage()    {}
 func (*Metrics_PTransform_ActiveElements) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_6ff4efbe085bd72e, []int{21, 0, 2}
+	return fileDescriptor_6d954c03a4758710, []int{21, 0, 2}
 }
 func (m *Metrics_PTransform_ActiveElements) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Metrics_PTransform_ActiveElements.Unmarshal(m, b)
@@ -2732,8 +2819,8 @@ func (m *Metrics_PTransform_ActiveElements) XXX_Unmarshal(b []byte) error {
 func (m *Metrics_PTransform_ActiveElements) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_Metrics_PTransform_ActiveElements.Marshal(b, m, deterministic)
 }
-func (dst *Metrics_PTransform_ActiveElements) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Metrics_PTransform_ActiveElements.Merge(dst, src)
+func (m *Metrics_PTransform_ActiveElements) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_Metrics_PTransform_ActiveElements.Merge(m, src)
 }
 func (m *Metrics_PTransform_ActiveElements) XXX_Size() int {
 	return xxx_messageInfo_Metrics_PTransform_ActiveElements.Size(m)
@@ -2785,7 +2872,7 @@ func (m *Metrics_User) Reset()         { *m = Metrics_User{} }
 func (m *Metrics_User) String() string { return proto.CompactTextString(m) }
 func (*Metrics_User) ProtoMessage()    {}
 func (*Metrics_User) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_6ff4efbe085bd72e, []int{21, 1}
+	return fileDescriptor_6d954c03a4758710, []int{21, 1}
 }
 func (m *Metrics_User) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Metrics_User.Unmarshal(m, b)
@@ -2793,8 +2880,8 @@ func (m *Metrics_User) XXX_Unmarshal(b []byte) error {
 func (m *Metrics_User) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_Metrics_User.Marshal(b, m, deterministic)
 }
-func (dst *Metrics_User) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Metrics_User.Merge(dst, src)
+func (m *Metrics_User) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_Metrics_User.Merge(m, src)
 }
 func (m *Metrics_User) XXX_Size() int {
 	return xxx_messageInfo_Metrics_User.Size(m)
@@ -2805,6 +2892,13 @@ func (m *Metrics_User) XXX_DiscardUnknown() {
 
 var xxx_messageInfo_Metrics_User proto.InternalMessageInfo
 
+func (m *Metrics_User) GetMetricName() *Metrics_User_MetricName {
+	if m != nil {
+		return m.MetricName
+	}
+	return nil
+}
+
 type isMetrics_User_Data interface {
 	isMetrics_User_Data()
 }
@@ -2812,16 +2906,20 @@ type isMetrics_User_Data interface {
 type Metrics_User_CounterData_ struct {
 	CounterData *Metrics_User_CounterData `protobuf:"bytes,1001,opt,name=counter_data,json=counterData,proto3,oneof"`
 }
+
 type Metrics_User_DistributionData_ struct {
 	DistributionData *Metrics_User_DistributionData `protobuf:"bytes,1002,opt,name=distribution_data,json=distributionData,proto3,oneof"`
 }
+
 type Metrics_User_GaugeData_ struct {
 	GaugeData *Metrics_User_GaugeData `protobuf:"bytes,1003,opt,name=gauge_data,json=gaugeData,proto3,oneof"`
 }
 
-func (*Metrics_User_CounterData_) isMetrics_User_Data()      {}
+func (*Metrics_User_CounterData_) isMetrics_User_Data() {}
+
 func (*Metrics_User_DistributionData_) isMetrics_User_Data() {}
-func (*Metrics_User_GaugeData_) isMetrics_User_Data()        {}
+
+func (*Metrics_User_GaugeData_) isMetrics_User_Data() {}
 
 func (m *Metrics_User) GetData() isMetrics_User_Data {
 	if m != nil {
@@ -2830,13 +2928,6 @@ func (m *Metrics_User) GetData() isMetrics_User_Data {
 	return nil
 }
 
-func (m *Metrics_User) GetMetricName() *Metrics_User_MetricName {
-	if m != nil {
-		return m.MetricName
-	}
-	return nil
-}
-
 func (m *Metrics_User) GetCounterData() *Metrics_User_CounterData {
 	if x, ok := m.GetData().(*Metrics_User_CounterData_); ok {
 		return x.CounterData
@@ -2966,7 +3057,7 @@ func (m *Metrics_User_MetricName) Reset()         { *m = Metrics_User_MetricName
 func (m *Metrics_User_MetricName) String() string { return proto.CompactTextString(m) }
 func (*Metrics_User_MetricName) ProtoMessage()    {}
 func (*Metrics_User_MetricName) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_6ff4efbe085bd72e, []int{21, 1, 0}
+	return fileDescriptor_6d954c03a4758710, []int{21, 1, 0}
 }
 func (m *Metrics_User_MetricName) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Metrics_User_MetricName.Unmarshal(m, b)
@@ -2974,8 +3065,8 @@ func (m *Metrics_User_MetricName) XXX_Unmarshal(b []byte) error {
 func (m *Metrics_User_MetricName) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_Metrics_User_MetricName.Marshal(b, m, deterministic)
 }
-func (dst *Metrics_User_MetricName) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Metrics_User_MetricName.Merge(dst, src)
+func (m *Metrics_User_MetricName) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_Metrics_User_MetricName.Merge(m, src)
 }
 func (m *Metrics_User_MetricName) XXX_Size() int {
 	return xxx_messageInfo_Metrics_User_MetricName.Size(m)
@@ -3012,7 +3103,7 @@ func (m *Metrics_User_CounterData) Reset()         { *m = Metrics_User_CounterDa
 func (m *Metrics_User_CounterData) String() string { return proto.CompactTextString(m) }
 func (*Metrics_User_CounterData) ProtoMessage()    {}
 func (*Metrics_User_CounterData) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_6ff4efbe085bd72e, []int{21, 1, 1}
+	return fileDescriptor_6d954c03a4758710, []int{21, 1, 1}
 }
 func (m *Metrics_User_CounterData) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Metrics_User_CounterData.Unmarshal(m, b)
@@ -3020,8 +3111,8 @@ func (m *Metrics_User_CounterData) XXX_Unmarshal(b []byte) error {
 func (m *Metrics_User_CounterData) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_Metrics_User_CounterData.Marshal(b, m, deterministic)
 }
-func (dst *Metrics_User_CounterData) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Metrics_User_CounterData.Merge(dst, src)
+func (m *Metrics_User_CounterData) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_Metrics_User_CounterData.Merge(m, src)
 }
 func (m *Metrics_User_CounterData) XXX_Size() int {
 	return xxx_messageInfo_Metrics_User_CounterData.Size(m)
@@ -3054,7 +3145,7 @@ func (m *Metrics_User_DistributionData) Reset()         { *m = Metrics_User_Dist
 func (m *Metrics_User_DistributionData) String() string { return proto.CompactTextString(m) }
 func (*Metrics_User_DistributionData) ProtoMessage()    {}
 func (*Metrics_User_DistributionData) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_6ff4efbe085bd72e, []int{21, 1, 2}
+	return fileDescriptor_6d954c03a4758710, []int{21, 1, 2}
 }
 func (m *Metrics_User_DistributionData) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Metrics_User_DistributionData.Unmarshal(m, b)
@@ -3062,8 +3153,8 @@ func (m *Metrics_User_DistributionData) XXX_Unmarshal(b []byte) error {
 func (m *Metrics_User_DistributionData) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_Metrics_User_DistributionData.Marshal(b, m, deterministic)
 }
-func (dst *Metrics_User_DistributionData) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Metrics_User_DistributionData.Merge(dst, src)
+func (m *Metrics_User_DistributionData) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_Metrics_User_DistributionData.Merge(m, src)
 }
 func (m *Metrics_User_DistributionData) XXX_Size() int {
 	return xxx_messageInfo_Metrics_User_DistributionData.Size(m)
@@ -3115,7 +3206,7 @@ func (m *Metrics_User_GaugeData) Reset()         { *m = Metrics_User_GaugeData{}
 func (m *Metrics_User_GaugeData) String() string { return proto.CompactTextString(m) }
 func (*Metrics_User_GaugeData) ProtoMessage()    {}
 func (*Metrics_User_GaugeData) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_6ff4efbe085bd72e, []int{21, 1, 3}
+	return fileDescriptor_6d954c03a4758710, []int{21, 1, 3}
 }
 func (m *Metrics_User_GaugeData) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Metrics_User_GaugeData.Unmarshal(m, b)
@@ -3123,8 +3214,8 @@ func (m *Metrics_User_GaugeData) XXX_Unmarshal(b []byte) error {
 func (m *Metrics_User_GaugeData) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_Metrics_User_GaugeData.Marshal(b, m, deterministic)
 }
-func (dst *Metrics_User_GaugeData) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Metrics_User_GaugeData.Merge(dst, src)
+func (m *Metrics_User_GaugeData) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_Metrics_User_GaugeData.Merge(m, src)
 }
 func (m *Metrics_User_GaugeData) XXX_Size() int {
 	return xxx_messageInfo_Metrics_User_GaugeData.Size(m)
@@ -3164,7 +3255,7 @@ func (m *ProcessBundleProgressResponse) Reset()         { *m = ProcessBundleProg
 func (m *ProcessBundleProgressResponse) String() string { return proto.CompactTextString(m) }
 func (*ProcessBundleProgressResponse) ProtoMessage()    {}
 func (*ProcessBundleProgressResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_6ff4efbe085bd72e, []int{22}
+	return fileDescriptor_6d954c03a4758710, []int{22}
 }
 func (m *ProcessBundleProgressResponse) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_ProcessBundleProgressResponse.Unmarshal(m, b)
@@ -3172,8 +3263,8 @@ func (m *ProcessBundleProgressResponse) XXX_Unmarshal(b []byte) error {
 func (m *ProcessBundleProgressResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_ProcessBundleProgressResponse.Marshal(b, m, deterministic)
 }
-func (dst *ProcessBundleProgressResponse) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ProcessBundleProgressResponse.Merge(dst, src)
+func (m *ProcessBundleProgressResponse) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_ProcessBundleProgressResponse.Merge(m, src)
 }
 func (m *ProcessBundleProgressResponse) XXX_Size() int {
 	return xxx_messageInfo_ProcessBundleProgressResponse.Size(m)
@@ -3223,7 +3314,7 @@ func (m *ProcessBundleSplitRequest) Reset()         { *m = ProcessBundleSplitReq
 func (m *ProcessBundleSplitRequest) String() string { return proto.CompactTextString(m) }
 func (*ProcessBundleSplitRequest) ProtoMessage()    {}
 func (*ProcessBundleSplitRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_6ff4efbe085bd72e, []int{23}
+	return fileDescriptor_6d954c03a4758710, []int{23}
 }
 func (m *ProcessBundleSplitRequest) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_ProcessBundleSplitRequest.Unmarshal(m, b)
@@ -3231,8 +3322,8 @@ func (m *ProcessBundleSplitRequest) XXX_Unmarshal(b []byte) error {
 func (m *ProcessBundleSplitRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_ProcessBundleSplitRequest.Marshal(b, m, deterministic)
 }
-func (dst *ProcessBundleSplitRequest) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ProcessBundleSplitRequest.Merge(dst, src)
+func (m *ProcessBundleSplitRequest) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_ProcessBundleSplitRequest.Merge(m, src)
 }
 func (m *ProcessBundleSplitRequest) XXX_Size() int {
 	return xxx_messageInfo_ProcessBundleSplitRequest.Size(m)
@@ -3267,7 +3358,7 @@ func (m *ProcessBundleSplitResponse) Reset()         { *m = ProcessBundleSplitRe
 func (m *ProcessBundleSplitResponse) String() string { return proto.CompactTextString(m) }
 func (*ProcessBundleSplitResponse) ProtoMessage()    {}
 func (*ProcessBundleSplitResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_6ff4efbe085bd72e, []int{24}
+	return fileDescriptor_6d954c03a4758710, []int{24}
 }
 func (m *ProcessBundleSplitResponse) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_ProcessBundleSplitResponse.Unmarshal(m, b)
@@ -3275,8 +3366,8 @@ func (m *ProcessBundleSplitResponse) XXX_Unmarshal(b []byte) error {
 func (m *ProcessBundleSplitResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_ProcessBundleSplitResponse.Marshal(b, m, deterministic)
 }
-func (dst *ProcessBundleSplitResponse) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ProcessBundleSplitResponse.Merge(dst, src)
+func (m *ProcessBundleSplitResponse) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_ProcessBundleSplitResponse.Merge(m, src)
 }
 func (m *ProcessBundleSplitResponse) XXX_Size() int {
 	return xxx_messageInfo_ProcessBundleSplitResponse.Size(m)
@@ -3301,7 +3392,7 @@ func (m *Elements) Reset()         { *m = Elements{} }
 func (m *Elements) String() string { return proto.CompactTextString(m) }
 func (*Elements) ProtoMessage()    {}
 func (*Elements) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_6ff4efbe085bd72e, []int{25}
+	return fileDescriptor_6d954c03a4758710, []int{25}
 }
 func (m *Elements) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Elements.Unmarshal(m, b)
@@ -3309,8 +3400,8 @@ func (m *Elements) XXX_Unmarshal(b []byte) error {
 func (m *Elements) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_Elements.Marshal(b, m, deterministic)
 }
-func (dst *Elements) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Elements.Merge(dst, src)
+func (m *Elements) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_Elements.Merge(m, src)
 }
 func (m *Elements) XXX_Size() int {
 	return xxx_messageInfo_Elements.Size(m)
@@ -3360,7 +3451,7 @@ func (m *Elements_Data) Reset()         { *m = Elements_Data{} }
 func (m *Elements_Data) String() string { return proto.CompactTextString(m) }
 func (*Elements_Data) ProtoMessage()    {}
 func (*Elements_Data) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_6ff4efbe085bd72e, []int{25, 0}
+	return fileDescriptor_6d954c03a4758710, []int{25, 0}
 }
 func (m *Elements_Data) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Elements_Data.Unmarshal(m, b)
@@ -3368,8 +3459,8 @@ func (m *Elements_Data) XXX_Unmarshal(b []byte) error {
 func (m *Elements_Data) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_Elements_Data.Marshal(b, m, deterministic)
 }
-func (dst *Elements_Data) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Elements_Data.Merge(dst, src)
+func (m *Elements_Data) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_Elements_Data.Merge(m, src)
 }
 func (m *Elements_Data) XXX_Size() int {
 	return xxx_messageInfo_Elements_Data.Size(m)
@@ -3428,7 +3519,7 @@ func (m *StateRequest) Reset()         { *m = StateRequest{} }
 func (m *StateRequest) String() string { return proto.CompactTextString(m) }
 func (*StateRequest) ProtoMessage()    {}
 func (*StateRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_6ff4efbe085bd72e, []int{26}
+	return fileDescriptor_6d954c03a4758710, []int{26}
 }
 func (m *StateRequest) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_StateRequest.Unmarshal(m, b)
@@ -3436,8 +3527,8 @@ func (m *StateRequest) XXX_Unmarshal(b []byte) error {
 func (m *StateRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_StateRequest.Marshal(b, m, deterministic)
 }
-func (dst *StateRequest) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_StateRequest.Merge(dst, src)
+func (m *StateRequest) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_StateRequest.Merge(m, src)
 }
 func (m *StateRequest) XXX_Size() int {
 	return xxx_messageInfo_StateRequest.Size(m)
@@ -3448,6 +3539,27 @@ func (m *StateRequest) XXX_DiscardUnknown() {
 
 var xxx_messageInfo_StateRequest proto.InternalMessageInfo
 
+func (m *StateRequest) GetId() string {
+	if m != nil {
+		return m.Id
+	}
+	return ""
+}
+
+func (m *StateRequest) GetInstructionReference() string {
+	if m != nil {
+		return m.InstructionReference
+	}
+	return ""
+}
+
+func (m *StateRequest) GetStateKey() *StateKey {
+	if m != nil {
+		return m.StateKey
+	}
+	return nil
+}
+
 type isStateRequest_Request interface {
 	isStateRequest_Request()
 }
@@ -3455,16 +3567,20 @@ type isStateRequest_Request interface {
 type StateRequest_Get struct {
 	Get *StateGetRequest `protobuf:"bytes,1000,opt,name=get,proto3,oneof"`
 }
+
 type StateRequest_Append struct {
 	Append *StateAppendRequest `protobuf:"bytes,1001,opt,name=append,proto3,oneof"`
 }
+
 type StateRequest_Clear struct {
 	Clear *StateClearRequest `protobuf:"bytes,1002,opt,name=clear,proto3,oneof"`
 }
 
-func (*StateRequest_Get) isStateRequest_Request()    {}
+func (*StateRequest_Get) isStateRequest_Request() {}
+
 func (*StateRequest_Append) isStateRequest_Request() {}
-func (*StateRequest_Clear) isStateRequest_Request()  {}
+
+func (*StateRequest_Clear) isStateRequest_Request() {}
 
 func (m *StateRequest) GetRequest() isStateRequest_Request {
 	if m != nil {
@@ -3473,27 +3589,6 @@ func (m *StateRequest) GetRequest() isStateRequest_Request {
 	return nil
 }
 
-func (m *StateRequest) GetId() string {
-	if m != nil {
-		return m.Id
-	}
-	return ""
-}
-
-func (m *StateRequest) GetInstructionReference() string {
-	if m != nil {
-		return m.InstructionReference
-	}
-	return ""
-}
-
-func (m *StateRequest) GetStateKey() *StateKey {
-	if m != nil {
-		return m.StateKey
-	}
-	return nil
-}
-
 func (m *StateRequest) GetGet() *StateGetRequest {
 	if x, ok := m.GetRequest().(*StateRequest_Get); ok {
 		return x.Get
@@ -3636,7 +3731,7 @@ func (m *StateResponse) Reset()         { *m = StateResponse{} }
 func (m *StateResponse) String() string { return proto.CompactTextString(m) }
 func (*StateResponse) ProtoMessage()    {}
 func (*StateResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_6ff4efbe085bd72e, []int{27}
+	return fileDescriptor_6d954c03a4758710, []int{27}
 }
 func (m *StateResponse) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_StateResponse.Unmarshal(m, b)
@@ -3644,8 +3739,8 @@ func (m *StateResponse) XXX_Unmarshal(b []byte) error {
 func (m *StateResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_StateResponse.Marshal(b, m, deterministic)
 }
-func (dst *StateResponse) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_StateResponse.Merge(dst, src)
+func (m *StateResponse) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_StateResponse.Merge(m, src)
 }
 func (m *StateResponse) XXX_Size() int {
 	return xxx_messageInfo_StateResponse.Size(m)
@@ -3656,6 +3751,27 @@ func (m *StateResponse) XXX_DiscardUnknown() {
 
 var xxx_messageInfo_StateResponse proto.InternalMessageInfo
 
+func (m *StateResponse) GetId() string {
+	if m != nil {
+		return m.Id
+	}
+	return ""
+}
+
+func (m *StateResponse) GetError() string {
+	if m != nil {
+		return m.Error
+	}
+	return ""
+}
+
+func (m *StateResponse) GetCacheToken() []byte {
+	if m != nil {
+		return m.CacheToken
+	}
+	return nil
+}
+
 type isStateResponse_Response interface {
 	isStateResponse_Response()
 }
@@ -3663,16 +3779,20 @@ type isStateResponse_Response interface {
 type StateResponse_Get struct {
 	Get *StateGetResponse `protobuf:"bytes,1000,opt,name=get,proto3,oneof"`
 }
+
 type StateResponse_Append struct {
 	Append *StateAppendResponse `protobuf:"bytes,1001,opt,name=append,proto3,oneof"`
 }
+
 type StateResponse_Clear struct {
 	Clear *StateClearResponse `protobuf:"bytes,1002,opt,name=clear,proto3,oneof"`
 }
 
-func (*StateResponse_Get) isStateResponse_Response()    {}
+func (*StateResponse_Get) isStateResponse_Response() {}
+
 func (*StateResponse_Append) isStateResponse_Response() {}
-func (*StateResponse_Clear) isStateResponse_Response()  {}
+
+func (*StateResponse_Clear) isStateResponse_Response() {}
 
 func (m *StateResponse) GetResponse() isStateResponse_Response {
 	if m != nil {
@@ -3681,27 +3801,6 @@ func (m *StateResponse) GetResponse() isStateResponse_Response {
 	return nil
 }
 
-func (m *StateResponse) GetId() string {
-	if m != nil {
-		return m.Id
-	}
-	return ""
-}
-
-func (m *StateResponse) GetError() string {
-	if m != nil {
-		return m.Error
-	}
-	return ""
-}
-
-func (m *StateResponse) GetCacheToken() []byte {
-	if m != nil {
-		return m.CacheToken
-	}
-	return nil
-}
-
 func (m *StateResponse) GetGet() *StateGetResponse {
 	if x, ok := m.GetResponse().(*StateResponse_Get); ok {
 		return x.Get
@@ -3833,7 +3932,7 @@ func (m *StateKey) Reset()         { *m = StateKey{} }
 func (m *StateKey) String() string { return proto.CompactTextString(m) }
 func (*StateKey) ProtoMessage()    {}
 func (*StateKey) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_6ff4efbe085bd72e, []int{28}
+	return fileDescriptor_6d954c03a4758710, []int{28}
 }
 func (m *StateKey) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_StateKey.Unmarshal(m, b)
@@ -3841,8 +3940,8 @@ func (m *StateKey) XXX_Unmarshal(b []byte) error {
 func (m *StateKey) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_StateKey.Marshal(b, m, deterministic)
 }
-func (dst *StateKey) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_StateKey.Merge(dst, src)
+func (m *StateKey) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_StateKey.Merge(m, src)
 }
 func (m *StateKey) XXX_Size() int {
 	return xxx_messageInfo_StateKey.Size(m)
@@ -3860,16 +3959,20 @@ type isStateKey_Type interface {
 type StateKey_Runner_ struct {
 	Runner *StateKey_Runner `protobuf:"bytes,1,opt,name=runner,proto3,oneof"`
 }
+
 type StateKey_MultimapSideInput_ struct {
 	MultimapSideInput *StateKey_MultimapSideInput `protobuf:"bytes,2,opt,name=multimap_side_input,json=multimapSideInput,proto3,oneof"`
 }
+
 type StateKey_BagUserState_ struct {
 	BagUserState *StateKey_BagUserState `protobuf:"bytes,3,opt,name=bag_user_state,json=bagUserState,proto3,oneof"`
 }
 
-func (*StateKey_Runner_) isStateKey_Type()            {}
+func (*StateKey_Runner_) isStateKey_Type() {}
+
 func (*StateKey_MultimapSideInput_) isStateKey_Type() {}
-func (*StateKey_BagUserState_) isStateKey_Type()      {}
+
+func (*StateKey_BagUserState_) isStateKey_Type() {}
 
 func (m *StateKey) GetType() isStateKey_Type {
 	if m != nil {
@@ -4005,7 +4108,7 @@ func (m *StateKey_Runner) Reset()         { *m = StateKey_Runner{} }
 func (m *StateKey_Runner) String() string { return proto.CompactTextString(m) }
 func (*StateKey_Runner) ProtoMessage()    {}
 func (*StateKey_Runner) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_6ff4efbe085bd72e, []int{28, 0}
+	return fileDescriptor_6d954c03a4758710, []int{28, 0}
 }
 func (m *StateKey_Runner) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_StateKey_Runner.Unmarshal(m, b)
@@ -4013,8 +4116,8 @@ func (m *StateKey_Runner) XXX_Unmarshal(b []byte) error {
 func (m *StateKey_Runner) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_StateKey_Runner.Marshal(b, m, deterministic)
 }
-func (dst *StateKey_Runner) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_StateKey_Runner.Merge(dst, src)
+func (m *StateKey_Runner) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_StateKey_Runner.Merge(m, src)
 }
 func (m *StateKey_Runner) XXX_Size() int {
 	return xxx_messageInfo_StateKey_Runner.Size(m)
@@ -4051,7 +4154,7 @@ func (m *StateKey_MultimapSideInput) Reset()         { *m = StateKey_MultimapSid
 func (m *StateKey_MultimapSideInput) String() string { return proto.CompactTextString(m) }
 func (*StateKey_MultimapSideInput) ProtoMessage()    {}
 func (*StateKey_MultimapSideInput) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_6ff4efbe085bd72e, []int{28, 1}
+	return fileDescriptor_6d954c03a4758710, []int{28, 1}
 }
 func (m *StateKey_MultimapSideInput) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_StateKey_MultimapSideInput.Unmarshal(m, b)
@@ -4059,8 +4162,8 @@ func (m *StateKey_MultimapSideInput) XXX_Unmarshal(b []byte) error {
 func (m *StateKey_MultimapSideInput) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_StateKey_MultimapSideInput.Marshal(b, m, deterministic)
 }
-func (dst *StateKey_MultimapSideInput) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_StateKey_MultimapSideInput.Merge(dst, src)
+func (m *StateKey_MultimapSideInput) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_StateKey_MultimapSideInput.Merge(m, src)
 }
 func (m *StateKey_MultimapSideInput) XXX_Size() int {
 	return xxx_messageInfo_StateKey_MultimapSideInput.Size(m)
@@ -4118,7 +4221,7 @@ func (m *StateKey_BagUserState) Reset()         { *m = StateKey_BagUserState{} }
 func (m *StateKey_BagUserState) String() string { return proto.CompactTextString(m) }
 func (*StateKey_BagUserState) ProtoMessage()    {}
 func (*StateKey_BagUserState) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_6ff4efbe085bd72e, []int{28, 2}
+	return fileDescriptor_6d954c03a4758710, []int{28, 2}
 }
 func (m *StateKey_BagUserState) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_StateKey_BagUserState.Unmarshal(m, b)
@@ -4126,8 +4229,8 @@ func (m *StateKey_BagUserState) XXX_Unmarshal(b []byte) error {
 func (m *StateKey_BagUserState) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_StateKey_BagUserState.Marshal(b, m, deterministic)
 }
-func (dst *StateKey_BagUserState) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_StateKey_BagUserState.Merge(dst, src)
+func (m *StateKey_BagUserState) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_StateKey_BagUserState.Merge(m, src)
 }
 func (m *StateKey_BagUserState) XXX_Size() int {
 	return xxx_messageInfo_StateKey_BagUserState.Size(m)
@@ -4183,7 +4286,7 @@ func (m *StateGetRequest) Reset()         { *m = StateGetRequest{} }
 func (m *StateGetRequest) String() string { return proto.CompactTextString(m) }
 func (*StateGetRequest) ProtoMessage()    {}
 func (*StateGetRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_6ff4efbe085bd72e, []int{29}
+	return fileDescriptor_6d954c03a4758710, []int{29}
 }
 func (m *StateGetRequest) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_StateGetRequest.Unmarshal(m, b)
@@ -4191,8 +4294,8 @@ func (m *StateGetRequest) XXX_Unmarshal(b []byte) error {
 func (m *StateGetRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_StateGetRequest.Marshal(b, m, deterministic)
 }
-func (dst *StateGetRequest) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_StateGetRequest.Merge(dst, src)
+func (m *StateGetRequest) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_StateGetRequest.Merge(m, src)
 }
 func (m *StateGetRequest) XXX_Size() int {
 	return xxx_messageInfo_StateGetRequest.Size(m)
@@ -4230,7 +4333,7 @@ func (m *StateGetResponse) Reset()         { *m = StateGetResponse{} }
 func (m *StateGetResponse) String() string { return proto.CompactTextString(m) }
 func (*StateGetResponse) ProtoMessage()    {}
 func (*StateGetResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_6ff4efbe085bd72e, []int{30}
+	return fileDescriptor_6d954c03a4758710, []int{30}
 }
 func (m *StateGetResponse) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_StateGetResponse.Unmarshal(m, b)
@@ -4238,8 +4341,8 @@ func (m *StateGetResponse) XXX_Unmarshal(b []byte) error {
 func (m *StateGetResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_StateGetResponse.Marshal(b, m, deterministic)
 }
-func (dst *StateGetResponse) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_StateGetResponse.Merge(dst, src)
+func (m *StateGetResponse) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_StateGetResponse.Merge(m, src)
 }
 func (m *StateGetResponse) XXX_Size() int {
 	return xxx_messageInfo_StateGetResponse.Size(m)
@@ -4279,7 +4382,7 @@ func (m *StateAppendRequest) Reset()         { *m = StateAppendRequest{} }
 func (m *StateAppendRequest) String() string { return proto.CompactTextString(m) }
 func (*StateAppendRequest) ProtoMessage()    {}
 func (*StateAppendRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_6ff4efbe085bd72e, []int{31}
+	return fileDescriptor_6d954c03a4758710, []int{31}
 }
 func (m *StateAppendRequest) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_StateAppendRequest.Unmarshal(m, b)
@@ -4287,8 +4390,8 @@ func (m *StateAppendRequest) XXX_Unmarshal(b []byte) error {
 func (m *StateAppendRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_StateAppendRequest.Marshal(b, m, deterministic)
 }
-func (dst *StateAppendRequest) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_StateAppendRequest.Merge(dst, src)
+func (m *StateAppendRequest) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_StateAppendRequest.Merge(m, src)
 }
 func (m *StateAppendRequest) XXX_Size() int {
 	return xxx_messageInfo_StateAppendRequest.Size(m)
@@ -4317,7 +4420,7 @@ func (m *StateAppendResponse) Reset()         { *m = StateAppendResponse{} }
 func (m *StateAppendResponse) String() string { return proto.CompactTextString(m) }
 func (*StateAppendResponse) ProtoMessage()    {}
 func (*StateAppendResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_6ff4efbe085bd72e, []int{32}
+	return fileDescriptor_6d954c03a4758710, []int{32}
 }
 func (m *StateAppendResponse) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_StateAppendResponse.Unmarshal(m, b)
@@ -4325,8 +4428,8 @@ func (m *StateAppendResponse) XXX_Unmarshal(b []byte) error {
 func (m *StateAppendResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_StateAppendResponse.Marshal(b, m, deterministic)
 }
-func (dst *StateAppendResponse) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_StateAppendResponse.Merge(dst, src)
+func (m *StateAppendResponse) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_StateAppendResponse.Merge(m, src)
 }
 func (m *StateAppendResponse) XXX_Size() int {
 	return xxx_messageInfo_StateAppendResponse.Size(m)
@@ -4348,7 +4451,7 @@ func (m *StateClearRequest) Reset()         { *m = StateClearRequest{} }
 func (m *StateClearRequest) String() string { return proto.CompactTextString(m) }
 func (*StateClearRequest) ProtoMessage()    {}
 func (*StateClearRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_6ff4efbe085bd72e, []int{33}
+	return fileDescriptor_6d954c03a4758710, []int{33}
 }
 func (m *StateClearRequest) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_StateClearRequest.Unmarshal(m, b)
@@ -4356,8 +4459,8 @@ func (m *StateClearRequest) XXX_Unmarshal(b []byte) error {
 func (m *StateClearRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_StateClearRequest.Marshal(b, m, deterministic)
 }
-func (dst *StateClearRequest) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_StateClearRequest.Merge(dst, src)
+func (m *StateClearRequest) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_StateClearRequest.Merge(m, src)
 }
 func (m *StateClearRequest) XXX_Size() int {
 	return xxx_messageInfo_StateClearRequest.Size(m)
@@ -4379,7 +4482,7 @@ func (m *StateClearResponse) Reset()         { *m = StateClearResponse{} }
 func (m *StateClearResponse) String() string { return proto.CompactTextString(m) }
 func (*StateClearResponse) ProtoMessage()    {}
 func (*StateClearResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_6ff4efbe085bd72e, []int{34}
+	return fileDescriptor_6d954c03a4758710, []int{34}
 }
 func (m *StateClearResponse) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_StateClearResponse.Unmarshal(m, b)
@@ -4387,8 +4490,8 @@ func (m *StateClearResponse) XXX_Unmarshal(b []byte) error {
 func (m *StateClearResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_StateClearResponse.Marshal(b, m, deterministic)
 }
-func (dst *StateClearResponse) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_StateClearResponse.Merge(dst, src)
+func (m *StateClearResponse) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_StateClearResponse.Merge(m, src)
 }
 func (m *StateClearResponse) XXX_Size() int {
 	return xxx_messageInfo_StateClearResponse.Size(m)
@@ -4435,7 +4538,7 @@ func (m *LogEntry) Reset()         { *m = LogEntry{} }
 func (m *LogEntry) String() string { return proto.CompactTextString(m) }
 func (*LogEntry) ProtoMessage()    {}
 func (*LogEntry) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_6ff4efbe085bd72e, []int{35}
+	return fileDescriptor_6d954c03a4758710, []int{35}
 }
 func (m *LogEntry) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_LogEntry.Unmarshal(m, b)
@@ -4443,8 +4546,8 @@ func (m *LogEntry) XXX_Unmarshal(b []byte) error {
 func (m *LogEntry) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_LogEntry.Marshal(b, m, deterministic)
 }
-func (dst *LogEntry) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_LogEntry.Merge(dst, src)
+func (m *LogEntry) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_LogEntry.Merge(m, src)
 }
 func (m *LogEntry) XXX_Size() int {
 	return xxx_messageInfo_LogEntry.Size(m)
@@ -4525,7 +4628,7 @@ func (m *LogEntry_List) Reset()         { *m = LogEntry_List{} }
 func (m *LogEntry_List) String() string { return proto.CompactTextString(m) }
 func (*LogEntry_List) ProtoMessage()    {}
 func (*LogEntry_List) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_6ff4efbe085bd72e, []int{35, 0}
+	return fileDescriptor_6d954c03a4758710, []int{35, 0}
 }
 func (m *LogEntry_List) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_LogEntry_List.Unmarshal(m, b)
@@ -4533,8 +4636,8 @@ func (m *LogEntry_List) XXX_Unmarshal(b []byte) error {
 func (m *LogEntry_List) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_LogEntry_List.Marshal(b, m, deterministic)
 }
-func (dst *LogEntry_List) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_LogEntry_List.Merge(dst, src)
+func (m *LogEntry_List) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_LogEntry_List.Merge(m, src)
 }
 func (m *LogEntry_List) XXX_Size() int {
 	return xxx_messageInfo_LogEntry_List.Size(m)
@@ -4575,7 +4678,7 @@ func (m *LogEntry_Severity) Reset()         { *m = LogEntry_Severity{} }
 func (m *LogEntry_Severity) String() string { return proto.CompactTextString(m) }
 func (*LogEntry_Severity) ProtoMessage()    {}
 func (*LogEntry_Severity) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_6ff4efbe085bd72e, []int{35, 1}
+	return fileDescriptor_6d954c03a4758710, []int{35, 1}
 }
 func (m *LogEntry_Severity) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_LogEntry_Severity.Unmarshal(m, b)
@@ -4583,8 +4686,8 @@ func (m *LogEntry_Severity) XXX_Unmarshal(b []byte) error {
 func (m *LogEntry_Severity) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_LogEntry_Severity.Marshal(b, m, deterministic)
 }
-func (dst *LogEntry_Severity) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_LogEntry_Severity.Merge(dst, src)
+func (m *LogEntry_Severity) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_LogEntry_Severity.Merge(m, src)
 }
 func (m *LogEntry_Severity) XXX_Size() int {
 	return xxx_messageInfo_LogEntry_Severity.Size(m)
@@ -4605,7 +4708,7 @@ func (m *LogControl) Reset()         { *m = LogControl{} }
 func (m *LogControl) String() string { return proto.CompactTextString(m) }
 func (*LogControl) ProtoMessage()    {}
 func (*LogControl) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_6ff4efbe085bd72e, []int{36}
+	return fileDescriptor_6d954c03a4758710, []int{36}
 }
 func (m *LogControl) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_LogControl.Unmarshal(m, b)
@@ -4613,8 +4716,8 @@ func (m *LogControl) XXX_Unmarshal(b []byte) error {
 func (m *LogControl) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_LogControl.Marshal(b, m, deterministic)
 }
-func (dst *LogControl) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_LogControl.Merge(dst, src)
+func (m *LogControl) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_LogControl.Merge(m, src)
 }
 func (m *LogControl) XXX_Size() int {
 	return xxx_messageInfo_LogControl.Size(m)
@@ -4642,6 +4745,7 @@ func init() {
 	proto.RegisterType((*BundleSplit)(nil), "org.apache.beam.model.fn_execution.v1.BundleSplit")
 	proto.RegisterType((*BundleSplit_Application)(nil), "org.apache.beam.model.fn_execution.v1.BundleSplit.Application")
 	proto.RegisterMapType((map[string]int64)(nil), "org.apache.beam.model.fn_execution.v1.BundleSplit.Application.OutputWatermarksEntry")
+	proto.RegisterType((*BundleSplit_DelayedApplication)(nil), "org.apache.beam.model.fn_execution.v1.BundleSplit.DelayedApplication")
 	proto.RegisterType((*ProcessBundleRequest)(nil), "org.apache.beam.model.fn_execution.v1.ProcessBundleRequest")
 	proto.RegisterType((*ProcessBundleResponse)(nil), "org.apache.beam.model.fn_execution.v1.ProcessBundleResponse")
 	proto.RegisterType((*ProcessBundleProgressRequest)(nil), "org.apache.beam.model.fn_execution.v1.ProcessBundleProgressRequest")
@@ -5102,207 +5206,209 @@ var _BeamFnLogging_serviceDesc = grpc.ServiceDesc{
 	Metadata: "beam_fn_api.proto",
 }
 
-func init() { proto.RegisterFile("beam_fn_api.proto", fileDescriptor_beam_fn_api_6ff4efbe085bd72e) }
-
-var fileDescriptor_beam_fn_api_6ff4efbe085bd72e = []byte{
-	// 3170 bytes of a gzipped FileDescriptorProto
-	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xcc, 0x5a, 0xcb, 0x6f, 0x23, 0xc7,
-	0xd1, 0xd7, 0x90, 0x14, 0x25, 0x16, 0x29, 0x8a, 0x6a, 0x49, 0xbb, 0xdc, 0xf9, 0xd6, 0x9f, 0xd7,
-	0xb3, 0x31, 0xb0, 0x17, 0x73, 0xe3, 0xdd, 0x8d, 0xbd, 0xbb, 0xb1, 0xd7, 0xa6, 0x28, 0xae, 0x44,
-	0x5b, 0xaf, 0x8c, 0xb8, 0x56, 0x6c, 0x03, 0x1e, 0x8c, 0x38, 0x2d, 0x7a, 0xbc, 0xe4, 0xcc, 0xb8,
-	0xa7, 0xa9, 0x87, 0x61, 0x24, 0x30, 0x02, 0x38, 0x0f, 0x24, 0xf0, 0x2d, 0x31, 0x72, 0x8b, 0x73,
-	0x4b, 0x0e, 0x39, 0xe4, 0xaf, 0xc8, 0x29, 0x08, 0x90, 0x5b, 0x72, 0x34, 0x10, 0x27, 0x41, 0xee,
-	0x01, 0x72, 0x09, 0xfa, 0x31, 0x0f, 0xbe, 0xb4, 0x1c, 0x6a, 0x13, 0xe4, 0x36, 0x5d, 0xdd, 0x55,
-	0xbf, 0xea, 0xea, 0xea, 0xea, 0xaa, 0x9e, 0x86, 0xa5, 0x43, 0x6c, 0x76, 0x8d, 0x23, 0xc7, 0x30,
-	0x3d, 0xbb, 0xe2, 0x11, 0x97, 0xba, 0xe8, 0x79, 0x97, 0xb4, 0x2b, 0xa6, 0x67, 0xb6, 0xde, 0xc7,
-	0x15, 0xd6, 0x5b, 0xe9, 0xba, 0x16, 0xee, 0x54, 0x8e, 0x1c, 0x03, 0x9f, 0xe2, 0x56, 0x8f, 0xda,
-	0xae, 0x53, 0x39, 0x7e, 0x51, 0x5d, 0xe5, 0x9c, 0xa4, 0xe7, 0x38, 0x98, 0x44, 0xdc, 0xea, 0x22,
-	0x76, 0x2c, 0xcf, 0xb5, 0x1d, 0xea, 0x4b, 0xc2, 0xb3, 0x6d, 0xd7, 0x6d, 0x77, 0xf0, 0x4d, 0xde,
-	0x3a, 0xec, 0x1d, 0xdd, 0xa4, 0x76, 0x17, 0xfb, 0xd4, 0xec, 0x7a, 0x72, 0xc0, 0xff, 0x0f, 0x0e,
-	0x38, 0x21, 0xa6, 0xe7, 0x61, 0x22, 0x05, 0x68, 0xbf, 0x51, 0x20, 0xdb, 0x34, 0x49, 0x1b, 0x53,
-	0xb4, 0x06, 0xcf, 0x78, 0xc4, 0xee, 0xda, 0xd4, 0x3e, 0xc6, 0x06, 0x25, 0xa6, 0xe3, 0x1f, 0xb9,
-	0xa4, 0x6b, 0x10, 0x7c, 0x84, 0x09, 0x76, 0x5a, 0xb8, 0xac, 0x5c, 0x53, 0x6e, 0xe4, 0xf4, 0xff,
-	0x0b, 0x07, 0x35, 0x83, 0x31, 0x7a, 0x30, 0x04, 0x21, 0xc8, 0x38, 0x66, 0x17, 0x97, 0x53, 0x7c,
-	0x28, 0xff, 0x56, 0xb7, 0x21, 0xb3, 0x65, 0xfb, 0x14, 0xd5, 0x21, 0x4b, 0x39, 0x52, 0x59, 0xb9,
-	0x96, 0xbe, 0x91, 0xbf, 0xf5, 0x42, 0x65, 0x22, 0x5b, 0x54, 0x84, 0x7a, 0xba, 0x64, 0xd6, 0x7e,
-	0xae, 0x40, 0x51, 0xc7, 0x5d, 0x97, 0xe2, 0x0d, 0xe2, 0xb5, 0xf6, 0x5c, 0x42, 0x51, 0x17, 0x2e,
-	0x99, 0x9e, 0x6d, 0xf8, 0x98, 0x1c, 0xdb, 0x2d, 0x6c, 0x58, 0xd8, 0x6f, 0x11, 0xdb, 0xa3, 0x2e,
-	0xe1, 0x2a, 0xe7, 0x6f, 0xbd, 0x3c, 0x06, 0xc9, 0xb3, 0x3d, 0xdc, 0xb1, 0x1d, 0xcc, 0x50, 0xaa,
-	0x9e, 0xbd, 0x2f, 0xf8, 0xd7, 0x43, 0x76, 0x7d, 0xc5, 0x1c, 0x41, 0x45, 0x57, 0x60, 0xbe, 0xe5,
-	0x5a, 0x98, 0x18, 0xb6, 0x25, 0x27, 0x3a, 0xc7, 0xdb, 0x0d, 0x4b, 0xfb, 0x57, 0x1a, 0x50, 0xc3,
-	0xf1, 0x29, 0xe9, 0xb5, 0x98, 0xfa, 0x3a, 0xfe, 0xb0, 0x87, 0x7d, 0x8a, 0x9e, 0x87, 0xa2, 0x1d,
-	0x51, 0x19, 0x9f, 0xb0, 0xe5, 0x42, 0x8c, 0xda, 0xb0, 0xd0, 0x23, 0x98, 0x27, 0xb8, 0x6d, 0xfb,
-	0x14, 0x93, 0xf2, 0x57, 0x73, 0x5c, 0xf5, 0x97, 0x26, 0x34, 0x92, 0x2e, 0xf9, 0x24, 0xe2, 0xe6,
-	0x8c, 0x1e, 0x8a, 0x42, 0x18, 0x8a, 0x1e, 0x71, 0x5b, 0xd8, 0xf7, 0x8d, 0xc3, 0x9e, 0x63, 0x75,
-	0x70, 0xf9, 0xaf, 0x42, 0xf8, 0x37, 0x27, 0x14, 0xbe, 0x27, 0xb8, 0xd7, 0x38, 0x73, 0x84, 0xb0,
-	0xe0, 0xc5, 0xe9, 0xe8, 0x3b, 0x70, 0xb9, 0x1f, 0xc6, 0xf0, 0x88, 0xdb, 0x26, 0xd8, 0xf7, 0xcb,
-	0x7f, 0x13, 0x78, 0xb5, 0x69, 0xf0, 0xf6, 0xa4, 0x90, 0x08, 0x77, 0xd5, 0x1b, 0xd5, 0x8f, 0x7a,
-	0xb0, 0x32, 0x80, 0xef, 0x7b, 0x1d, 0x9b, 0x96, 0xff, 0x2e, 0xc0, 0x5f, 0x9f, 0x06, 0x7c, 0x9f,
-	0x49, 0x88, 0x90, 0x91, 0x37, 0xd4, 0xb9, 0x96, 0x83, 0x39, 0x22, 0x06, 0x68, 0x3f, 0xcb, 0xc0,
-	0x72, 0xdf, 0xea, 0xfb, 0x9e, 0xeb, 0xf8, 0x78, 0xd2, 0xe5, 0x5f, 0x81, 0x59, 0x4c, 0x88, 0x4b,
-	0xa4, 0x53, 0x89, 0x06, 0x7a, 0x6b, 0xd8, 0x29, 0x5e, 0x4e, 0xec, 0x14, 0x42, 0x91, 0x3e, 0xaf,
-	0x38, 0x1a, 0xe7, 0x15, 0xaf, 0x4c, 0xe7, 0x15, 0x21, 0xc4, 0x80, 0x5b, 0x7c, 0xf7, 0x89, 0x6e,
-	0xb1, 0x7e, 0x31, 0xb7, 0x08, 0x81, 0xc7, 0xf8, 0xc5, 0xf1, 0xf9, 0x7e, 0x51, 0xbd, 0x80, 0x5f,
-	0x84, 0xd0, 0xa3, 0x1c, 0x03, 0xd8, 0xc2, 0x89, 0x11, 0xda, 0x4f, 0x14, 0x58, 0x1c, 0xd8, 0xa2,
-	0xe8, 0x23, 0xb8, 0x32, 0xa0, 0x57, 0x5f, 0xe0, 0x62, 0x21, 0xf2, 0xc1, 0x34, 0xba, 0xc5, 0xe2,
-	0xd7, 0x65, 0x6f, 0x74, 0x87, 0x86, 0xa0, 0x34, 0xe8, 0x1c, 0xda, 0x2f, 0x01, 0x2e, 0x8f, 0x11,
-	0x84, 0x8a, 0x90, 0x0a, 0xbd, 0x36, 0x65, 0x5b, 0xc8, 0x01, 0x08, 0x4f, 0x08, 0xbf, 0x9c, 0xe2,
-	0xca, 0xee, 0x5c, 0x4c, 0xd9, 0x4a, 0x78, 0x9c, 0xf8, 0x75, 0x87, 0x92, 0x33, 0x3d, 0x86, 0x80,
-	0x28, 0x14, 0xbc, 0x96, 0xdb, 0xe9, 0x60, 0xbe, 0x57, 0xfc, 0x72, 0x9a, 0x23, 0xee, 0x5d, 0x10,
-	0x71, 0x2f, 0x26, 0x52, 0x60, 0xf6, 0xa1, 0xa0, 0x1f, 0x29, 0xb0, 0x72, 0x62, 0x3b, 0x96, 0x7b,
-	0x62, 0x3b, 0x6d, 0xc3, 0xa7, 0xc4, 0xa4, 0xb8, 0x6d, 0x63, 0xbf, 0x9c, 0xe1, 0xf0, 0x07, 0x17,
-	0x84, 0x3f, 0x08, 0x44, 0xef, 0x87, 0x92, 0x85, 0x16, 0xcb, 0x27, 0xc3, 0x3d, 0xe8, 0x10, 0xb2,
-	0xfc, 0x94, 0xf1, 0xcb, 0xb3, 0x1c, 0xfd, 0x8d, 0x0b, 0xa2, 0xd7, 0xb8, 0x30, 0x01, 0x28, 0x25,
-	0x33, 0x33, 0x63, 0xe7, 0xd8, 0x26, 0xae, 0xd3, 0xc5, 0x0e, 0xf5, 0xcb, 0xd9, 0xa7, 0x62, 0xe6,
-	0x7a, 0x4c, 0xa4, 0x34, 0x73, 0x1c, 0x05, 0x9d, 0xc2, 0x55, 0x9f, 0x9a, 0x14, 0x1b, 0x63, 0x0e,
-	0xf1, 0xb9, 0x8b, 0x1d, 0xe2, 0x57, 0xb8, 0xf0, 0x51, 0x5d, 0x6a, 0x07, 0x16, 0x07, 0xbc, 0x0e,
-	0x95, 0x20, 0xfd, 0x18, 0x9f, 0x49, 0x57, 0x67, 0x9f, 0xa8, 0x06, 0xb3, 0xc7, 0x66, 0xa7, 0x27,
-	0x92, 0x9a, 0xf1, 0x69, 0x4b, 0x5c, 0x8f, 0xbd, 0x28, 0x35, 0x12, 0xbc, 0xf7, 0x53, 0x77, 0x15,
-	0xd5, 0x85, 0xa5, 0x21, 0x8f, 0x1b, 0x81, 0xb7, 0xde, 0x8f, 0x57, 0x99, 0x04, 0xaf, 0x16, 0x8a,
-	0x8d, 0x03, 0x7e, 0x0c, 0xe5, 0x71, 0x3e, 0x36, 0x02, 0xf7, 0x8d, 0x7e, 0xdc, 0x3b, 0x13, 0xe0,
-	0x0e, 0x4a, 0x3f, 0x8b, 0xa3, 0xb7, 0x20, 0x1f, 0xf3, 0xb1, 0x11, 0x80, 0x0f, 0xfa, 0x01, 0x6f,
-	0x4c, 0x00, 0xc8, 0x05, 0x0e, 0xd8, 0x74, 0xc8, 0xbd, 0x9e, 0x8e, 0x4d, 0x63, 0x62, 0x63, 0x80,
-	0xda, 0x1f, 0x33, 0x90, 0x8f, 0x45, 0x79, 0xd4, 0x82, 0x05, 0x96, 0x10, 0x9b, 0xe4, 0xcc, 0x20,
-	0xae, 0x4b, 0xfd, 0x84, 0x91, 0x3b, 0x26, 0xaa, 0x52, 0xf5, 0xbc, 0x8e, 0xdd, 0x32, 0xf9, 0x2a,
-	0x16, 0xa4, 0x50, 0x9d, 0xc9, 0x64, 0x19, 0x1c, 0xc1, 0xbe, 0x6d, 0xf5, 0xcc, 0x8e, 0x44, 0x49,
-	0x3d, 0x15, 0x94, 0x85, 0x40, 0x2a, 0x87, 0x51, 0xff, 0x99, 0x82, 0x7c, 0xac, 0x1b, 0x5d, 0x87,
-	0x05, 0x2f, 0x2a, 0x04, 0xc2, 0x03, 0xa0, 0x10, 0x11, 0x1b, 0x16, 0xcb, 0x86, 0x6d, 0xc7, 0xeb,
-	0xd1, 0x58, 0x36, 0xcc, 0xdb, 0x0d, 0x0b, 0x95, 0x61, 0x0e, 0x77, 0x30, 0xb3, 0x60, 0x39, 0x7d,
-	0x4d, 0xb9, 0x51, 0xd0, 0x83, 0x26, 0xfa, 0x44, 0x81, 0x25, 0xb7, 0x47, 0x19, 0xdb, 0x89, 0x49,
-	0x31, 0xe9, 0x9a, 0xe4, 0x71, 0x10, 0x56, 0x9b, 0x17, 0x9b, 0x54, 0x65, 0x97, 0xcb, 0x3d, 0x08,
-	0xc5, 0x8a, 0x90, 0x53, 0x72, 0x07, 0xc8, 0xe8, 0x21, 0x94, 0x8e, 0x88, 0x29, 0x52, 0x32, 0xf7,
-	0xc8, 0x38, 0x71, 0xc9, 0xe3, 0xf2, 0x2c, 0x77, 0x8f, 0xab, 0x15, 0x51, 0x35, 0x55, 0x82, 0xaa,
-	0xa9, 0xb2, 0xee, 0xf6, 0x0e, 0x3b, 0xf8, 0x2d, 0xe6, 0x05, 0x7a, 0x31, 0xe0, 0xda, 0x3d, 0x3a,
-	0x70, 0xc9, 0x63, 0xb5, 0x06, 0xab, 0x23, 0x21, 0x47, 0xb8, 0xe1, 0x4a, 0xdc, 0x0d, 0xd3, 0x71,
-	0xb7, 0xfa, 0x81, 0x02, 0x2b, 0xa3, 0xd2, 0x6c, 0xb4, 0x0d, 0xd7, 0xc7, 0x66, 0x09, 0x43, 0xb5,
-	0xd9, 0xb5, 0x31, 0xe7, 0x7d, 0x54, 0xa0, 0x3d, 0x07, 0x85, 0x16, 0x33, 0xac, 0x41, 0xdd, 0xc7,
-	0xd8, 0x11, 0x7e, 0x54, 0xd0, 0xf3, 0x9c, 0xd6, 0xe4, 0x24, 0xed, 0xd7, 0x0a, 0xac, 0x8e, 0xcc,
-	0xed, 0xd0, 0x26, 0xcc, 0x75, 0x31, 0x25, 0x76, 0xcb, 0x97, 0x85, 0x55, 0x65, 0xc2, 0xa5, 0xda,
-	0x16, 0x5c, 0x7a, 0xc0, 0x8e, 0x36, 0x61, 0x56, 0x24, 0x61, 0x62, 0x3f, 0xde, 0x4a, 0xbe, 0xe4,
-	0xba, 0x10, 0xa0, 0xed, 0xc3, 0xd5, 0xf3, 0xca, 0x05, 0x74, 0x1b, 0x56, 0xe3, 0xb9, 0xf7, 0xa0,
-	0xc5, 0x56, 0xec, 0x78, 0xbe, 0x2e, 0xfb, 0x58, 0x19, 0x57, 0xdc, 0x76, 0x1d, 0x9b, 0xba, 0xc4,
-	0x76, 0xda, 0x0d, 0xe7, 0xc8, 0x65, 0x8b, 0xd9, 0x23, 0x4e, 0xb0, 0x98, 0x3d, 0xe2, 0xb0, 0x5a,
-	0x97, 0x9e, 0x79, 0x61, 0xad, 0xcb, 0xbe, 0x91, 0x07, 0xab, 0xdd, 0x90, 0xcf, 0xa0, 0xe6, 0x21,
-	0x5b, 0x2f, 0x93, 0x9a, 0xdc, 0xff, 0xf3, 0xb7, 0xee, 0x4f, 0x6a, 0xaf, 0x50, 0x46, 0x93, 0x89,
-	0x58, 0x37, 0xa9, 0xb9, 0x39, 0xa3, 0x2f, 0x77, 0x87, 0xc9, 0x68, 0x03, 0xb2, 0xc2, 0xa8, 0xe5,
-	0xcc, 0xb9, 0xc7, 0xd3, 0xe8, 0x25, 0xd9, 0x9c, 0xd1, 0x25, 0x3b, 0x7a, 0x1b, 0xb2, 0x1d, 0xf3,
-	0x10, 0x77, 0x82, 0xfc, 0xa2, 0x9a, 0x58, 0x57, 0x66, 0xa7, 0xca, 0x16, 0x97, 0x21, 0xd3, 0x0a,
-	0x21, 0x50, 0xbd, 0x07, 0xf9, 0x18, 0xf9, 0x49, 0xfb, 0x22, 0x17, 0xdf, 0x17, 0x1f, 0xc0, 0x4a,
-	0x3f, 0x80, 0x10, 0x84, 0x16, 0x20, 0xd7, 0xd4, 0xab, 0x3b, 0xfb, 0x0f, 0x77, 0xf5, 0xed, 0xd2,
-	0x0c, 0x5a, 0x84, 0xfc, 0x5e, 0x6d, 0x77, 0x6b, 0xab, 0x5e, 0x6b, 0x36, 0x76, 0x77, 0x4a, 0x0a,
-	0xba, 0x04, 0xe8, 0xa0, 0xb1, 0xb3, 0xbe, 0x7b, 0xd0, 0xd8, 0xd9, 0x30, 0xf6, 0x9b, 0x7a, 0xb5,
-	0x59, 0xdf, 0x78, 0xbb, 0x94, 0x42, 0x39, 0x98, 0xad, 0xed, 0xae, 0xd7, 0xf5, 0x52, 0x9a, 0xf1,
-	0xd4, 0x77, 0xde, 0x6a, 0xe8, 0xbb, 0x3b, 0xdb, 0xf5, 0x9d, 0x66, 0x29, 0xb3, 0x96, 0x85, 0x0c,
-	0x5b, 0x2b, 0xed, 0x57, 0x29, 0xc8, 0x0a, 0xf3, 0xa0, 0x03, 0x28, 0xb4, 0xdc, 0x9e, 0x43, 0x31,
-	0x11, 0xcb, 0xa8, 0x24, 0x72, 0xd7, 0x9a, 0x60, 0x95, 0xcb, 0x97, 0x6f, 0x45, 0x4d, 0x74, 0x04,
-	0x4b, 0x96, 0xed, 0x53, 0x62, 0x1f, 0xf2, 0xd1, 0x42, 0x7a, 0x2a, 0x51, 0x75, 0xb7, 0x1e, 0xe3,
-	0x97, 0x10, 0x25, 0x6b, 0x80, 0xc6, 0x26, 0x80, 0x4f, 0x29, 0xc1, 0x5d, 0x33, 0xee, 0x87, 0x93,
-	0x4e, 0xa0, 0x2e, 0x58, 0x83, 0x09, 0xe0, 0xa8, 0x19, 0x1a, 0xeb, 0x7b, 0x0a, 0x3b, 0xe6, 0xa3,
-	0x89, 0x3d, 0x07, 0x79, 0xdb, 0xa1, 0x2f, 0xdd, 0x31, 0xc4, 0x82, 0x32, 0x83, 0xa5, 0x37, 0x67,
-	0x74, 0xe0, 0x44, 0x1e, 0x40, 0xd1, 0x75, 0x28, 0x58, 0x3c, 0x9e, 0x1a, 0xd1, 0xa2, 0x2b, 0x4c,
-	0xbe, 0x15, 0x45, 0x59, 0x36, 0x88, 0x4d, 0xc5, 0x69, 0xcb, 0x41, 0x4c, 0xf1, 0x1c, 0x1b, 0x24,
-	0xa8, 0x7c, 0xd0, 0xda, 0x9c, 0xf4, 0x1b, 0xed, 0x2f, 0x0a, 0xe4, 0x63, 0xca, 0x22, 0x13, 0x4a,
-	0xb6, 0x43, 0x8d, 0xbe, 0xa9, 0x8b, 0xb5, 0xfb, 0xc6, 0x84, 0x53, 0x6f, 0x38, 0xb4, 0x7f, 0xf6,
-	0x45, 0xbb, 0x8f, 0x82, 0x3e, 0x80, 0x65, 0x39, 0x8b, 0x3e, 0x14, 0xb1, 0x86, 0x77, 0x27, 0x5d,
-	0x43, 0x2e, 0xa1, 0x1f, 0x68, 0xc9, 0x1a, 0x24, 0xae, 0xe5, 0x60, 0x4e, 0x82, 0x68, 0x37, 0xa1,
-	0xd8, 0xaf, 0x1a, 0x7a, 0x06, 0x98, 0x71, 0x85, 0x99, 0x44, 0xfa, 0x91, 0xd6, 0x73, 0xb6, 0x43,
-	0xb9, 0x89, 0x7c, 0xed, 0x2e, 0x2c, 0x0d, 0xa1, 0xb0, 0x93, 0x3d, 0xbe, 0x04, 0xe2, 0x1c, 0x50,
-	0xf4, 0x42, 0x6c, 0x05, 0x7c, 0xed, 0xb3, 0x14, 0x94, 0x06, 0x9d, 0x8c, 0x45, 0x38, 0x86, 0x36,
-	0xec, 0xbc, 0x4a, 0xa2, 0x08, 0xd7, 0x70, 0xe8, 0x08, 0xff, 0x5d, 0xb6, 0x87, 0xc9, 0xe8, 0x0c,
-	0xca, 0x52, 0xd7, 0x71, 0x3b, 0xe6, 0xd5, 0x44, 0xd6, 0x1e, 0x81, 0x7b, 0xc9, 0x1a, 0xd9, 0xb3,
-	0x56, 0x84, 0x42, 0x1c, 0x53, 0x33, 0x61, 0x79, 0x84, 0xe2, 0x2c, 0x7c, 0xf1, 0xbd, 0x2d, 0xbc,
-	0x5d, 0x17, 0x0d, 0x16, 0xe6, 0xfc, 0x5e, 0x57, 0x1e, 0xf5, 0xec, 0x93, 0x51, 0xba, 0xb6, 0xc3,
-	0x5d, 0x39, 0xad, 0xb3, 0x4f, 0x4e, 0x31, 0x4f, 0x79, 0xe8, 0x66, 0x14, 0xf3, 0x54, 0xb3, 0xe0,
-	0xd2, 0x68, 0x35, 0x9f, 0x8c, 0xa2, 0x0c, 0xa1, 0x28, 0x43, 0x28, 0x8a, 0x40, 0xf9, 0x32, 0x0d,
-	0xcb, 0x23, 0x0e, 0x19, 0x9e, 0x1e, 0xb8, 0x9d, 0x5e, 0xd7, 0x31, 0x1c, 0xb3, 0x2b, 0xbd, 0x29,
-	0xc7, 0x22, 0x17, 0xa3, 0xed, 0x30, 0x12, 0x6a, 0xc1, 0x3c, 0x71, 0x4f, 0x02, 0xf3, 0xb3, 0x93,
-	0x62, 0x73, 0xfa, 0x53, 0x2d, 0x46, 0xd3, 0xdd, 0x13, 0x7d, 0x8e, 0xb8, 0x27, 0x8c, 0xac, 0xfe,
-	0x4e, 0x81, 0xd5, 0xa8, 0xab, 0xc6, 0xe1, 0x45, 0x5c, 0xf8, 0x6f, 0xc6, 0x17, 0x74, 0x1f, 0x72,
-	0xe1, 0x85, 0xba, 0x3c, 0x5f, 0xd5, 0xa1, 0xdc, 0xb0, 0x19, 0x8c, 0xd8, 0x9c, 0xd1, 0xa3, 0xe1,
-	0x61, 0x6c, 0x52, 0xcf, 0x60, 0xa1, 0x6f, 0x96, 0xe8, 0x7d, 0xc8, 0xc6, 0x36, 0xeb, 0xe4, 0xf5,
-	0xf5, 0xf9, 0xf6, 0x8b, 0x19, 0x49, 0x97, 0xf2, 0xb5, 0xcf, 0x97, 0x61, 0x4e, 0xe6, 0x5e, 0xc8,
-	0x84, 0xbc, 0x17, 0xbb, 0xb3, 0x11, 0xd0, 0xaf, 0x25, 0x4b, 0xe0, 0x2a, 0x7b, 0x74, 0xe0, 0x92,
-	0x26, 0x2e, 0x53, 0xfd, 0x73, 0x1e, 0x20, 0x2a, 0x7d, 0xd1, 0x47, 0x10, 0x5c, 0x8b, 0x61, 0xcb,
-	0x90, 0x99, 0x7f, 0x90, 0x39, 0xbe, 0x99, 0x14, 0x38, 0x14, 0x1b, 0x5c, 0x33, 0x60, 0xab, 0x2e,
-	0x45, 0xea, 0x4b, 0xde, 0x20, 0x09, 0x7d, 0x08, 0x8b, 0x2c, 0x49, 0x3f, 0xc6, 0x11, 0xb0, 0x88,
-	0x15, 0x9b, 0xd3, 0x03, 0x57, 0xb9, 0xc0, 0x10, 0xb5, 0x68, 0xf6, 0xb5, 0x91, 0x0d, 0x10, 0xab,
-	0x65, 0xc4, 0x0d, 0x55, 0x63, 0x7a, 0xb4, 0xc1, 0x02, 0x26, 0x26, 0x1c, 0x6d, 0x40, 0xa6, 0xe7,
-	0x63, 0x22, 0x0b, 0xa6, 0xdb, 0x09, 0x41, 0x1e, 0xf9, 0x98, 0xe8, 0x5c, 0x80, 0xfa, 0x65, 0x1a,
-	0xe6, 0xb7, 0xb1, 0xe9, 0xf7, 0x08, 0xb6, 0xd0, 0x8f, 0x15, 0x58, 0x11, 0xa5, 0x9c, 0xb4, 0x99,
-	0xc1, 0x43, 0x4c, 0xe0, 0x2b, 0xef, 0x4c, 0x3f, 0x97, 0x00, 0xa2, 0xd2, 0x60, 0xe2, 0xa5, 0xc5,
-	0x78, 0xd6, 0x20, 0x27, 0x87, 0xec, 0xa1, 0x0e, 0xf4, 0x99, 0x02, 0xab, 0xb2, 0x46, 0x1c, 0xd0,
-	0x47, 0x84, 0x9d, 0x77, 0x9f, 0x82, 0x3e, 0xa2, 0x70, 0x1b, 0xa1, 0xd0, 0xb2, 0x3b, 0xdc, 0x83,
-	0x6e, 0x40, 0x89, 0xba, 0xd4, 0xec, 0x18, 0x6c, 0x97, 0x1b, 0xbe, 0x17, 0x14, 0xb6, 0x8a, 0x5e,
-	0xe4, 0x74, 0x16, 0x09, 0xf6, 0x19, 0x55, 0xad, 0xc3, 0xe5, 0x31, 0x53, 0x4d, 0x52, 0x15, 0xaa,
-	0x0f, 0xa1, 0x3c, 0x4e, 0xc3, 0x44, 0x72, 0x7c, 0x58, 0x1a, 0xda, 0x35, 0xe8, 0x3d, 0x98, 0xef,
-	0x4a, 0x3b, 0xc8, 0x4d, 0xb9, 0x76, 0x71, 0x8b, 0xea, 0xa1, 0x4c, 0xf5, 0xb3, 0x34, 0x14, 0xfb,
-	0xb7, 0xcc, 0x7f, 0x1a, 0x12, 0xbd, 0x00, 0x28, 0x2c, 0xe9, 0x59, 0xae, 0x63, 0x3b, 0xb6, 0xd3,
-	0x96, 0x67, 0xe3, 0x52, 0xd0, 0xa3, 0x07, 0x1d, 0xe8, 0x17, 0x0a, 0x5c, 0xe9, 0xf7, 0x30, 0x3f,
-	0xc6, 0x26, 0x76, 0x30, 0x7e, 0x5a, 0xf1, 0xa2, 0xdf, 0xd7, 0xfc, 0x50, 0x0b, 0xe1, 0x6f, 0x97,
-	0xdd, 0xd1, 0xbd, 0xea, 0x1b, 0x70, 0xf5, 0x3c, 0xc6, 0x44, 0x6e, 0xf0, 0x2a, 0x2c, 0x5e, 0xe0,
-	0x8e, 0x42, 0xfd, 0xfd, 0x2c, 0x64, 0x58, 0xec, 0x40, 0x06, 0xe4, 0x45, 0xd1, 0xc8, 0xb3, 0x04,
-	0xb9, 0x92, 0x0f, 0xa6, 0x88, 0x42, 0xb2, 0xc1, 0x12, 0x0b, 0x1d, 0xba, 0xe1, 0x37, 0xc2, 0x03,
-	0x65, 0x97, 0xfc, 0x33, 0xf5, 0xda, 0x34, 0x10, 0xe7, 0x14, 0x61, 0x74, 0x54, 0x11, 0x96, 0xf0,
-	0xa7, 0x54, 0x1f, 0xd6, 0x44, 0x25, 0x99, 0x01, 0xd0, 0x36, 0x7b, 0x6d, 0x79, 0x31, 0x20, 0xff,
-	0x42, 0xbd, 0x3a, 0x0d, 0xdc, 0x06, 0x13, 0x23, 0x71, 0x72, 0xed, 0xa0, 0xa1, 0x3e, 0x00, 0x88,
-	0xec, 0x8a, 0xae, 0x42, 0x8e, 0xe7, 0x72, 0x9e, 0xd9, 0x0a, 0xea, 0xeb, 0x88, 0x10, 0xfe, 0xb0,
-	0x4f, 0xc7, 0x7e, 0xd8, 0x5f, 0xef, 0xaf, 0xe8, 0x42, 0x87, 0x50, 0x62, 0x0e, 0xa1, 0xbe, 0x37,
-	0xa2, 0x36, 0x78, 0x8a, 0x79, 0xb0, 0xfa, 0x2e, 0xe4, 0xc2, 0xe9, 0x8d, 0x56, 0x01, 0xdd, 0x8d,
-	0x67, 0x67, 0xa9, 0x27, 0x65, 0x67, 0xf1, 0xdc, 0x4c, 0x16, 0xaf, 0xea, 0x19, 0x94, 0x06, 0x33,
-	0x9a, 0x11, 0x3b, 0x62, 0xb7, 0xff, 0xf2, 0xf8, 0xde, 0xd4, 0x11, 0x21, 0x7e, 0xb1, 0xf1, 0x5b,
-	0x05, 0x9e, 0x39, 0xf7, 0x87, 0xe6, 0xff, 0xe4, 0x6d, 0xdb, 0x17, 0x0a, 0x5c, 0x19, 0xfb, 0x83,
-	0x7c, 0xaa, 0xbb, 0x36, 0xb4, 0x07, 0xab, 0xf1, 0x6b, 0x58, 0x11, 0x7f, 0x2d, 0x4c, 0xa4, 0xb2,
-	0xe7, 0xdf, 0xc5, 0x2e, 0x47, 0x77, 0xb1, 0x7a, 0xc0, 0xa8, 0x5d, 0x05, 0x75, 0xfc, 0xcf, 0x5a,
-	0xed, 0x1f, 0x0a, 0xcc, 0x87, 0x07, 0xd2, 0xa6, 0x70, 0x04, 0x99, 0xe1, 0xdc, 0x99, 0xf4, 0x5a,
-	0x24, 0x08, 0xe9, 0xcc, 0x39, 0x75, 0xe1, 0x4a, 0x3f, 0x55, 0x20, 0xc3, 0x7d, 0x75, 0x2a, 0x23,
-	0x44, 0x6f, 0x63, 0x52, 0x89, 0x6e, 0xf1, 0xfa, 0xdf, 0xc6, 0xb0, 0xdd, 0x1c, 0xde, 0xf2, 0x14,
-	0x84, 0x62, 0xda, 0x17, 0x69, 0x28, 0xec, 0x53, 0x93, 0x86, 0x37, 0xca, 0x83, 0xff, 0x72, 0xc7,
-	0x2a, 0x9c, 0x3a, 0x47, 0xe1, 0x2d, 0xc8, 0x89, 0x7f, 0x76, 0x6c, 0xaf, 0x88, 0x4b, 0xa5, 0x9b,
-	0x13, 0xea, 0xcc, 0x95, 0x79, 0x13, 0x9f, 0xe9, 0xf3, 0xbe, 0xfc, 0x42, 0x6f, 0x42, 0x9a, 0xcd,
-	0x3d, 0xe1, 0x9b, 0x17, 0x2e, 0x68, 0x03, 0xc7, 0xde, 0x67, 0x30, 0x29, 0xa8, 0x09, 0x59, 0xd3,
-	0xf3, 0xb0, 0x63, 0x05, 0xc7, 0xc6, 0xbd, 0x24, 0xf2, 0xaa, 0x9c, 0x35, 0x12, 0x29, 0x65, 0xa1,
-	0x6f, 0xc1, 0x6c, 0xab, 0x83, 0x4d, 0x12, 0x9c, 0x0f, 0x77, 0x93, 0x08, 0xad, 0x31, 0xce, 0x48,
-	0xa6, 0x90, 0x14, 0x7f, 0x39, 0xf2, 0xa7, 0x14, 0x2c, 0xc8, 0x45, 0x92, 0xbb, 0x7f, 0x70, 0x95,
-	0x46, 0x3f, 0x0e, 0x79, 0x16, 0xf2, 0xb1, 0xeb, 0x7c, 0xb9, 0xee, 0x10, 0xdd, 0xe6, 0xa3, 0xad,
-	0x3e, 0xcb, 0xbe, 0x9c, 0xd8, 0xb2, 0xe1, 0x0b, 0x07, 0x6e, 0xda, 0x47, 0x83, 0xa6, 0xbd, 0x3f,
-	0x8d, 0x69, 0x43, 0x99, 0x81, 0x6d, 0xf5, 0x01, 0xdb, 0xde, 0x9b, 0xc2, 0xb6, 0xa1, 0x50, 0x69,
-	0xdc, 0xf8, 0xeb, 0x8b, 0xaf, 0x32, 0x30, 0x1f, 0x78, 0x1d, 0xda, 0x83, 0xac, 0x78, 0x57, 0x27,
-	0xa3, 0xea, 0x4b, 0x09, 0xdd, 0xb6, 0xa2, 0x73, 0x6e, 0xa6, 0xbe, 0x90, 0x83, 0x7c, 0x58, 0xee,
-	0xf6, 0x3a, 0xd4, 0xee, 0x9a, 0x9e, 0xe1, 0xdb, 0x16, 0x36, 0x78, 0x31, 0x23, 0x77, 0x72, 0x35,
-	0xa9, 0xf8, 0x6d, 0x29, 0x6a, 0xdf, 0xb6, 0x30, 0xaf, 0x21, 0x36, 0x67, 0xf4, 0xa5, 0xee, 0x20,
-	0x11, 0x59, 0x50, 0x3c, 0x34, 0xdb, 0x06, 0xab, 0xe2, 0x0c, 0xbe, 0x8f, 0xe4, 0x2e, 0x7c, 0x25,
-	0x29, 0xde, 0x9a, 0xd9, 0x66, 0xc9, 0x04, 0x6f, 0x6f, 0xce, 0xe8, 0x85, 0xc3, 0x58, 0x5b, 0x55,
-	0x21, 0x2b, 0xa6, 0x1b, 0x3f, 0x16, 0x0b, 0xfc, 0x58, 0x54, 0x3f, 0x55, 0x60, 0x69, 0x48, 0xd9,
-	0xc9, 0xfe, 0x19, 0x6a, 0xb0, 0x10, 0x19, 0x2a, 0xfa, 0x71, 0x98, 0xf7, 0x03, 0x31, 0x0d, 0x0b,
-	0x5d, 0x82, 0xac, 0x78, 0x06, 0x21, 0xbd, 0x5a, 0xb6, 0x02, 0x45, 0x32, 0x91, 0x22, 0x9f, 0x28,
-	0x50, 0x88, 0xcf, 0x62, 0x62, 0x1d, 0x22, 0xe3, 0xc5, 0x74, 0xe8, 0x05, 0x62, 0x92, 0xe8, 0xc0,
-	0x32, 0x0a, 0x7a, 0xe6, 0x61, 0xed, 0x75, 0x58, 0x1c, 0x08, 0x4b, 0xac, 0x28, 0x69, 0xb9, 0x0e,
-	0xb5, 0x9d, 0x1e, 0xff, 0x3f, 0x29, 0xb7, 0xaa, 0x30, 0xe4, 0x52, 0xbc, 0x87, 0xef, 0x58, 0xed,
-	0x11, 0x94, 0x06, 0xb7, 0x5f, 0x42, 0x11, 0xe1, 0x31, 0x90, 0x8a, 0x1d, 0x03, 0x37, 0x00, 0x0d,
-	0xc7, 0xb7, 0x70, 0xa4, 0x12, 0x1b, 0xb9, 0x0a, 0xcb, 0x23, 0xb6, 0xab, 0xb6, 0x0c, 0x4b, 0x43,
-	0xb1, 0x4c, 0x5b, 0x91, 0x52, 0xfb, 0x36, 0xa1, 0xf6, 0x87, 0x0c, 0xcc, 0x6f, 0xb9, 0xb2, 0x40,
-	0x79, 0x1b, 0xe6, 0x7d, 0x7c, 0x8c, 0x89, 0x4d, 0x85, 0xf7, 0x14, 0x27, 0xce, 0x75, 0x03, 0x11,
-	0x95, 0x7d, 0xc9, 0x5f, 0xa9, 0x3b, 0xbd, 0xae, 0x1e, 0x8a, 0x9b, 0x3e, 0x01, 0x44, 0x65, 0x96,
-	0x5b, 0xf9, 0xbe, 0xd9, 0x0e, 0x32, 0xdf, 0xa0, 0xc9, 0xe2, 0x2c, 0x25, 0x2c, 0x55, 0xce, 0x88,
-	0x38, 0xcb, 0x1b, 0xe3, 0xcf, 0xc8, 0xd9, 0x73, 0xce, 0xc8, 0x27, 0x3e, 0xa8, 0xcd, 0x3e, 0xf9,
-	0x41, 0xed, 0x73, 0x50, 0xe8, 0xb8, 0x6d, 0xa3, 0xe3, 0x8a, 0x9f, 0xdb, 0xfc, 0x2d, 0x4c, 0x4e,
-	0xcf, 0x77, 0xdc, 0xf6, 0x96, 0x24, 0x31, 0x27, 0xa5, 0xef, 0x13, 0x6c, 0x5a, 0xe5, 0x79, 0xde,
-	0x29, 0x5b, 0xea, 0xb7, 0xe5, 0xbb, 0xdb, 0x3d, 0x60, 0xc3, 0x0d, 0xec, 0x50, 0x62, 0x87, 0x77,
-	0x8e, 0x37, 0x13, 0xae, 0x81, 0x0e, 0x1d, 0xf1, 0x65, 0x63, 0x5f, 0x25, 0x30, 0x1f, 0x2c, 0x89,
-	0x76, 0x04, 0x19, 0xb6, 0x2a, 0x68, 0x11, 0xf2, 0x8f, 0x76, 0xf6, 0xf7, 0xea, 0xb5, 0xc6, 0xc3,
-	0x46, 0x7d, 0xbd, 0x34, 0x83, 0x72, 0x30, 0xdb, 0xd4, 0xab, 0xb5, 0x7a, 0x49, 0x61, 0x9f, 0xeb,
-	0xf5, 0xb5, 0x47, 0x1b, 0xa5, 0x14, 0x9a, 0x87, 0x4c, 0x63, 0xe7, 0xe1, 0x6e, 0x29, 0x8d, 0x00,
-	0xb2, 0x3b, 0xbb, 0xcd, 0x46, 0xad, 0x5e, 0xca, 0x30, 0xea, 0x41, 0x55, 0xdf, 0x29, 0xcd, 0xb2,
-	0xa1, 0x75, 0x5d, 0xdf, 0xd5, 0x4b, 0x59, 0x54, 0x80, 0xf9, 0x9a, 0xde, 0x68, 0x36, 0x6a, 0xd5,
-	0xad, 0xd2, 0x9c, 0x56, 0x00, 0xd8, 0x72, 0xdb, 0x35, 0xd7, 0xa1, 0xc4, 0xed, 0xdc, 0xfa, 0x5c,
-	0x81, 0x85, 0x35, 0x6c, 0x76, 0x1f, 0x3a, 0x92, 0x82, 0x3e, 0x55, 0x60, 0x2e, 0xf8, 0x9e, 0xfc,
-	0x27, 0xc4, 0xd0, 0x5b, 0x4d, 0xf5, 0xde, 0x34, 0xbc, 0x62, 0x33, 0xcc, 0xdc, 0x50, 0xbe, 0xae,
-	0xdc, 0xfa, 0x18, 0x40, 0x68, 0xc6, 0xb3, 0x41, 0x47, 0x66, 0x85, 0x37, 0x13, 0xa6, 0x96, 0x6a,
-	0x52, 0x06, 0x89, 0xfe, 0x7d, 0x05, 0xf2, 0x02, 0x5e, 0x84, 0xc2, 0x53, 0x98, 0x15, 0x1f, 0xb7,
-	0x93, 0x9c, 0x0b, 0x72, 0x46, 0xea, 0x9d, 0x64, 0x4c, 0x72, 0xfb, 0x0b, 0x4d, 0x7e, 0x18, 0x2e,
-	0xd1, 0x96, 0xdb, 0x6e, 0xdb, 0x4e, 0x1b, 0x9d, 0xc2, 0x5c, 0xf0, 0x79, 0x27, 0x69, 0x08, 0x60,
-	0x9e, 0xac, 0xbe, 0x38, 0x39, 0x97, 0x74, 0x05, 0xa1, 0xcb, 0x5a, 0x15, 0xbe, 0x36, 0x8e, 0x33,
-	0xce, 0xb8, 0x96, 0x13, 0x0a, 0x57, 0x3d, 0xfb, 0x9d, 0x62, 0xac, 0xcb, 0x38, 0x7e, 0xf1, 0x30,
-	0xcb, 0x03, 0xca, 0xed, 0x7f, 0x07, 0x00, 0x00, 0xff, 0xff, 0x46, 0x47, 0xa1, 0x3b, 0xd4, 0x2f,
-	0x00, 0x00,
+func init() { proto.RegisterFile("beam_fn_api.proto", fileDescriptor_6d954c03a4758710) }
+
+var fileDescriptor_6d954c03a4758710 = []byte{
+	// 3213 bytes of a gzipped FileDescriptorProto
+	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xcc, 0x5a, 0xcd, 0x6f, 0x1b, 0xc7,
+	0x15, 0xd7, 0x92, 0x14, 0x45, 0x3e, 0x52, 0x14, 0x35, 0x92, 0x6c, 0x7a, 0xe3, 0x34, 0xce, 0xba,
+	0x01, 0x7c, 0x09, 0xdd, 0xd8, 0x6e, 0x62, 0xbb, 0x89, 0x13, 0x89, 0xa2, 0x25, 0x26, 0xfa, 0xea,
+	0x8a, 0x8e, 0x9a, 0x04, 0xc8, 0x76, 0xc5, 0x1d, 0x31, 0x1b, 0x93, 0xbb, 0x9b, 0xd9, 0xa5, 0x64,
+	0x05, 0x41, 0x8b, 0xa0, 0x40, 0xfa, 0x81, 0x16, 0x39, 0x14, 0x68, 0x83, 0xde, 0x9a, 0xde, 0xda,
+	0x43, 0x0f, 0xfd, 0x2b, 0x7a, 0x2a, 0x7a, 0x6e, 0x8f, 0x01, 0x9a, 0xb6, 0xe8, 0xa1, 0xb7, 0x02,
+	0xbd, 0x14, 0xf3, 0xb1, 0xbb, 0xc3, 0x2f, 0x99, 0x4b, 0xb9, 0x45, 0x6f, 0x3b, 0x6f, 0xe6, 0xbd,
+	0xdf, 0x9b, 0x37, 0x6f, 0xde, 0xbc, 0x37, 0x3b, 0xb0, 0x78, 0x88, 0xcd, 0xae, 0x71, 0xe4, 0x18,
+	0xa6, 0x67, 0x57, 0x3d, 0xe2, 0x06, 0x2e, 0x7a, 0xce, 0x25, 0xed, 0xaa, 0xe9, 0x99, 0xad, 0xf7,
+	0x70, 0x95, 0xf6, 0x56, 0xbb, 0xae, 0x85, 0x3b, 0xd5, 0x23, 0xc7, 0xc0, 0x8f, 0x70, 0xab, 0x17,
+	0xd8, 0xae, 0x53, 0x3d, 0x7e, 0x41, 0x5d, 0x61, 0x9c, 0xa4, 0xe7, 0x38, 0x98, 0xc4, 0xdc, 0xea,
+	0x02, 0x76, 0x2c, 0xcf, 0xb5, 0x9d, 0xc0, 0x17, 0x84, 0x67, 0xda, 0xae, 0xdb, 0xee, 0xe0, 0xeb,
+	0xac, 0x75, 0xd8, 0x3b, 0xba, 0x1e, 0xd8, 0x5d, 0xec, 0x07, 0x66, 0xd7, 0x13, 0x03, 0xbe, 0x32,
+	0x38, 0xe0, 0x84, 0x98, 0x9e, 0x87, 0x89, 0x10, 0xa0, 0xfd, 0x56, 0x81, 0x6c, 0xd3, 0x24, 0x6d,
+	0x1c, 0xa0, 0x35, 0x78, 0xda, 0x23, 0x76, 0xd7, 0x0e, 0xec, 0x63, 0x6c, 0x04, 0xc4, 0x74, 0xfc,
+	0x23, 0x97, 0x74, 0x0d, 0x82, 0x8f, 0x30, 0xc1, 0x4e, 0x0b, 0x57, 0x94, 0x2b, 0xca, 0xb5, 0xbc,
+	0xfe, 0x54, 0x34, 0xa8, 0x19, 0x8e, 0xd1, 0xc3, 0x21, 0x08, 0x41, 0xc6, 0x31, 0xbb, 0xb8, 0x92,
+	0x62, 0x43, 0xd9, 0xb7, 0xba, 0x0d, 0x99, 0x2d, 0xdb, 0x0f, 0x50, 0x1d, 0xb2, 0x01, 0x43, 0xaa,
+	0x28, 0x57, 0xd2, 0xd7, 0x0a, 0x37, 0x9e, 0xaf, 0x4e, 0x64, 0x8b, 0x2a, 0x57, 0x4f, 0x17, 0xcc,
+	0xda, 0x2f, 0x14, 0x28, 0xe9, 0xb8, 0xeb, 0x06, 0x78, 0x83, 0x78, 0xad, 0x3d, 0x97, 0x04, 0xa8,
+	0x0b, 0x17, 0x4c, 0xcf, 0x36, 0x7c, 0x4c, 0x8e, 0xed, 0x16, 0x36, 0x2c, 0xec, 0xb7, 0x88, 0xed,
+	0x05, 0x2e, 0x61, 0x2a, 0x17, 0x6e, 0xbc, 0x34, 0x06, 0xc9, 0xb3, 0x3d, 0xdc, 0xb1, 0x1d, 0x4c,
+	0x51, 0x56, 0x3d, 0x7b, 0x9f, 0xf3, 0xaf, 0x47, 0xec, 0xfa, 0xb2, 0x39, 0x82, 0x8a, 0x2e, 0x41,
+	0xae, 0xe5, 0x5a, 0x98, 0x18, 0xb6, 0x25, 0x26, 0x3a, 0xc7, 0xda, 0x0d, 0x4b, 0xfb, 0x77, 0x1a,
+	0x50, 0xc3, 0xf1, 0x03, 0xd2, 0x6b, 0x51, 0xf5, 0x75, 0xfc, 0x41, 0x0f, 0xfb, 0x01, 0x7a, 0x0e,
+	0x4a, 0x76, 0x4c, 0xa5, 0x7c, 0xdc, 0x96, 0xf3, 0x12, 0xb5, 0x61, 0xa1, 0x07, 0x90, 0x23, 0xb8,
+	0x6d, 0xfb, 0x01, 0x26, 0x95, 0x2f, 0xe7, 0x98, 0xea, 0x2f, 0x4e, 0x68, 0x24, 0x5d, 0xf0, 0x09,
+	0xc4, 0xcd, 0x19, 0x3d, 0x12, 0x85, 0x30, 0x94, 0x3c, 0xe2, 0xb6, 0xb0, 0xef, 0x1b, 0x87, 0x3d,
+	0xc7, 0xea, 0xe0, 0xca, 0x5f, 0xb9, 0xf0, 0x6f, 0x4c, 0x28, 0x7c, 0x8f, 0x73, 0xaf, 0x31, 0xe6,
+	0x18, 0x61, 0xde, 0x93, 0xe9, 0xe8, 0x3b, 0x70, 0xb1, 0x1f, 0xc6, 0xf0, 0x88, 0xdb, 0x26, 0xd8,
+	0xf7, 0x2b, 0x7f, 0xe3, 0x78, 0xb5, 0x69, 0xf0, 0xf6, 0x84, 0x90, 0x18, 0x77, 0xc5, 0x1b, 0xd5,
+	0x8f, 0x7a, 0xb0, 0x3c, 0x80, 0xef, 0x7b, 0x1d, 0x3b, 0xa8, 0xfc, 0x9d, 0x83, 0xbf, 0x36, 0x0d,
+	0xf8, 0x3e, 0x95, 0x10, 0x23, 0x23, 0x6f, 0xa8, 0x73, 0x2d, 0x0f, 0x73, 0x84, 0x0f, 0xd0, 0x7e,
+	0x9e, 0x81, 0xa5, 0xbe, 0xd5, 0xf7, 0x3d, 0xd7, 0xf1, 0xf1, 0xa4, 0xcb, 0xbf, 0x0c, 0xb3, 0x98,
+	0x10, 0x97, 0x08, 0xa7, 0xe2, 0x0d, 0xf4, 0xe6, 0xb0, 0x53, 0xbc, 0x94, 0xd8, 0x29, 0xb8, 0x22,
+	0x7d, 0x5e, 0x71, 0x34, 0xce, 0x2b, 0x5e, 0x9e, 0xce, 0x2b, 0x22, 0x88, 0x01, 0xb7, 0xf8, 0xee,
+	0x63, 0xdd, 0x62, 0xfd, 0x7c, 0x6e, 0x11, 0x01, 0x8f, 0xf1, 0x8b, 0xe3, 0xb3, 0xfd, 0x62, 0xf5,
+	0x1c, 0x7e, 0x11, 0x41, 0x8f, 0x72, 0x0c, 0xa0, 0x0b, 0xc7, 0x47, 0x68, 0x3f, 0x51, 0x60, 0x61,
+	0x60, 0x8b, 0xa2, 0x0f, 0xe1, 0xd2, 0x80, 0x5e, 0x7d, 0x81, 0x8b, 0x86, 0xc8, 0x7b, 0xd3, 0xe8,
+	0x26, 0xc5, 0xaf, 0x8b, 0xde, 0xe8, 0x0e, 0x0d, 0x41, 0x79, 0xd0, 0x39, 0xb4, 0x5f, 0x01, 0x5c,
+	0x1c, 0x23, 0x08, 0x95, 0x20, 0x15, 0x79, 0x6d, 0xca, 0xb6, 0x90, 0x03, 0x10, 0x9d, 0x10, 0x7e,
+	0x25, 0xc5, 0x94, 0xdd, 0x39, 0x9f, 0xb2, 0xd5, 0xe8, 0x38, 0xf1, 0xeb, 0x4e, 0x40, 0x4e, 0x75,
+	0x09, 0x01, 0x05, 0x50, 0xf4, 0x5a, 0x6e, 0xa7, 0x83, 0xd9, 0x5e, 0xf1, 0x2b, 0x69, 0x86, 0xb8,
+	0x77, 0x4e, 0xc4, 0x3d, 0x49, 0x24, 0xc7, 0xec, 0x43, 0x41, 0x3f, 0x52, 0x60, 0xf9, 0xc4, 0x76,
+	0x2c, 0xf7, 0xc4, 0x76, 0xda, 0x86, 0x1f, 0x10, 0x33, 0xc0, 0x6d, 0x1b, 0xfb, 0x95, 0x0c, 0x83,
+	0x3f, 0x38, 0x27, 0xfc, 0x41, 0x28, 0x7a, 0x3f, 0x92, 0xcc, 0xb5, 0x58, 0x3a, 0x19, 0xee, 0x41,
+	0x87, 0x90, 0x65, 0xa7, 0x8c, 0x5f, 0x99, 0x65, 0xe8, 0xaf, 0x9f, 0x13, 0xbd, 0xc6, 0x84, 0x71,
+	0x40, 0x21, 0x99, 0x9a, 0x19, 0x3b, 0xc7, 0x36, 0x71, 0x9d, 0x2e, 0x76, 0x02, 0xbf, 0x92, 0x7d,
+	0x22, 0x66, 0xae, 0x4b, 0x22, 0x85, 0x99, 0x65, 0x14, 0xf4, 0x08, 0x2e, 0xfb, 0x81, 0x19, 0x60,
+	0x63, 0xcc, 0x21, 0x3e, 0x77, 0xbe, 0x43, 0xfc, 0x12, 0x13, 0x3e, 0xaa, 0x4b, 0xed, 0xc0, 0xc2,
+	0x80, 0xd7, 0xa1, 0x32, 0xa4, 0x1f, 0xe2, 0x53, 0xe1, 0xea, 0xf4, 0x13, 0xd5, 0x60, 0xf6, 0xd8,
+	0xec, 0xf4, 0x78, 0x52, 0x33, 0x3e, 0x6d, 0x91, 0xf5, 0xd8, 0x8b, 0x53, 0x23, 0xce, 0x7b, 0x37,
+	0x75, 0x5b, 0x51, 0x5d, 0x58, 0x1c, 0xf2, 0xb8, 0x11, 0x78, 0xeb, 0xfd, 0x78, 0xd5, 0x49, 0xf0,
+	0x6a, 0x91, 0x58, 0x19, 0xf0, 0x23, 0xa8, 0x8c, 0xf3, 0xb1, 0x11, 0xb8, 0xaf, 0xf7, 0xe3, 0xde,
+	0x9a, 0x00, 0x77, 0x50, 0xfa, 0xa9, 0x8c, 0xde, 0x82, 0x82, 0xe4, 0x63, 0x23, 0x00, 0xef, 0xf5,
+	0x03, 0x5e, 0x9b, 0x00, 0x90, 0x09, 0x1c, 0xb0, 0xe9, 0x90, 0x7b, 0x3d, 0x19, 0x9b, 0x4a, 0x62,
+	0x25, 0x40, 0xed, 0x9f, 0xb3, 0x50, 0x90, 0xa2, 0x3c, 0x6a, 0xc1, 0x3c, 0x4d, 0x88, 0x4d, 0x72,
+	0x6a, 0x10, 0xd7, 0x0d, 0xfc, 0x84, 0x91, 0x5b, 0x12, 0x55, 0x5d, 0xf5, 0xbc, 0x8e, 0xdd, 0x32,
+	0xd9, 0x2a, 0x16, 0x85, 0x50, 0x9d, 0xca, 0x44, 0x1d, 0x28, 0x11, 0xec, 0xdb, 0x56, 0xcf, 0xec,
+	0x08, 0x14, 0x1e, 0x72, 0xeb, 0x53, 0xa0, 0xac, 0xe3, 0x8e, 0x79, 0x8a, 0x2d, 0x19, 0x6c, 0x3e,
+	0x14, 0xce, 0xd0, 0xd4, 0x7f, 0xa5, 0xa0, 0x20, 0x75, 0xa3, 0xab, 0x30, 0xef, 0xc5, 0xf5, 0x40,
+	0x74, 0x0e, 0x14, 0x63, 0x62, 0xc3, 0xa2, 0x49, 0xb1, 0xed, 0x78, 0xbd, 0x40, 0x4a, 0x8a, 0x59,
+	0xbb, 0x61, 0xa1, 0x0a, 0xcc, 0xe1, 0x0e, 0xa6, 0x86, 0xac, 0xa4, 0xaf, 0x28, 0xd7, 0x8a, 0x7a,
+	0xd8, 0x44, 0x1f, 0x2b, 0xb0, 0xe8, 0xf6, 0x02, 0xca, 0x76, 0x62, 0x06, 0x98, 0x74, 0x4d, 0xf2,
+	0x30, 0x8c, 0xae, 0xcd, 0xf3, 0x59, 0xb0, 0xba, 0xcb, 0xe4, 0x1e, 0x44, 0x62, 0x79, 0xe4, 0x29,
+	0xbb, 0x03, 0x64, 0x74, 0x1f, 0xca, 0x47, 0xc4, 0xe4, 0x99, 0x99, 0x7b, 0x64, 0x9c, 0xb8, 0xe4,
+	0x61, 0x65, 0x96, 0x79, 0xc9, 0xe5, 0x2a, 0x2f, 0x9e, 0xaa, 0x61, 0xf1, 0x54, 0x5d, 0x77, 0x7b,
+	0x87, 0x1d, 0xfc, 0x26, 0x75, 0x06, 0xbd, 0x14, 0x72, 0xed, 0x1e, 0x1d, 0xb8, 0xe4, 0xa1, 0x5a,
+	0x83, 0x95, 0x91, 0x90, 0x23, 0xbc, 0x71, 0x59, 0xf6, 0xc6, 0xb4, 0xec, 0xce, 0x3f, 0x55, 0x00,
+	0x0d, 0x2f, 0x10, 0x7a, 0x0a, 0xf2, 0x16, 0xa5, 0x1a, 0x3e, 0x6e, 0x31, 0x41, 0x8a, 0x9e, 0x63,
+	0x84, 0x7d, 0xdc, 0x42, 0xdf, 0x86, 0x82, 0x19, 0x8f, 0x15, 0x1e, 0x7e, 0x5e, 0xff, 0x93, 0x45,
+	0x6a, 0x3f, 0x50, 0x60, 0x79, 0x54, 0x0d, 0x80, 0xb6, 0xe1, 0xea, 0xd8, 0x14, 0x66, 0xa8, 0x70,
+	0xbc, 0x32, 0x26, 0x19, 0x89, 0xab, 0xc7, 0x67, 0xa1, 0xd8, 0xa2, 0x0a, 0x1b, 0x81, 0xfb, 0x10,
+	0x3b, 0xdc, 0xc9, 0x8b, 0x7a, 0x81, 0xd1, 0x9a, 0x8c, 0xa4, 0xfd, 0x46, 0x81, 0x95, 0x91, 0x89,
+	0x27, 0xda, 0x84, 0xb9, 0x2e, 0x0e, 0x88, 0xdd, 0xf2, 0x45, 0xd5, 0x57, 0x9d, 0xd0, 0x04, 0xdb,
+	0x9c, 0x4b, 0x0f, 0xd9, 0xd1, 0x26, 0xcc, 0xf2, 0x0c, 0x91, 0x9b, 0xf2, 0x46, 0x72, 0x53, 0xea,
+	0x5c, 0x80, 0xb6, 0x0f, 0x97, 0xcf, 0xaa, 0x65, 0xd0, 0x4d, 0x58, 0x91, 0x0b, 0x83, 0x41, 0x8b,
+	0x2d, 0xdb, 0x72, 0x31, 0x21, 0xfa, 0x68, 0x8d, 0x59, 0xda, 0x76, 0x1d, 0x3b, 0x70, 0x89, 0xed,
+	0xb4, 0x1b, 0xce, 0x91, 0x4b, 0x5d, 0xac, 0x47, 0x9c, 0xd0, 0xc5, 0x7a, 0xc4, 0xa1, 0x85, 0x78,
+	0x70, 0xea, 0x45, 0x85, 0x38, 0xfd, 0x46, 0x1e, 0xac, 0x74, 0x23, 0x3e, 0x23, 0x30, 0x0f, 0xe9,
+	0x7a, 0x99, 0x81, 0xc9, 0x76, 0x65, 0xe1, 0xc6, 0xdd, 0x49, 0xed, 0x15, 0xc9, 0x68, 0x52, 0x11,
+	0xeb, 0x66, 0x60, 0x6e, 0xce, 0xe8, 0x4b, 0xdd, 0x61, 0x32, 0xda, 0x80, 0x2c, 0x37, 0x6a, 0x25,
+	0x73, 0xe6, 0xd9, 0x39, 0x7a, 0x49, 0x36, 0x67, 0x74, 0xc1, 0x8e, 0xde, 0x82, 0x6c, 0xc7, 0x3c,
+	0xc4, 0x9d, 0x30, 0xf9, 0x59, 0x4d, 0xac, 0x2b, 0xb5, 0x53, 0x75, 0x8b, 0xc9, 0x10, 0x39, 0x0f,
+	0x17, 0xa8, 0xde, 0x81, 0x82, 0x44, 0x7e, 0xdc, 0x6e, 0xcd, 0xcb, 0x67, 0xc1, 0xfb, 0xb0, 0xdc,
+	0x0f, 0xc0, 0x05, 0xa1, 0x79, 0xc8, 0x37, 0xf5, 0xd5, 0x9d, 0xfd, 0xfb, 0xbb, 0xfa, 0x76, 0x79,
+	0x06, 0x2d, 0x40, 0x61, 0xaf, 0xb6, 0xbb, 0xb5, 0x55, 0xaf, 0x35, 0x1b, 0xbb, 0x3b, 0x65, 0x05,
+	0x5d, 0x00, 0x74, 0xd0, 0xd8, 0x59, 0xdf, 0x3d, 0x68, 0xec, 0x6c, 0x18, 0xfb, 0x4d, 0x7d, 0xb5,
+	0x59, 0xdf, 0x78, 0xab, 0x9c, 0x42, 0x79, 0x98, 0xad, 0xed, 0xae, 0xd7, 0xf5, 0x72, 0x9a, 0xf2,
+	0xd4, 0x77, 0xde, 0x6c, 0xe8, 0xbb, 0x3b, 0xdb, 0xf5, 0x9d, 0x66, 0x39, 0xb3, 0x96, 0x85, 0x0c,
+	0x5d, 0x2b, 0xed, 0xd7, 0x29, 0xc8, 0x72, 0xf3, 0xa0, 0x03, 0x28, 0xb6, 0xdc, 0x9e, 0x13, 0x60,
+	0xc2, 0x97, 0x51, 0x49, 0xe4, 0xae, 0x35, 0xce, 0x2a, 0x96, 0xaf, 0xd0, 0x8a, 0x9b, 0xe8, 0x08,
+	0x16, 0x2d, 0xdb, 0x0f, 0x88, 0x7d, 0xc8, 0x46, 0x73, 0xe9, 0xa9, 0x44, 0xa5, 0xe7, 0xba, 0xc4,
+	0x2f, 0x20, 0xca, 0xd6, 0x00, 0x8d, 0x4e, 0x00, 0x3f, 0x0a, 0x08, 0xee, 0x9a, 0xb2, 0x1f, 0x4e,
+	0x3a, 0x81, 0x3a, 0x67, 0x0d, 0x27, 0x80, 0xe3, 0x66, 0x64, 0xac, 0xef, 0x29, 0x34, 0x07, 0x89,
+	0x27, 0xf6, 0x2c, 0x14, 0x6c, 0x27, 0x78, 0xf1, 0x96, 0xc1, 0x17, 0x94, 0x1a, 0x2c, 0xbd, 0x39,
+	0xa3, 0x03, 0x23, 0xb2, 0xb0, 0x8e, 0xae, 0x42, 0xd1, 0x62, 0x51, 0xde, 0x88, 0x17, 0x5d, 0xa1,
+	0xf2, 0xad, 0x38, 0xf6, 0xd3, 0x41, 0x74, 0x2a, 0x4e, 0x5b, 0x0c, 0xa2, 0x8a, 0xe7, 0xe9, 0x20,
+	0x4e, 0x65, 0x83, 0xd6, 0xe6, 0x84, 0xdf, 0x68, 0x7f, 0x51, 0xa0, 0x20, 0x29, 0x8b, 0x4c, 0x28,
+	0xdb, 0x4e, 0x60, 0xf4, 0x4d, 0x9d, 0xaf, 0xdd, 0xd7, 0x27, 0x9c, 0x7a, 0xc3, 0x09, 0xfa, 0x67,
+	0x5f, 0xb2, 0xfb, 0x28, 0xe8, 0x7d, 0x58, 0x12, 0xb3, 0xe8, 0x43, 0xe1, 0x6b, 0x78, 0x7b, 0xd2,
+	0x35, 0x64, 0x12, 0xfa, 0x81, 0x16, 0xad, 0x41, 0xe2, 0x5a, 0x1e, 0xe6, 0x04, 0x88, 0x76, 0x1d,
+	0x4a, 0xfd, 0xaa, 0xa1, 0xa7, 0x81, 0x1a, 0x97, 0x9b, 0x89, 0xe7, 0x46, 0x69, 0x3d, 0x6f, 0x3b,
+	0x01, 0x33, 0x91, 0xaf, 0xdd, 0x86, 0xc5, 0x21, 0x14, 0x9a, 0x6f, 0xc8, 0x4b, 0xc0, 0xcf, 0x01,
+	0x45, 0x2f, 0x4a, 0x2b, 0xe0, 0x6b, 0x9f, 0xa6, 0xa0, 0x3c, 0xe8, 0x64, 0x34, 0xc2, 0x51, 0xb4,
+	0x61, 0xe7, 0x55, 0x12, 0x45, 0xb8, 0x86, 0x13, 0x8c, 0xf0, 0xdf, 0x25, 0x7b, 0x98, 0x8c, 0x4e,
+	0xa1, 0x22, 0x74, 0x1d, 0xb7, 0x63, 0x5e, 0x49, 0x64, 0xed, 0x11, 0xb8, 0x17, 0xac, 0x91, 0x3d,
+	0x6b, 0x25, 0x28, 0xca, 0x98, 0x9a, 0x09, 0x4b, 0x23, 0x14, 0xa7, 0xe1, 0x8b, 0xed, 0x6d, 0xee,
+	0xed, 0x3a, 0x6f, 0xd0, 0x30, 0xe7, 0xf7, 0xba, 0x22, 0x01, 0xa1, 0x9f, 0x94, 0xd2, 0xb5, 0x1d,
+	0xe6, 0xca, 0x69, 0x9d, 0x7e, 0x32, 0x8a, 0xf9, 0x88, 0x85, 0x6e, 0x4a, 0x31, 0x1f, 0x69, 0x16,
+	0x5c, 0x18, 0xad, 0xe6, 0xe3, 0x51, 0x94, 0x21, 0x14, 0x65, 0x08, 0x45, 0xe1, 0x28, 0x5f, 0xa4,
+	0x61, 0x69, 0xc4, 0x21, 0xc3, 0xd2, 0x03, 0xb7, 0xd3, 0xeb, 0x3a, 0x86, 0x63, 0x76, 0x85, 0x37,
+	0xe5, 0x69, 0xe4, 0xa2, 0xb4, 0x1d, 0x4a, 0x42, 0x2d, 0xc8, 0x11, 0xf7, 0x24, 0x34, 0x3f, 0x3d,
+	0x29, 0x36, 0xa7, 0x3f, 0xd5, 0x24, 0x9a, 0xee, 0x9e, 0xe8, 0x73, 0xc4, 0x3d, 0xa1, 0x64, 0xf5,
+	0xf7, 0x0a, 0xac, 0xc4, 0x5d, 0x35, 0x06, 0xcf, 0xe3, 0xc2, 0xff, 0x32, 0xbe, 0xa0, 0xbb, 0x90,
+	0x8f, 0x6e, 0xfb, 0xc5, 0xf9, 0xaa, 0x0e, 0x65, 0xac, 0xcd, 0x70, 0xc4, 0xe6, 0x8c, 0x1e, 0x0f,
+	0x8f, 0x62, 0x93, 0x7a, 0x0a, 0xf3, 0x7d, 0xb3, 0x44, 0xef, 0x41, 0x56, 0xda, 0xac, 0x93, 0x17,
+	0xff, 0x67, 0xdb, 0x4f, 0x32, 0x92, 0x2e, 0xe4, 0x6b, 0x9f, 0x2d, 0xc1, 0x9c, 0xc8, 0xbd, 0x90,
+	0x09, 0x05, 0x4f, 0xba, 0x50, 0xe2, 0xd0, 0xaf, 0x26, 0x4b, 0xe0, 0xaa, 0x7b, 0xc1, 0xc0, 0x0d,
+	0x92, 0x2c, 0x53, 0xfd, 0x73, 0x01, 0x20, 0xae, 0xcb, 0xd1, 0x87, 0x10, 0xde, 0xd9, 0x61, 0xcb,
+	0x10, 0xf5, 0x48, 0x98, 0x39, 0xbe, 0x91, 0x14, 0x38, 0x12, 0x1b, 0xde, 0x81, 0x60, 0xab, 0x2e,
+	0x44, 0xea, 0x8b, 0xde, 0x20, 0x09, 0x7d, 0x00, 0x0b, 0xb4, 0x74, 0x38, 0xc6, 0x31, 0x30, 0x8f,
+	0x15, 0x9b, 0xd3, 0x03, 0xaf, 0x32, 0x81, 0x11, 0x6a, 0xc9, 0xec, 0x6b, 0x23, 0x1b, 0x40, 0xaa,
+	0xb0, 0xf8, 0xf5, 0x59, 0x63, 0x7a, 0xb4, 0xc1, 0xb2, 0x4a, 0x12, 0x8e, 0x36, 0x20, 0xd3, 0xf3,
+	0x31, 0x11, 0x65, 0xdc, 0xcd, 0x84, 0x20, 0x0f, 0x7c, 0x4c, 0x74, 0x26, 0x40, 0xfd, 0x22, 0x0d,
+	0xb9, 0x6d, 0x6c, 0xfa, 0x3d, 0x82, 0x2d, 0xf4, 0x63, 0x05, 0x96, 0x79, 0x81, 0x29, 0x6c, 0x66,
+	0xb0, 0x10, 0x13, 0xfa, 0xca, 0xdb, 0xd3, 0xcf, 0x25, 0x84, 0xa8, 0x36, 0xa8, 0x78, 0x61, 0x31,
+	0x96, 0x35, 0x88, 0xc9, 0x21, 0x7b, 0xa8, 0x03, 0x7d, 0xaa, 0xc0, 0x8a, 0xa8, 0x5c, 0x07, 0xf4,
+	0xe1, 0x61, 0xe7, 0x9d, 0x27, 0xa0, 0x0f, 0x2f, 0x27, 0x47, 0x28, 0xb4, 0xe4, 0x0e, 0xf7, 0xa0,
+	0x6b, 0x50, 0x0e, 0xdc, 0xc0, 0xec, 0x18, 0x74, 0x97, 0x1b, 0xbe, 0x17, 0x96, 0xdb, 0x8a, 0x5e,
+	0x62, 0x74, 0x1a, 0x09, 0xf6, 0x29, 0x55, 0xad, 0xc3, 0xc5, 0x31, 0x53, 0x4d, 0x54, 0xab, 0xde,
+	0x87, 0xca, 0x38, 0x0d, 0x13, 0xc9, 0xf1, 0x61, 0x71, 0x68, 0xd7, 0xa0, 0x77, 0x21, 0xd7, 0x15,
+	0x76, 0x10, 0x9b, 0x72, 0xed, 0xfc, 0x16, 0xd5, 0x23, 0x99, 0xea, 0xa7, 0x69, 0x28, 0xf5, 0x6f,
+	0x99, 0xff, 0x36, 0x24, 0x7a, 0x1e, 0x50, 0x74, 0xd1, 0x40, 0x73, 0x1d, 0xdb, 0xb1, 0x9d, 0xb6,
+	0x38, 0x1b, 0x17, 0xc3, 0x1e, 0x3d, 0xec, 0x40, 0xbf, 0x54, 0xe0, 0x52, 0xbf, 0x87, 0xf9, 0x12,
+	0x1b, 0xdf, 0xc1, 0xf8, 0x49, 0xc5, 0x8b, 0x7e, 0x5f, 0xf3, 0x23, 0x2d, 0xb8, 0xbf, 0x5d, 0x74,
+	0x47, 0xf7, 0xaa, 0xaf, 0xc3, 0xe5, 0xb3, 0x18, 0x13, 0xb9, 0xc1, 0x2b, 0xb0, 0x70, 0x9e, 0x9b,
+	0x93, 0x3f, 0xcc, 0x42, 0x86, 0xc6, 0x0e, 0x64, 0x40, 0x81, 0x17, 0x8d, 0x2c, 0x4b, 0x10, 0x2b,
+	0x79, 0x6f, 0x8a, 0x28, 0x24, 0x1a, 0x34, 0xb1, 0xd0, 0xa1, 0x1b, 0x7d, 0x23, 0x3c, 0x50, 0x76,
+	0x89, 0xdf, 0x66, 0xaf, 0x4e, 0x03, 0x71, 0x46, 0x11, 0x16, 0x8c, 0x2a, 0xc2, 0x12, 0xfe, 0x31,
+	0xeb, 0xc3, 0x9a, 0xa8, 0x24, 0x33, 0x00, 0xda, 0x66, 0xaf, 0x2d, 0x2e, 0x06, 0xc4, 0x2f, 0xb2,
+	0x57, 0xa6, 0x81, 0xdb, 0xa0, 0x62, 0x04, 0x4e, 0xbe, 0x1d, 0x36, 0xd4, 0x7b, 0x00, 0xb1, 0x5d,
+	0xd1, 0x65, 0xc8, 0xb3, 0x5c, 0xce, 0x33, 0x5b, 0x61, 0x7d, 0x1d, 0x13, 0xa2, 0xd7, 0x04, 0x69,
+	0xe9, 0x35, 0xc1, 0xd5, 0xfe, 0x8a, 0x2e, 0x72, 0x08, 0x45, 0x72, 0x08, 0xf5, 0xdd, 0x11, 0xb5,
+	0xc1, 0x13, 0xcc, 0x83, 0xd5, 0x77, 0x20, 0x1f, 0x4d, 0x6f, 0xb4, 0x0a, 0xe8, 0xb6, 0x9c, 0x9d,
+	0xa5, 0x1e, 0x97, 0x9d, 0xc9, 0xb9, 0x99, 0x28, 0x5e, 0xd5, 0x53, 0x28, 0x0f, 0x66, 0x34, 0x23,
+	0x76, 0xc4, 0x6e, 0xff, 0xcd, 0xf6, 0x9d, 0xa9, 0x23, 0x82, 0x7c, 0xb1, 0xf1, 0x3b, 0x05, 0x9e,
+	0x3e, 0xf3, 0x6f, 0xeb, 0xff, 0xe5, 0x6d, 0xdb, 0xe7, 0x0a, 0x5c, 0x1a, 0xfb, 0xf7, 0x7e, 0xaa,
+	0xbb, 0x36, 0xb4, 0x07, 0x2b, 0xf2, 0xe5, 0x30, 0x8f, 0xbf, 0x16, 0x26, 0x42, 0xd9, 0xb3, 0x6f,
+	0x88, 0x97, 0xe2, 0x1b, 0x62, 0x3d, 0x64, 0xd4, 0x2e, 0x83, 0x3a, 0xfe, 0x4f, 0xb2, 0xf6, 0x0f,
+	0x05, 0x72, 0xd1, 0x81, 0xb4, 0xc9, 0x1d, 0x41, 0x64, 0x38, 0xb7, 0x26, 0xbd, 0x16, 0x09, 0x43,
+	0x3a, 0x75, 0x4e, 0x9d, 0xbb, 0xd2, 0xcf, 0x14, 0xc8, 0x30, 0x5f, 0x9d, 0xca, 0x08, 0xf1, 0xc3,
+	0x9d, 0x54, 0xa2, 0x5b, 0xbc, 0xfe, 0x87, 0x3b, 0x74, 0x37, 0x47, 0xb7, 0x3c, 0x45, 0xae, 0x98,
+	0xf6, 0x79, 0x1a, 0x8a, 0xfb, 0x81, 0x19, 0x44, 0x37, 0xca, 0x83, 0x3f, 0x9a, 0xc7, 0x2a, 0x9c,
+	0x3a, 0x43, 0xe1, 0x2d, 0xc8, 0xf3, 0x1f, 0x8a, 0x74, 0xaf, 0xf0, 0x4b, 0xa5, 0xeb, 0x13, 0xea,
+	0xcc, 0x94, 0x79, 0x03, 0x9f, 0xea, 0x39, 0x5f, 0x7c, 0xa1, 0x37, 0x20, 0x4d, 0xe7, 0x9e, 0xf0,
+	0x41, 0x0e, 0x13, 0xb4, 0x81, 0xa5, 0xc7, 0x23, 0x54, 0x0a, 0x6a, 0x42, 0xd6, 0xf4, 0x3c, 0xec,
+	0x58, 0xe1, 0xb1, 0x71, 0x27, 0x89, 0xbc, 0x55, 0xc6, 0x1a, 0x8b, 0x14, 0xb2, 0xd0, 0x37, 0x61,
+	0xb6, 0xd5, 0xc1, 0x26, 0x09, 0xcf, 0x87, 0xdb, 0x49, 0x84, 0xd6, 0x28, 0x67, 0x2c, 0x93, 0x4b,
+	0x92, 0x9f, 0xb5, 0xfc, 0x29, 0x05, 0xf3, 0x62, 0x91, 0xc4, 0xee, 0x1f, 0x5c, 0xa5, 0xd1, 0x2f,
+	0x57, 0x9e, 0x81, 0x82, 0x74, 0x9d, 0x2f, 0xd6, 0x1d, 0xe2, 0xdb, 0x7c, 0xb4, 0xd5, 0x67, 0xd9,
+	0x97, 0x12, 0x5b, 0x36, 0x7a, 0x7e, 0xc1, 0x4c, 0xfb, 0x60, 0xd0, 0xb4, 0x77, 0xa7, 0x31, 0x6d,
+	0x24, 0x33, 0xb4, 0xad, 0x3e, 0x60, 0xdb, 0x3b, 0x53, 0xd8, 0x36, 0x12, 0x2a, 0x8c, 0x2b, 0x3f,
+	0x0d, 0xf9, 0x32, 0x03, 0xb9, 0xd0, 0xeb, 0xd0, 0x1e, 0x64, 0xf9, 0xa3, 0x3f, 0x11, 0x55, 0x5f,
+	0x4c, 0xe8, 0xb6, 0x55, 0x9d, 0x71, 0x53, 0xf5, 0xb9, 0x1c, 0xe4, 0xc3, 0x52, 0xb7, 0xd7, 0x09,
+	0xec, 0xae, 0xe9, 0x19, 0xbe, 0x6d, 0x61, 0x83, 0x15, 0x33, 0x62, 0x27, 0xaf, 0x26, 0x15, 0xbf,
+	0x2d, 0x44, 0xed, 0xdb, 0x16, 0x66, 0x35, 0xc4, 0xe6, 0x8c, 0xbe, 0xd8, 0x1d, 0x24, 0x22, 0x0b,
+	0x4a, 0x87, 0x66, 0xdb, 0xa0, 0x55, 0x9c, 0xc1, 0xf6, 0x91, 0xd8, 0x85, 0x2f, 0x27, 0xc5, 0x5b,
+	0x33, 0xdb, 0x34, 0x99, 0x60, 0xed, 0xcd, 0x19, 0xbd, 0x78, 0x28, 0xb5, 0x55, 0x15, 0xb2, 0x7c,
+	0xba, 0xf2, 0xb1, 0x58, 0x64, 0xc7, 0xa2, 0xfa, 0x89, 0x02, 0x8b, 0x43, 0xca, 0x4e, 0xf6, 0x27,
+	0x53, 0x83, 0xf9, 0xd8, 0x50, 0xf1, 0xef, 0xcc, 0x82, 0x1f, 0x8a, 0x69, 0x58, 0xe8, 0x02, 0x64,
+	0xf9, 0x1b, 0x0d, 0xe1, 0xd5, 0xa2, 0x15, 0x2a, 0x92, 0x89, 0x15, 0xf9, 0x58, 0x81, 0xa2, 0x3c,
+	0x8b, 0x89, 0x75, 0x88, 0x8d, 0x27, 0xe9, 0xd0, 0x0b, 0xc5, 0x24, 0xd1, 0x81, 0x66, 0x14, 0xc1,
+	0xa9, 0x87, 0xb5, 0xd7, 0x60, 0x61, 0x20, 0x2c, 0xd1, 0xa2, 0xa4, 0xe5, 0x3a, 0x81, 0xed, 0xf4,
+	0xd8, 0xaf, 0x3e, 0xb1, 0x55, 0xb9, 0x21, 0x17, 0xe5, 0x1e, 0xb6, 0x63, 0xb5, 0x07, 0x50, 0x1e,
+	0xdc, 0x7e, 0x09, 0x45, 0x44, 0xc7, 0x40, 0x4a, 0x3a, 0x06, 0xae, 0x01, 0x1a, 0x8e, 0x6f, 0xd1,
+	0x48, 0x45, 0x1a, 0xb9, 0x02, 0x4b, 0x23, 0xb6, 0xab, 0xb6, 0x04, 0x8b, 0x43, 0xb1, 0x4c, 0x5b,
+	0x16, 0x52, 0xfb, 0x36, 0xa1, 0xf6, 0xc7, 0x0c, 0xe4, 0xb6, 0x5c, 0x51, 0xa0, 0xbc, 0x05, 0x39,
+	0x1f, 0x1f, 0x63, 0x62, 0x07, 0xdc, 0x7b, 0x4a, 0x13, 0xe7, 0xba, 0xa1, 0x88, 0xea, 0xbe, 0xe0,
+	0xaf, 0xd6, 0x9d, 0x5e, 0x57, 0x8f, 0xc4, 0x4d, 0x9f, 0x00, 0xa2, 0x0a, 0xcd, 0xad, 0x7c, 0xdf,
+	0x6c, 0x87, 0x99, 0x6f, 0xd8, 0xa4, 0x71, 0x36, 0x20, 0x34, 0x55, 0xce, 0xf0, 0x38, 0xcb, 0x1a,
+	0xe3, 0xcf, 0xc8, 0xd9, 0x33, 0xce, 0xc8, 0xc7, 0xbe, 0xf6, 0xcd, 0x3e, 0xfe, 0xb5, 0xef, 0xb3,
+	0x50, 0xec, 0xb8, 0x6d, 0xa3, 0xe3, 0x8a, 0x5f, 0xcf, 0x73, 0xdc, 0x49, 0x3b, 0x6e, 0x7b, 0x4b,
+	0x90, 0xa8, 0x93, 0x06, 0xef, 0x11, 0x6c, 0x5a, 0x95, 0x1c, 0xeb, 0x14, 0x2d, 0xf5, 0x5b, 0xe2,
+	0x51, 0xf0, 0x1e, 0xd0, 0xe1, 0x06, 0x76, 0x02, 0x62, 0x47, 0x77, 0x8e, 0xd7, 0x13, 0xae, 0x81,
+	0x0e, 0x1d, 0xfe, 0x65, 0x63, 0x5f, 0x25, 0x90, 0x0b, 0x97, 0x44, 0x3b, 0x82, 0x0c, 0x5d, 0x15,
+	0xb4, 0x00, 0x85, 0x07, 0x3b, 0xfb, 0x7b, 0xf5, 0x5a, 0xe3, 0x7e, 0xa3, 0xbe, 0x5e, 0x9e, 0x41,
+	0x79, 0x98, 0x6d, 0xea, 0xab, 0xb5, 0x7a, 0x59, 0xa1, 0x9f, 0xeb, 0xf5, 0xb5, 0x07, 0x1b, 0xe5,
+	0x14, 0xca, 0x41, 0xa6, 0xb1, 0x73, 0x7f, 0xb7, 0x9c, 0x46, 0x00, 0xd9, 0x9d, 0xdd, 0x66, 0xa3,
+	0x56, 0x2f, 0x67, 0x28, 0xf5, 0x60, 0x55, 0xdf, 0x29, 0xcf, 0xd2, 0xa1, 0x75, 0x5d, 0xdf, 0xd5,
+	0xcb, 0x59, 0x54, 0x84, 0x5c, 0x4d, 0x6f, 0x34, 0x1b, 0xb5, 0xd5, 0xad, 0xf2, 0x9c, 0x56, 0x04,
+	0xd8, 0x72, 0xdb, 0x35, 0xd7, 0x09, 0x88, 0xdb, 0xb9, 0xf1, 0x99, 0x02, 0xf3, 0x6b, 0xd8, 0xec,
+	0xde, 0x77, 0x04, 0x05, 0x7d, 0xa2, 0xc0, 0x5c, 0xf8, 0x3d, 0xf9, 0x4f, 0x88, 0xa1, 0x87, 0xa4,
+	0xea, 0x9d, 0x69, 0x78, 0xf9, 0x66, 0x98, 0xb9, 0xa6, 0x7c, 0x4d, 0xb9, 0xf1, 0x11, 0x00, 0xd7,
+	0x8c, 0x65, 0x83, 0x8e, 0xc8, 0x0a, 0xaf, 0x27, 0x4c, 0x2d, 0xd5, 0xa4, 0x0c, 0x02, 0xfd, 0xfb,
+	0x0a, 0x14, 0x38, 0x3c, 0x0f, 0x85, 0x8f, 0x60, 0x96, 0x7f, 0xdc, 0x4c, 0x72, 0x2e, 0x88, 0x19,
+	0xa9, 0xb7, 0x92, 0x31, 0x89, 0xed, 0xcf, 0x35, 0xf9, 0x61, 0xb4, 0x44, 0x5b, 0x6e, 0xbb, 0x6d,
+	0x3b, 0x6d, 0xf4, 0x08, 0xe6, 0xc2, 0xcf, 0x5b, 0x49, 0x43, 0x00, 0xf5, 0x64, 0xf5, 0x85, 0xc9,
+	0xb9, 0x84, 0x2b, 0x70, 0x5d, 0xd6, 0x56, 0xe1, 0xab, 0xe3, 0x38, 0x65, 0xc6, 0xb5, 0x3c, 0x57,
+	0x78, 0xd5, 0xb3, 0xdf, 0x2e, 0x49, 0x5d, 0xc6, 0xf1, 0x0b, 0x87, 0x59, 0x16, 0x50, 0x6e, 0xfe,
+	0x27, 0x00, 0x00, 0xff, 0xff, 0xba, 0xe2, 0x47, 0x6e, 0x71, 0x30, 0x00, 0x00,
 }
diff --git a/sdks/go/pkg/beam/model/fnexecution_v1/beam_provision_api.pb.go b/sdks/go/pkg/beam/model/fnexecution_v1/beam_provision_api.pb.go
index a0c31ca4be5..29fe011d557 100644
--- a/sdks/go/pkg/beam/model/fnexecution_v1/beam_provision_api.pb.go
+++ b/sdks/go/pkg/beam/model/fnexecution_v1/beam_provision_api.pb.go
@@ -3,10 +3,12 @@
 
 package fnexecution_v1
 
-import proto "github.com/golang/protobuf/proto"
-import fmt "fmt"
-import math "math"
-import _struct "github.com/golang/protobuf/ptypes/struct"
+import (
+	fmt "fmt"
+	proto "github.com/golang/protobuf/proto"
+	_struct "github.com/golang/protobuf/ptypes/struct"
+	math "math"
+)
 
 import (
 	context "golang.org/x/net/context"
@@ -35,7 +37,7 @@ func (m *GetProvisionInfoRequest) Reset()         { *m = GetProvisionInfoRequest
 func (m *GetProvisionInfoRequest) String() string { return proto.CompactTextString(m) }
 func (*GetProvisionInfoRequest) ProtoMessage()    {}
 func (*GetProvisionInfoRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_provision_api_6d75e94da57739fd, []int{0}
+	return fileDescriptor_92e393e5933c7d6f, []int{0}
 }
 func (m *GetProvisionInfoRequest) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_GetProvisionInfoRequest.Unmarshal(m, b)
@@ -43,8 +45,8 @@ func (m *GetProvisionInfoRequest) XXX_Unmarshal(b []byte) error {
 func (m *GetProvisionInfoRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_GetProvisionInfoRequest.Marshal(b, m, deterministic)
 }
-func (dst *GetProvisionInfoRequest) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_GetProvisionInfoRequest.Merge(dst, src)
+func (m *GetProvisionInfoRequest) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_GetProvisionInfoRequest.Merge(m, src)
 }
 func (m *GetProvisionInfoRequest) XXX_Size() int {
 	return xxx_messageInfo_GetProvisionInfoRequest.Size(m)
@@ -67,7 +69,7 @@ func (m *GetProvisionInfoResponse) Reset()         { *m = GetProvisionInfoRespon
 func (m *GetProvisionInfoResponse) String() string { return proto.CompactTextString(m) }
 func (*GetProvisionInfoResponse) ProtoMessage()    {}
 func (*GetProvisionInfoResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_provision_api_6d75e94da57739fd, []int{1}
+	return fileDescriptor_92e393e5933c7d6f, []int{1}
 }
 func (m *GetProvisionInfoResponse) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_GetProvisionInfoResponse.Unmarshal(m, b)
@@ -75,8 +77,8 @@ func (m *GetProvisionInfoResponse) XXX_Unmarshal(b []byte) error {
 func (m *GetProvisionInfoResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_GetProvisionInfoResponse.Marshal(b, m, deterministic)
 }
-func (dst *GetProvisionInfoResponse) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_GetProvisionInfoResponse.Merge(dst, src)
+func (m *GetProvisionInfoResponse) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_GetProvisionInfoResponse.Merge(m, src)
 }
 func (m *GetProvisionInfoResponse) XXX_Size() int {
 	return xxx_messageInfo_GetProvisionInfoResponse.Size(m)
@@ -124,7 +126,7 @@ func (m *ProvisionInfo) Reset()         { *m = ProvisionInfo{} }
 func (m *ProvisionInfo) String() string { return proto.CompactTextString(m) }
 func (*ProvisionInfo) ProtoMessage()    {}
 func (*ProvisionInfo) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_provision_api_6d75e94da57739fd, []int{2}
+	return fileDescriptor_92e393e5933c7d6f, []int{2}
 }
 func (m *ProvisionInfo) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_ProvisionInfo.Unmarshal(m, b)
@@ -132,8 +134,8 @@ func (m *ProvisionInfo) XXX_Unmarshal(b []byte) error {
 func (m *ProvisionInfo) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_ProvisionInfo.Marshal(b, m, deterministic)
 }
-func (dst *ProvisionInfo) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ProvisionInfo.Merge(dst, src)
+func (m *ProvisionInfo) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_ProvisionInfo.Merge(m, src)
 }
 func (m *ProvisionInfo) XXX_Size() int {
 	return xxx_messageInfo_ProvisionInfo.Size(m)
@@ -205,7 +207,7 @@ func (m *Resources) Reset()         { *m = Resources{} }
 func (m *Resources) String() string { return proto.CompactTextString(m) }
 func (*Resources) ProtoMessage()    {}
 func (*Resources) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_provision_api_6d75e94da57739fd, []int{3}
+	return fileDescriptor_92e393e5933c7d6f, []int{3}
 }
 func (m *Resources) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Resources.Unmarshal(m, b)
@@ -213,8 +215,8 @@ func (m *Resources) XXX_Unmarshal(b []byte) error {
 func (m *Resources) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_Resources.Marshal(b, m, deterministic)
 }
-func (dst *Resources) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Resources.Merge(dst, src)
+func (m *Resources) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_Resources.Merge(m, src)
 }
 func (m *Resources) XXX_Size() int {
 	return xxx_messageInfo_Resources.Size(m)
@@ -259,7 +261,7 @@ func (m *Resources_Memory) Reset()         { *m = Resources_Memory{} }
 func (m *Resources_Memory) String() string { return proto.CompactTextString(m) }
 func (*Resources_Memory) ProtoMessage()    {}
 func (*Resources_Memory) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_provision_api_6d75e94da57739fd, []int{3, 0}
+	return fileDescriptor_92e393e5933c7d6f, []int{3, 0}
 }
 func (m *Resources_Memory) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Resources_Memory.Unmarshal(m, b)
@@ -267,8 +269,8 @@ func (m *Resources_Memory) XXX_Unmarshal(b []byte) error {
 func (m *Resources_Memory) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_Resources_Memory.Marshal(b, m, deterministic)
 }
-func (dst *Resources_Memory) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Resources_Memory.Merge(dst, src)
+func (m *Resources_Memory) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_Resources_Memory.Merge(m, src)
 }
 func (m *Resources_Memory) XXX_Size() int {
 	return xxx_messageInfo_Resources_Memory.Size(m)
@@ -300,7 +302,7 @@ func (m *Resources_Cpu) Reset()         { *m = Resources_Cpu{} }
 func (m *Resources_Cpu) String() string { return proto.CompactTextString(m) }
 func (*Resources_Cpu) ProtoMessage()    {}
 func (*Resources_Cpu) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_provision_api_6d75e94da57739fd, []int{3, 1}
+	return fileDescriptor_92e393e5933c7d6f, []int{3, 1}
 }
 func (m *Resources_Cpu) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Resources_Cpu.Unmarshal(m, b)
@@ -308,8 +310,8 @@ func (m *Resources_Cpu) XXX_Unmarshal(b []byte) error {
 func (m *Resources_Cpu) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_Resources_Cpu.Marshal(b, m, deterministic)
 }
-func (dst *Resources_Cpu) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Resources_Cpu.Merge(dst, src)
+func (m *Resources_Cpu) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_Resources_Cpu.Merge(m, src)
 }
 func (m *Resources_Cpu) XXX_Size() int {
 	return xxx_messageInfo_Resources_Cpu.Size(m)
@@ -340,7 +342,7 @@ func (m *Resources_Disk) Reset()         { *m = Resources_Disk{} }
 func (m *Resources_Disk) String() string { return proto.CompactTextString(m) }
 func (*Resources_Disk) ProtoMessage()    {}
 func (*Resources_Disk) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_provision_api_6d75e94da57739fd, []int{3, 2}
+	return fileDescriptor_92e393e5933c7d6f, []int{3, 2}
 }
 func (m *Resources_Disk) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Resources_Disk.Unmarshal(m, b)
@@ -348,8 +350,8 @@ func (m *Resources_Disk) XXX_Unmarshal(b []byte) error {
 func (m *Resources_Disk) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_Resources_Disk.Marshal(b, m, deterministic)
 }
-func (dst *Resources_Disk) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Resources_Disk.Merge(dst, src)
+func (m *Resources_Disk) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_Resources_Disk.Merge(m, src)
 }
 func (m *Resources_Disk) XXX_Size() int {
 	return xxx_messageInfo_Resources_Disk.Size(m)
@@ -451,11 +453,9 @@ var _ProvisionService_serviceDesc = grpc.ServiceDesc{
 	Metadata: "beam_provision_api.proto",
 }
 
-func init() {
-	proto.RegisterFile("beam_provision_api.proto", fileDescriptor_beam_provision_api_6d75e94da57739fd)
-}
+func init() { proto.RegisterFile("beam_provision_api.proto", fileDescriptor_92e393e5933c7d6f) }
 
-var fileDescriptor_beam_provision_api_6d75e94da57739fd = []byte{
+var fileDescriptor_92e393e5933c7d6f = []byte{
 	// 506 bytes of a gzipped FileDescriptorProto
 	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x9c, 0x93, 0xdb, 0x6e, 0xd3, 0x40,
 	0x10, 0x86, 0x95, 0x43, 0x4d, 0x33, 0x40, 0x1a, 0xad, 0x80, 0xba, 0xa6, 0x48, 0x28, 0x02, 0xc1,
diff --git a/sdks/go/pkg/beam/model/jobmanagement_v1/beam_artifact_api.pb.go b/sdks/go/pkg/beam/model/jobmanagement_v1/beam_artifact_api.pb.go
index 18b13e35161..8aee3ffe6b4 100644
--- a/sdks/go/pkg/beam/model/jobmanagement_v1/beam_artifact_api.pb.go
+++ b/sdks/go/pkg/beam/model/jobmanagement_v1/beam_artifact_api.pb.go
@@ -3,9 +3,11 @@
 
 package jobmanagement_v1
 
-import proto "github.com/golang/protobuf/proto"
-import fmt "fmt"
-import math "math"
+import (
+	fmt "fmt"
+	proto "github.com/golang/protobuf/proto"
+	math "math"
+)
 
 import (
 	context "golang.org/x/net/context"
@@ -41,7 +43,7 @@ func (m *ArtifactMetadata) Reset()         { *m = ArtifactMetadata{} }
 func (m *ArtifactMetadata) String() string { return proto.CompactTextString(m) }
 func (*ArtifactMetadata) ProtoMessage()    {}
 func (*ArtifactMetadata) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_artifact_api_e055cefa38e30fa8, []int{0}
+	return fileDescriptor_8ef4db42c81e3972, []int{0}
 }
 func (m *ArtifactMetadata) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_ArtifactMetadata.Unmarshal(m, b)
@@ -49,8 +51,8 @@ func (m *ArtifactMetadata) XXX_Unmarshal(b []byte) error {
 func (m *ArtifactMetadata) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_ArtifactMetadata.Marshal(b, m, deterministic)
 }
-func (dst *ArtifactMetadata) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ArtifactMetadata.Merge(dst, src)
+func (m *ArtifactMetadata) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_ArtifactMetadata.Merge(m, src)
 }
 func (m *ArtifactMetadata) XXX_Size() int {
 	return xxx_messageInfo_ArtifactMetadata.Size(m)
@@ -94,7 +96,7 @@ func (m *Manifest) Reset()         { *m = Manifest{} }
 func (m *Manifest) String() string { return proto.CompactTextString(m) }
 func (*Manifest) ProtoMessage()    {}
 func (*Manifest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_artifact_api_e055cefa38e30fa8, []int{1}
+	return fileDescriptor_8ef4db42c81e3972, []int{1}
 }
 func (m *Manifest) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Manifest.Unmarshal(m, b)
@@ -102,8 +104,8 @@ func (m *Manifest) XXX_Unmarshal(b []byte) error {
 func (m *Manifest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_Manifest.Marshal(b, m, deterministic)
 }
-func (dst *Manifest) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Manifest.Merge(dst, src)
+func (m *Manifest) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_Manifest.Merge(m, src)
 }
 func (m *Manifest) XXX_Size() int {
 	return xxx_messageInfo_Manifest.Size(m)
@@ -134,7 +136,7 @@ func (m *ProxyManifest) Reset()         { *m = ProxyManifest{} }
 func (m *ProxyManifest) String() string { return proto.CompactTextString(m) }
 func (*ProxyManifest) ProtoMessage()    {}
 func (*ProxyManifest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_artifact_api_e055cefa38e30fa8, []int{2}
+	return fileDescriptor_8ef4db42c81e3972, []int{2}
 }
 func (m *ProxyManifest) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_ProxyManifest.Unmarshal(m, b)
@@ -142,8 +144,8 @@ func (m *ProxyManifest) XXX_Unmarshal(b []byte) error {
 func (m *ProxyManifest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_ProxyManifest.Marshal(b, m, deterministic)
 }
-func (dst *ProxyManifest) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ProxyManifest.Merge(dst, src)
+func (m *ProxyManifest) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_ProxyManifest.Merge(m, src)
 }
 func (m *ProxyManifest) XXX_Size() int {
 	return xxx_messageInfo_ProxyManifest.Size(m)
@@ -180,7 +182,7 @@ func (m *ProxyManifest_Location) Reset()         { *m = ProxyManifest_Location{}
 func (m *ProxyManifest_Location) String() string { return proto.CompactTextString(m) }
 func (*ProxyManifest_Location) ProtoMessage()    {}
 func (*ProxyManifest_Location) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_artifact_api_e055cefa38e30fa8, []int{2, 0}
+	return fileDescriptor_8ef4db42c81e3972, []int{2, 0}
 }
 func (m *ProxyManifest_Location) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_ProxyManifest_Location.Unmarshal(m, b)
@@ -188,8 +190,8 @@ func (m *ProxyManifest_Location) XXX_Unmarshal(b []byte) error {
 func (m *ProxyManifest_Location) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_ProxyManifest_Location.Marshal(b, m, deterministic)
 }
-func (dst *ProxyManifest_Location) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ProxyManifest_Location.Merge(dst, src)
+func (m *ProxyManifest_Location) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_ProxyManifest_Location.Merge(m, src)
 }
 func (m *ProxyManifest_Location) XXX_Size() int {
 	return xxx_messageInfo_ProxyManifest_Location.Size(m)
@@ -228,7 +230,7 @@ func (m *GetManifestRequest) Reset()         { *m = GetManifestRequest{} }
 func (m *GetManifestRequest) String() string { return proto.CompactTextString(m) }
 func (*GetManifestRequest) ProtoMessage()    {}
 func (*GetManifestRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_artifact_api_e055cefa38e30fa8, []int{3}
+	return fileDescriptor_8ef4db42c81e3972, []int{3}
 }
 func (m *GetManifestRequest) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_GetManifestRequest.Unmarshal(m, b)
@@ -236,8 +238,8 @@ func (m *GetManifestRequest) XXX_Unmarshal(b []byte) error {
 func (m *GetManifestRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_GetManifestRequest.Marshal(b, m, deterministic)
 }
-func (dst *GetManifestRequest) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_GetManifestRequest.Merge(dst, src)
+func (m *GetManifestRequest) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_GetManifestRequest.Merge(m, src)
 }
 func (m *GetManifestRequest) XXX_Size() int {
 	return xxx_messageInfo_GetManifestRequest.Size(m)
@@ -267,7 +269,7 @@ func (m *GetManifestResponse) Reset()         { *m = GetManifestResponse{} }
 func (m *GetManifestResponse) String() string { return proto.CompactTextString(m) }
 func (*GetManifestResponse) ProtoMessage()    {}
 func (*GetManifestResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_artifact_api_e055cefa38e30fa8, []int{4}
+	return fileDescriptor_8ef4db42c81e3972, []int{4}
 }
 func (m *GetManifestResponse) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_GetManifestResponse.Unmarshal(m, b)
@@ -275,8 +277,8 @@ func (m *GetManifestResponse) XXX_Unmarshal(b []byte) error {
 func (m *GetManifestResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_GetManifestResponse.Marshal(b, m, deterministic)
 }
-func (dst *GetManifestResponse) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_GetManifestResponse.Merge(dst, src)
+func (m *GetManifestResponse) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_GetManifestResponse.Merge(m, src)
 }
 func (m *GetManifestResponse) XXX_Size() int {
 	return xxx_messageInfo_GetManifestResponse.Size(m)
@@ -310,7 +312,7 @@ func (m *GetArtifactRequest) Reset()         { *m = GetArtifactRequest{} }
 func (m *GetArtifactRequest) String() string { return proto.CompactTextString(m) }
 func (*GetArtifactRequest) ProtoMessage()    {}
 func (*GetArtifactRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_artifact_api_e055cefa38e30fa8, []int{5}
+	return fileDescriptor_8ef4db42c81e3972, []int{5}
 }
 func (m *GetArtifactRequest) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_GetArtifactRequest.Unmarshal(m, b)
@@ -318,8 +320,8 @@ func (m *GetArtifactRequest) XXX_Unmarshal(b []byte) error {
 func (m *GetArtifactRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_GetArtifactRequest.Marshal(b, m, deterministic)
 }
-func (dst *GetArtifactRequest) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_GetArtifactRequest.Merge(dst, src)
+func (m *GetArtifactRequest) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_GetArtifactRequest.Merge(m, src)
 }
 func (m *GetArtifactRequest) XXX_Size() int {
 	return xxx_messageInfo_GetArtifactRequest.Size(m)
@@ -356,7 +358,7 @@ func (m *ArtifactChunk) Reset()         { *m = ArtifactChunk{} }
 func (m *ArtifactChunk) String() string { return proto.CompactTextString(m) }
 func (*ArtifactChunk) ProtoMessage()    {}
 func (*ArtifactChunk) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_artifact_api_e055cefa38e30fa8, []int{6}
+	return fileDescriptor_8ef4db42c81e3972, []int{6}
 }
 func (m *ArtifactChunk) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_ArtifactChunk.Unmarshal(m, b)
@@ -364,8 +366,8 @@ func (m *ArtifactChunk) XXX_Unmarshal(b []byte) error {
 func (m *ArtifactChunk) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_ArtifactChunk.Marshal(b, m, deterministic)
 }
-func (dst *ArtifactChunk) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ArtifactChunk.Merge(dst, src)
+func (m *ArtifactChunk) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_ArtifactChunk.Merge(m, src)
 }
 func (m *ArtifactChunk) XXX_Size() int {
 	return xxx_messageInfo_ArtifactChunk.Size(m)
@@ -398,7 +400,7 @@ func (m *PutArtifactMetadata) Reset()         { *m = PutArtifactMetadata{} }
 func (m *PutArtifactMetadata) String() string { return proto.CompactTextString(m) }
 func (*PutArtifactMetadata) ProtoMessage()    {}
 func (*PutArtifactMetadata) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_artifact_api_e055cefa38e30fa8, []int{7}
+	return fileDescriptor_8ef4db42c81e3972, []int{7}
 }
 func (m *PutArtifactMetadata) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_PutArtifactMetadata.Unmarshal(m, b)
@@ -406,8 +408,8 @@ func (m *PutArtifactMetadata) XXX_Unmarshal(b []byte) error {
 func (m *PutArtifactMetadata) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_PutArtifactMetadata.Marshal(b, m, deterministic)
 }
-func (dst *PutArtifactMetadata) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_PutArtifactMetadata.Merge(dst, src)
+func (m *PutArtifactMetadata) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_PutArtifactMetadata.Merge(m, src)
 }
 func (m *PutArtifactMetadata) XXX_Size() int {
 	return xxx_messageInfo_PutArtifactMetadata.Size(m)
@@ -449,7 +451,7 @@ func (m *PutArtifactRequest) Reset()         { *m = PutArtifactRequest{} }
 func (m *PutArtifactRequest) String() string { return proto.CompactTextString(m) }
 func (*PutArtifactRequest) ProtoMessage()    {}
 func (*PutArtifactRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_artifact_api_e055cefa38e30fa8, []int{8}
+	return fileDescriptor_8ef4db42c81e3972, []int{8}
 }
 func (m *PutArtifactRequest) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_PutArtifactRequest.Unmarshal(m, b)
@@ -457,8 +459,8 @@ func (m *PutArtifactRequest) XXX_Unmarshal(b []byte) error {
 func (m *PutArtifactRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_PutArtifactRequest.Marshal(b, m, deterministic)
 }
-func (dst *PutArtifactRequest) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_PutArtifactRequest.Merge(dst, src)
+func (m *PutArtifactRequest) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_PutArtifactRequest.Merge(m, src)
 }
 func (m *PutArtifactRequest) XXX_Size() int {
 	return xxx_messageInfo_PutArtifactRequest.Size(m)
@@ -476,12 +478,14 @@ type isPutArtifactRequest_Content interface {
 type PutArtifactRequest_Metadata struct {
 	Metadata *PutArtifactMetadata `protobuf:"bytes,1,opt,name=metadata,proto3,oneof"`
 }
+
 type PutArtifactRequest_Data struct {
 	Data *ArtifactChunk `protobuf:"bytes,2,opt,name=data,proto3,oneof"`
 }
 
 func (*PutArtifactRequest_Metadata) isPutArtifactRequest_Content() {}
-func (*PutArtifactRequest_Data) isPutArtifactRequest_Content()     {}
+
+func (*PutArtifactRequest_Data) isPutArtifactRequest_Content() {}
 
 func (m *PutArtifactRequest) GetContent() isPutArtifactRequest_Content {
 	if m != nil {
@@ -588,7 +592,7 @@ func (m *PutArtifactResponse) Reset()         { *m = PutArtifactResponse{} }
 func (m *PutArtifactResponse) String() string { return proto.CompactTextString(m) }
 func (*PutArtifactResponse) ProtoMessage()    {}
 func (*PutArtifactResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_artifact_api_e055cefa38e30fa8, []int{9}
+	return fileDescriptor_8ef4db42c81e3972, []int{9}
 }
 func (m *PutArtifactResponse) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_PutArtifactResponse.Unmarshal(m, b)
@@ -596,8 +600,8 @@ func (m *PutArtifactResponse) XXX_Unmarshal(b []byte) error {
 func (m *PutArtifactResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_PutArtifactResponse.Marshal(b, m, deterministic)
 }
-func (dst *PutArtifactResponse) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_PutArtifactResponse.Merge(dst, src)
+func (m *PutArtifactResponse) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_PutArtifactResponse.Merge(m, src)
 }
 func (m *PutArtifactResponse) XXX_Size() int {
 	return xxx_messageInfo_PutArtifactResponse.Size(m)
@@ -625,7 +629,7 @@ func (m *CommitManifestRequest) Reset()         { *m = CommitManifestRequest{} }
 func (m *CommitManifestRequest) String() string { return proto.CompactTextString(m) }
 func (*CommitManifestRequest) ProtoMessage()    {}
 func (*CommitManifestRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_artifact_api_e055cefa38e30fa8, []int{10}
+	return fileDescriptor_8ef4db42c81e3972, []int{10}
 }
 func (m *CommitManifestRequest) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_CommitManifestRequest.Unmarshal(m, b)
@@ -633,8 +637,8 @@ func (m *CommitManifestRequest) XXX_Unmarshal(b []byte) error {
 func (m *CommitManifestRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_CommitManifestRequest.Marshal(b, m, deterministic)
 }
-func (dst *CommitManifestRequest) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_CommitManifestRequest.Merge(dst, src)
+func (m *CommitManifestRequest) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_CommitManifestRequest.Merge(m, src)
 }
 func (m *CommitManifestRequest) XXX_Size() int {
 	return xxx_messageInfo_CommitManifestRequest.Size(m)
@@ -674,7 +678,7 @@ func (m *CommitManifestResponse) Reset()         { *m = CommitManifestResponse{}
 func (m *CommitManifestResponse) String() string { return proto.CompactTextString(m) }
 func (*CommitManifestResponse) ProtoMessage()    {}
 func (*CommitManifestResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_artifact_api_e055cefa38e30fa8, []int{11}
+	return fileDescriptor_8ef4db42c81e3972, []int{11}
 }
 func (m *CommitManifestResponse) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_CommitManifestResponse.Unmarshal(m, b)
@@ -682,8 +686,8 @@ func (m *CommitManifestResponse) XXX_Unmarshal(b []byte) error {
 func (m *CommitManifestResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_CommitManifestResponse.Marshal(b, m, deterministic)
 }
-func (dst *CommitManifestResponse) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_CommitManifestResponse.Merge(dst, src)
+func (m *CommitManifestResponse) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_CommitManifestResponse.Merge(m, src)
 }
 func (m *CommitManifestResponse) XXX_Size() int {
 	return xxx_messageInfo_CommitManifestResponse.Size(m)
@@ -1000,11 +1004,9 @@ var _ArtifactRetrievalService_serviceDesc = grpc.ServiceDesc{
 	Metadata: "beam_artifact_api.proto",
 }
 
-func init() {
-	proto.RegisterFile("beam_artifact_api.proto", fileDescriptor_beam_artifact_api_e055cefa38e30fa8)
-}
+func init() { proto.RegisterFile("beam_artifact_api.proto", fileDescriptor_8ef4db42c81e3972) }
 
-var fileDescriptor_beam_artifact_api_e055cefa38e30fa8 = []byte{
+var fileDescriptor_8ef4db42c81e3972 = []byte{
 	// 613 bytes of a gzipped FileDescriptorProto
 	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xac, 0x56, 0xc1, 0x6e, 0xd3, 0x4c,
 	0x10, 0xee, 0x3a, 0xbf, 0x7e, 0xd2, 0x31, 0x2d, 0xd1, 0x56, 0x2d, 0x56, 0x4e, 0x91, 0x91, 0x68,
diff --git a/sdks/go/pkg/beam/model/jobmanagement_v1/beam_job_api.pb.go b/sdks/go/pkg/beam/model/jobmanagement_v1/beam_job_api.pb.go
index 466209040f4..e3f1ed84a0c 100644
--- a/sdks/go/pkg/beam/model/jobmanagement_v1/beam_job_api.pb.go
+++ b/sdks/go/pkg/beam/model/jobmanagement_v1/beam_job_api.pb.go
@@ -3,11 +3,13 @@
 
 package jobmanagement_v1
 
-import proto "github.com/golang/protobuf/proto"
-import fmt "fmt"
-import math "math"
-import pipeline_v1 "github.com/apache/beam/sdks/go/pkg/beam/model/pipeline_v1"
-import _struct "github.com/golang/protobuf/ptypes/struct"
+import (
+	fmt "fmt"
+	pipeline_v1 "github.com/apache/beam/sdks/go/pkg/beam/model/pipeline_v1"
+	proto "github.com/golang/protobuf/proto"
+	_struct "github.com/golang/protobuf/ptypes/struct"
+	math "math"
+)
 
 import (
 	context "golang.org/x/net/context"
@@ -44,6 +46,7 @@ var JobMessage_MessageImportance_name = map[int32]string{
 	4: "JOB_MESSAGE_WARNING",
 	5: "JOB_MESSAGE_ERROR",
 }
+
 var JobMessage_MessageImportance_value = map[string]int32{
 	"MESSAGE_IMPORTANCE_UNSPECIFIED": 0,
 	"JOB_MESSAGE_DEBUG":              1,
@@ -56,8 +59,9 @@ var JobMessage_MessageImportance_value = map[string]int32{
 func (x JobMessage_MessageImportance) String() string {
 	return proto.EnumName(JobMessage_MessageImportance_name, int32(x))
 }
+
 func (JobMessage_MessageImportance) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_beam_job_api_1fb9125c215633d2, []int{9, 0}
+	return fileDescriptor_97c7b84f742157ae, []int{9, 0}
 }
 
 type JobState_Enum int32
@@ -89,6 +93,7 @@ var JobState_Enum_name = map[int32]string{
 	9:  "STARTING",
 	10: "CANCELLING",
 }
+
 var JobState_Enum_value = map[string]int32{
 	"UNSPECIFIED": 0,
 	"STOPPED":     1,
@@ -106,8 +111,9 @@ var JobState_Enum_value = map[string]int32{
 func (x JobState_Enum) String() string {
 	return proto.EnumName(JobState_Enum_name, int32(x))
 }
+
 func (JobState_Enum) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_beam_job_api_1fb9125c215633d2, []int{11, 0}
+	return fileDescriptor_97c7b84f742157ae, []int{11, 0}
 }
 
 // Prepare is a synchronous request that returns a preparationId back
@@ -127,7 +133,7 @@ func (m *PrepareJobRequest) Reset()         { *m = PrepareJobRequest{} }
 func (m *PrepareJobRequest) String() string { return proto.CompactTextString(m) }
 func (*PrepareJobRequest) ProtoMessage()    {}
 func (*PrepareJobRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_job_api_1fb9125c215633d2, []int{0}
+	return fileDescriptor_97c7b84f742157ae, []int{0}
 }
 func (m *PrepareJobRequest) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_PrepareJobRequest.Unmarshal(m, b)
@@ -135,8 +141,8 @@ func (m *PrepareJobRequest) XXX_Unmarshal(b []byte) error {
 func (m *PrepareJobRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_PrepareJobRequest.Marshal(b, m, deterministic)
 }
-func (dst *PrepareJobRequest) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_PrepareJobRequest.Merge(dst, src)
+func (m *PrepareJobRequest) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_PrepareJobRequest.Merge(m, src)
 }
 func (m *PrepareJobRequest) XXX_Size() int {
 	return xxx_messageInfo_PrepareJobRequest.Size(m)
@@ -187,7 +193,7 @@ func (m *PrepareJobResponse) Reset()         { *m = PrepareJobResponse{} }
 func (m *PrepareJobResponse) String() string { return proto.CompactTextString(m) }
 func (*PrepareJobResponse) ProtoMessage()    {}
 func (*PrepareJobResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_job_api_1fb9125c215633d2, []int{1}
+	return fileDescriptor_97c7b84f742157ae, []int{1}
 }
 func (m *PrepareJobResponse) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_PrepareJobResponse.Unmarshal(m, b)
@@ -195,8 +201,8 @@ func (m *PrepareJobResponse) XXX_Unmarshal(b []byte) error {
 func (m *PrepareJobResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_PrepareJobResponse.Marshal(b, m, deterministic)
 }
-func (dst *PrepareJobResponse) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_PrepareJobResponse.Merge(dst, src)
+func (m *PrepareJobResponse) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_PrepareJobResponse.Merge(m, src)
 }
 func (m *PrepareJobResponse) XXX_Size() int {
 	return xxx_messageInfo_PrepareJobResponse.Size(m)
@@ -248,7 +254,7 @@ func (m *RunJobRequest) Reset()         { *m = RunJobRequest{} }
 func (m *RunJobRequest) String() string { return proto.CompactTextString(m) }
 func (*RunJobRequest) ProtoMessage()    {}
 func (*RunJobRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_job_api_1fb9125c215633d2, []int{2}
+	return fileDescriptor_97c7b84f742157ae, []int{2}
 }
 func (m *RunJobRequest) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_RunJobRequest.Unmarshal(m, b)
@@ -256,8 +262,8 @@ func (m *RunJobRequest) XXX_Unmarshal(b []byte) error {
 func (m *RunJobRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_RunJobRequest.Marshal(b, m, deterministic)
 }
-func (dst *RunJobRequest) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_RunJobRequest.Merge(dst, src)
+func (m *RunJobRequest) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_RunJobRequest.Merge(m, src)
 }
 func (m *RunJobRequest) XXX_Size() int {
 	return xxx_messageInfo_RunJobRequest.Size(m)
@@ -293,7 +299,7 @@ func (m *RunJobResponse) Reset()         { *m = RunJobResponse{} }
 func (m *RunJobResponse) String() string { return proto.CompactTextString(m) }
 func (*RunJobResponse) ProtoMessage()    {}
 func (*RunJobResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_job_api_1fb9125c215633d2, []int{3}
+	return fileDescriptor_97c7b84f742157ae, []int{3}
 }
 func (m *RunJobResponse) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_RunJobResponse.Unmarshal(m, b)
@@ -301,8 +307,8 @@ func (m *RunJobResponse) XXX_Unmarshal(b []byte) error {
 func (m *RunJobResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_RunJobResponse.Marshal(b, m, deterministic)
 }
-func (dst *RunJobResponse) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_RunJobResponse.Merge(dst, src)
+func (m *RunJobResponse) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_RunJobResponse.Merge(m, src)
 }
 func (m *RunJobResponse) XXX_Size() int {
 	return xxx_messageInfo_RunJobResponse.Size(m)
@@ -334,7 +340,7 @@ func (m *CancelJobRequest) Reset()         { *m = CancelJobRequest{} }
 func (m *CancelJobRequest) String() string { return proto.CompactTextString(m) }
 func (*CancelJobRequest) ProtoMessage()    {}
 func (*CancelJobRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_job_api_1fb9125c215633d2, []int{4}
+	return fileDescriptor_97c7b84f742157ae, []int{4}
 }
 func (m *CancelJobRequest) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_CancelJobRequest.Unmarshal(m, b)
@@ -342,8 +348,8 @@ func (m *CancelJobRequest) XXX_Unmarshal(b []byte) error {
 func (m *CancelJobRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_CancelJobRequest.Marshal(b, m, deterministic)
 }
-func (dst *CancelJobRequest) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_CancelJobRequest.Merge(dst, src)
+func (m *CancelJobRequest) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_CancelJobRequest.Merge(m, src)
 }
 func (m *CancelJobRequest) XXX_Size() int {
 	return xxx_messageInfo_CancelJobRequest.Size(m)
@@ -373,7 +379,7 @@ func (m *CancelJobResponse) Reset()         { *m = CancelJobResponse{} }
 func (m *CancelJobResponse) String() string { return proto.CompactTextString(m) }
 func (*CancelJobResponse) ProtoMessage()    {}
 func (*CancelJobResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_job_api_1fb9125c215633d2, []int{5}
+	return fileDescriptor_97c7b84f742157ae, []int{5}
 }
 func (m *CancelJobResponse) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_CancelJobResponse.Unmarshal(m, b)
@@ -381,8 +387,8 @@ func (m *CancelJobResponse) XXX_Unmarshal(b []byte) error {
 func (m *CancelJobResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_CancelJobResponse.Marshal(b, m, deterministic)
 }
-func (dst *CancelJobResponse) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_CancelJobResponse.Merge(dst, src)
+func (m *CancelJobResponse) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_CancelJobResponse.Merge(m, src)
 }
 func (m *CancelJobResponse) XXX_Size() int {
 	return xxx_messageInfo_CancelJobResponse.Size(m)
@@ -414,7 +420,7 @@ func (m *GetJobStateRequest) Reset()         { *m = GetJobStateRequest{} }
 func (m *GetJobStateRequest) String() string { return proto.CompactTextString(m) }
 func (*GetJobStateRequest) ProtoMessage()    {}
 func (*GetJobStateRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_job_api_1fb9125c215633d2, []int{6}
+	return fileDescriptor_97c7b84f742157ae, []int{6}
 }
 func (m *GetJobStateRequest) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_GetJobStateRequest.Unmarshal(m, b)
@@ -422,8 +428,8 @@ func (m *GetJobStateRequest) XXX_Unmarshal(b []byte) error {
 func (m *GetJobStateRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_GetJobStateRequest.Marshal(b, m, deterministic)
 }
-func (dst *GetJobStateRequest) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_GetJobStateRequest.Merge(dst, src)
+func (m *GetJobStateRequest) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_GetJobStateRequest.Merge(m, src)
 }
 func (m *GetJobStateRequest) XXX_Size() int {
 	return xxx_messageInfo_GetJobStateRequest.Size(m)
@@ -452,7 +458,7 @@ func (m *GetJobStateResponse) Reset()         { *m = GetJobStateResponse{} }
 func (m *GetJobStateResponse) String() string { return proto.CompactTextString(m) }
 func (*GetJobStateResponse) ProtoMessage()    {}
 func (*GetJobStateResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_job_api_1fb9125c215633d2, []int{7}
+	return fileDescriptor_97c7b84f742157ae, []int{7}
 }
 func (m *GetJobStateResponse) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_GetJobStateResponse.Unmarshal(m, b)
@@ -460,8 +466,8 @@ func (m *GetJobStateResponse) XXX_Unmarshal(b []byte) error {
 func (m *GetJobStateResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_GetJobStateResponse.Marshal(b, m, deterministic)
 }
-func (dst *GetJobStateResponse) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_GetJobStateResponse.Merge(dst, src)
+func (m *GetJobStateResponse) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_GetJobStateResponse.Merge(m, src)
 }
 func (m *GetJobStateResponse) XXX_Size() int {
 	return xxx_messageInfo_GetJobStateResponse.Size(m)
@@ -494,7 +500,7 @@ func (m *JobMessagesRequest) Reset()         { *m = JobMessagesRequest{} }
 func (m *JobMessagesRequest) String() string { return proto.CompactTextString(m) }
 func (*JobMessagesRequest) ProtoMessage()    {}
 func (*JobMessagesRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_job_api_1fb9125c215633d2, []int{8}
+	return fileDescriptor_97c7b84f742157ae, []int{8}
 }
 func (m *JobMessagesRequest) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_JobMessagesRequest.Unmarshal(m, b)
@@ -502,8 +508,8 @@ func (m *JobMessagesRequest) XXX_Unmarshal(b []byte) error {
 func (m *JobMessagesRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_JobMessagesRequest.Marshal(b, m, deterministic)
 }
-func (dst *JobMessagesRequest) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_JobMessagesRequest.Merge(dst, src)
+func (m *JobMessagesRequest) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_JobMessagesRequest.Merge(m, src)
 }
 func (m *JobMessagesRequest) XXX_Size() int {
 	return xxx_messageInfo_JobMessagesRequest.Size(m)
@@ -535,7 +541,7 @@ func (m *JobMessage) Reset()         { *m = JobMessage{} }
 func (m *JobMessage) String() string { return proto.CompactTextString(m) }
 func (*JobMessage) ProtoMessage()    {}
 func (*JobMessage) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_job_api_1fb9125c215633d2, []int{9}
+	return fileDescriptor_97c7b84f742157ae, []int{9}
 }
 func (m *JobMessage) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_JobMessage.Unmarshal(m, b)
@@ -543,8 +549,8 @@ func (m *JobMessage) XXX_Unmarshal(b []byte) error {
 func (m *JobMessage) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_JobMessage.Marshal(b, m, deterministic)
 }
-func (dst *JobMessage) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_JobMessage.Merge(dst, src)
+func (m *JobMessage) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_JobMessage.Merge(m, src)
 }
 func (m *JobMessage) XXX_Size() int {
 	return xxx_messageInfo_JobMessage.Size(m)
@@ -597,7 +603,7 @@ func (m *JobMessagesResponse) Reset()         { *m = JobMessagesResponse{} }
 func (m *JobMessagesResponse) String() string { return proto.CompactTextString(m) }
 func (*JobMessagesResponse) ProtoMessage()    {}
 func (*JobMessagesResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_job_api_1fb9125c215633d2, []int{10}
+	return fileDescriptor_97c7b84f742157ae, []int{10}
 }
 func (m *JobMessagesResponse) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_JobMessagesResponse.Unmarshal(m, b)
@@ -605,8 +611,8 @@ func (m *JobMessagesResponse) XXX_Unmarshal(b []byte) error {
 func (m *JobMessagesResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_JobMessagesResponse.Marshal(b, m, deterministic)
 }
-func (dst *JobMessagesResponse) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_JobMessagesResponse.Merge(dst, src)
+func (m *JobMessagesResponse) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_JobMessagesResponse.Merge(m, src)
 }
 func (m *JobMessagesResponse) XXX_Size() int {
 	return xxx_messageInfo_JobMessagesResponse.Size(m)
@@ -624,12 +630,14 @@ type isJobMessagesResponse_Response interface {
 type JobMessagesResponse_MessageResponse struct {
 	MessageResponse *JobMessage `protobuf:"bytes,1,opt,name=message_response,json=messageResponse,proto3,oneof"`
 }
+
 type JobMessagesResponse_StateResponse struct {
 	StateResponse *GetJobStateResponse `protobuf:"bytes,2,opt,name=state_response,json=stateResponse,proto3,oneof"`
 }
 
 func (*JobMessagesResponse_MessageResponse) isJobMessagesResponse_Response() {}
-func (*JobMessagesResponse_StateResponse) isJobMessagesResponse_Response()   {}
+
+func (*JobMessagesResponse_StateResponse) isJobMessagesResponse_Response() {}
 
 func (m *JobMessagesResponse) GetResponse() isJobMessagesResponse_Response {
 	if m != nil {
@@ -737,7 +745,7 @@ func (m *JobState) Reset()         { *m = JobState{} }
 func (m *JobState) String() string { return proto.CompactTextString(m) }
 func (*JobState) ProtoMessage()    {}
 func (*JobState) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_job_api_1fb9125c215633d2, []int{11}
+	return fileDescriptor_97c7b84f742157ae, []int{11}
 }
 func (m *JobState) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_JobState.Unmarshal(m, b)
@@ -745,8 +753,8 @@ func (m *JobState) XXX_Unmarshal(b []byte) error {
 func (m *JobState) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_JobState.Marshal(b, m, deterministic)
 }
-func (dst *JobState) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_JobState.Merge(dst, src)
+func (m *JobState) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_JobState.Merge(m, src)
 }
 func (m *JobState) XXX_Size() int {
 	return xxx_messageInfo_JobState.Size(m)
@@ -1080,9 +1088,9 @@ var _JobService_serviceDesc = grpc.ServiceDesc{
 	Metadata: "beam_job_api.proto",
 }
 
-func init() { proto.RegisterFile("beam_job_api.proto", fileDescriptor_beam_job_api_1fb9125c215633d2) }
+func init() { proto.RegisterFile("beam_job_api.proto", fileDescriptor_97c7b84f742157ae) }
 
-var fileDescriptor_beam_job_api_1fb9125c215633d2 = []byte{
+var fileDescriptor_97c7b84f742157ae = []byte{
 	// 954 bytes of a gzipped FileDescriptorProto
 	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x56, 0x4f, 0x6f, 0xe3, 0x44,
 	0x14, 0xaf, 0xd3, 0x34, 0x4d, 0x5e, 0xb7, 0x89, 0x3b, 0xa5, 0x6a, 0x36, 0x02, 0xb4, 0x58, 0x82,
diff --git a/sdks/go/pkg/beam/model/pipeline_v1/beam_runner_api.pb.go b/sdks/go/pkg/beam/model/pipeline_v1/beam_runner_api.pb.go
index d760f81b694..fec282fb8b3 100644
--- a/sdks/go/pkg/beam/model/pipeline_v1/beam_runner_api.pb.go
+++ b/sdks/go/pkg/beam/model/pipeline_v1/beam_runner_api.pb.go
@@ -3,11 +3,13 @@
 
 package pipeline_v1
 
-import proto "github.com/golang/protobuf/proto"
-import fmt "fmt"
-import math "math"
-import descriptor "github.com/golang/protobuf/protoc-gen-go/descriptor"
-import any "github.com/golang/protobuf/ptypes/any"
+import (
+	fmt "fmt"
+	proto "github.com/golang/protobuf/proto"
+	descriptor "github.com/golang/protobuf/protoc-gen-go/descriptor"
+	any "github.com/golang/protobuf/ptypes/any"
+	math "math"
+)
 
 // Reference imports to suppress errors if they are not otherwise used.
 var _ = proto.Marshal
@@ -55,6 +57,7 @@ var StandardPTransforms_Primitives_name = map[int32]string{
 	5: "TEST_STREAM",
 	6: "MAP_WINDOWS",
 }
+
 var StandardPTransforms_Primitives_value = map[string]int32{
 	"PAR_DO":         0,
 	"FLATTEN":        1,
@@ -68,8 +71,9 @@ var StandardPTransforms_Primitives_value = map[string]int32{
 func (x StandardPTransforms_Primitives) String() string {
 	return proto.EnumName(StandardPTransforms_Primitives_name, int32(x))
 }
+
 func (StandardPTransforms_Primitives) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_92025e5caa638397, []int{4, 0}
+	return fileDescriptor_cf57597c3a9659a9, []int{4, 0}
 }
 
 type StandardPTransforms_DeprecatedPrimitives int32
@@ -87,6 +91,7 @@ var StandardPTransforms_DeprecatedPrimitives_name = map[int32]string{
 	0: "READ",
 	1: "CREATE_VIEW",
 }
+
 var StandardPTransforms_DeprecatedPrimitives_value = map[string]int32{
 	"READ":        0,
 	"CREATE_VIEW": 1,
@@ -95,8 +100,9 @@ var StandardPTransforms_DeprecatedPrimitives_value = map[string]int32{
 func (x StandardPTransforms_DeprecatedPrimitives) String() string {
 	return proto.EnumName(StandardPTransforms_DeprecatedPrimitives_name, int32(x))
 }
+
 func (StandardPTransforms_DeprecatedPrimitives) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_92025e5caa638397, []int{4, 1}
+	return fileDescriptor_cf57597c3a9659a9, []int{4, 1}
 }
 
 type StandardPTransforms_Composites int32
@@ -112,37 +118,32 @@ const (
 	// each of CombineComponents.
 	// Payload: CombinePayload
 	StandardPTransforms_COMBINE_GLOBALLY StandardPTransforms_Composites = 1
-	// Represents the Combine.groupedValues() operation.
-	// If this is produced by an SDK, it is assumed that the SDK understands
-	// each of CombineComponents.
-	// Payload: CombinePayload
-	StandardPTransforms_COMBINE_GROUPED_VALUES StandardPTransforms_Composites = 2
 	// Represents the Reshuffle operation.
-	StandardPTransforms_RESHUFFLE StandardPTransforms_Composites = 3
+	StandardPTransforms_RESHUFFLE StandardPTransforms_Composites = 2
 	// Less well-known. Payload: WriteFilesPayload.
-	StandardPTransforms_WRITE_FILES StandardPTransforms_Composites = 4
+	StandardPTransforms_WRITE_FILES StandardPTransforms_Composites = 3
 )
 
 var StandardPTransforms_Composites_name = map[int32]string{
 	0: "COMBINE_PER_KEY",
 	1: "COMBINE_GLOBALLY",
-	2: "COMBINE_GROUPED_VALUES",
-	3: "RESHUFFLE",
-	4: "WRITE_FILES",
+	2: "RESHUFFLE",
+	3: "WRITE_FILES",
 }
+
 var StandardPTransforms_Composites_value = map[string]int32{
-	"COMBINE_PER_KEY":        0,
-	"COMBINE_GLOBALLY":       1,
-	"COMBINE_GROUPED_VALUES": 2,
-	"RESHUFFLE":              3,
-	"WRITE_FILES":            4,
+	"COMBINE_PER_KEY":  0,
+	"COMBINE_GLOBALLY": 1,
+	"RESHUFFLE":        2,
+	"WRITE_FILES":      3,
 }
 
 func (x StandardPTransforms_Composites) String() string {
 	return proto.EnumName(StandardPTransforms_Composites_name, int32(x))
 }
+
 func (StandardPTransforms_Composites) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_92025e5caa638397, []int{4, 2}
+	return fileDescriptor_cf57597c3a9659a9, []int{4, 2}
 }
 
 // Payload for all of these: CombinePayload
@@ -155,15 +156,23 @@ const (
 	// Represents the Pre-Combine part of a lifted Combine Per Key, as described
 	// in the following document:
 	// https://s.apache.org/beam-runner-api-combine-model#heading=h.ta0g6ase8z07
+	// Payload: CombinePayload
 	StandardPTransforms_COMBINE_PER_KEY_PRECOMBINE StandardPTransforms_CombineComponents = 3
 	// Represents the Merge Accumulators part of a lifted Combine Per Key, as
 	// described in the following document:
 	// https://s.apache.org/beam-runner-api-combine-model#heading=h.jco9rvatld5m
+	// Payload: CombinePayload
 	StandardPTransforms_COMBINE_PER_KEY_MERGE_ACCUMULATORS StandardPTransforms_CombineComponents = 4
 	// Represents the Extract Outputs part of a lifted Combine Per Key, as
 	// described in the following document:
 	// https://s.apache.org/beam-runner-api-combine-model#heading=h.i9i6p8gtl6ku
+	// Payload: CombinePayload
 	StandardPTransforms_COMBINE_PER_KEY_EXTRACT_OUTPUTS StandardPTransforms_CombineComponents = 5
+	// Represents the Combine Grouped Values transform, as described in the
+	// following document:
+	// https://s.apache.org/beam-runner-api-combine-model#heading=h.aj86ew4v1wk
+	// Payload: CombinePayload
+	StandardPTransforms_COMBINE_GROUPED_VALUES StandardPTransforms_CombineComponents = 6
 )
 
 var StandardPTransforms_CombineComponents_name = map[int32]string{
@@ -173,7 +182,9 @@ var StandardPTransforms_CombineComponents_name = map[int32]string{
 	3: "COMBINE_PER_KEY_PRECOMBINE",
 	4: "COMBINE_PER_KEY_MERGE_ACCUMULATORS",
 	5: "COMBINE_PER_KEY_EXTRACT_OUTPUTS",
+	6: "COMBINE_GROUPED_VALUES",
 }
+
 var StandardPTransforms_CombineComponents_value = map[string]int32{
 	"COMBINE_PGBKCV":                     0,
 	"COMBINE_MERGE_ACCUMULATORS":         1,
@@ -181,40 +192,61 @@ var StandardPTransforms_CombineComponents_value = map[string]int32{
 	"COMBINE_PER_KEY_PRECOMBINE":         3,
 	"COMBINE_PER_KEY_MERGE_ACCUMULATORS": 4,
 	"COMBINE_PER_KEY_EXTRACT_OUTPUTS":    5,
+	"COMBINE_GROUPED_VALUES":             6,
 }
 
 func (x StandardPTransforms_CombineComponents) String() string {
 	return proto.EnumName(StandardPTransforms_CombineComponents_name, int32(x))
 }
+
 func (StandardPTransforms_CombineComponents) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_92025e5caa638397, []int{4, 3}
+	return fileDescriptor_cf57597c3a9659a9, []int{4, 3}
 }
 
 // Payload for all of these: ParDoPayload containing the user's SDF
 type StandardPTransforms_SplittableParDoComponents int32
 
 const (
-	StandardPTransforms_PAIR_WITH_RESTRICTION  StandardPTransforms_SplittableParDoComponents = 0
-	StandardPTransforms_SPLIT_RESTRICTION      StandardPTransforms_SplittableParDoComponents = 1
+	// Pairs the input element with its initial restriction.
+	// Input: element; output: KV(element, restriction).
+	StandardPTransforms_PAIR_WITH_RESTRICTION StandardPTransforms_SplittableParDoComponents = 0
+	// Splits the restriction inside an element/restriction pair.
+	// Input: KV(element, restriction); output: KV(element, restriction).
+	StandardPTransforms_SPLIT_RESTRICTION StandardPTransforms_SplittableParDoComponents = 1
+	// Applies the DoFn to every element/restriction pair in a uniquely keyed
+	// collection, in a splittable fashion.
+	// Input: KV(bytes, KV(element, restriction)); output: DoFn's output.
+	// The first "bytes" is an opaque unique key using the standard bytes coder.
+	// Typically a runner would rewrite this into a runner-specific grouping
+	// operation supporting state and timers, followed by PROCESS_ELEMENTS,
+	// with some runner-specific glue code in between.
 	StandardPTransforms_PROCESS_KEYED_ELEMENTS StandardPTransforms_SplittableParDoComponents = 2
+	// Like PROCESS_KEYED_ELEMENTS, but without the unique key - just elements
+	// and restrictions.
+	// Input: KV(element, restriction); output: DoFn's output.
+	StandardPTransforms_PROCESS_ELEMENTS StandardPTransforms_SplittableParDoComponents = 3
 )
 
 var StandardPTransforms_SplittableParDoComponents_name = map[int32]string{
 	0: "PAIR_WITH_RESTRICTION",
 	1: "SPLIT_RESTRICTION",
 	2: "PROCESS_KEYED_ELEMENTS",
+	3: "PROCESS_ELEMENTS",
 }
+
 var StandardPTransforms_SplittableParDoComponents_value = map[string]int32{
 	"PAIR_WITH_RESTRICTION":  0,
 	"SPLIT_RESTRICTION":      1,
 	"PROCESS_KEYED_ELEMENTS": 2,
+	"PROCESS_ELEMENTS":       3,
 }
 
 func (x StandardPTransforms_SplittableParDoComponents) String() string {
 	return proto.EnumName(StandardPTransforms_SplittableParDoComponents_name, int32(x))
 }
+
 func (StandardPTransforms_SplittableParDoComponents) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_92025e5caa638397, []int{4, 4}
+	return fileDescriptor_cf57597c3a9659a9, []int{4, 4}
 }
 
 type StandardSideInputTypes_Enum int32
@@ -228,6 +260,7 @@ var StandardSideInputTypes_Enum_name = map[int32]string{
 	0: "ITERABLE",
 	1: "MULTIMAP",
 }
+
 var StandardSideInputTypes_Enum_value = map[string]int32{
 	"ITERABLE": 0,
 	"MULTIMAP": 1,
@@ -236,8 +269,9 @@ var StandardSideInputTypes_Enum_value = map[string]int32{
 func (x StandardSideInputTypes_Enum) String() string {
 	return proto.EnumName(StandardSideInputTypes_Enum_name, int32(x))
 }
+
 func (StandardSideInputTypes_Enum) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_92025e5caa638397, []int{5, 0}
+	return fileDescriptor_cf57597c3a9659a9, []int{5, 0}
 }
 
 type Parameter_Type_Enum int32
@@ -255,6 +289,7 @@ var Parameter_Type_Enum_name = map[int32]string{
 	2: "PIPELINE_OPTIONS",
 	3: "RESTRICTION_TRACKER",
 }
+
 var Parameter_Type_Enum_value = map[string]int32{
 	"UNSPECIFIED":         0,
 	"WINDOW":              1,
@@ -265,8 +300,9 @@ var Parameter_Type_Enum_value = map[string]int32{
 func (x Parameter_Type_Enum) String() string {
 	return proto.EnumName(Parameter_Type_Enum_name, int32(x))
 }
+
 func (Parameter_Type_Enum) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_92025e5caa638397, []int{9, 0, 0}
+	return fileDescriptor_cf57597c3a9659a9, []int{9, 0, 0}
 }
 
 type IsBounded_Enum int32
@@ -282,6 +318,7 @@ var IsBounded_Enum_name = map[int32]string{
 	1: "UNBOUNDED",
 	2: "BOUNDED",
 }
+
 var IsBounded_Enum_value = map[string]int32{
 	"UNSPECIFIED": 0,
 	"UNBOUNDED":   1,
@@ -291,8 +328,9 @@ var IsBounded_Enum_value = map[string]int32{
 func (x IsBounded_Enum) String() string {
 	return proto.EnumName(IsBounded_Enum_name, int32(x))
 }
+
 func (IsBounded_Enum) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_92025e5caa638397, []int{17, 0}
+	return fileDescriptor_cf57597c3a9659a9, []int{17, 0}
 }
 
 type StandardCoders_Enum int32
@@ -307,8 +345,24 @@ const (
 	// Encodes an iterable of elements.
 	// Components: Coder for a single element.
 	StandardCoders_ITERABLE StandardCoders_Enum = 3
-	// Components: None
-	StandardCoders_TIMESTAMP StandardCoders_Enum = 4
+	// Encodes a timer containing a timestamp and a user specified payload.
+	// The encoding is represented as: timestamp payload
+	//   timestamp - a big endian 8 byte integer representing millis-since-epoch.
+	//     The encoded representation is shifted so that the byte representation of
+	//     negative values are lexicographically ordered before the byte representation
+	//     of positive values. This is typically done by subtracting -9223372036854775808
+	//     from the value and encoding it as a signed big endian integer. Example values:
+	//
+	//     -9223372036854775808: 00 00 00 00 00 00 00 00
+	//                     -255: 7F FF FF FF FF FF FF 01
+	//                       -1: 7F FF FF FF FF FF FF FF
+	//                        0: 80 00 00 00 00 00 00 00
+	//                        1: 80 00 00 00 00 00 00 01
+	//                      256: 80 00 00 00 00 00 01 00
+	//      9223372036854775807: FF FF FF FF FF FF FF FF
+	//   payload - user defined data, uses the component coder
+	// Components: Coder for the payload.
+	StandardCoders_TIMER StandardCoders_Enum = 4
 	// Components: None
 	StandardCoders_INTERVAL_WINDOW StandardCoders_Enum = 5
 	// Components: The coder to attach a length prefix to
@@ -326,18 +380,19 @@ var StandardCoders_Enum_name = map[int32]string{
 	1: "KV",
 	2: "VARINT",
 	3: "ITERABLE",
-	4: "TIMESTAMP",
+	4: "TIMER",
 	5: "INTERVAL_WINDOW",
 	6: "LENGTH_PREFIX",
 	7: "GLOBAL_WINDOW",
 	8: "WINDOWED_VALUE",
 }
+
 var StandardCoders_Enum_value = map[string]int32{
 	"BYTES":           0,
 	"KV":              1,
 	"VARINT":          2,
 	"ITERABLE":        3,
-	"TIMESTAMP":       4,
+	"TIMER":           4,
 	"INTERVAL_WINDOW": 5,
 	"LENGTH_PREFIX":   6,
 	"GLOBAL_WINDOW":   7,
@@ -347,8 +402,9 @@ var StandardCoders_Enum_value = map[string]int32{
 func (x StandardCoders_Enum) String() string {
 	return proto.EnumName(StandardCoders_Enum_name, int32(x))
 }
+
 func (StandardCoders_Enum) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_92025e5caa638397, []int{24, 0}
+	return fileDescriptor_cf57597c3a9659a9, []int{24, 0}
 }
 
 type MergeStatus_Enum int32
@@ -374,6 +430,7 @@ var MergeStatus_Enum_name = map[int32]string{
 	2: "NEEDS_MERGE",
 	3: "ALREADY_MERGED",
 }
+
 var MergeStatus_Enum_value = map[string]int32{
 	"UNSPECIFIED":    0,
 	"NON_MERGING":    1,
@@ -384,8 +441,9 @@ var MergeStatus_Enum_value = map[string]int32{
 func (x MergeStatus_Enum) String() string {
 	return proto.EnumName(MergeStatus_Enum_name, int32(x))
 }
+
 func (MergeStatus_Enum) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_92025e5caa638397, []int{26, 0}
+	return fileDescriptor_cf57597c3a9659a9, []int{26, 0}
 }
 
 type AccumulationMode_Enum int32
@@ -403,6 +461,7 @@ var AccumulationMode_Enum_name = map[int32]string{
 	1: "DISCARDING",
 	2: "ACCUMULATING",
 }
+
 var AccumulationMode_Enum_value = map[string]int32{
 	"UNSPECIFIED":  0,
 	"DISCARDING":   1,
@@ -412,8 +471,9 @@ var AccumulationMode_Enum_value = map[string]int32{
 func (x AccumulationMode_Enum) String() string {
 	return proto.EnumName(AccumulationMode_Enum_name, int32(x))
 }
+
 func (AccumulationMode_Enum) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_92025e5caa638397, []int{27, 0}
+	return fileDescriptor_cf57597c3a9659a9, []int{27, 0}
 }
 
 type ClosingBehavior_Enum int32
@@ -432,6 +492,7 @@ var ClosingBehavior_Enum_name = map[int32]string{
 	1: "EMIT_ALWAYS",
 	2: "EMIT_IF_NONEMPTY",
 }
+
 var ClosingBehavior_Enum_value = map[string]int32{
 	"UNSPECIFIED":      0,
 	"EMIT_ALWAYS":      1,
@@ -441,8 +502,9 @@ var ClosingBehavior_Enum_value = map[string]int32{
 func (x ClosingBehavior_Enum) String() string {
 	return proto.EnumName(ClosingBehavior_Enum_name, int32(x))
 }
+
 func (ClosingBehavior_Enum) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_92025e5caa638397, []int{28, 0}
+	return fileDescriptor_cf57597c3a9659a9, []int{28, 0}
 }
 
 type OnTimeBehavior_Enum int32
@@ -461,6 +523,7 @@ var OnTimeBehavior_Enum_name = map[int32]string{
 	1: "FIRE_ALWAYS",
 	2: "FIRE_IF_NONEMPTY",
 }
+
 var OnTimeBehavior_Enum_value = map[string]int32{
 	"UNSPECIFIED":      0,
 	"FIRE_ALWAYS":      1,
@@ -470,8 +533,9 @@ var OnTimeBehavior_Enum_value = map[string]int32{
 func (x OnTimeBehavior_Enum) String() string {
 	return proto.EnumName(OnTimeBehavior_Enum_name, int32(x))
 }
+
 func (OnTimeBehavior_Enum) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_92025e5caa638397, []int{29, 0}
+	return fileDescriptor_cf57597c3a9659a9, []int{29, 0}
 }
 
 type OutputTime_Enum int32
@@ -494,6 +558,7 @@ var OutputTime_Enum_name = map[int32]string{
 	2: "LATEST_IN_PANE",
 	3: "EARLIEST_IN_PANE",
 }
+
 var OutputTime_Enum_value = map[string]int32{
 	"UNSPECIFIED":      0,
 	"END_OF_WINDOW":    1,
@@ -504,8 +569,9 @@ var OutputTime_Enum_value = map[string]int32{
 func (x OutputTime_Enum) String() string {
 	return proto.EnumName(OutputTime_Enum_name, int32(x))
 }
+
 func (OutputTime_Enum) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_92025e5caa638397, []int{30, 0}
+	return fileDescriptor_cf57597c3a9659a9, []int{30, 0}
 }
 
 type TimeDomain_Enum int32
@@ -531,6 +597,7 @@ var TimeDomain_Enum_name = map[int32]string{
 	2: "PROCESSING_TIME",
 	3: "SYNCHRONIZED_PROCESSING_TIME",
 }
+
 var TimeDomain_Enum_value = map[string]int32{
 	"UNSPECIFIED":                  0,
 	"EVENT_TIME":                   1,
@@ -541,8 +608,37 @@ var TimeDomain_Enum_value = map[string]int32{
 func (x TimeDomain_Enum) String() string {
 	return proto.EnumName(TimeDomain_Enum_name, int32(x))
 }
+
 func (TimeDomain_Enum) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_92025e5caa638397, []int{31, 0}
+	return fileDescriptor_cf57597c3a9659a9, []int{31, 0}
+}
+
+type StandardEnvironments_Environments int32
+
+const (
+	StandardEnvironments_DOCKER   StandardEnvironments_Environments = 0
+	StandardEnvironments_PROCESS  StandardEnvironments_Environments = 1
+	StandardEnvironments_EXTERNAL StandardEnvironments_Environments = 2
+)
+
+var StandardEnvironments_Environments_name = map[int32]string{
+	0: "DOCKER",
+	1: "PROCESS",
+	2: "EXTERNAL",
+}
+
+var StandardEnvironments_Environments_value = map[string]int32{
+	"DOCKER":   0,
+	"PROCESS":  1,
+	"EXTERNAL": 2,
+}
+
+func (x StandardEnvironments_Environments) String() string {
+	return proto.EnumName(StandardEnvironments_Environments_name, int32(x))
+}
+
+func (StandardEnvironments_Environments) EnumDescriptor() ([]byte, []int) {
+	return fileDescriptor_cf57597c3a9659a9, []int{36, 0}
 }
 
 type DisplayData_Type_Enum int32
@@ -568,6 +664,7 @@ var DisplayData_Type_Enum_name = map[int32]string{
 	6: "DURATION",
 	7: "JAVA_CLASS",
 }
+
 var DisplayData_Type_Enum_value = map[string]int32{
 	"UNSPECIFIED": 0,
 	"STRING":      1,
@@ -582,8 +679,9 @@ var DisplayData_Type_Enum_value = map[string]int32{
 func (x DisplayData_Type_Enum) String() string {
 	return proto.EnumName(DisplayData_Type_Enum_name, int32(x))
 }
+
 func (DisplayData_Type_Enum) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_92025e5caa638397, []int{38, 2, 0}
+	return fileDescriptor_cf57597c3a9659a9, []int{41, 2, 0}
 }
 
 // A set of mappings from id to message. This is included as an optional field
@@ -608,7 +706,7 @@ func (m *Components) Reset()         { *m = Components{} }
 func (m *Components) String() string { return proto.CompactTextString(m) }
 func (*Components) ProtoMessage()    {}
 func (*Components) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_92025e5caa638397, []int{0}
+	return fileDescriptor_cf57597c3a9659a9, []int{0}
 }
 func (m *Components) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Components.Unmarshal(m, b)
@@ -616,8 +714,8 @@ func (m *Components) XXX_Unmarshal(b []byte) error {
 func (m *Components) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_Components.Marshal(b, m, deterministic)
 }
-func (dst *Components) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Components.Merge(dst, src)
+func (m *Components) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_Components.Merge(m, src)
 }
 func (m *Components) XXX_Size() int {
 	return xxx_messageInfo_Components.Size(m)
@@ -697,7 +795,7 @@ func (m *MessageWithComponents) Reset()         { *m = MessageWithComponents{} }
 func (m *MessageWithComponents) String() string { return proto.CompactTextString(m) }
 func (*MessageWithComponents) ProtoMessage()    {}
 func (*MessageWithComponents) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_92025e5caa638397, []int{1}
+	return fileDescriptor_cf57597c3a9659a9, []int{1}
 }
 func (m *MessageWithComponents) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_MessageWithComponents.Unmarshal(m, b)
@@ -705,8 +803,8 @@ func (m *MessageWithComponents) XXX_Unmarshal(b []byte) error {
 func (m *MessageWithComponents) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_MessageWithComponents.Marshal(b, m, deterministic)
 }
-func (dst *MessageWithComponents) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_MessageWithComponents.Merge(dst, src)
+func (m *MessageWithComponents) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_MessageWithComponents.Merge(m, src)
 }
 func (m *MessageWithComponents) XXX_Size() int {
 	return xxx_messageInfo_MessageWithComponents.Size(m)
@@ -717,6 +815,13 @@ func (m *MessageWithComponents) XXX_DiscardUnknown() {
 
 var xxx_messageInfo_MessageWithComponents proto.InternalMessageInfo
 
+func (m *MessageWithComponents) GetComponents() *Components {
+	if m != nil {
+		return m.Components
+	}
+	return nil
+}
+
 type isMessageWithComponents_Root interface {
 	isMessageWithComponents_Root()
 }
@@ -724,48 +829,68 @@ type isMessageWithComponents_Root interface {
 type MessageWithComponents_Coder struct {
 	Coder *Coder `protobuf:"bytes,2,opt,name=coder,proto3,oneof"`
 }
+
 type MessageWithComponents_CombinePayload struct {
 	CombinePayload *CombinePayload `protobuf:"bytes,3,opt,name=combine_payload,json=combinePayload,proto3,oneof"`
 }
+
 type MessageWithComponents_SdkFunctionSpec struct {
 	SdkFunctionSpec *SdkFunctionSpec `protobuf:"bytes,4,opt,name=sdk_function_spec,json=sdkFunctionSpec,proto3,oneof"`
 }
+
 type MessageWithComponents_ParDoPayload struct {
 	ParDoPayload *ParDoPayload `protobuf:"bytes,6,opt,name=par_do_payload,json=parDoPayload,proto3,oneof"`
 }
+
 type MessageWithComponents_Ptransform struct {
 	Ptransform *PTransform `protobuf:"bytes,7,opt,name=ptransform,proto3,oneof"`
 }
+
 type MessageWithComponents_Pcollection struct {
 	Pcollection *PCollection `protobuf:"bytes,8,opt,name=pcollection,proto3,oneof"`
 }
+
 type MessageWithComponents_ReadPayload struct {
 	ReadPayload *ReadPayload `protobuf:"bytes,9,opt,name=read_payload,json=readPayload,proto3,oneof"`
 }
+
 type MessageWithComponents_SideInput struct {
 	SideInput *SideInput `protobuf:"bytes,11,opt,name=side_input,json=sideInput,proto3,oneof"`
 }
+
 type MessageWithComponents_WindowIntoPayload struct {
 	WindowIntoPayload *WindowIntoPayload `protobuf:"bytes,12,opt,name=window_into_payload,json=windowIntoPayload,proto3,oneof"`
 }
+
 type MessageWithComponents_WindowingStrategy struct {
 	WindowingStrategy *WindowingStrategy `protobuf:"bytes,13,opt,name=windowing_strategy,json=windowingStrategy,proto3,oneof"`
 }
+
 type MessageWithComponents_FunctionSpec struct {
 	FunctionSpec *FunctionSpec `protobuf:"bytes,14,opt,name=function_spec,json=functionSpec,proto3,oneof"`
 }
 
-func (*MessageWithComponents_Coder) isMessageWithComponents_Root()             {}
-func (*MessageWithComponents_CombinePayload) isMessageWithComponents_Root()    {}
-func (*MessageWithComponents_SdkFunctionSpec) isMessageWithComponents_Root()   {}
-func (*MessageWithComponents_ParDoPayload) isMessageWithComponents_Root()      {}
-func (*MessageWithComponents_Ptransform) isMessageWithComponents_Root()        {}
-func (*MessageWithComponents_Pcollection) isMessageWithComponents_Root()       {}
-func (*MessageWithComponents_ReadPayload) isMessageWithComponents_Root()       {}
-func (*MessageWithComponents_SideInput) isMessageWithComponents_Root()         {}
+func (*MessageWithComponents_Coder) isMessageWithComponents_Root() {}
+
+func (*MessageWithComponents_CombinePayload) isMessageWithComponents_Root() {}
+
+func (*MessageWithComponents_SdkFunctionSpec) isMessageWithComponents_Root() {}
+
+func (*MessageWithComponents_ParDoPayload) isMessageWithComponents_Root() {}
+
+func (*MessageWithComponents_Ptransform) isMessageWithComponents_Root() {}
+
+func (*MessageWithComponents_Pcollection) isMessageWithComponents_Root() {}
+
+func (*MessageWithComponents_ReadPayload) isMessageWithComponents_Root() {}
+
+func (*MessageWithComponents_SideInput) isMessageWithComponents_Root() {}
+
 func (*MessageWithComponents_WindowIntoPayload) isMessageWithComponents_Root() {}
+
 func (*MessageWithComponents_WindowingStrategy) isMessageWithComponents_Root() {}
-func (*MessageWithComponents_FunctionSpec) isMessageWithComponents_Root()      {}
+
+func (*MessageWithComponents_FunctionSpec) isMessageWithComponents_Root() {}
 
 func (m *MessageWithComponents) GetRoot() isMessageWithComponents_Root {
 	if m != nil {
@@ -774,13 +899,6 @@ func (m *MessageWithComponents) GetRoot() isMessageWithComponents_Root {
 	return nil
 }
 
-func (m *MessageWithComponents) GetComponents() *Components {
-	if m != nil {
-		return m.Components
-	}
-	return nil
-}
-
 func (m *MessageWithComponents) GetCoder() *Coder {
 	if x, ok := m.GetRoot().(*MessageWithComponents_Coder); ok {
 		return x.Coder
@@ -1132,7 +1250,7 @@ func (m *Pipeline) Reset()         { *m = Pipeline{} }
 func (m *Pipeline) String() string { return proto.CompactTextString(m) }
 func (*Pipeline) ProtoMessage()    {}
 func (*Pipeline) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_92025e5caa638397, []int{2}
+	return fileDescriptor_cf57597c3a9659a9, []int{2}
 }
 func (m *Pipeline) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Pipeline.Unmarshal(m, b)
@@ -1140,8 +1258,8 @@ func (m *Pipeline) XXX_Unmarshal(b []byte) error {
 func (m *Pipeline) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_Pipeline.Marshal(b, m, deterministic)
 }
-func (dst *Pipeline) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Pipeline.Merge(dst, src)
+func (m *Pipeline) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_Pipeline.Merge(m, src)
 }
 func (m *Pipeline) XXX_Size() int {
 	return xxx_messageInfo_Pipeline.Size(m)
@@ -1246,7 +1364,7 @@ func (m *PTransform) Reset()         { *m = PTransform{} }
 func (m *PTransform) String() string { return proto.CompactTextString(m) }
 func (*PTransform) ProtoMessage()    {}
 func (*PTransform) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_92025e5caa638397, []int{3}
+	return fileDescriptor_cf57597c3a9659a9, []int{3}
 }
 func (m *PTransform) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_PTransform.Unmarshal(m, b)
@@ -1254,8 +1372,8 @@ func (m *PTransform) XXX_Unmarshal(b []byte) error {
 func (m *PTransform) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_PTransform.Marshal(b, m, deterministic)
 }
-func (dst *PTransform) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_PTransform.Merge(dst, src)
+func (m *PTransform) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_PTransform.Merge(m, src)
 }
 func (m *PTransform) XXX_Size() int {
 	return xxx_messageInfo_PTransform.Size(m)
@@ -1318,7 +1436,7 @@ func (m *StandardPTransforms) Reset()         { *m = StandardPTransforms{} }
 func (m *StandardPTransforms) String() string { return proto.CompactTextString(m) }
 func (*StandardPTransforms) ProtoMessage()    {}
 func (*StandardPTransforms) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_92025e5caa638397, []int{4}
+	return fileDescriptor_cf57597c3a9659a9, []int{4}
 }
 func (m *StandardPTransforms) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_StandardPTransforms.Unmarshal(m, b)
@@ -1326,8 +1444,8 @@ func (m *StandardPTransforms) XXX_Unmarshal(b []byte) error {
 func (m *StandardPTransforms) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_StandardPTransforms.Marshal(b, m, deterministic)
 }
-func (dst *StandardPTransforms) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_StandardPTransforms.Merge(dst, src)
+func (m *StandardPTransforms) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_StandardPTransforms.Merge(m, src)
 }
 func (m *StandardPTransforms) XXX_Size() int {
 	return xxx_messageInfo_StandardPTransforms.Size(m)
@@ -1348,7 +1466,7 @@ func (m *StandardSideInputTypes) Reset()         { *m = StandardSideInputTypes{}
 func (m *StandardSideInputTypes) String() string { return proto.CompactTextString(m) }
 func (*StandardSideInputTypes) ProtoMessage()    {}
 func (*StandardSideInputTypes) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_92025e5caa638397, []int{5}
+	return fileDescriptor_cf57597c3a9659a9, []int{5}
 }
 func (m *StandardSideInputTypes) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_StandardSideInputTypes.Unmarshal(m, b)
@@ -1356,8 +1474,8 @@ func (m *StandardSideInputTypes) XXX_Unmarshal(b []byte) error {
 func (m *StandardSideInputTypes) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_StandardSideInputTypes.Marshal(b, m, deterministic)
 }
-func (dst *StandardSideInputTypes) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_StandardSideInputTypes.Merge(dst, src)
+func (m *StandardSideInputTypes) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_StandardSideInputTypes.Merge(m, src)
 }
 func (m *StandardSideInputTypes) XXX_Size() int {
 	return xxx_messageInfo_StandardSideInputTypes.Size(m)
@@ -1398,7 +1516,7 @@ func (m *PCollection) Reset()         { *m = PCollection{} }
 func (m *PCollection) String() string { return proto.CompactTextString(m) }
 func (*PCollection) ProtoMessage()    {}
 func (*PCollection) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_92025e5caa638397, []int{6}
+	return fileDescriptor_cf57597c3a9659a9, []int{6}
 }
 func (m *PCollection) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_PCollection.Unmarshal(m, b)
@@ -1406,8 +1524,8 @@ func (m *PCollection) XXX_Unmarshal(b []byte) error {
 func (m *PCollection) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_PCollection.Marshal(b, m, deterministic)
 }
-func (dst *PCollection) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_PCollection.Merge(dst, src)
+func (m *PCollection) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_PCollection.Merge(m, src)
 }
 func (m *PCollection) XXX_Size() int {
 	return xxx_messageInfo_PCollection.Size(m)
@@ -1464,7 +1582,7 @@ type ExecutableStagePayload struct {
 	// (Required) Input PCollection id. This must be present as a value in the inputs of any
 	// PTransform the ExecutableStagePayload is the payload of.
 	Input string `protobuf:"bytes,2,opt,name=input,proto3" json:"input,omitempty"`
-	// The side inputs required for this executable stage. Each Side Input of each PTransform within
+	// The side inputs required for this executable stage. Each side input of each PTransform within
 	// this ExecutableStagePayload must be represented within this field.
 	SideInputs []*ExecutableStagePayload_SideInputId `protobuf:"bytes,3,rep,name=side_inputs,json=sideInputs,proto3" json:"side_inputs,omitempty"`
 	// PTransform ids contained within this executable stage. This must contain at least one
@@ -1475,17 +1593,23 @@ type ExecutableStagePayload struct {
 	Outputs []string `protobuf:"bytes,5,rep,name=outputs,proto3" json:"outputs,omitempty"`
 	// (Required) The components for the Executable Stage. This must contain all of the Transforms
 	// in transforms, and the closure of all of the components they recognize.
-	Components           *Components `protobuf:"bytes,6,opt,name=components,proto3" json:"components,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}    `json:"-"`
-	XXX_unrecognized     []byte      `json:"-"`
-	XXX_sizecache        int32       `json:"-"`
+	Components *Components `protobuf:"bytes,6,opt,name=components,proto3" json:"components,omitempty"`
+	// The user states required for this executable stage. Each user state of each PTransform within
+	// this ExecutableStagePayload must be represented within this field.
+	UserStates []*ExecutableStagePayload_UserStateId `protobuf:"bytes,7,rep,name=user_states,json=userStates,proto3" json:"user_states,omitempty"`
+	// The timers required for this executable stage. Each timer of each PTransform within
+	// this ExecutableStagePayload must be represented within this field.
+	Timers               []*ExecutableStagePayload_TimerId `protobuf:"bytes,8,rep,name=timers,proto3" json:"timers,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}                          `json:"-"`
+	XXX_unrecognized     []byte                            `json:"-"`
+	XXX_sizecache        int32                             `json:"-"`
 }
 
 func (m *ExecutableStagePayload) Reset()         { *m = ExecutableStagePayload{} }
 func (m *ExecutableStagePayload) String() string { return proto.CompactTextString(m) }
 func (*ExecutableStagePayload) ProtoMessage()    {}
 func (*ExecutableStagePayload) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_92025e5caa638397, []int{7}
+	return fileDescriptor_cf57597c3a9659a9, []int{7}
 }
 func (m *ExecutableStagePayload) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_ExecutableStagePayload.Unmarshal(m, b)
@@ -1493,8 +1617,8 @@ func (m *ExecutableStagePayload) XXX_Unmarshal(b []byte) error {
 func (m *ExecutableStagePayload) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_ExecutableStagePayload.Marshal(b, m, deterministic)
 }
-func (dst *ExecutableStagePayload) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ExecutableStagePayload.Merge(dst, src)
+func (m *ExecutableStagePayload) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_ExecutableStagePayload.Merge(m, src)
 }
 func (m *ExecutableStagePayload) XXX_Size() int {
 	return xxx_messageInfo_ExecutableStagePayload.Size(m)
@@ -1547,6 +1671,20 @@ func (m *ExecutableStagePayload) GetComponents() *Components {
 	return nil
 }
 
+func (m *ExecutableStagePayload) GetUserStates() []*ExecutableStagePayload_UserStateId {
+	if m != nil {
+		return m.UserStates
+	}
+	return nil
+}
+
+func (m *ExecutableStagePayload) GetTimers() []*ExecutableStagePayload_TimerId {
+	if m != nil {
+		return m.Timers
+	}
+	return nil
+}
+
 // A reference to a side input. Side inputs are uniquely identified by PTransform id and
 // local name.
 type ExecutableStagePayload_SideInputId struct {
@@ -1563,7 +1701,7 @@ func (m *ExecutableStagePayload_SideInputId) Reset()         { *m = ExecutableSt
 func (m *ExecutableStagePayload_SideInputId) String() string { return proto.CompactTextString(m) }
 func (*ExecutableStagePayload_SideInputId) ProtoMessage()    {}
 func (*ExecutableStagePayload_SideInputId) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_92025e5caa638397, []int{7, 0}
+	return fileDescriptor_cf57597c3a9659a9, []int{7, 0}
 }
 func (m *ExecutableStagePayload_SideInputId) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_ExecutableStagePayload_SideInputId.Unmarshal(m, b)
@@ -1571,8 +1709,8 @@ func (m *ExecutableStagePayload_SideInputId) XXX_Unmarshal(b []byte) error {
 func (m *ExecutableStagePayload_SideInputId) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_ExecutableStagePayload_SideInputId.Marshal(b, m, deterministic)
 }
-func (dst *ExecutableStagePayload_SideInputId) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ExecutableStagePayload_SideInputId.Merge(dst, src)
+func (m *ExecutableStagePayload_SideInputId) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_ExecutableStagePayload_SideInputId.Merge(m, src)
 }
 func (m *ExecutableStagePayload_SideInputId) XXX_Size() int {
 	return xxx_messageInfo_ExecutableStagePayload_SideInputId.Size(m)
@@ -1597,6 +1735,106 @@ func (m *ExecutableStagePayload_SideInputId) GetLocalName() string {
 	return ""
 }
 
+// A reference to user state. User states are uniquely identified by PTransform id and
+// local name.
+type ExecutableStagePayload_UserStateId struct {
+	// (Required) The id of the PTransform that references this user state.
+	TransformId string `protobuf:"bytes,1,opt,name=transform_id,json=transformId,proto3" json:"transform_id,omitempty"`
+	// (Required) The local name of this user state for the PTransform that references it.
+	LocalName            string   `protobuf:"bytes,2,opt,name=local_name,json=localName,proto3" json:"local_name,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *ExecutableStagePayload_UserStateId) Reset()         { *m = ExecutableStagePayload_UserStateId{} }
+func (m *ExecutableStagePayload_UserStateId) String() string { return proto.CompactTextString(m) }
+func (*ExecutableStagePayload_UserStateId) ProtoMessage()    {}
+func (*ExecutableStagePayload_UserStateId) Descriptor() ([]byte, []int) {
+	return fileDescriptor_cf57597c3a9659a9, []int{7, 1}
+}
+func (m *ExecutableStagePayload_UserStateId) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_ExecutableStagePayload_UserStateId.Unmarshal(m, b)
+}
+func (m *ExecutableStagePayload_UserStateId) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_ExecutableStagePayload_UserStateId.Marshal(b, m, deterministic)
+}
+func (m *ExecutableStagePayload_UserStateId) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_ExecutableStagePayload_UserStateId.Merge(m, src)
+}
+func (m *ExecutableStagePayload_UserStateId) XXX_Size() int {
+	return xxx_messageInfo_ExecutableStagePayload_UserStateId.Size(m)
+}
+func (m *ExecutableStagePayload_UserStateId) XXX_DiscardUnknown() {
+	xxx_messageInfo_ExecutableStagePayload_UserStateId.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_ExecutableStagePayload_UserStateId proto.InternalMessageInfo
+
+func (m *ExecutableStagePayload_UserStateId) GetTransformId() string {
+	if m != nil {
+		return m.TransformId
+	}
+	return ""
+}
+
+func (m *ExecutableStagePayload_UserStateId) GetLocalName() string {
+	if m != nil {
+		return m.LocalName
+	}
+	return ""
+}
+
+// A reference to a timer. Timers are uniquely identified by PTransform id and
+// local name.
+type ExecutableStagePayload_TimerId struct {
+	// (Required) The id of the PTransform that references this timer.
+	TransformId string `protobuf:"bytes,1,opt,name=transform_id,json=transformId,proto3" json:"transform_id,omitempty"`
+	// (Required) The local name of this timer for the PTransform that references it.
+	LocalName            string   `protobuf:"bytes,2,opt,name=local_name,json=localName,proto3" json:"local_name,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *ExecutableStagePayload_TimerId) Reset()         { *m = ExecutableStagePayload_TimerId{} }
+func (m *ExecutableStagePayload_TimerId) String() string { return proto.CompactTextString(m) }
+func (*ExecutableStagePayload_TimerId) ProtoMessage()    {}
+func (*ExecutableStagePayload_TimerId) Descriptor() ([]byte, []int) {
+	return fileDescriptor_cf57597c3a9659a9, []int{7, 2}
+}
+func (m *ExecutableStagePayload_TimerId) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_ExecutableStagePayload_TimerId.Unmarshal(m, b)
+}
+func (m *ExecutableStagePayload_TimerId) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_ExecutableStagePayload_TimerId.Marshal(b, m, deterministic)
+}
+func (m *ExecutableStagePayload_TimerId) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_ExecutableStagePayload_TimerId.Merge(m, src)
+}
+func (m *ExecutableStagePayload_TimerId) XXX_Size() int {
+	return xxx_messageInfo_ExecutableStagePayload_TimerId.Size(m)
+}
+func (m *ExecutableStagePayload_TimerId) XXX_DiscardUnknown() {
+	xxx_messageInfo_ExecutableStagePayload_TimerId.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_ExecutableStagePayload_TimerId proto.InternalMessageInfo
+
+func (m *ExecutableStagePayload_TimerId) GetTransformId() string {
+	if m != nil {
+		return m.TransformId
+	}
+	return ""
+}
+
+func (m *ExecutableStagePayload_TimerId) GetLocalName() string {
+	if m != nil {
+		return m.LocalName
+	}
+	return ""
+}
+
 // The payload for the primitive ParDo transform.
 type ParDoPayload struct {
 	// (Required) The SdkFunctionSpec of the DoFn.
@@ -1625,7 +1863,7 @@ func (m *ParDoPayload) Reset()         { *m = ParDoPayload{} }
 func (m *ParDoPayload) String() string { return proto.CompactTextString(m) }
 func (*ParDoPayload) ProtoMessage()    {}
 func (*ParDoPayload) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_92025e5caa638397, []int{8}
+	return fileDescriptor_cf57597c3a9659a9, []int{8}
 }
 func (m *ParDoPayload) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_ParDoPayload.Unmarshal(m, b)
@@ -1633,8 +1871,8 @@ func (m *ParDoPayload) XXX_Unmarshal(b []byte) error {
 func (m *ParDoPayload) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_ParDoPayload.Marshal(b, m, deterministic)
 }
-func (dst *ParDoPayload) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ParDoPayload.Merge(dst, src)
+func (m *ParDoPayload) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_ParDoPayload.Merge(m, src)
 }
 func (m *ParDoPayload) XXX_Size() int {
 	return xxx_messageInfo_ParDoPayload.Size(m)
@@ -1719,7 +1957,7 @@ func (m *Parameter) Reset()         { *m = Parameter{} }
 func (m *Parameter) String() string { return proto.CompactTextString(m) }
 func (*Parameter) ProtoMessage()    {}
 func (*Parameter) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_92025e5caa638397, []int{9}
+	return fileDescriptor_cf57597c3a9659a9, []int{9}
 }
 func (m *Parameter) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Parameter.Unmarshal(m, b)
@@ -1727,8 +1965,8 @@ func (m *Parameter) XXX_Unmarshal(b []byte) error {
 func (m *Parameter) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_Parameter.Marshal(b, m, deterministic)
 }
-func (dst *Parameter) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Parameter.Merge(dst, src)
+func (m *Parameter) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_Parameter.Merge(m, src)
 }
 func (m *Parameter) XXX_Size() int {
 	return xxx_messageInfo_Parameter.Size(m)
@@ -1756,7 +1994,7 @@ func (m *Parameter_Type) Reset()         { *m = Parameter_Type{} }
 func (m *Parameter_Type) String() string { return proto.CompactTextString(m) }
 func (*Parameter_Type) ProtoMessage()    {}
 func (*Parameter_Type) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_92025e5caa638397, []int{9, 0}
+	return fileDescriptor_cf57597c3a9659a9, []int{9, 0}
 }
 func (m *Parameter_Type) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Parameter_Type.Unmarshal(m, b)
@@ -1764,8 +2002,8 @@ func (m *Parameter_Type) XXX_Unmarshal(b []byte) error {
 func (m *Parameter_Type) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_Parameter_Type.Marshal(b, m, deterministic)
 }
-func (dst *Parameter_Type) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Parameter_Type.Merge(dst, src)
+func (m *Parameter_Type) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_Parameter_Type.Merge(m, src)
 }
 func (m *Parameter_Type) XXX_Size() int {
 	return xxx_messageInfo_Parameter_Type.Size(m)
@@ -1793,7 +2031,7 @@ func (m *StateSpec) Reset()         { *m = StateSpec{} }
 func (m *StateSpec) String() string { return proto.CompactTextString(m) }
 func (*StateSpec) ProtoMessage()    {}
 func (*StateSpec) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_92025e5caa638397, []int{10}
+	return fileDescriptor_cf57597c3a9659a9, []int{10}
 }
 func (m *StateSpec) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_StateSpec.Unmarshal(m, b)
@@ -1801,8 +2039,8 @@ func (m *StateSpec) XXX_Unmarshal(b []byte) error {
 func (m *StateSpec) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_StateSpec.Marshal(b, m, deterministic)
 }
-func (dst *StateSpec) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_StateSpec.Merge(dst, src)
+func (m *StateSpec) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_StateSpec.Merge(m, src)
 }
 func (m *StateSpec) XXX_Size() int {
 	return xxx_messageInfo_StateSpec.Size(m)
@@ -1820,24 +2058,32 @@ type isStateSpec_Spec interface {
 type StateSpec_ValueSpec struct {
 	ValueSpec *ValueStateSpec `protobuf:"bytes,1,opt,name=value_spec,json=valueSpec,proto3,oneof"`
 }
+
 type StateSpec_BagSpec struct {
 	BagSpec *BagStateSpec `protobuf:"bytes,2,opt,name=bag_spec,json=bagSpec,proto3,oneof"`
 }
+
 type StateSpec_CombiningSpec struct {
 	CombiningSpec *CombiningStateSpec `protobuf:"bytes,3,opt,name=combining_spec,json=combiningSpec,proto3,oneof"`
 }
+
 type StateSpec_MapSpec struct {
 	MapSpec *MapStateSpec `protobuf:"bytes,4,opt,name=map_spec,json=mapSpec,proto3,oneof"`
 }
+
 type StateSpec_SetSpec struct {
 	SetSpec *SetStateSpec `protobuf:"bytes,5,opt,name=set_spec,json=setSpec,proto3,oneof"`
 }
 
-func (*StateSpec_ValueSpec) isStateSpec_Spec()     {}
-func (*StateSpec_BagSpec) isStateSpec_Spec()       {}
+func (*StateSpec_ValueSpec) isStateSpec_Spec() {}
+
+func (*StateSpec_BagSpec) isStateSpec_Spec() {}
+
 func (*StateSpec_CombiningSpec) isStateSpec_Spec() {}
-func (*StateSpec_MapSpec) isStateSpec_Spec()       {}
-func (*StateSpec_SetSpec) isStateSpec_Spec()       {}
+
+func (*StateSpec_MapSpec) isStateSpec_Spec() {}
+
+func (*StateSpec_SetSpec) isStateSpec_Spec() {}
 
 func (m *StateSpec) GetSpec() isStateSpec_Spec {
 	if m != nil {
@@ -2023,7 +2269,7 @@ func (m *ValueStateSpec) Reset()         { *m = ValueStateSpec{} }
 func (m *ValueStateSpec) String() string { return proto.CompactTextString(m) }
 func (*ValueStateSpec) ProtoMessage()    {}
 func (*ValueStateSpec) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_92025e5caa638397, []int{11}
+	return fileDescriptor_cf57597c3a9659a9, []int{11}
 }
 func (m *ValueStateSpec) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_ValueStateSpec.Unmarshal(m, b)
@@ -2031,8 +2277,8 @@ func (m *ValueStateSpec) XXX_Unmarshal(b []byte) error {
 func (m *ValueStateSpec) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_ValueStateSpec.Marshal(b, m, deterministic)
 }
-func (dst *ValueStateSpec) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ValueStateSpec.Merge(dst, src)
+func (m *ValueStateSpec) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_ValueStateSpec.Merge(m, src)
 }
 func (m *ValueStateSpec) XXX_Size() int {
 	return xxx_messageInfo_ValueStateSpec.Size(m)
@@ -2061,7 +2307,7 @@ func (m *BagStateSpec) Reset()         { *m = BagStateSpec{} }
 func (m *BagStateSpec) String() string { return proto.CompactTextString(m) }
 func (*BagStateSpec) ProtoMessage()    {}
 func (*BagStateSpec) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_92025e5caa638397, []int{12}
+	return fileDescriptor_cf57597c3a9659a9, []int{12}
 }
 func (m *BagStateSpec) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_BagStateSpec.Unmarshal(m, b)
@@ -2069,8 +2315,8 @@ func (m *BagStateSpec) XXX_Unmarshal(b []byte) error {
 func (m *BagStateSpec) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_BagStateSpec.Marshal(b, m, deterministic)
 }
-func (dst *BagStateSpec) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_BagStateSpec.Merge(dst, src)
+func (m *BagStateSpec) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_BagStateSpec.Merge(m, src)
 }
 func (m *BagStateSpec) XXX_Size() int {
 	return xxx_messageInfo_BagStateSpec.Size(m)
@@ -2100,7 +2346,7 @@ func (m *CombiningStateSpec) Reset()         { *m = CombiningStateSpec{} }
 func (m *CombiningStateSpec) String() string { return proto.CompactTextString(m) }
 func (*CombiningStateSpec) ProtoMessage()    {}
 func (*CombiningStateSpec) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_92025e5caa638397, []int{13}
+	return fileDescriptor_cf57597c3a9659a9, []int{13}
 }
 func (m *CombiningStateSpec) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_CombiningStateSpec.Unmarshal(m, b)
@@ -2108,8 +2354,8 @@ func (m *CombiningStateSpec) XXX_Unmarshal(b []byte) error {
 func (m *CombiningStateSpec) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_CombiningStateSpec.Marshal(b, m, deterministic)
 }
-func (dst *CombiningStateSpec) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_CombiningStateSpec.Merge(dst, src)
+func (m *CombiningStateSpec) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_CombiningStateSpec.Merge(m, src)
 }
 func (m *CombiningStateSpec) XXX_Size() int {
 	return xxx_messageInfo_CombiningStateSpec.Size(m)
@@ -2146,7 +2392,7 @@ func (m *MapStateSpec) Reset()         { *m = MapStateSpec{} }
 func (m *MapStateSpec) String() string { return proto.CompactTextString(m) }
 func (*MapStateSpec) ProtoMessage()    {}
 func (*MapStateSpec) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_92025e5caa638397, []int{14}
+	return fileDescriptor_cf57597c3a9659a9, []int{14}
 }
 func (m *MapStateSpec) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_MapStateSpec.Unmarshal(m, b)
@@ -2154,8 +2400,8 @@ func (m *MapStateSpec) XXX_Unmarshal(b []byte) error {
 func (m *MapStateSpec) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_MapStateSpec.Marshal(b, m, deterministic)
 }
-func (dst *MapStateSpec) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_MapStateSpec.Merge(dst, src)
+func (m *MapStateSpec) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_MapStateSpec.Merge(m, src)
 }
 func (m *MapStateSpec) XXX_Size() int {
 	return xxx_messageInfo_MapStateSpec.Size(m)
@@ -2191,7 +2437,7 @@ func (m *SetStateSpec) Reset()         { *m = SetStateSpec{} }
 func (m *SetStateSpec) String() string { return proto.CompactTextString(m) }
 func (*SetStateSpec) ProtoMessage()    {}
 func (*SetStateSpec) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_92025e5caa638397, []int{15}
+	return fileDescriptor_cf57597c3a9659a9, []int{15}
 }
 func (m *SetStateSpec) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_SetStateSpec.Unmarshal(m, b)
@@ -2199,8 +2445,8 @@ func (m *SetStateSpec) XXX_Unmarshal(b []byte) error {
 func (m *SetStateSpec) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_SetStateSpec.Marshal(b, m, deterministic)
 }
-func (dst *SetStateSpec) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_SetStateSpec.Merge(dst, src)
+func (m *SetStateSpec) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_SetStateSpec.Merge(m, src)
 }
 func (m *SetStateSpec) XXX_Size() int {
 	return xxx_messageInfo_SetStateSpec.Size(m)
@@ -2229,7 +2475,7 @@ func (m *TimerSpec) Reset()         { *m = TimerSpec{} }
 func (m *TimerSpec) String() string { return proto.CompactTextString(m) }
 func (*TimerSpec) ProtoMessage()    {}
 func (*TimerSpec) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_92025e5caa638397, []int{16}
+	return fileDescriptor_cf57597c3a9659a9, []int{16}
 }
 func (m *TimerSpec) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_TimerSpec.Unmarshal(m, b)
@@ -2237,8 +2483,8 @@ func (m *TimerSpec) XXX_Unmarshal(b []byte) error {
 func (m *TimerSpec) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_TimerSpec.Marshal(b, m, deterministic)
 }
-func (dst *TimerSpec) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_TimerSpec.Merge(dst, src)
+func (m *TimerSpec) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_TimerSpec.Merge(m, src)
 }
 func (m *TimerSpec) XXX_Size() int {
 	return xxx_messageInfo_TimerSpec.Size(m)
@@ -2266,7 +2512,7 @@ func (m *IsBounded) Reset()         { *m = IsBounded{} }
 func (m *IsBounded) String() string { return proto.CompactTextString(m) }
 func (*IsBounded) ProtoMessage()    {}
 func (*IsBounded) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_92025e5caa638397, []int{17}
+	return fileDescriptor_cf57597c3a9659a9, []int{17}
 }
 func (m *IsBounded) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_IsBounded.Unmarshal(m, b)
@@ -2274,8 +2520,8 @@ func (m *IsBounded) XXX_Unmarshal(b []byte) error {
 func (m *IsBounded) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_IsBounded.Marshal(b, m, deterministic)
 }
-func (dst *IsBounded) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_IsBounded.Merge(dst, src)
+func (m *IsBounded) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_IsBounded.Merge(m, src)
 }
 func (m *IsBounded) XXX_Size() int {
 	return xxx_messageInfo_IsBounded.Size(m)
@@ -2301,7 +2547,7 @@ func (m *ReadPayload) Reset()         { *m = ReadPayload{} }
 func (m *ReadPayload) String() string { return proto.CompactTextString(m) }
 func (*ReadPayload) ProtoMessage()    {}
 func (*ReadPayload) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_92025e5caa638397, []int{18}
+	return fileDescriptor_cf57597c3a9659a9, []int{18}
 }
 func (m *ReadPayload) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_ReadPayload.Unmarshal(m, b)
@@ -2309,8 +2555,8 @@ func (m *ReadPayload) XXX_Unmarshal(b []byte) error {
 func (m *ReadPayload) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_ReadPayload.Marshal(b, m, deterministic)
 }
-func (dst *ReadPayload) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ReadPayload.Merge(dst, src)
+func (m *ReadPayload) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_ReadPayload.Merge(m, src)
 }
 func (m *ReadPayload) XXX_Size() int {
 	return xxx_messageInfo_ReadPayload.Size(m)
@@ -2348,7 +2594,7 @@ func (m *WindowIntoPayload) Reset()         { *m = WindowIntoPayload{} }
 func (m *WindowIntoPayload) String() string { return proto.CompactTextString(m) }
 func (*WindowIntoPayload) ProtoMessage()    {}
 func (*WindowIntoPayload) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_92025e5caa638397, []int{19}
+	return fileDescriptor_cf57597c3a9659a9, []int{19}
 }
 func (m *WindowIntoPayload) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_WindowIntoPayload.Unmarshal(m, b)
@@ -2356,8 +2602,8 @@ func (m *WindowIntoPayload) XXX_Unmarshal(b []byte) error {
 func (m *WindowIntoPayload) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_WindowIntoPayload.Marshal(b, m, deterministic)
 }
-func (dst *WindowIntoPayload) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_WindowIntoPayload.Merge(dst, src)
+func (m *WindowIntoPayload) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_WindowIntoPayload.Merge(m, src)
 }
 func (m *WindowIntoPayload) XXX_Size() int {
 	return xxx_messageInfo_WindowIntoPayload.Size(m)
@@ -2390,7 +2636,7 @@ func (m *CombinePayload) Reset()         { *m = CombinePayload{} }
 func (m *CombinePayload) String() string { return proto.CompactTextString(m) }
 func (*CombinePayload) ProtoMessage()    {}
 func (*CombinePayload) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_92025e5caa638397, []int{20}
+	return fileDescriptor_cf57597c3a9659a9, []int{20}
 }
 func (m *CombinePayload) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_CombinePayload.Unmarshal(m, b)
@@ -2398,8 +2644,8 @@ func (m *CombinePayload) XXX_Unmarshal(b []byte) error {
 func (m *CombinePayload) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_CombinePayload.Marshal(b, m, deterministic)
 }
-func (dst *CombinePayload) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_CombinePayload.Merge(dst, src)
+func (m *CombinePayload) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_CombinePayload.Merge(m, src)
 }
 func (m *CombinePayload) XXX_Size() int {
 	return xxx_messageInfo_CombinePayload.Size(m)
@@ -2438,7 +2684,7 @@ func (m *TestStreamPayload) Reset()         { *m = TestStreamPayload{} }
 func (m *TestStreamPayload) String() string { return proto.CompactTextString(m) }
 func (*TestStreamPayload) ProtoMessage()    {}
 func (*TestStreamPayload) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_92025e5caa638397, []int{21}
+	return fileDescriptor_cf57597c3a9659a9, []int{21}
 }
 func (m *TestStreamPayload) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_TestStreamPayload.Unmarshal(m, b)
@@ -2446,8 +2692,8 @@ func (m *TestStreamPayload) XXX_Unmarshal(b []byte) error {
 func (m *TestStreamPayload) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_TestStreamPayload.Marshal(b, m, deterministic)
 }
-func (dst *TestStreamPayload) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_TestStreamPayload.Merge(dst, src)
+func (m *TestStreamPayload) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_TestStreamPayload.Merge(m, src)
 }
 func (m *TestStreamPayload) XXX_Size() int {
 	return xxx_messageInfo_TestStreamPayload.Size(m)
@@ -2487,7 +2733,7 @@ func (m *TestStreamPayload_Event) Reset()         { *m = TestStreamPayload_Event
 func (m *TestStreamPayload_Event) String() string { return proto.CompactTextString(m) }
 func (*TestStreamPayload_Event) ProtoMessage()    {}
 func (*TestStreamPayload_Event) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_92025e5caa638397, []int{21, 0}
+	return fileDescriptor_cf57597c3a9659a9, []int{21, 0}
 }
 func (m *TestStreamPayload_Event) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_TestStreamPayload_Event.Unmarshal(m, b)
@@ -2495,8 +2741,8 @@ func (m *TestStreamPayload_Event) XXX_Unmarshal(b []byte) error {
 func (m *TestStreamPayload_Event) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_TestStreamPayload_Event.Marshal(b, m, deterministic)
 }
-func (dst *TestStreamPayload_Event) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_TestStreamPayload_Event.Merge(dst, src)
+func (m *TestStreamPayload_Event) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_TestStreamPayload_Event.Merge(m, src)
 }
 func (m *TestStreamPayload_Event) XXX_Size() int {
 	return xxx_messageInfo_TestStreamPayload_Event.Size(m)
@@ -2514,16 +2760,20 @@ type isTestStreamPayload_Event_Event interface {
 type TestStreamPayload_Event_WatermarkEvent struct {
 	WatermarkEvent *TestStreamPayload_Event_AdvanceWatermark `protobuf:"bytes,1,opt,name=watermark_event,json=watermarkEvent,proto3,oneof"`
 }
+
 type TestStreamPayload_Event_ProcessingTimeEvent struct {
 	ProcessingTimeEvent *TestStreamPayload_Event_AdvanceProcessingTime `protobuf:"bytes,2,opt,name=processing_time_event,json=processingTimeEvent,proto3,oneof"`
 }
+
 type TestStreamPayload_Event_ElementEvent struct {
 	ElementEvent *TestStreamPayload_Event_AddElements `protobuf:"bytes,3,opt,name=element_event,json=elementEvent,proto3,oneof"`
 }
 
-func (*TestStreamPayload_Event_WatermarkEvent) isTestStreamPayload_Event_Event()      {}
+func (*TestStreamPayload_Event_WatermarkEvent) isTestStreamPayload_Event_Event() {}
+
 func (*TestStreamPayload_Event_ProcessingTimeEvent) isTestStreamPayload_Event_Event() {}
-func (*TestStreamPayload_Event_ElementEvent) isTestStreamPayload_Event_Event()        {}
+
+func (*TestStreamPayload_Event_ElementEvent) isTestStreamPayload_Event_Event() {}
 
 func (m *TestStreamPayload_Event) GetEvent() isTestStreamPayload_Event_Event {
 	if m != nil {
@@ -2659,7 +2909,7 @@ func (m *TestStreamPayload_Event_AdvanceWatermark) Reset() {
 func (m *TestStreamPayload_Event_AdvanceWatermark) String() string { return proto.CompactTextString(m) }
 func (*TestStreamPayload_Event_AdvanceWatermark) ProtoMessage()    {}
 func (*TestStreamPayload_Event_AdvanceWatermark) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_92025e5caa638397, []int{21, 0, 0}
+	return fileDescriptor_cf57597c3a9659a9, []int{21, 0, 0}
 }
 func (m *TestStreamPayload_Event_AdvanceWatermark) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_TestStreamPayload_Event_AdvanceWatermark.Unmarshal(m, b)
@@ -2667,8 +2917,8 @@ func (m *TestStreamPayload_Event_AdvanceWatermark) XXX_Unmarshal(b []byte) error
 func (m *TestStreamPayload_Event_AdvanceWatermark) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_TestStreamPayload_Event_AdvanceWatermark.Marshal(b, m, deterministic)
 }
-func (dst *TestStreamPayload_Event_AdvanceWatermark) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_TestStreamPayload_Event_AdvanceWatermark.Merge(dst, src)
+func (m *TestStreamPayload_Event_AdvanceWatermark) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_TestStreamPayload_Event_AdvanceWatermark.Merge(m, src)
 }
 func (m *TestStreamPayload_Event_AdvanceWatermark) XXX_Size() int {
 	return xxx_messageInfo_TestStreamPayload_Event_AdvanceWatermark.Size(m)
@@ -2701,7 +2951,7 @@ func (m *TestStreamPayload_Event_AdvanceProcessingTime) String() string {
 }
 func (*TestStreamPayload_Event_AdvanceProcessingTime) ProtoMessage() {}
 func (*TestStreamPayload_Event_AdvanceProcessingTime) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_92025e5caa638397, []int{21, 0, 1}
+	return fileDescriptor_cf57597c3a9659a9, []int{21, 0, 1}
 }
 func (m *TestStreamPayload_Event_AdvanceProcessingTime) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_TestStreamPayload_Event_AdvanceProcessingTime.Unmarshal(m, b)
@@ -2709,8 +2959,8 @@ func (m *TestStreamPayload_Event_AdvanceProcessingTime) XXX_Unmarshal(b []byte)
 func (m *TestStreamPayload_Event_AdvanceProcessingTime) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_TestStreamPayload_Event_AdvanceProcessingTime.Marshal(b, m, deterministic)
 }
-func (dst *TestStreamPayload_Event_AdvanceProcessingTime) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_TestStreamPayload_Event_AdvanceProcessingTime.Merge(dst, src)
+func (m *TestStreamPayload_Event_AdvanceProcessingTime) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_TestStreamPayload_Event_AdvanceProcessingTime.Merge(m, src)
 }
 func (m *TestStreamPayload_Event_AdvanceProcessingTime) XXX_Size() int {
 	return xxx_messageInfo_TestStreamPayload_Event_AdvanceProcessingTime.Size(m)
@@ -2739,7 +2989,7 @@ func (m *TestStreamPayload_Event_AddElements) Reset()         { *m = TestStreamP
 func (m *TestStreamPayload_Event_AddElements) String() string { return proto.CompactTextString(m) }
 func (*TestStreamPayload_Event_AddElements) ProtoMessage()    {}
 func (*TestStreamPayload_Event_AddElements) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_92025e5caa638397, []int{21, 0, 2}
+	return fileDescriptor_cf57597c3a9659a9, []int{21, 0, 2}
 }
 func (m *TestStreamPayload_Event_AddElements) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_TestStreamPayload_Event_AddElements.Unmarshal(m, b)
@@ -2747,8 +2997,8 @@ func (m *TestStreamPayload_Event_AddElements) XXX_Unmarshal(b []byte) error {
 func (m *TestStreamPayload_Event_AddElements) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_TestStreamPayload_Event_AddElements.Marshal(b, m, deterministic)
 }
-func (dst *TestStreamPayload_Event_AddElements) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_TestStreamPayload_Event_AddElements.Merge(dst, src)
+func (m *TestStreamPayload_Event_AddElements) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_TestStreamPayload_Event_AddElements.Merge(m, src)
 }
 func (m *TestStreamPayload_Event_AddElements) XXX_Size() int {
 	return xxx_messageInfo_TestStreamPayload_Event_AddElements.Size(m)
@@ -2778,7 +3028,7 @@ func (m *TestStreamPayload_TimestampedElement) Reset()         { *m = TestStream
 func (m *TestStreamPayload_TimestampedElement) String() string { return proto.CompactTextString(m) }
 func (*TestStreamPayload_TimestampedElement) ProtoMessage()    {}
 func (*TestStreamPayload_TimestampedElement) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_92025e5caa638397, []int{21, 1}
+	return fileDescriptor_cf57597c3a9659a9, []int{21, 1}
 }
 func (m *TestStreamPayload_TimestampedElement) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_TestStreamPayload_TimestampedElement.Unmarshal(m, b)
@@ -2786,8 +3036,8 @@ func (m *TestStreamPayload_TimestampedElement) XXX_Unmarshal(b []byte) error {
 func (m *TestStreamPayload_TimestampedElement) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_TestStreamPayload_TimestampedElement.Marshal(b, m, deterministic)
 }
-func (dst *TestStreamPayload_TimestampedElement) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_TestStreamPayload_TimestampedElement.Merge(dst, src)
+func (m *TestStreamPayload_TimestampedElement) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_TestStreamPayload_TimestampedElement.Merge(m, src)
 }
 func (m *TestStreamPayload_TimestampedElement) XXX_Size() int {
 	return xxx_messageInfo_TestStreamPayload_TimestampedElement.Size(m)
@@ -2830,7 +3080,7 @@ func (m *WriteFilesPayload) Reset()         { *m = WriteFilesPayload{} }
 func (m *WriteFilesPayload) String() string { return proto.CompactTextString(m) }
 func (*WriteFilesPayload) ProtoMessage()    {}
 func (*WriteFilesPayload) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_92025e5caa638397, []int{22}
+	return fileDescriptor_cf57597c3a9659a9, []int{22}
 }
 func (m *WriteFilesPayload) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_WriteFilesPayload.Unmarshal(m, b)
@@ -2838,8 +3088,8 @@ func (m *WriteFilesPayload) XXX_Unmarshal(b []byte) error {
 func (m *WriteFilesPayload) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_WriteFilesPayload.Marshal(b, m, deterministic)
 }
-func (dst *WriteFilesPayload) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_WriteFilesPayload.Merge(dst, src)
+func (m *WriteFilesPayload) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_WriteFilesPayload.Merge(m, src)
 }
 func (m *WriteFilesPayload) XXX_Size() int {
 	return xxx_messageInfo_WriteFilesPayload.Size(m)
@@ -2907,7 +3157,7 @@ func (m *Coder) Reset()         { *m = Coder{} }
 func (m *Coder) String() string { return proto.CompactTextString(m) }
 func (*Coder) ProtoMessage()    {}
 func (*Coder) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_92025e5caa638397, []int{23}
+	return fileDescriptor_cf57597c3a9659a9, []int{23}
 }
 func (m *Coder) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Coder.Unmarshal(m, b)
@@ -2915,8 +3165,8 @@ func (m *Coder) XXX_Unmarshal(b []byte) error {
 func (m *Coder) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_Coder.Marshal(b, m, deterministic)
 }
-func (dst *Coder) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Coder.Merge(dst, src)
+func (m *Coder) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_Coder.Merge(m, src)
 }
 func (m *Coder) XXX_Size() int {
 	return xxx_messageInfo_Coder.Size(m)
@@ -2951,7 +3201,7 @@ func (m *StandardCoders) Reset()         { *m = StandardCoders{} }
 func (m *StandardCoders) String() string { return proto.CompactTextString(m) }
 func (*StandardCoders) ProtoMessage()    {}
 func (*StandardCoders) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_92025e5caa638397, []int{24}
+	return fileDescriptor_cf57597c3a9659a9, []int{24}
 }
 func (m *StandardCoders) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_StandardCoders.Unmarshal(m, b)
@@ -2959,8 +3209,8 @@ func (m *StandardCoders) XXX_Unmarshal(b []byte) error {
 func (m *StandardCoders) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_StandardCoders.Marshal(b, m, deterministic)
 }
-func (dst *StandardCoders) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_StandardCoders.Merge(dst, src)
+func (m *StandardCoders) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_StandardCoders.Merge(m, src)
 }
 func (m *StandardCoders) XXX_Size() int {
 	return xxx_messageInfo_StandardCoders.Size(m)
@@ -3022,7 +3272,7 @@ func (m *WindowingStrategy) Reset()         { *m = WindowingStrategy{} }
 func (m *WindowingStrategy) String() string { return proto.CompactTextString(m) }
 func (*WindowingStrategy) ProtoMessage()    {}
 func (*WindowingStrategy) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_92025e5caa638397, []int{25}
+	return fileDescriptor_cf57597c3a9659a9, []int{25}
 }
 func (m *WindowingStrategy) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_WindowingStrategy.Unmarshal(m, b)
@@ -3030,8 +3280,8 @@ func (m *WindowingStrategy) XXX_Unmarshal(b []byte) error {
 func (m *WindowingStrategy) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_WindowingStrategy.Marshal(b, m, deterministic)
 }
-func (dst *WindowingStrategy) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_WindowingStrategy.Merge(dst, src)
+func (m *WindowingStrategy) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_WindowingStrategy.Merge(m, src)
 }
 func (m *WindowingStrategy) XXX_Size() int {
 	return xxx_messageInfo_WindowingStrategy.Size(m)
@@ -3125,7 +3375,7 @@ func (m *MergeStatus) Reset()         { *m = MergeStatus{} }
 func (m *MergeStatus) String() string { return proto.CompactTextString(m) }
 func (*MergeStatus) ProtoMessage()    {}
 func (*MergeStatus) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_92025e5caa638397, []int{26}
+	return fileDescriptor_cf57597c3a9659a9, []int{26}
 }
 func (m *MergeStatus) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_MergeStatus.Unmarshal(m, b)
@@ -3133,8 +3383,8 @@ func (m *MergeStatus) XXX_Unmarshal(b []byte) error {
 func (m *MergeStatus) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_MergeStatus.Marshal(b, m, deterministic)
 }
-func (dst *MergeStatus) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_MergeStatus.Merge(dst, src)
+func (m *MergeStatus) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_MergeStatus.Merge(m, src)
 }
 func (m *MergeStatus) XXX_Size() int {
 	return xxx_messageInfo_MergeStatus.Size(m)
@@ -3158,7 +3408,7 @@ func (m *AccumulationMode) Reset()         { *m = AccumulationMode{} }
 func (m *AccumulationMode) String() string { return proto.CompactTextString(m) }
 func (*AccumulationMode) ProtoMessage()    {}
 func (*AccumulationMode) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_92025e5caa638397, []int{27}
+	return fileDescriptor_cf57597c3a9659a9, []int{27}
 }
 func (m *AccumulationMode) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_AccumulationMode.Unmarshal(m, b)
@@ -3166,8 +3416,8 @@ func (m *AccumulationMode) XXX_Unmarshal(b []byte) error {
 func (m *AccumulationMode) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_AccumulationMode.Marshal(b, m, deterministic)
 }
-func (dst *AccumulationMode) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_AccumulationMode.Merge(dst, src)
+func (m *AccumulationMode) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_AccumulationMode.Merge(m, src)
 }
 func (m *AccumulationMode) XXX_Size() int {
 	return xxx_messageInfo_AccumulationMode.Size(m)
@@ -3190,7 +3440,7 @@ func (m *ClosingBehavior) Reset()         { *m = ClosingBehavior{} }
 func (m *ClosingBehavior) String() string { return proto.CompactTextString(m) }
 func (*ClosingBehavior) ProtoMessage()    {}
 func (*ClosingBehavior) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_92025e5caa638397, []int{28}
+	return fileDescriptor_cf57597c3a9659a9, []int{28}
 }
 func (m *ClosingBehavior) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_ClosingBehavior.Unmarshal(m, b)
@@ -3198,8 +3448,8 @@ func (m *ClosingBehavior) XXX_Unmarshal(b []byte) error {
 func (m *ClosingBehavior) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_ClosingBehavior.Marshal(b, m, deterministic)
 }
-func (dst *ClosingBehavior) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ClosingBehavior.Merge(dst, src)
+func (m *ClosingBehavior) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_ClosingBehavior.Merge(m, src)
 }
 func (m *ClosingBehavior) XXX_Size() int {
 	return xxx_messageInfo_ClosingBehavior.Size(m)
@@ -3222,7 +3472,7 @@ func (m *OnTimeBehavior) Reset()         { *m = OnTimeBehavior{} }
 func (m *OnTimeBehavior) String() string { return proto.CompactTextString(m) }
 func (*OnTimeBehavior) ProtoMessage()    {}
 func (*OnTimeBehavior) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_92025e5caa638397, []int{29}
+	return fileDescriptor_cf57597c3a9659a9, []int{29}
 }
 func (m *OnTimeBehavior) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_OnTimeBehavior.Unmarshal(m, b)
@@ -3230,8 +3480,8 @@ func (m *OnTimeBehavior) XXX_Unmarshal(b []byte) error {
 func (m *OnTimeBehavior) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_OnTimeBehavior.Marshal(b, m, deterministic)
 }
-func (dst *OnTimeBehavior) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OnTimeBehavior.Merge(dst, src)
+func (m *OnTimeBehavior) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OnTimeBehavior.Merge(m, src)
 }
 func (m *OnTimeBehavior) XXX_Size() int {
 	return xxx_messageInfo_OnTimeBehavior.Size(m)
@@ -3254,7 +3504,7 @@ func (m *OutputTime) Reset()         { *m = OutputTime{} }
 func (m *OutputTime) String() string { return proto.CompactTextString(m) }
 func (*OutputTime) ProtoMessage()    {}
 func (*OutputTime) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_92025e5caa638397, []int{30}
+	return fileDescriptor_cf57597c3a9659a9, []int{30}
 }
 func (m *OutputTime) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_OutputTime.Unmarshal(m, b)
@@ -3262,8 +3512,8 @@ func (m *OutputTime) XXX_Unmarshal(b []byte) error {
 func (m *OutputTime) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_OutputTime.Marshal(b, m, deterministic)
 }
-func (dst *OutputTime) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OutputTime.Merge(dst, src)
+func (m *OutputTime) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OutputTime.Merge(m, src)
 }
 func (m *OutputTime) XXX_Size() int {
 	return xxx_messageInfo_OutputTime.Size(m)
@@ -3285,7 +3535,7 @@ func (m *TimeDomain) Reset()         { *m = TimeDomain{} }
 func (m *TimeDomain) String() string { return proto.CompactTextString(m) }
 func (*TimeDomain) ProtoMessage()    {}
 func (*TimeDomain) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_92025e5caa638397, []int{31}
+	return fileDescriptor_cf57597c3a9659a9, []int{31}
 }
 func (m *TimeDomain) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_TimeDomain.Unmarshal(m, b)
@@ -3293,8 +3543,8 @@ func (m *TimeDomain) XXX_Unmarshal(b []byte) error {
 func (m *TimeDomain) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_TimeDomain.Marshal(b, m, deterministic)
 }
-func (dst *TimeDomain) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_TimeDomain.Merge(dst, src)
+func (m *TimeDomain) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_TimeDomain.Merge(m, src)
 }
 func (m *TimeDomain) XXX_Size() int {
 	return xxx_messageInfo_TimeDomain.Size(m)
@@ -3335,7 +3585,7 @@ func (m *Trigger) Reset()         { *m = Trigger{} }
 func (m *Trigger) String() string { return proto.CompactTextString(m) }
 func (*Trigger) ProtoMessage()    {}
 func (*Trigger) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_92025e5caa638397, []int{32}
+	return fileDescriptor_cf57597c3a9659a9, []int{32}
 }
 func (m *Trigger) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Trigger.Unmarshal(m, b)
@@ -3343,8 +3593,8 @@ func (m *Trigger) XXX_Unmarshal(b []byte) error {
 func (m *Trigger) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_Trigger.Marshal(b, m, deterministic)
 }
-func (dst *Trigger) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Trigger.Merge(dst, src)
+func (m *Trigger) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_Trigger.Merge(m, src)
 }
 func (m *Trigger) XXX_Size() int {
 	return xxx_messageInfo_Trigger.Size(m)
@@ -3362,52 +3612,74 @@ type isTrigger_Trigger interface {
 type Trigger_AfterAll_ struct {
 	AfterAll *Trigger_AfterAll `protobuf:"bytes,1,opt,name=after_all,json=afterAll,proto3,oneof"`
 }
+
 type Trigger_AfterAny_ struct {
 	AfterAny *Trigger_AfterAny `protobuf:"bytes,2,opt,name=after_any,json=afterAny,proto3,oneof"`
 }
+
 type Trigger_AfterEach_ struct {
 	AfterEach *Trigger_AfterEach `protobuf:"bytes,3,opt,name=after_each,json=afterEach,proto3,oneof"`
 }
+
 type Trigger_AfterEndOfWindow_ struct {
 	AfterEndOfWindow *Trigger_AfterEndOfWindow `protobuf:"bytes,4,opt,name=after_end_of_window,json=afterEndOfWindow,proto3,oneof"`
 }
+
 type Trigger_AfterProcessingTime_ struct {
 	AfterProcessingTime *Trigger_AfterProcessingTime `protobuf:"bytes,5,opt,name=after_processing_time,json=afterProcessingTime,proto3,oneof"`
 }
+
 type Trigger_AfterSynchronizedProcessingTime_ struct {
 	AfterSynchronizedProcessingTime *Trigger_AfterSynchronizedProcessingTime `protobuf:"bytes,6,opt,name=after_synchronized_processing_time,json=afterSynchronizedProcessingTime,proto3,oneof"`
 }
+
 type Trigger_Always_ struct {
 	Always *Trigger_Always `protobuf:"bytes,12,opt,name=always,proto3,oneof"`
 }
+
 type Trigger_Default_ struct {
 	Default *Trigger_Default `protobuf:"bytes,7,opt,name=default,proto3,oneof"`
 }
+
 type Trigger_ElementCount_ struct {
 	ElementCount *Trigger_ElementCount `protobuf:"bytes,8,opt,name=element_count,json=elementCount,proto3,oneof"`
 }
+
 type Trigger_Never_ struct {
 	Never *Trigger_Never `protobuf:"bytes,9,opt,name=never,proto3,oneof"`
 }
+
 type Trigger_OrFinally_ struct {
 	OrFinally *Trigger_OrFinally `protobuf:"bytes,10,opt,name=or_finally,json=orFinally,proto3,oneof"`
 }
+
 type Trigger_Repeat_ struct {
 	Repeat *Trigger_Repeat `protobuf:"bytes,11,opt,name=repeat,proto3,oneof"`
 }
 
-func (*Trigger_AfterAll_) isTrigger_Trigger()                        {}
-func (*Trigger_AfterAny_) isTrigger_Trigger()                        {}
-func (*Trigger_AfterEach_) isTrigger_Trigger()                       {}
-func (*Trigger_AfterEndOfWindow_) isTrigger_Trigger()                {}
-func (*Trigger_AfterProcessingTime_) isTrigger_Trigger()             {}
+func (*Trigger_AfterAll_) isTrigger_Trigger() {}
+
+func (*Trigger_AfterAny_) isTrigger_Trigger() {}
+
+func (*Trigger_AfterEach_) isTrigger_Trigger() {}
+
+func (*Trigger_AfterEndOfWindow_) isTrigger_Trigger() {}
+
+func (*Trigger_AfterProcessingTime_) isTrigger_Trigger() {}
+
 func (*Trigger_AfterSynchronizedProcessingTime_) isTrigger_Trigger() {}
-func (*Trigger_Always_) isTrigger_Trigger()                          {}
-func (*Trigger_Default_) isTrigger_Trigger()                         {}
-func (*Trigger_ElementCount_) isTrigger_Trigger()                    {}
-func (*Trigger_Never_) isTrigger_Trigger()                           {}
-func (*Trigger_OrFinally_) isTrigger_Trigger()                       {}
-func (*Trigger_Repeat_) isTrigger_Trigger()                          {}
+
+func (*Trigger_Always_) isTrigger_Trigger() {}
+
+func (*Trigger_Default_) isTrigger_Trigger() {}
+
+func (*Trigger_ElementCount_) isTrigger_Trigger() {}
+
+func (*Trigger_Never_) isTrigger_Trigger() {}
+
+func (*Trigger_OrFinally_) isTrigger_Trigger() {}
+
+func (*Trigger_Repeat_) isTrigger_Trigger() {}
 
 func (m *Trigger) GetTrigger() isTrigger_Trigger {
 	if m != nil {
@@ -3776,7 +4048,7 @@ func (m *Trigger_AfterAll) Reset()         { *m = Trigger_AfterAll{} }
 func (m *Trigger_AfterAll) String() string { return proto.CompactTextString(m) }
 func (*Trigger_AfterAll) ProtoMessage()    {}
 func (*Trigger_AfterAll) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_92025e5caa638397, []int{32, 0}
+	return fileDescriptor_cf57597c3a9659a9, []int{32, 0}
 }
 func (m *Trigger_AfterAll) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Trigger_AfterAll.Unmarshal(m, b)
@@ -3784,8 +4056,8 @@ func (m *Trigger_AfterAll) XXX_Unmarshal(b []byte) error {
 func (m *Trigger_AfterAll) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_Trigger_AfterAll.Marshal(b, m, deterministic)
 }
-func (dst *Trigger_AfterAll) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Trigger_AfterAll.Merge(dst, src)
+func (m *Trigger_AfterAll) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_Trigger_AfterAll.Merge(m, src)
 }
 func (m *Trigger_AfterAll) XXX_Size() int {
 	return xxx_messageInfo_Trigger_AfterAll.Size(m)
@@ -3815,7 +4087,7 @@ func (m *Trigger_AfterAny) Reset()         { *m = Trigger_AfterAny{} }
 func (m *Trigger_AfterAny) String() string { return proto.CompactTextString(m) }
 func (*Trigger_AfterAny) ProtoMessage()    {}
 func (*Trigger_AfterAny) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_92025e5caa638397, []int{32, 1}
+	return fileDescriptor_cf57597c3a9659a9, []int{32, 1}
 }
 func (m *Trigger_AfterAny) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Trigger_AfterAny.Unmarshal(m, b)
@@ -3823,8 +4095,8 @@ func (m *Trigger_AfterAny) XXX_Unmarshal(b []byte) error {
 func (m *Trigger_AfterAny) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_Trigger_AfterAny.Marshal(b, m, deterministic)
 }
-func (dst *Trigger_AfterAny) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Trigger_AfterAny.Merge(dst, src)
+func (m *Trigger_AfterAny) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_Trigger_AfterAny.Merge(m, src)
 }
 func (m *Trigger_AfterAny) XXX_Size() int {
 	return xxx_messageInfo_Trigger_AfterAny.Size(m)
@@ -3855,7 +4127,7 @@ func (m *Trigger_AfterEach) Reset()         { *m = Trigger_AfterEach{} }
 func (m *Trigger_AfterEach) String() string { return proto.CompactTextString(m) }
 func (*Trigger_AfterEach) ProtoMessage()    {}
 func (*Trigger_AfterEach) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_92025e5caa638397, []int{32, 2}
+	return fileDescriptor_cf57597c3a9659a9, []int{32, 2}
 }
 func (m *Trigger_AfterEach) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Trigger_AfterEach.Unmarshal(m, b)
@@ -3863,8 +4135,8 @@ func (m *Trigger_AfterEach) XXX_Unmarshal(b []byte) error {
 func (m *Trigger_AfterEach) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_Trigger_AfterEach.Marshal(b, m, deterministic)
 }
-func (dst *Trigger_AfterEach) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Trigger_AfterEach.Merge(dst, src)
+func (m *Trigger_AfterEach) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_Trigger_AfterEach.Merge(m, src)
 }
 func (m *Trigger_AfterEach) XXX_Size() int {
 	return xxx_messageInfo_Trigger_AfterEach.Size(m)
@@ -3901,7 +4173,7 @@ func (m *Trigger_AfterEndOfWindow) Reset()         { *m = Trigger_AfterEndOfWind
 func (m *Trigger_AfterEndOfWindow) String() string { return proto.CompactTextString(m) }
 func (*Trigger_AfterEndOfWindow) ProtoMessage()    {}
 func (*Trigger_AfterEndOfWindow) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_92025e5caa638397, []int{32, 3}
+	return fileDescriptor_cf57597c3a9659a9, []int{32, 3}
 }
 func (m *Trigger_AfterEndOfWindow) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Trigger_AfterEndOfWindow.Unmarshal(m, b)
@@ -3909,8 +4181,8 @@ func (m *Trigger_AfterEndOfWindow) XXX_Unmarshal(b []byte) error {
 func (m *Trigger_AfterEndOfWindow) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_Trigger_AfterEndOfWindow.Marshal(b, m, deterministic)
 }
-func (dst *Trigger_AfterEndOfWindow) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Trigger_AfterEndOfWindow.Merge(dst, src)
+func (m *Trigger_AfterEndOfWindow) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_Trigger_AfterEndOfWindow.Merge(m, src)
 }
 func (m *Trigger_AfterEndOfWindow) XXX_Size() int {
 	return xxx_messageInfo_Trigger_AfterEndOfWindow.Size(m)
@@ -3949,7 +4221,7 @@ func (m *Trigger_AfterProcessingTime) Reset()         { *m = Trigger_AfterProces
 func (m *Trigger_AfterProcessingTime) String() string { return proto.CompactTextString(m) }
 func (*Trigger_AfterProcessingTime) ProtoMessage()    {}
 func (*Trigger_AfterProcessingTime) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_92025e5caa638397, []int{32, 4}
+	return fileDescriptor_cf57597c3a9659a9, []int{32, 4}
 }
 func (m *Trigger_AfterProcessingTime) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Trigger_AfterProcessingTime.Unmarshal(m, b)
@@ -3957,8 +4229,8 @@ func (m *Trigger_AfterProcessingTime) XXX_Unmarshal(b []byte) error {
 func (m *Trigger_AfterProcessingTime) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_Trigger_AfterProcessingTime.Marshal(b, m, deterministic)
 }
-func (dst *Trigger_AfterProcessingTime) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Trigger_AfterProcessingTime.Merge(dst, src)
+func (m *Trigger_AfterProcessingTime) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_Trigger_AfterProcessingTime.Merge(m, src)
 }
 func (m *Trigger_AfterProcessingTime) XXX_Size() int {
 	return xxx_messageInfo_Trigger_AfterProcessingTime.Size(m)
@@ -3990,7 +4262,7 @@ func (m *Trigger_AfterSynchronizedProcessingTime) Reset() {
 func (m *Trigger_AfterSynchronizedProcessingTime) String() string { return proto.CompactTextString(m) }
 func (*Trigger_AfterSynchronizedProcessingTime) ProtoMessage()    {}
 func (*Trigger_AfterSynchronizedProcessingTime) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_92025e5caa638397, []int{32, 5}
+	return fileDescriptor_cf57597c3a9659a9, []int{32, 5}
 }
 func (m *Trigger_AfterSynchronizedProcessingTime) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Trigger_AfterSynchronizedProcessingTime.Unmarshal(m, b)
@@ -3998,8 +4270,8 @@ func (m *Trigger_AfterSynchronizedProcessingTime) XXX_Unmarshal(b []byte) error
 func (m *Trigger_AfterSynchronizedProcessingTime) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_Trigger_AfterSynchronizedProcessingTime.Marshal(b, m, deterministic)
 }
-func (dst *Trigger_AfterSynchronizedProcessingTime) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Trigger_AfterSynchronizedProcessingTime.Merge(dst, src)
+func (m *Trigger_AfterSynchronizedProcessingTime) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_Trigger_AfterSynchronizedProcessingTime.Merge(m, src)
 }
 func (m *Trigger_AfterSynchronizedProcessingTime) XXX_Size() int {
 	return xxx_messageInfo_Trigger_AfterSynchronizedProcessingTime.Size(m)
@@ -4022,7 +4294,7 @@ func (m *Trigger_Default) Reset()         { *m = Trigger_Default{} }
 func (m *Trigger_Default) String() string { return proto.CompactTextString(m) }
 func (*Trigger_Default) ProtoMessage()    {}
 func (*Trigger_Default) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_92025e5caa638397, []int{32, 6}
+	return fileDescriptor_cf57597c3a9659a9, []int{32, 6}
 }
 func (m *Trigger_Default) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Trigger_Default.Unmarshal(m, b)
@@ -4030,8 +4302,8 @@ func (m *Trigger_Default) XXX_Unmarshal(b []byte) error {
 func (m *Trigger_Default) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_Trigger_Default.Marshal(b, m, deterministic)
 }
-func (dst *Trigger_Default) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Trigger_Default.Merge(dst, src)
+func (m *Trigger_Default) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_Trigger_Default.Merge(m, src)
 }
 func (m *Trigger_Default) XXX_Size() int {
 	return xxx_messageInfo_Trigger_Default.Size(m)
@@ -4054,7 +4326,7 @@ func (m *Trigger_ElementCount) Reset()         { *m = Trigger_ElementCount{} }
 func (m *Trigger_ElementCount) String() string { return proto.CompactTextString(m) }
 func (*Trigger_ElementCount) ProtoMessage()    {}
 func (*Trigger_ElementCount) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_92025e5caa638397, []int{32, 7}
+	return fileDescriptor_cf57597c3a9659a9, []int{32, 7}
 }
 func (m *Trigger_ElementCount) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Trigger_ElementCount.Unmarshal(m, b)
@@ -4062,8 +4334,8 @@ func (m *Trigger_ElementCount) XXX_Unmarshal(b []byte) error {
 func (m *Trigger_ElementCount) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_Trigger_ElementCount.Marshal(b, m, deterministic)
 }
-func (dst *Trigger_ElementCount) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Trigger_ElementCount.Merge(dst, src)
+func (m *Trigger_ElementCount) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_Trigger_ElementCount.Merge(m, src)
 }
 func (m *Trigger_ElementCount) XXX_Size() int {
 	return xxx_messageInfo_Trigger_ElementCount.Size(m)
@@ -4093,7 +4365,7 @@ func (m *Trigger_Never) Reset()         { *m = Trigger_Never{} }
 func (m *Trigger_Never) String() string { return proto.CompactTextString(m) }
 func (*Trigger_Never) ProtoMessage()    {}
 func (*Trigger_Never) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_92025e5caa638397, []int{32, 8}
+	return fileDescriptor_cf57597c3a9659a9, []int{32, 8}
 }
 func (m *Trigger_Never) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Trigger_Never.Unmarshal(m, b)
@@ -4101,8 +4373,8 @@ func (m *Trigger_Never) XXX_Unmarshal(b []byte) error {
 func (m *Trigger_Never) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_Trigger_Never.Marshal(b, m, deterministic)
 }
-func (dst *Trigger_Never) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Trigger_Never.Merge(dst, src)
+func (m *Trigger_Never) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_Trigger_Never.Merge(m, src)
 }
 func (m *Trigger_Never) XXX_Size() int {
 	return xxx_messageInfo_Trigger_Never.Size(m)
@@ -4125,7 +4397,7 @@ func (m *Trigger_Always) Reset()         { *m = Trigger_Always{} }
 func (m *Trigger_Always) String() string { return proto.CompactTextString(m) }
 func (*Trigger_Always) ProtoMessage()    {}
 func (*Trigger_Always) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_92025e5caa638397, []int{32, 9}
+	return fileDescriptor_cf57597c3a9659a9, []int{32, 9}
 }
 func (m *Trigger_Always) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Trigger_Always.Unmarshal(m, b)
@@ -4133,8 +4405,8 @@ func (m *Trigger_Always) XXX_Unmarshal(b []byte) error {
 func (m *Trigger_Always) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_Trigger_Always.Marshal(b, m, deterministic)
 }
-func (dst *Trigger_Always) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Trigger_Always.Merge(dst, src)
+func (m *Trigger_Always) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_Trigger_Always.Merge(m, src)
 }
 func (m *Trigger_Always) XXX_Size() int {
 	return xxx_messageInfo_Trigger_Always.Size(m)
@@ -4161,7 +4433,7 @@ func (m *Trigger_OrFinally) Reset()         { *m = Trigger_OrFinally{} }
 func (m *Trigger_OrFinally) String() string { return proto.CompactTextString(m) }
 func (*Trigger_OrFinally) ProtoMessage()    {}
 func (*Trigger_OrFinally) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_92025e5caa638397, []int{32, 10}
+	return fileDescriptor_cf57597c3a9659a9, []int{32, 10}
 }
 func (m *Trigger_OrFinally) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Trigger_OrFinally.Unmarshal(m, b)
@@ -4169,8 +4441,8 @@ func (m *Trigger_OrFinally) XXX_Unmarshal(b []byte) error {
 func (m *Trigger_OrFinally) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_Trigger_OrFinally.Marshal(b, m, deterministic)
 }
-func (dst *Trigger_OrFinally) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Trigger_OrFinally.Merge(dst, src)
+func (m *Trigger_OrFinally) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_Trigger_OrFinally.Merge(m, src)
 }
 func (m *Trigger_OrFinally) XXX_Size() int {
 	return xxx_messageInfo_Trigger_OrFinally.Size(m)
@@ -4209,7 +4481,7 @@ func (m *Trigger_Repeat) Reset()         { *m = Trigger_Repeat{} }
 func (m *Trigger_Repeat) String() string { return proto.CompactTextString(m) }
 func (*Trigger_Repeat) ProtoMessage()    {}
 func (*Trigger_Repeat) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_92025e5caa638397, []int{32, 11}
+	return fileDescriptor_cf57597c3a9659a9, []int{32, 11}
 }
 func (m *Trigger_Repeat) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Trigger_Repeat.Unmarshal(m, b)
@@ -4217,8 +4489,8 @@ func (m *Trigger_Repeat) XXX_Unmarshal(b []byte) error {
 func (m *Trigger_Repeat) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_Trigger_Repeat.Marshal(b, m, deterministic)
 }
-func (dst *Trigger_Repeat) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Trigger_Repeat.Merge(dst, src)
+func (m *Trigger_Repeat) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_Trigger_Repeat.Merge(m, src)
 }
 func (m *Trigger_Repeat) XXX_Size() int {
 	return xxx_messageInfo_Trigger_Repeat.Size(m)
@@ -4254,7 +4526,7 @@ func (m *TimestampTransform) Reset()         { *m = TimestampTransform{} }
 func (m *TimestampTransform) String() string { return proto.CompactTextString(m) }
 func (*TimestampTransform) ProtoMessage()    {}
 func (*TimestampTransform) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_92025e5caa638397, []int{33}
+	return fileDescriptor_cf57597c3a9659a9, []int{33}
 }
 func (m *TimestampTransform) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_TimestampTransform.Unmarshal(m, b)
@@ -4262,8 +4534,8 @@ func (m *TimestampTransform) XXX_Unmarshal(b []byte) error {
 func (m *TimestampTransform) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_TimestampTransform.Marshal(b, m, deterministic)
 }
-func (dst *TimestampTransform) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_TimestampTransform.Merge(dst, src)
+func (m *TimestampTransform) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_TimestampTransform.Merge(m, src)
 }
 func (m *TimestampTransform) XXX_Size() int {
 	return xxx_messageInfo_TimestampTransform.Size(m)
@@ -4281,11 +4553,13 @@ type isTimestampTransform_TimestampTransform interface {
 type TimestampTransform_Delay_ struct {
 	Delay *TimestampTransform_Delay `protobuf:"bytes,1,opt,name=delay,proto3,oneof"`
 }
+
 type TimestampTransform_AlignTo_ struct {
 	AlignTo *TimestampTransform_AlignTo `protobuf:"bytes,2,opt,name=align_to,json=alignTo,proto3,oneof"`
 }
 
-func (*TimestampTransform_Delay_) isTimestampTransform_TimestampTransform()   {}
+func (*TimestampTransform_Delay_) isTimestampTransform_TimestampTransform() {}
+
 func (*TimestampTransform_AlignTo_) isTimestampTransform_TimestampTransform() {}
 
 func (m *TimestampTransform) GetTimestampTransform() isTimestampTransform_TimestampTransform {
@@ -4395,7 +4669,7 @@ func (m *TimestampTransform_Delay) Reset()         { *m = TimestampTransform_Del
 func (m *TimestampTransform_Delay) String() string { return proto.CompactTextString(m) }
 func (*TimestampTransform_Delay) ProtoMessage()    {}
 func (*TimestampTransform_Delay) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_92025e5caa638397, []int{33, 0}
+	return fileDescriptor_cf57597c3a9659a9, []int{33, 0}
 }
 func (m *TimestampTransform_Delay) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_TimestampTransform_Delay.Unmarshal(m, b)
@@ -4403,8 +4677,8 @@ func (m *TimestampTransform_Delay) XXX_Unmarshal(b []byte) error {
 func (m *TimestampTransform_Delay) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_TimestampTransform_Delay.Marshal(b, m, deterministic)
 }
-func (dst *TimestampTransform_Delay) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_TimestampTransform_Delay.Merge(dst, src)
+func (m *TimestampTransform_Delay) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_TimestampTransform_Delay.Merge(m, src)
 }
 func (m *TimestampTransform_Delay) XXX_Size() int {
 	return xxx_messageInfo_TimestampTransform_Delay.Size(m)
@@ -4438,7 +4712,7 @@ func (m *TimestampTransform_AlignTo) Reset()         { *m = TimestampTransform_A
 func (m *TimestampTransform_AlignTo) String() string { return proto.CompactTextString(m) }
 func (*TimestampTransform_AlignTo) ProtoMessage()    {}
 func (*TimestampTransform_AlignTo) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_92025e5caa638397, []int{33, 1}
+	return fileDescriptor_cf57597c3a9659a9, []int{33, 1}
 }
 func (m *TimestampTransform_AlignTo) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_TimestampTransform_AlignTo.Unmarshal(m, b)
@@ -4446,8 +4720,8 @@ func (m *TimestampTransform_AlignTo) XXX_Unmarshal(b []byte) error {
 func (m *TimestampTransform_AlignTo) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_TimestampTransform_AlignTo.Marshal(b, m, deterministic)
 }
-func (dst *TimestampTransform_AlignTo) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_TimestampTransform_AlignTo.Merge(dst, src)
+func (m *TimestampTransform_AlignTo) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_TimestampTransform_AlignTo.Merge(m, src)
 }
 func (m *TimestampTransform_AlignTo) XXX_Size() int {
 	return xxx_messageInfo_TimestampTransform_AlignTo.Size(m)
@@ -4506,7 +4780,7 @@ func (m *SideInput) Reset()         { *m = SideInput{} }
 func (m *SideInput) String() string { return proto.CompactTextString(m) }
 func (*SideInput) ProtoMessage()    {}
 func (*SideInput) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_92025e5caa638397, []int{34}
+	return fileDescriptor_cf57597c3a9659a9, []int{34}
 }
 func (m *SideInput) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_SideInput.Unmarshal(m, b)
@@ -4514,8 +4788,8 @@ func (m *SideInput) XXX_Unmarshal(b []byte) error {
 func (m *SideInput) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_SideInput.Marshal(b, m, deterministic)
 }
-func (dst *SideInput) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_SideInput.Merge(dst, src)
+func (m *SideInput) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_SideInput.Merge(m, src)
 }
 func (m *SideInput) XXX_Size() int {
 	return xxx_messageInfo_SideInput.Size(m)
@@ -4547,15 +4821,16 @@ func (m *SideInput) GetWindowMappingFn() *SdkFunctionSpec {
 	return nil
 }
 
-// An environment for executing UDFs. Generally an SDK container URL, but
-// there can be many for a single SDK, for example to provide dependency
-// isolation.
+// An environment for executing UDFs. By default, an SDK container URL, but
+// can also be a process forked by a command, or an externally managed process.
 type Environment struct {
-	// (Required) The URL of a container
-	//
-	// TODO: reconcile with Fn API's DockerContainer structure by
-	// adding adequate metadata to know how to interpret the container
-	Url                  string   `protobuf:"bytes,1,opt,name=url,proto3" json:"url,omitempty"`
+	// Deprecated
+	Url string `protobuf:"bytes,1,opt,name=url,proto3" json:"url,omitempty"`
+	// (Required) The URN of the payload
+	Urn string `protobuf:"bytes,2,opt,name=urn,proto3" json:"urn,omitempty"`
+	// (Optional) The data specifying any parameters to the URN. If
+	// the URN does not require any arguments, this may be omitted.
+	Payload              []byte   `protobuf:"bytes,3,opt,name=payload,proto3" json:"payload,omitempty"`
 	XXX_NoUnkeyedLiteral struct{} `json:"-"`
 	XXX_unrecognized     []byte   `json:"-"`
 	XXX_sizecache        int32    `json:"-"`
@@ -4565,7 +4840,7 @@ func (m *Environment) Reset()         { *m = Environment{} }
 func (m *Environment) String() string { return proto.CompactTextString(m) }
 func (*Environment) ProtoMessage()    {}
 func (*Environment) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_92025e5caa638397, []int{35}
+	return fileDescriptor_cf57597c3a9659a9, []int{35}
 }
 func (m *Environment) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Environment.Unmarshal(m, b)
@@ -4573,8 +4848,8 @@ func (m *Environment) XXX_Unmarshal(b []byte) error {
 func (m *Environment) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_Environment.Marshal(b, m, deterministic)
 }
-func (dst *Environment) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Environment.Merge(dst, src)
+func (m *Environment) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_Environment.Merge(m, src)
 }
 func (m *Environment) XXX_Size() int {
 	return xxx_messageInfo_Environment.Size(m)
@@ -4592,6 +4867,151 @@ func (m *Environment) GetUrl() string {
 	return ""
 }
 
+func (m *Environment) GetUrn() string {
+	if m != nil {
+		return m.Urn
+	}
+	return ""
+}
+
+func (m *Environment) GetPayload() []byte {
+	if m != nil {
+		return m.Payload
+	}
+	return nil
+}
+
+type StandardEnvironments struct {
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *StandardEnvironments) Reset()         { *m = StandardEnvironments{} }
+func (m *StandardEnvironments) String() string { return proto.CompactTextString(m) }
+func (*StandardEnvironments) ProtoMessage()    {}
+func (*StandardEnvironments) Descriptor() ([]byte, []int) {
+	return fileDescriptor_cf57597c3a9659a9, []int{36}
+}
+func (m *StandardEnvironments) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_StandardEnvironments.Unmarshal(m, b)
+}
+func (m *StandardEnvironments) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_StandardEnvironments.Marshal(b, m, deterministic)
+}
+func (m *StandardEnvironments) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_StandardEnvironments.Merge(m, src)
+}
+func (m *StandardEnvironments) XXX_Size() int {
+	return xxx_messageInfo_StandardEnvironments.Size(m)
+}
+func (m *StandardEnvironments) XXX_DiscardUnknown() {
+	xxx_messageInfo_StandardEnvironments.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_StandardEnvironments proto.InternalMessageInfo
+
+// The payload of a Docker image
+type DockerPayload struct {
+	ContainerImage       string   `protobuf:"bytes,1,opt,name=container_image,json=containerImage,proto3" json:"container_image,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *DockerPayload) Reset()         { *m = DockerPayload{} }
+func (m *DockerPayload) String() string { return proto.CompactTextString(m) }
+func (*DockerPayload) ProtoMessage()    {}
+func (*DockerPayload) Descriptor() ([]byte, []int) {
+	return fileDescriptor_cf57597c3a9659a9, []int{37}
+}
+func (m *DockerPayload) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_DockerPayload.Unmarshal(m, b)
+}
+func (m *DockerPayload) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_DockerPayload.Marshal(b, m, deterministic)
+}
+func (m *DockerPayload) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_DockerPayload.Merge(m, src)
+}
+func (m *DockerPayload) XXX_Size() int {
+	return xxx_messageInfo_DockerPayload.Size(m)
+}
+func (m *DockerPayload) XXX_DiscardUnknown() {
+	xxx_messageInfo_DockerPayload.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_DockerPayload proto.InternalMessageInfo
+
+func (m *DockerPayload) GetContainerImage() string {
+	if m != nil {
+		return m.ContainerImage
+	}
+	return ""
+}
+
+type ProcessPayload struct {
+	Os                   string            `protobuf:"bytes,1,opt,name=os,proto3" json:"os,omitempty"`
+	Arch                 string            `protobuf:"bytes,2,opt,name=arch,proto3" json:"arch,omitempty"`
+	Command              string            `protobuf:"bytes,3,opt,name=command,proto3" json:"command,omitempty"`
+	Env                  map[string]string `protobuf:"bytes,4,rep,name=env,proto3" json:"env,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"`
+	XXX_NoUnkeyedLiteral struct{}          `json:"-"`
+	XXX_unrecognized     []byte            `json:"-"`
+	XXX_sizecache        int32             `json:"-"`
+}
+
+func (m *ProcessPayload) Reset()         { *m = ProcessPayload{} }
+func (m *ProcessPayload) String() string { return proto.CompactTextString(m) }
+func (*ProcessPayload) ProtoMessage()    {}
+func (*ProcessPayload) Descriptor() ([]byte, []int) {
+	return fileDescriptor_cf57597c3a9659a9, []int{38}
+}
+func (m *ProcessPayload) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_ProcessPayload.Unmarshal(m, b)
+}
+func (m *ProcessPayload) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_ProcessPayload.Marshal(b, m, deterministic)
+}
+func (m *ProcessPayload) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_ProcessPayload.Merge(m, src)
+}
+func (m *ProcessPayload) XXX_Size() int {
+	return xxx_messageInfo_ProcessPayload.Size(m)
+}
+func (m *ProcessPayload) XXX_DiscardUnknown() {
+	xxx_messageInfo_ProcessPayload.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_ProcessPayload proto.InternalMessageInfo
+
+func (m *ProcessPayload) GetOs() string {
+	if m != nil {
+		return m.Os
+	}
+	return ""
+}
+
+func (m *ProcessPayload) GetArch() string {
+	if m != nil {
+		return m.Arch
+	}
+	return ""
+}
+
+func (m *ProcessPayload) GetCommand() string {
+	if m != nil {
+		return m.Command
+	}
+	return ""
+}
+
+func (m *ProcessPayload) GetEnv() map[string]string {
+	if m != nil {
+		return m.Env
+	}
+	return nil
+}
+
 // A specification of a user defined function.
 //
 type SdkFunctionSpec struct {
@@ -4609,7 +5029,7 @@ func (m *SdkFunctionSpec) Reset()         { *m = SdkFunctionSpec{} }
 func (m *SdkFunctionSpec) String() string { return proto.CompactTextString(m) }
 func (*SdkFunctionSpec) ProtoMessage()    {}
 func (*SdkFunctionSpec) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_92025e5caa638397, []int{36}
+	return fileDescriptor_cf57597c3a9659a9, []int{39}
 }
 func (m *SdkFunctionSpec) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_SdkFunctionSpec.Unmarshal(m, b)
@@ -4617,8 +5037,8 @@ func (m *SdkFunctionSpec) XXX_Unmarshal(b []byte) error {
 func (m *SdkFunctionSpec) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_SdkFunctionSpec.Marshal(b, m, deterministic)
 }
-func (dst *SdkFunctionSpec) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_SdkFunctionSpec.Merge(dst, src)
+func (m *SdkFunctionSpec) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_SdkFunctionSpec.Merge(m, src)
 }
 func (m *SdkFunctionSpec) XXX_Size() int {
 	return xxx_messageInfo_SdkFunctionSpec.Size(m)
@@ -4688,7 +5108,7 @@ func (m *FunctionSpec) Reset()         { *m = FunctionSpec{} }
 func (m *FunctionSpec) String() string { return proto.CompactTextString(m) }
 func (*FunctionSpec) ProtoMessage()    {}
 func (*FunctionSpec) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_92025e5caa638397, []int{37}
+	return fileDescriptor_cf57597c3a9659a9, []int{40}
 }
 func (m *FunctionSpec) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_FunctionSpec.Unmarshal(m, b)
@@ -4696,8 +5116,8 @@ func (m *FunctionSpec) XXX_Unmarshal(b []byte) error {
 func (m *FunctionSpec) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_FunctionSpec.Marshal(b, m, deterministic)
 }
-func (dst *FunctionSpec) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_FunctionSpec.Merge(dst, src)
+func (m *FunctionSpec) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_FunctionSpec.Merge(m, src)
 }
 func (m *FunctionSpec) XXX_Size() int {
 	return xxx_messageInfo_FunctionSpec.Size(m)
@@ -4735,7 +5155,7 @@ func (m *DisplayData) Reset()         { *m = DisplayData{} }
 func (m *DisplayData) String() string { return proto.CompactTextString(m) }
 func (*DisplayData) ProtoMessage()    {}
 func (*DisplayData) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_92025e5caa638397, []int{38}
+	return fileDescriptor_cf57597c3a9659a9, []int{41}
 }
 func (m *DisplayData) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_DisplayData.Unmarshal(m, b)
@@ -4743,8 +5163,8 @@ func (m *DisplayData) XXX_Unmarshal(b []byte) error {
 func (m *DisplayData) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_DisplayData.Marshal(b, m, deterministic)
 }
-func (dst *DisplayData) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_DisplayData.Merge(dst, src)
+func (m *DisplayData) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_DisplayData.Merge(m, src)
 }
 func (m *DisplayData) XXX_Size() int {
 	return xxx_messageInfo_DisplayData.Size(m)
@@ -4779,7 +5199,7 @@ func (m *DisplayData_Identifier) Reset()         { *m = DisplayData_Identifier{}
 func (m *DisplayData_Identifier) String() string { return proto.CompactTextString(m) }
 func (*DisplayData_Identifier) ProtoMessage()    {}
 func (*DisplayData_Identifier) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_92025e5caa638397, []int{38, 0}
+	return fileDescriptor_cf57597c3a9659a9, []int{41, 0}
 }
 func (m *DisplayData_Identifier) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_DisplayData_Identifier.Unmarshal(m, b)
@@ -4787,8 +5207,8 @@ func (m *DisplayData_Identifier) XXX_Unmarshal(b []byte) error {
 func (m *DisplayData_Identifier) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_DisplayData_Identifier.Marshal(b, m, deterministic)
 }
-func (dst *DisplayData_Identifier) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_DisplayData_Identifier.Merge(dst, src)
+func (m *DisplayData_Identifier) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_DisplayData_Identifier.Merge(m, src)
 }
 func (m *DisplayData_Identifier) XXX_Size() int {
 	return xxx_messageInfo_DisplayData_Identifier.Size(m)
@@ -4843,7 +5263,7 @@ func (m *DisplayData_Item) Reset()         { *m = DisplayData_Item{} }
 func (m *DisplayData_Item) String() string { return proto.CompactTextString(m) }
 func (*DisplayData_Item) ProtoMessage()    {}
 func (*DisplayData_Item) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_92025e5caa638397, []int{38, 1}
+	return fileDescriptor_cf57597c3a9659a9, []int{41, 1}
 }
 func (m *DisplayData_Item) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_DisplayData_Item.Unmarshal(m, b)
@@ -4851,8 +5271,8 @@ func (m *DisplayData_Item) XXX_Unmarshal(b []byte) error {
 func (m *DisplayData_Item) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_DisplayData_Item.Marshal(b, m, deterministic)
 }
-func (dst *DisplayData_Item) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_DisplayData_Item.Merge(dst, src)
+func (m *DisplayData_Item) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_DisplayData_Item.Merge(m, src)
 }
 func (m *DisplayData_Item) XXX_Size() int {
 	return xxx_messageInfo_DisplayData_Item.Size(m)
@@ -4915,7 +5335,7 @@ func (m *DisplayData_Type) Reset()         { *m = DisplayData_Type{} }
 func (m *DisplayData_Type) String() string { return proto.CompactTextString(m) }
 func (*DisplayData_Type) ProtoMessage()    {}
 func (*DisplayData_Type) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_92025e5caa638397, []int{38, 2}
+	return fileDescriptor_cf57597c3a9659a9, []int{41, 2}
 }
 func (m *DisplayData_Type) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_DisplayData_Type.Unmarshal(m, b)
@@ -4923,8 +5343,8 @@ func (m *DisplayData_Type) XXX_Unmarshal(b []byte) error {
 func (m *DisplayData_Type) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_DisplayData_Type.Marshal(b, m, deterministic)
 }
-func (dst *DisplayData_Type) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_DisplayData_Type.Merge(dst, src)
+func (m *DisplayData_Type) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_DisplayData_Type.Merge(m, src)
 }
 func (m *DisplayData_Type) XXX_Size() int {
 	return xxx_messageInfo_DisplayData_Type.Size(m)
@@ -4961,6 +5381,8 @@ func init() {
 	proto.RegisterType((*PCollection)(nil), "org.apache.beam.model.pipeline.v1.PCollection")
 	proto.RegisterType((*ExecutableStagePayload)(nil), "org.apache.beam.model.pipeline.v1.ExecutableStagePayload")
 	proto.RegisterType((*ExecutableStagePayload_SideInputId)(nil), "org.apache.beam.model.pipeline.v1.ExecutableStagePayload.SideInputId")
+	proto.RegisterType((*ExecutableStagePayload_UserStateId)(nil), "org.apache.beam.model.pipeline.v1.ExecutableStagePayload.UserStateId")
+	proto.RegisterType((*ExecutableStagePayload_TimerId)(nil), "org.apache.beam.model.pipeline.v1.ExecutableStagePayload.TimerId")
 	proto.RegisterType((*ParDoPayload)(nil), "org.apache.beam.model.pipeline.v1.ParDoPayload")
 	proto.RegisterMapType((map[string]*SideInput)(nil), "org.apache.beam.model.pipeline.v1.ParDoPayload.SideInputsEntry")
 	proto.RegisterMapType((map[string]*StateSpec)(nil), "org.apache.beam.model.pipeline.v1.ParDoPayload.StateSpecsEntry")
@@ -5013,6 +5435,10 @@ func init() {
 	proto.RegisterType((*TimestampTransform_AlignTo)(nil), "org.apache.beam.model.pipeline.v1.TimestampTransform.AlignTo")
 	proto.RegisterType((*SideInput)(nil), "org.apache.beam.model.pipeline.v1.SideInput")
 	proto.RegisterType((*Environment)(nil), "org.apache.beam.model.pipeline.v1.Environment")
+	proto.RegisterType((*StandardEnvironments)(nil), "org.apache.beam.model.pipeline.v1.StandardEnvironments")
+	proto.RegisterType((*DockerPayload)(nil), "org.apache.beam.model.pipeline.v1.DockerPayload")
+	proto.RegisterType((*ProcessPayload)(nil), "org.apache.beam.model.pipeline.v1.ProcessPayload")
+	proto.RegisterMapType((map[string]string)(nil), "org.apache.beam.model.pipeline.v1.ProcessPayload.EnvEntry")
 	proto.RegisterType((*SdkFunctionSpec)(nil), "org.apache.beam.model.pipeline.v1.SdkFunctionSpec")
 	proto.RegisterType((*FunctionSpec)(nil), "org.apache.beam.model.pipeline.v1.FunctionSpec")
 	proto.RegisterType((*DisplayData)(nil), "org.apache.beam.model.pipeline.v1.DisplayData")
@@ -5034,295 +5460,310 @@ func init() {
 	proto.RegisterEnum("org.apache.beam.model.pipeline.v1.OnTimeBehavior_Enum", OnTimeBehavior_Enum_name, OnTimeBehavior_Enum_value)
 	proto.RegisterEnum("org.apache.beam.model.pipeline.v1.OutputTime_Enum", OutputTime_Enum_name, OutputTime_Enum_value)
 	proto.RegisterEnum("org.apache.beam.model.pipeline.v1.TimeDomain_Enum", TimeDomain_Enum_name, TimeDomain_Enum_value)
+	proto.RegisterEnum("org.apache.beam.model.pipeline.v1.StandardEnvironments_Environments", StandardEnvironments_Environments_name, StandardEnvironments_Environments_value)
 	proto.RegisterEnum("org.apache.beam.model.pipeline.v1.DisplayData_Type_Enum", DisplayData_Type_Enum_name, DisplayData_Type_Enum_value)
 	proto.RegisterExtension(E_BeamUrn)
 }
 
-func init() {
-	proto.RegisterFile("beam_runner_api.proto", fileDescriptor_beam_runner_api_92025e5caa638397)
-}
+func init() { proto.RegisterFile("beam_runner_api.proto", fileDescriptor_cf57597c3a9659a9) }
 
-var fileDescriptor_beam_runner_api_92025e5caa638397 = []byte{
-	// 4490 bytes of a gzipped FileDescriptorProto
+var fileDescriptor_cf57597c3a9659a9 = []byte{
+	// 4747 bytes of a gzipped FileDescriptorProto
 	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xbc, 0x5b, 0xcd, 0x6f, 0xe3, 0x58,
-	0x72, 0xd7, 0xb7, 0xad, 0x92, 0x2c, 0xd3, 0xcf, 0xdd, 0x3d, 0x6e, 0x66, 0x66, 0xba, 0x9b, 0x33,
-	0x99, 0xe9, 0x9d, 0xed, 0x51, 0x8f, 0xbb, 0xe7, 0xd3, 0xb3, 0xdb, 0xb3, 0xb2, 0x45, 0x59, 0xec,
-	0xd6, 0xd7, 0x50, 0xb2, 0x3d, 0x3d, 0x3b, 0x19, 0x0e, 0x2d, 0x3e, 0xc9, 0x84, 0x29, 0x52, 0x21,
-	0x29, 0x7b, 0x14, 0x64, 0xb1, 0x40, 0x0e, 0x49, 0x80, 0x5c, 0x92, 0x43, 0x0e, 0x7b, 0xdd, 0x00,
-	0xb9, 0x24, 0x40, 0x90, 0x6c, 0x10, 0x20, 0x97, 0x04, 0xd8, 0xe4, 0x2f, 0x48, 0x6e, 0x39, 0xe7,
-	0x1f, 0x08, 0x82, 0x20, 0x40, 0x2e, 0x09, 0xde, 0x07, 0x29, 0x4a, 0xb2, 0xbb, 0x25, 0xbb, 0x91,
-	0x9b, 0x58, 0x7c, 0xf5, 0xab, 0x62, 0xbd, 0x7a, 0xf5, 0xaa, 0xea, 0x3d, 0xc1, 0xcd, 0x63, 0xac,
-	0x0f, 0x34, 0x77, 0x64, 0xdb, 0xd8, 0xd5, 0xf4, 0xa1, 0x59, 0x1c, 0xba, 0x8e, 0xef, 0xa0, 0x7b,
-	0x8e, 0xdb, 0x2f, 0xea, 0x43, 0xbd, 0x7b, 0x82, 0x8b, 0x64, 0x44, 0x71, 0xe0, 0x18, 0xd8, 0x2a,
-	0x0e, 0xcd, 0x21, 0xb6, 0x4c, 0x1b, 0x17, 0xcf, 0xb6, 0xc5, 0xdb, 0x7d, 0xc7, 0xe9, 0x5b, 0xf8,
-	0x21, 0x65, 0x38, 0x1e, 0xf5, 0x1e, 0xea, 0xf6, 0x98, 0x71, 0x8b, 0x77, 0x67, 0x5f, 0x19, 0xd8,
-	0xeb, 0xba, 0xe6, 0xd0, 0x77, 0x5c, 0x36, 0x42, 0xfa, 0xbb, 0x55, 0x80, 0x3d, 0x67, 0x30, 0x74,
-	0x6c, 0x6c, 0xfb, 0x1e, 0xfa, 0x2d, 0x00, 0xdf, 0xd5, 0x6d, 0xaf, 0xe7, 0xb8, 0x03, 0x6f, 0x2b,
-	0x7e, 0x37, 0x79, 0x3f, 0xf7, 0xe8, 0xc7, 0xc5, 0x97, 0xea, 0x50, 0x9c, 0x40, 0x14, 0x3b, 0x21,
-	0xbf, 0x6c, 0xfb, 0xee, 0x58, 0x8d, 0x00, 0xa2, 0x2e, 0xe4, 0x87, 0x5d, 0xc7, 0xb2, 0x70, 0xd7,
-	0x37, 0x1d, 0xdb, 0xdb, 0x4a, 0x50, 0x01, 0x5f, 0x2c, 0x27, 0xa0, 0x15, 0x41, 0x60, 0x22, 0xa6,
-	0x40, 0xd1, 0x18, 0x6e, 0x9c, 0x9b, 0xb6, 0xe1, 0x9c, 0x9b, 0x76, 0x5f, 0xf3, 0x7c, 0x57, 0xf7,
-	0x71, 0xdf, 0xc4, 0xde, 0x56, 0x92, 0x0a, 0xab, 0x2c, 0x27, 0xec, 0x28, 0x40, 0x6a, 0x87, 0x40,
-	0x4c, 0xe6, 0xe6, 0xf9, 0xfc, 0x1b, 0xf4, 0x25, 0x64, 0xba, 0x8e, 0x81, 0x5d, 0x6f, 0x2b, 0x45,
-	0x85, 0x7d, 0xb6, 0x9c, 0xb0, 0x3d, 0xca, 0xcb, 0xf0, 0x39, 0x10, 0x31, 0x19, 0xb6, 0xcf, 0x4c,
-	0xd7, 0xb1, 0x07, 0x64, 0xcc, 0x56, 0xfa, 0x2a, 0x26, 0x93, 0x23, 0x08, 0xdc, 0x64, 0x51, 0x50,
-	0xd1, 0x82, 0xf5, 0x99, 0x69, 0x43, 0x02, 0x24, 0x4f, 0xf1, 0x78, 0x2b, 0x7e, 0x37, 0x7e, 0x3f,
-	0xab, 0x92, 0x9f, 0x68, 0x0f, 0xd2, 0x67, 0xba, 0x35, 0xc2, 0x5b, 0x89, 0xbb, 0xf1, 0xfb, 0xb9,
-	0x47, 0xef, 0x2f, 0xa0, 0x42, 0x2b, 0x44, 0x55, 0x19, 0xef, 0x4e, 0xe2, 0xd3, 0xb8, 0xe8, 0xc0,
-	0xc6, 0xdc, 0x1c, 0x5e, 0x20, 0xaf, 0x3c, 0x2d, 0xaf, 0xb8, 0x88, 0xbc, 0xbd, 0x10, 0x36, 0x2a,
-	0xf0, 0x77, 0x61, 0xeb, 0xb2, 0x79, 0xbc, 0x40, 0xee, 0xd3, 0x69, 0xb9, 0x1f, 0x2e, 0x20, 0x77,
-	0x16, 0x7d, 0x1c, 0x95, 0xde, 0x85, 0x5c, 0x64, 0x62, 0x2f, 0x10, 0xf8, 0x64, 0x5a, 0xe0, 0xfd,
-	0x85, 0xe6, 0xd6, 0xc0, 0xee, 0x8c, 0x4d, 0xe7, 0x26, 0xf9, 0xd5, 0xd8, 0x34, 0x02, 0x1b, 0x11,
-	0x28, 0xfd, 0xfd, 0x2a, 0xdc, 0xac, 0x63, 0xcf, 0xd3, 0xfb, 0xf8, 0xc8, 0xf4, 0x4f, 0x22, 0x31,
-	0xa4, 0x0e, 0xd0, 0x0d, 0x9f, 0xa8, 0xf0, 0xc5, 0x9c, 0x65, 0x02, 0xa1, 0x46, 0x00, 0xd0, 0x4f,
-	0x20, 0x4d, 0x97, 0xc2, 0xb2, 0xd6, 0xa9, 0xc6, 0x54, 0xc6, 0x88, 0xbe, 0x81, 0xf5, 0xae, 0x33,
-	0x38, 0x36, 0x6d, 0xac, 0x0d, 0xf5, 0xb1, 0xe5, 0xe8, 0xc6, 0x56, 0x92, 0x62, 0x6d, 0x2f, 0xa6,
-	0x15, 0xe1, 0x6c, 0x31, 0xc6, 0x6a, 0x4c, 0x2d, 0x74, 0xa7, 0x28, 0xe8, 0x3b, 0xd8, 0xf0, 0x8c,
-	0x53, 0xad, 0x37, 0xb2, 0xa9, 0xdf, 0x69, 0xde, 0x10, 0x77, 0xb7, 0x52, 0x14, 0xff, 0xd1, 0x02,
-	0xf8, 0x6d, 0xe3, 0xb4, 0xc2, 0x59, 0xdb, 0x43, 0xdc, 0xad, 0xc6, 0xd4, 0x75, 0x6f, 0x9a, 0x84,
-	0x8e, 0xa0, 0x30, 0xd4, 0x5d, 0xcd, 0x70, 0x42, 0xf5, 0x33, 0x14, 0xfe, 0xe1, 0x22, 0x2b, 0x42,
-	0x77, 0xcb, 0xce, 0x44, 0xf9, 0xfc, 0x30, 0xf2, 0x8c, 0x9a, 0x00, 0xc3, 0x30, 0x3a, 0x6f, 0xad,
-	0x5c, 0x61, 0x59, 0x57, 0x63, 0x6a, 0x04, 0x02, 0xa9, 0x90, 0x8b, 0x84, 0xe2, 0xad, 0xd5, 0xab,
-	0x2c, 0xdc, 0x6a, 0x4c, 0x8d, 0x82, 0xa0, 0x36, 0xe4, 0x5d, 0xac, 0x1b, 0xe1, 0xb7, 0x67, 0x17,
-	0x06, 0x55, 0xb1, 0x6e, 0x4c, 0x3e, 0x3d, 0xe7, 0x4e, 0x1e, 0x89, 0x8f, 0x7a, 0xa6, 0x81, 0x35,
-	0xd3, 0x1e, 0x8e, 0xfc, 0xad, 0x1c, 0x85, 0x7c, 0xb0, 0xc8, 0x6c, 0x99, 0x06, 0x56, 0x08, 0x4f,
-	0x35, 0xa6, 0x66, 0xbd, 0xe0, 0x01, 0xf5, 0x80, 0x6f, 0x07, 0x9a, 0x69, 0xfb, 0x93, 0x69, 0xca,
-	0x2f, 0x19, 0x40, 0x14, 0xdb, 0x8f, 0xcc, 0xd5, 0xc6, 0xf9, 0x2c, 0x11, 0x61, 0x40, 0x73, 0x5b,
-	0xdb, 0x78, 0x6b, 0xed, 0xea, 0x71, 0x6a, 0x22, 0x26, 0x42, 0x44, 0x87, 0xb0, 0x36, 0xed, 0xce,
-	0x85, 0x85, 0xfd, 0x6d, 0xc6, 0x97, 0xf3, 0xbd, 0xc8, 0xf3, 0x6e, 0x06, 0x52, 0xae, 0xe3, 0xf8,
-	0xd2, 0xbf, 0xc5, 0x61, 0xb5, 0xc5, 0x99, 0x5e, 0x75, 0xb8, 0x78, 0x00, 0x88, 0xc8, 0xd0, 0x42,
-	0xa7, 0xd4, 0x4c, 0x83, 0x25, 0x1a, 0x59, 0x55, 0x20, 0x6f, 0x42, 0xdf, 0x55, 0x0c, 0xb2, 0x61,
-	0xe7, 0x0d, 0xd3, 0x1b, 0x5a, 0xfa, 0x58, 0x33, 0x74, 0x5f, 0xe7, 0x71, 0x61, 0x11, 0xe7, 0x2a,
-	0x33, 0xb6, 0xb2, 0xee, 0xeb, 0x6a, 0xce, 0x98, 0x3c, 0x48, 0x7f, 0x94, 0x02, 0x98, 0x2c, 0x10,
-	0x74, 0x07, 0x72, 0x23, 0xdb, 0xfc, 0xed, 0x11, 0xd6, 0x6c, 0x7d, 0x80, 0xb7, 0xd2, 0x34, 0x16,
-	0x03, 0x23, 0x35, 0xf4, 0x01, 0x46, 0x7b, 0x90, 0xa2, 0x36, 0x8e, 0x5f, 0xc9, 0xc6, 0x2a, 0x65,
-	0x46, 0x6f, 0xc3, 0x9a, 0x37, 0x3a, 0x8e, 0xa4, 0x6e, 0xec, 0x83, 0xa7, 0x89, 0x24, 0x3d, 0xa1,
-	0x0e, 0x1f, 0xe4, 0x42, 0x9f, 0x2d, 0xb5, 0xd6, 0x8b, 0xd4, 0xd7, 0x83, 0xf4, 0x84, 0x01, 0xa1,
-	0x0e, 0xac, 0x38, 0x23, 0x9f, 0x62, 0xb2, 0x94, 0x67, 0x67, 0x39, 0xcc, 0x26, 0x63, 0x66, 0xa0,
-	0x01, 0xd4, 0xdc, 0xb4, 0x64, 0xae, 0x3d, 0x2d, 0xe2, 0x67, 0x90, 0x8b, 0xe8, 0x7f, 0xc1, 0xd6,
-	0x78, 0x23, 0xba, 0x35, 0x66, 0xa3, 0x7b, 0xeb, 0x0e, 0xe4, 0xa3, 0x6a, 0x2e, 0xc3, 0x2b, 0xfd,
-	0x4b, 0x1e, 0x36, 0xdb, 0xbe, 0x6e, 0x1b, 0xba, 0x6b, 0x4c, 0x3e, 0xdb, 0x93, 0xfe, 0x3b, 0x01,
-	0xd0, 0x72, 0xcd, 0x81, 0xe9, 0x9b, 0x67, 0xd8, 0x43, 0xef, 0x43, 0xa6, 0x55, 0x52, 0xb5, 0x72,
-	0x53, 0x88, 0x89, 0xf7, 0x7e, 0xf9, 0xb7, 0xff, 0xf3, 0xeb, 0xf4, 0x6f, 0x8c, 0x5c, 0x7b, 0x87,
-	0x7c, 0xe4, 0x4e, 0x38, 0x81, 0x3b, 0x43, 0xdd, 0x35, 0x9c, 0x9d, 0xb3, 0x6d, 0xf4, 0x00, 0x56,
-	0x2a, 0xb5, 0x52, 0xa7, 0x23, 0x37, 0x84, 0xb8, 0x78, 0x87, 0x8e, 0xbf, 0x3d, 0x33, 0xb6, 0x67,
-	0xe9, 0xbe, 0x8f, 0x6d, 0x32, 0xfa, 0x63, 0xc8, 0xef, 0xab, 0xcd, 0x83, 0x96, 0xb6, 0xfb, 0x5c,
-	0x7b, 0x26, 0x3f, 0x17, 0x12, 0xe2, 0xdb, 0x94, 0xe5, 0xcd, 0x19, 0x96, 0xbe, 0xeb, 0x8c, 0x86,
-	0xda, 0xf1, 0x58, 0x3b, 0xc5, 0x63, 0x2e, 0x45, 0xa9, 0xb7, 0x0e, 0x6a, 0x6d, 0x59, 0x48, 0x5e,
-	0x22, 0xc5, 0x1c, 0x0c, 0x47, 0x96, 0x87, 0xc9, 0xe8, 0x4f, 0xa0, 0x50, 0x6a, 0xb7, 0x95, 0xfd,
-	0x86, 0x76, 0xa4, 0x34, 0xca, 0xcd, 0xa3, 0xb6, 0x90, 0x12, 0xdf, 0xa2, 0x4c, 0x6f, 0xcc, 0x30,
-	0x45, 0x42, 0x25, 0x63, 0xcc, 0x75, 0xe4, 0x76, 0x47, 0x6b, 0x77, 0x54, 0xb9, 0x54, 0x17, 0xd2,
-	0xe2, 0x3b, 0x94, 0xeb, 0xee, 0x05, 0x06, 0xf0, 0xb1, 0xe7, 0x7b, 0xbe, 0x4b, 0x88, 0x67, 0xdb,
-	0xe8, 0x43, 0xc8, 0xd5, 0x4b, 0xad, 0x50, 0x5c, 0xe6, 0x12, 0x71, 0x03, 0x7d, 0xa8, 0x31, 0x91,
-	0xde, 0xce, 0xd9, 0xb6, 0xe4, 0xc3, 0x8d, 0x32, 0x1e, 0xba, 0xb8, 0xab, 0xfb, 0xd8, 0x88, 0x4c,
-	0xc1, 0x3b, 0x90, 0x52, 0xe5, 0x52, 0x59, 0x88, 0x89, 0xaf, 0x53, 0x98, 0x5b, 0x33, 0x30, 0x64,
-	0xfb, 0xe0, 0x52, 0xf7, 0x54, 0xb9, 0xd4, 0x91, 0xb5, 0x43, 0x45, 0x3e, 0x12, 0xe2, 0x97, 0x48,
-	0xed, 0xba, 0x58, 0xf7, 0xb1, 0x76, 0x66, 0xe2, 0x73, 0x22, 0xf5, 0x1f, 0x12, 0xbc, 0xce, 0xf2,
-	0x4c, 0x1f, 0x7b, 0xe8, 0x47, 0xb0, 0xbe, 0xd7, 0xac, 0xef, 0x2a, 0x0d, 0x59, 0x6b, 0xc9, 0x2a,
-	0x9d, 0x95, 0x98, 0xf8, 0x2e, 0x05, 0xba, 0x37, 0x0b, 0x14, 0xa4, 0x2e, 0xd8, 0x0d, 0x26, 0xe6,
-	0x09, 0x08, 0x01, 0xf7, 0x7e, 0xad, 0xb9, 0x5b, 0xaa, 0xd5, 0x9e, 0x0b, 0x71, 0xf1, 0x3e, 0x65,
-	0x97, 0x2e, 0x61, 0xef, 0x5b, 0xce, 0xb1, 0x6e, 0x59, 0x94, 0xff, 0x29, 0xdc, 0x0a, 0xf9, 0x89,
-	0x63, 0xc8, 0x65, 0xed, 0xb0, 0x54, 0x3b, 0x90, 0xdb, 0x42, 0x42, 0x2c, 0x52, 0x94, 0xfb, 0x97,
-	0xa1, 0x10, 0x17, 0xc1, 0x86, 0x46, 0x1d, 0x9c, 0x98, 0x13, 0x7d, 0x00, 0x59, 0x55, 0x6e, 0x57,
-	0x0f, 0x2a, 0x95, 0x1a, 0x71, 0x13, 0xee, 0xbc, 0x73, 0xb6, 0xf3, 0x4e, 0x46, 0xbd, 0x9e, 0x85,
-	0xb9, 0x01, 0x8f, 0x54, 0xa5, 0x23, 0x6b, 0x15, 0xa5, 0x26, 0xbf, 0xc0, 0x4b, 0x5c, 0xd3, 0xc7,
-	0x5a, 0xcf, 0xb4, 0xa8, 0x1c, 0xe9, 0x0f, 0x52, 0xb0, 0xc1, 0x73, 0xb1, 0x48, 0xae, 0xb9, 0x03,
-	0x85, 0xd0, 0x8e, 0xfb, 0xbb, 0xcf, 0xf6, 0x0e, 0x85, 0x58, 0xe0, 0x3e, 0x97, 0x99, 0xb1, 0x7f,
-	0x7c, 0xda, 0x3d, 0x23, 0x7a, 0xa8, 0x20, 0x06, 0xbc, 0x75, 0x59, 0xdd, 0x97, 0xb5, 0xd2, 0xde,
-	0xde, 0x41, 0xfd, 0xa0, 0x56, 0xea, 0x34, 0xd5, 0xb6, 0x10, 0x17, 0x1f, 0x51, 0x9c, 0x07, 0x97,
-	0xe0, 0x0c, 0xb0, 0xdb, 0xc7, 0x9a, 0xde, 0xed, 0x8e, 0x06, 0x23, 0x4b, 0xf7, 0x1d, 0x97, 0x5a,
-	0xa3, 0x06, 0xaf, 0x05, 0x98, 0xf2, 0x57, 0x1d, 0xb5, 0xb4, 0xd7, 0xd1, 0x9a, 0x07, 0x9d, 0xd6,
-	0x41, 0x87, 0x98, 0xf6, 0x21, 0x05, 0xfc, 0xc1, 0x25, 0x80, 0xf8, 0x7b, 0xdf, 0xd5, 0xbb, 0xbe,
-	0xc6, 0x63, 0xe0, 0x8c, 0x86, 0xdc, 0x4b, 0xb4, 0x96, 0x2a, 0x73, 0x92, 0x90, 0x7c, 0x89, 0x86,
-	0xdc, 0x61, 0x34, 0xe2, 0xeb, 0x8c, 0x44, 0x30, 0x8f, 0x41, 0x9a, 0xc5, 0xbc, 0xe0, 0xeb, 0x53,
-	0xe2, 0x0e, 0xc5, 0xfe, 0xf0, 0x25, 0xd8, 0x17, 0x5b, 0xe1, 0x1b, 0xb8, 0x33, 0x2b, 0x63, 0xd6,
-	0x1a, 0x69, 0xf1, 0x13, 0x2a, 0x60, 0xfb, 0x25, 0x02, 0xe6, 0xad, 0x22, 0xfd, 0x6f, 0x1c, 0x6e,
-	0xb7, 0x87, 0x96, 0xe9, 0xfb, 0xfa, 0xb1, 0x85, 0x69, 0x82, 0x1b, 0xf1, 0x88, 0x1a, 0xdc, 0x6c,
-	0x95, 0x14, 0x55, 0x3b, 0x52, 0x3a, 0x55, 0x4d, 0x95, 0xdb, 0x1d, 0x55, 0xd9, 0xeb, 0x28, 0xcd,
-	0x86, 0x10, 0x13, 0xb7, 0xa9, 0xc4, 0x1f, 0xce, 0x48, 0xf4, 0x8c, 0x9e, 0x36, 0xd4, 0x4d, 0x57,
-	0x3b, 0x37, 0xfd, 0x13, 0xcd, 0xc5, 0x9e, 0xef, 0x9a, 0x74, 0x7b, 0x25, 0x5f, 0x52, 0x86, 0x8d,
-	0x76, 0xab, 0xa6, 0x74, 0xa6, 0x90, 0xe2, 0xe2, 0xfb, 0x14, 0xe9, 0xdd, 0x0b, 0x90, 0x3c, 0xa2,
-	0xd8, 0x2c, 0x4a, 0x03, 0x6e, 0xb5, 0xd4, 0xe6, 0x9e, 0xdc, 0x6e, 0x13, 0x5b, 0xc8, 0x65, 0x4d,
-	0xae, 0xc9, 0x75, 0xb9, 0x41, 0x9d, 0xe2, 0xe2, 0x39, 0xa4, 0x4a, 0xb9, 0x4e, 0x17, 0x7b, 0x1e,
-	0x31, 0x03, 0x36, 0x34, 0x6c, 0x61, 0x5a, 0xd9, 0x11, 0x0b, 0xfc, 0x5e, 0x1c, 0x6e, 0x05, 0x9b,
-	0x4a, 0x98, 0x91, 0x76, 0xc6, 0x43, 0xec, 0x49, 0x27, 0x90, 0x92, 0xed, 0xd1, 0x00, 0x3d, 0x84,
-	0x55, 0xa5, 0x23, 0xab, 0xa5, 0xdd, 0x9a, 0x3c, 0xd9, 0x52, 0xa8, 0x90, 0x49, 0xde, 0xbb, 0x63,
-	0xfa, 0xd8, 0x25, 0x36, 0x24, 0x3a, 0x3e, 0x84, 0xd5, 0xfa, 0x41, 0xad, 0xa3, 0xd4, 0x4b, 0x2d,
-	0x21, 0x7e, 0x19, 0xc3, 0x60, 0x64, 0xf9, 0xe6, 0x40, 0x1f, 0x12, 0x25, 0x7e, 0x99, 0x80, 0x5c,
-	0x24, 0x6d, 0x9f, 0x4d, 0x74, 0xe2, 0x73, 0x89, 0xce, 0x6d, 0x58, 0xa5, 0xf5, 0x98, 0x66, 0x1a,
-	0x7c, 0x9f, 0x5c, 0xa1, 0xcf, 0x8a, 0x81, 0x5a, 0x00, 0xa6, 0xa7, 0x1d, 0x3b, 0x23, 0xdb, 0xc0,
-	0xac, 0x38, 0x2b, 0x2c, 0x54, 0x9c, 0x29, 0xde, 0x2e, 0xe3, 0x29, 0x92, 0x8f, 0x56, 0xb3, 0x66,
-	0xf0, 0x8c, 0x1e, 0xc1, 0xcd, 0xf9, 0x4c, 0x99, 0x48, 0x4e, 0x51, 0xc9, 0x73, 0xdd, 0x9b, 0xb1,
-	0x62, 0xcc, 0x65, 0x1d, 0xe9, 0xeb, 0x27, 0x83, 0xbf, 0x4e, 0xc2, 0x2d, 0xf9, 0x7b, 0xdc, 0x1d,
-	0x51, 0x5f, 0x6d, 0xfb, 0x7a, 0x3f, 0xac, 0x1b, 0x5b, 0x90, 0x8b, 0xf4, 0x60, 0x78, 0xfa, 0xb7,
-	0x6c, 0x41, 0x1e, 0x85, 0x20, 0x59, 0x08, 0xab, 0x67, 0x78, 0x16, 0x62, 0xf2, 0xda, 0x24, 0x37,
-	0x99, 0xc1, 0x20, 0xf3, 0x93, 0x17, 0x91, 0x73, 0xa1, 0xde, 0x93, 0x12, 0x48, 0x31, 0x54, 0x08,
-	0x4b, 0x20, 0x0f, 0xbd, 0x39, 0xd5, 0x3a, 0x4c, 0xd1, 0xfc, 0x33, 0xda, 0xfb, 0xdb, 0x9a, 0x64,
-	0x8a, 0x69, 0xfa, 0x32, 0xcc, 0xf6, 0xa6, 0x2b, 0x80, 0xcc, 0x35, 0x2b, 0x00, 0xb1, 0x09, 0xb9,
-	0x88, 0x8e, 0xe8, 0x1e, 0xe4, 0xa3, 0xb5, 0x00, 0x77, 0xcc, 0x9c, 0x3f, 0x29, 0x03, 0xd0, 0x1b,
-	0x00, 0x96, 0xd3, 0xd5, 0x2d, 0xe6, 0xb9, 0xcc, 0x7a, 0x59, 0x4a, 0x21, 0x8e, 0x2b, 0xfd, 0x7b,
-	0x06, 0xf2, 0xd1, 0x3a, 0x1a, 0xed, 0x43, 0xda, 0x70, 0xb4, 0x9e, 0xcd, 0x27, 0xed, 0x0a, 0x65,
-	0xbe, 0x9a, 0x32, 0x9c, 0x8a, 0x8d, 0x6a, 0x00, 0x43, 0xdd, 0xd5, 0x07, 0xd8, 0xc7, 0x6e, 0xd0,
-	0x0d, 0x7d, 0xb0, 0x58, 0x55, 0xcf, 0x98, 0xd4, 0x08, 0x3f, 0xfa, 0xee, 0xa2, 0x99, 0xfe, 0x62,
-	0xc9, 0x26, 0xc1, 0x64, 0x7e, 0x83, 0xfe, 0x6d, 0x64, 0x8e, 0x89, 0x04, 0x9f, 0xa4, 0x35, 0xa4,
-	0xe8, 0x08, 0x32, 0xfe, 0xe5, 0x25, 0x10, 0x08, 0x62, 0x85, 0x50, 0x42, 0x48, 0x20, 0x12, 0x7c,
-	0x73, 0x80, 0x5d, 0x2e, 0x21, 0x7d, 0x35, 0x09, 0x1d, 0x02, 0x11, 0x95, 0xe0, 0x87, 0x04, 0xe2,
-	0xa7, 0x5e, 0xb8, 0x7b, 0x50, 0x6f, 0x5b, 0x55, 0x23, 0x14, 0xf4, 0x01, 0xdc, 0x88, 0x84, 0x6f,
-	0x2d, 0x0c, 0x59, 0x2b, 0xd4, 0x2d, 0x50, 0xe4, 0xdd, 0x1e, 0x8b, 0x5e, 0xe2, 0x29, 0xac, 0xcf,
-	0x18, 0xed, 0x82, 0x12, 0x61, 0x77, 0xba, 0xf3, 0xb6, 0x54, 0xb3, 0x21, 0x5a, 0x8c, 0x10, 0x61,
-	0xd3, 0xf6, 0x7b, 0x45, 0xc2, 0x02, 0xd0, 0x19, 0x61, 0x33, 0xa6, 0x7c, 0x35, 0xc2, 0x42, 0xd0,
-	0x68, 0xa9, 0xf4, 0xab, 0x38, 0x64, 0x43, 0xc7, 0x46, 0x4f, 0x21, 0xe5, 0x8f, 0x87, 0x6c, 0x1f,
-	0x29, 0x3c, 0xfa, 0x78, 0x99, 0x45, 0x51, 0x24, 0x5b, 0x21, 0xdb, 0x11, 0x28, 0x86, 0xf8, 0x35,
-	0xa4, 0x08, 0x49, 0x52, 0xf9, 0xe6, 0xb8, 0x0e, 0xb9, 0x83, 0x46, 0xbb, 0x25, 0xef, 0x29, 0x15,
-	0x45, 0x2e, 0x0b, 0x31, 0x04, 0x90, 0x61, 0x55, 0x84, 0x10, 0x47, 0x37, 0x40, 0x68, 0x29, 0x2d,
-	0xb9, 0x46, 0xb2, 0x97, 0x66, 0x8b, 0x6c, 0xf8, 0x6d, 0x21, 0x81, 0x5e, 0x83, 0xcd, 0x48, 0x0a,
-	0xa0, 0x91, 0x64, 0xe6, 0x99, 0xac, 0x0a, 0x49, 0xe9, 0xaf, 0x93, 0x90, 0x0d, 0x6d, 0x87, 0x54,
-	0x00, 0xfa, 0x41, 0x5a, 0xa4, 0xa4, 0x5f, 0x64, 0x23, 0x3b, 0x24, 0x4c, 0x21, 0x4c, 0x35, 0xa6,
-	0x66, 0x29, 0x0c, 0xc5, 0xac, 0xc1, 0xea, 0xb1, 0xde, 0x67, 0x88, 0x89, 0x85, 0x9b, 0x04, 0xbb,
-	0x7a, 0x3f, 0x8a, 0xb7, 0x72, 0xac, 0xf7, 0x29, 0xda, 0xb7, 0xc0, 0x1b, 0x98, 0x74, 0x63, 0x24,
-	0x98, 0xac, 0xe7, 0xf1, 0xd1, 0xc2, 0xbd, 0x50, 0xba, 0x69, 0x4e, 0x90, 0xd7, 0x42, 0xb8, 0x40,
-	0x5b, 0x52, 0x70, 0x45, 0xba, 0xa0, 0x8b, 0x68, 0x5b, 0xd7, 0x87, 0x53, 0xda, 0x0e, 0xf4, 0x61,
-	0x80, 0xe6, 0x61, 0x9f, 0xa1, 0xa5, 0x17, 0x46, 0x6b, 0x63, 0x7f, 0x0a, 0xcd, 0xc3, 0x7e, 0xd0,
-	0x7f, 0x22, 0x48, 0xd2, 0x0f, 0xa1, 0x30, 0x6d, 0xf0, 0xa9, 0xdc, 0x24, 0x3e, 0x95, 0x9b, 0x48,
-	0x9f, 0x42, 0x3e, 0x6a, 0x4b, 0x74, 0x1f, 0x04, 0x9e, 0x8b, 0x69, 0x33, 0x2c, 0x05, 0x4e, 0xe7,
-	0x71, 0x41, 0xfa, 0x45, 0x1c, 0xd0, 0xbc, 0xc9, 0x48, 0x80, 0x89, 0x24, 0xcc, 0xb3, 0x20, 0x28,
-	0xf2, 0x8e, 0x03, 0xa1, 0x2f, 0xe9, 0x06, 0x49, 0xf3, 0xe2, 0x9e, 0xcd, 0x7d, 0xe0, 0x2a, 0x9b,
-	0x4e, 0x96, 0xa3, 0x54, 0x6c, 0xe9, 0x10, 0xf2, 0x51, 0x9b, 0xa3, 0xbb, 0x90, 0x27, 0xe9, 0xf6,
-	0x8c, 0x32, 0x70, 0x8a, 0xc7, 0x81, 0x12, 0x6f, 0x43, 0x81, 0xb9, 0xf6, 0x4c, 0x12, 0x97, 0xa7,
-	0xd4, 0xbd, 0x89, 0xb5, 0xa2, 0xd6, 0x5f, 0xc2, 0x5a, 0xdf, 0x41, 0x36, 0x0c, 0x0b, 0xa8, 0xcd,
-	0xb6, 0x01, 0xcd, 0x70, 0x06, 0xba, 0x69, 0xf3, 0x20, 0xf0, 0x68, 0xc1, 0xc8, 0x52, 0xa6, 0x4c,
-	0x2c, 0x00, 0xd0, 0xc8, 0xcf, 0x08, 0xd2, 0x4f, 0x20, 0x1b, 0x26, 0x8c, 0xd2, 0xe3, 0xcb, 0x62,
-	0xc1, 0x1a, 0x64, 0x0f, 0x1a, 0xbb, 0xcd, 0x83, 0x46, 0x59, 0x2e, 0x0b, 0x71, 0x94, 0x83, 0x95,
-	0xe0, 0x21, 0x21, 0xfd, 0x45, 0x1c, 0x72, 0x91, 0xae, 0x32, 0x7a, 0x0a, 0x19, 0xcf, 0x19, 0xb9,
-	0x5d, 0x7c, 0x8d, 0x4c, 0x80, 0x23, 0xcc, 0xe4, 0xc0, 0x89, 0xeb, 0xe7, 0xc0, 0x92, 0x01, 0x1b,
-	0x73, 0x7d, 0x65, 0xd4, 0x84, 0x2c, 0xef, 0xbf, 0x5c, 0x2b, 0x7f, 0x59, 0x65, 0x20, 0x15, 0x5b,
-	0xfa, 0xd3, 0x38, 0x14, 0xa6, 0x0f, 0x49, 0x66, 0xfc, 0x35, 0xfe, 0x0a, 0xfc, 0xf5, 0xd2, 0x45,
-	0x93, 0xb8, 0x6c, 0xd1, 0x48, 0xff, 0x9a, 0x81, 0x8d, 0x0e, 0xf6, 0xfc, 0x36, 0xed, 0x17, 0x05,
-	0xaa, 0x5d, 0xbe, 0xd0, 0x91, 0x0a, 0x19, 0x7c, 0x46, 0x53, 0xd0, 0xc4, 0xc2, 0x9d, 0xcc, 0x39,
-	0x01, 0x45, 0x99, 0x40, 0xa8, 0x1c, 0x49, 0xfc, 0x8f, 0x14, 0xa4, 0x29, 0x05, 0x9d, 0xc1, 0xfa,
-	0xb9, 0xee, 0x63, 0x77, 0xa0, 0xbb, 0xa7, 0x1a, 0x7d, 0xcb, 0x0d, 0xf3, 0xec, 0xea, 0x62, 0x8a,
-	0x25, 0xe3, 0x4c, 0xb7, 0xbb, 0xf8, 0x28, 0x00, 0xae, 0xc6, 0xd4, 0x42, 0x28, 0x85, 0xc9, 0xfd,
-	0xfd, 0x38, 0xdc, 0xe4, 0x95, 0x24, 0x89, 0xf8, 0x74, 0x55, 0x31, 0xf1, 0x2c, 0x8e, 0xb4, 0xae,
-	0x2f, 0xbe, 0x15, 0xc2, 0x93, 0xd5, 0x57, 0x8d, 0xa9, 0x9b, 0xc3, 0x29, 0x0a, 0x53, 0x64, 0x00,
-	0x6b, 0x41, 0x24, 0x60, 0xf2, 0xd9, 0xbe, 0x53, 0xb9, 0x96, 0x7c, 0x43, 0xe6, 0x65, 0x71, 0x35,
-	0xa6, 0xe6, 0x39, 0x3c, 0x7d, 0x27, 0x7e, 0x02, 0xc2, 0xac, 0x75, 0xd0, 0x5b, 0xb0, 0x66, 0xe3,
-	0x73, 0x2d, 0xb4, 0x10, 0x9d, 0x81, 0xa4, 0x9a, 0xb7, 0xf1, 0x79, 0x38, 0x48, 0xdc, 0x85, 0x9b,
-	0x17, 0x7e, 0x17, 0xfa, 0x01, 0x08, 0x3a, 0x7b, 0xa1, 0x19, 0x23, 0x57, 0xa7, 0x27, 0x5c, 0x0c,
-	0x60, 0x9d, 0xd3, 0xcb, 0x9c, 0x2c, 0xba, 0x90, 0x8b, 0xe8, 0x86, 0xba, 0xb0, 0x1a, 0x94, 0xef,
-	0xfc, 0x4e, 0xc5, 0xfe, 0x95, 0xbe, 0x9a, 0xa8, 0xe1, 0xf9, 0xfa, 0x60, 0x88, 0x03, 0x6c, 0x35,
-	0x04, 0xde, 0x5d, 0x81, 0x34, 0xb5, 0xab, 0xf8, 0x53, 0x40, 0xf3, 0x03, 0xd1, 0xbb, 0xb0, 0x8e,
-	0x6d, 0xe2, 0xea, 0x61, 0x2b, 0x81, 0x2a, 0x9f, 0x57, 0x0b, 0x9c, 0x1c, 0x0c, 0x7c, 0x1d, 0xb2,
-	0x7e, 0xc0, 0x4e, 0x7d, 0x24, 0xa9, 0x4e, 0x08, 0xd2, 0x7f, 0x26, 0x61, 0xe3, 0xc8, 0x35, 0x7d,
-	0x5c, 0x31, 0x2d, 0xec, 0x05, 0xab, 0xaa, 0x02, 0x29, 0xcf, 0xb4, 0x4f, 0xaf, 0x53, 0x0f, 0x11,
-	0x7e, 0xf4, 0x53, 0x58, 0x27, 0x25, 0x99, 0xee, 0x87, 0xc7, 0xa9, 0xd7, 0xd8, 0xed, 0x0a, 0x0c,
-	0x2a, 0xa0, 0x11, 0x0b, 0xb0, 0xa0, 0x85, 0x0d, 0x8d, 0x36, 0x17, 0x3d, 0xea, 0x82, 0xab, 0x6a,
-	0x21, 0x20, 0xd3, 0x0f, 0xf3, 0xd0, 0x8f, 0x40, 0xe4, 0xf7, 0x70, 0x0c, 0x92, 0x4e, 0x0e, 0x4c,
-	0x1b, 0x1b, 0x9a, 0x77, 0xa2, 0xbb, 0x86, 0x69, 0xf7, 0x69, 0x52, 0xb3, 0xaa, 0x6e, 0xb1, 0x11,
-	0xe5, 0x70, 0x40, 0x9b, 0xbf, 0x47, 0x78, 0xba, 0x0a, 0x63, 0x15, 0x4c, 0x79, 0x91, 0xc3, 0xb9,
-	0x59, 0xb3, 0xbe, 0xa8, 0x14, 0xfb, 0x7f, 0x2d, 0x3a, 0xa4, 0x9f, 0x43, 0x9a, 0x86, 0x55, 0x3a,
-	0xd1, 0x93, 0xcc, 0xf6, 0x6a, 0x13, 0x4d, 0xf6, 0xf7, 0x22, 0x6c, 0x86, 0x15, 0x7b, 0x18, 0xcc,
-	0x83, 0x53, 0xab, 0x8d, 0xf0, 0x15, 0x8f, 0xe5, 0x9e, 0xf4, 0x8f, 0x49, 0x28, 0x04, 0x1d, 0x2f,
-	0x76, 0x99, 0x42, 0xfa, 0xcb, 0x24, 0xdf, 0xc1, 0xdf, 0x86, 0xf4, 0xee, 0xf3, 0x8e, 0xdc, 0x16,
-	0x62, 0xe2, 0x6d, 0xda, 0xb6, 0xda, 0xa4, 0x6d, 0x2b, 0x8a, 0xba, 0x73, 0x3c, 0xf6, 0x59, 0x9f,
-	0xfa, 0x4d, 0x48, 0x3c, 0x3b, 0x14, 0xe2, 0xe2, 0x2d, 0x3a, 0x44, 0x88, 0x0c, 0x39, 0xa5, 0xdd,
-	0xe0, 0x77, 0x20, 0x73, 0x58, 0x52, 0x95, 0x46, 0x47, 0x48, 0x88, 0x22, 0x1d, 0x73, 0x23, 0x32,
-	0xe6, 0x4c, 0x77, 0x4d, 0xdb, 0x27, 0xe3, 0xde, 0x8b, 0x34, 0xd6, 0x92, 0x53, 0x47, 0x05, 0x6c,
-	0x64, 0xb4, 0xa7, 0xf6, 0x00, 0xb2, 0x1d, 0xa5, 0x2e, 0xb7, 0x3b, 0xa5, 0x7a, 0x4b, 0x48, 0x89,
-	0x6f, 0xd0, 0xc1, 0xaf, 0x45, 0x06, 0x87, 0xeb, 0x8a, 0x8c, 0xfe, 0x14, 0xd6, 0x95, 0x46, 0x47,
-	0x56, 0x0f, 0x4b, 0x35, 0x7e, 0xa6, 0x21, 0xa4, 0xa7, 0x7a, 0xe3, 0x5c, 0x80, 0xed, 0x63, 0xf7,
-	0x4c, 0xb7, 0xf8, 0x99, 0x06, 0xeb, 0xa8, 0xaf, 0xd5, 0xe4, 0xc6, 0x7e, 0xa7, 0xaa, 0xb5, 0x54,
-	0xb9, 0xa2, 0x7c, 0x25, 0x64, 0xa6, 0x1a, 0x78, 0x8c, 0xcf, 0xc2, 0x76, 0xdf, 0x3f, 0xd1, 0x86,
-	0x2e, 0xee, 0x99, 0xdf, 0x73, 0x2e, 0x76, 0x7a, 0x10, 0x48, 0x5b, 0xb9, 0x80, 0x8b, 0x1d, 0x1b,
-	0x44, 0x64, 0x7d, 0x0c, 0x05, 0x36, 0x3c, 0x38, 0x34, 0x10, 0x56, 0x45, 0x89, 0xb2, 0xbd, 0x1e,
-	0x61, 0x0b, 0x17, 0x1a, 0xf3, 0xa1, 0xb3, 0x6d, 0xe9, 0x0f, 0x33, 0x41, 0x36, 0x12, 0x3d, 0x69,
-	0x7e, 0xd5, 0xd9, 0x08, 0x3a, 0x84, 0x3c, 0xeb, 0x49, 0x7b, 0xbe, 0xee, 0x8f, 0x3c, 0x9e, 0x47,
-	0x3d, 0x5e, 0xa4, 0x04, 0x21, 0x6c, 0x6d, 0xca, 0xc5, 0x32, 0xa9, 0xdc, 0x60, 0x42, 0x41, 0xef,
-	0x04, 0xc1, 0x63, 0x92, 0x7a, 0x24, 0xe9, 0x3a, 0x5b, 0x63, 0xe4, 0x20, 0x4b, 0x2e, 0xc3, 0x8a,
-	0xef, 0x9a, 0xfd, 0x3e, 0x76, 0x79, 0xf5, 0xf3, 0xde, 0x22, 0x91, 0x9e, 0x71, 0xa8, 0x01, 0x2b,
-	0xc2, 0xb0, 0x11, 0x66, 0x34, 0xa6, 0x63, 0x6b, 0x84, 0x85, 0xd6, 0x3f, 0x85, 0x47, 0x9f, 0x2e,
-	0x80, 0x57, 0x8a, 0xf0, 0xd6, 0x1d, 0x83, 0xd7, 0xc2, 0x82, 0x3e, 0x43, 0x26, 0x59, 0x36, 0xeb,
-	0xc1, 0xd1, 0xb4, 0x80, 0xf6, 0x42, 0x16, 0xcb, 0xb2, 0xd9, 0x71, 0x28, 0xd9, 0x65, 0x78, 0x96,
-	0xed, 0x84, 0x04, 0x74, 0x0c, 0x42, 0xd7, 0x72, 0x68, 0xb2, 0x71, 0x8c, 0x4f, 0xf4, 0x33, 0xd3,
-	0x71, 0x69, 0xef, 0xa4, 0xf0, 0xe8, 0x93, 0x45, 0x4a, 0x4c, 0xc6, 0xba, 0xcb, 0x39, 0x19, 0xfc,
-	0x7a, 0x77, 0x9a, 0x4a, 0xb7, 0x62, 0xcb, 0xa2, 0x0e, 0x66, 0xe9, 0x3e, 0xb6, 0xb1, 0xe7, 0xd1,
-	0xcb, 0x26, 0x64, 0x2b, 0x66, 0xf4, 0x1a, 0x27, 0x93, 0x7a, 0xb7, 0x69, 0x13, 0xc5, 0x02, 0x66,
-	0x7a, 0x81, 0x64, 0xb1, 0x8e, 0xc2, 0x34, 0x23, 0xd3, 0x65, 0x06, 0x0d, 0x6d, 0xc3, 0x4d, 0xdd,
-	0xf3, 0xcc, 0xbe, 0xed, 0x69, 0xbe, 0xa3, 0x39, 0x36, 0xe6, 0x6b, 0x65, 0x0b, 0xe8, 0x3e, 0x81,
-	0xf8, 0xcb, 0x8e, 0xd3, 0xb4, 0x31, 0xf3, 0x7f, 0xe9, 0x1b, 0xc8, 0x45, 0x9c, 0x4d, 0xaa, 0x5f,
-	0x56, 0x89, 0xac, 0x43, 0xae, 0xd1, 0x6c, 0xd0, 0xe3, 0x19, 0xa5, 0xb1, 0x2f, 0xc4, 0x29, 0x41,
-	0x96, 0xcb, 0x6d, 0x76, 0x62, 0x23, 0x24, 0x10, 0x82, 0x42, 0xa9, 0xa6, 0xca, 0xa5, 0x32, 0x3f,
-	0xc4, 0x29, 0x0b, 0x49, 0xa9, 0x0e, 0xc2, 0xec, 0xfc, 0x4b, 0x9f, 0x5d, 0x26, 0xa2, 0x00, 0x50,
-	0x56, 0xda, 0x7b, 0x25, 0xb5, 0xcc, 0x24, 0x08, 0x90, 0x0f, 0xcf, 0x81, 0x08, 0x25, 0x21, 0x7d,
-	0x09, 0xeb, 0x33, 0x73, 0x22, 0x3d, 0x79, 0x81, 0xc2, 0x72, 0x5d, 0xe9, 0x68, 0xa5, 0xda, 0x51,
-	0xe9, 0x79, 0x9b, 0xf5, 0x52, 0x28, 0x41, 0xa9, 0x68, 0x8d, 0x66, 0x43, 0xae, 0xb7, 0x3a, 0xcf,
-	0x85, 0x84, 0xd4, 0x9a, 0x9d, 0x92, 0x17, 0x22, 0x56, 0x14, 0x55, 0x9e, 0x42, 0xa4, 0x84, 0x69,
-	0xc4, 0x63, 0x80, 0x89, 0x4b, 0x4a, 0x9d, 0xcb, 0xd0, 0x36, 0x60, 0x4d, 0x6e, 0x94, 0xb5, 0x66,
-	0x45, 0x0b, 0xbb, 0x3d, 0x08, 0x0a, 0xb5, 0x12, 0x3d, 0x7e, 0x56, 0x1a, 0x5a, 0xab, 0xd4, 0x20,
-	0x56, 0x25, 0x5a, 0x97, 0xd4, 0x9a, 0x12, 0xa5, 0x26, 0x25, 0x0b, 0x60, 0x52, 0x5c, 0x4a, 0xdf,
-	0xbe, 0xc0, 0xa2, 0xf2, 0xa1, 0xdc, 0xe8, 0x68, 0x24, 0xf2, 0x0b, 0x71, 0xb4, 0x09, 0xeb, 0xfc,
-	0xec, 0x47, 0x69, 0xec, 0x33, 0x62, 0x02, 0xdd, 0x85, 0xd7, 0xdb, 0xcf, 0x1b, 0x7b, 0x55, 0xb5,
-	0xd9, 0x50, 0xbe, 0x96, 0xcb, 0xda, 0xec, 0x88, 0xa4, 0xf4, 0x67, 0x02, 0xac, 0xf0, 0xb0, 0x80,
-	0x54, 0xc8, 0xea, 0x3d, 0x1f, 0xbb, 0x9a, 0x6e, 0x59, 0x3c, 0x48, 0x3e, 0x5e, 0x3c, 0xaa, 0x14,
-	0x4b, 0x84, 0xb7, 0x64, 0x59, 0xd5, 0x98, 0xba, 0xaa, 0xf3, 0xdf, 0x11, 0x4c, 0x7b, 0xcc, 0xb3,
-	0x83, 0xe5, 0x31, 0xed, 0xf1, 0x04, 0xd3, 0x1e, 0xa3, 0x03, 0x00, 0x86, 0x89, 0xf5, 0xee, 0x09,
-	0x4f, 0xef, 0x3f, 0x5c, 0x16, 0x54, 0xd6, 0xbb, 0x27, 0xd5, 0x98, 0xca, 0xb4, 0x23, 0x0f, 0xc8,
-	0x82, 0x4d, 0x0e, 0x6b, 0x1b, 0x9a, 0xd3, 0x0b, 0xd6, 0x17, 0x0b, 0xaf, 0x9f, 0x2f, 0x8d, 0x6f,
-	0x1b, 0xcd, 0x1e, 0x5b, 0x88, 0xd5, 0x98, 0x2a, 0xe8, 0x33, 0x34, 0xe4, 0xc3, 0x4d, 0x26, 0x6d,
-	0xa6, 0x68, 0xe2, 0xed, 0xa7, 0x27, 0xcb, 0xca, 0x9b, 0x2f, 0x8e, 0xf4, 0x79, 0x32, 0xfa, 0x45,
-	0x1c, 0x24, 0x26, 0xd6, 0x1b, 0xdb, 0xdd, 0x13, 0xd7, 0xb1, 0xcd, 0xdf, 0xc1, 0xc6, 0x9c, 0x0e,
-	0xec, 0x6c, 0xe4, 0xe9, 0xb2, 0x3a, 0xb4, 0x23, 0x98, 0x73, 0xfa, 0xdc, 0xd1, 0x5f, 0x3c, 0x04,
-	0x3d, 0x83, 0x8c, 0x6e, 0x9d, 0xeb, 0x63, 0x8f, 0xdf, 0x67, 0xdb, 0x5e, 0x46, 0x3c, 0x65, 0xac,
-	0xc6, 0x54, 0x0e, 0x81, 0x1a, 0xb0, 0x62, 0xe0, 0x9e, 0x3e, 0xb2, 0x7c, 0x7e, 0xdf, 0xf0, 0xd1,
-	0x12, 0x68, 0x65, 0xc6, 0x59, 0x8d, 0xa9, 0x01, 0x08, 0xfa, 0x76, 0x52, 0x55, 0x76, 0x9d, 0x91,
-	0xed, 0xf3, 0x3b, 0x87, 0x9f, 0x2c, 0x81, 0x2a, 0x07, 0x7d, 0xa8, 0x91, 0xed, 0x47, 0xca, 0x48,
-	0xfa, 0x8c, 0xaa, 0x90, 0xb6, 0xf1, 0x19, 0x76, 0xf9, 0xb5, 0xc3, 0x0f, 0x96, 0xc0, 0x6d, 0x10,
-	0xbe, 0x6a, 0x4c, 0x65, 0x00, 0x64, 0x75, 0x38, 0xae, 0xd6, 0x33, 0x6d, 0xdd, 0xb2, 0xc6, 0x74,
-	0x77, 0x58, 0x6e, 0x75, 0x34, 0xdd, 0x0a, 0xe3, 0x25, 0xab, 0xc3, 0x09, 0x1e, 0xc8, 0xec, 0xb8,
-	0x78, 0x88, 0xf5, 0xe0, 0x16, 0xe3, 0x32, 0xb3, 0xa3, 0x52, 0x46, 0x32, 0x3b, 0x0c, 0x42, 0xfc,
-	0x0a, 0x56, 0x83, 0x68, 0x81, 0x6a, 0x90, 0xa3, 0xb7, 0xc7, 0xe8, 0xd0, 0xa0, 0x6e, 0x5d, 0x26,
-	0x9b, 0x89, 0xb2, 0x4f, 0x90, 0xed, 0xf1, 0x2b, 0x46, 0x7e, 0x0e, 0xd9, 0x30, 0x70, 0xbc, 0x62,
-	0xe8, 0xbf, 0x89, 0x83, 0x30, 0x1b, 0x34, 0x50, 0x13, 0xd6, 0xb0, 0xee, 0x5a, 0x63, 0xad, 0x67,
-	0xba, 0xa6, 0xdd, 0x0f, 0xae, 0x2c, 0x2e, 0x23, 0x24, 0x4f, 0x01, 0x2a, 0x8c, 0x1f, 0xd5, 0x21,
-	0x4f, 0x92, 0x98, 0x10, 0x2f, 0xb1, 0x34, 0x5e, 0x8e, 0xf0, 0x73, 0x38, 0xf1, 0xe7, 0xb0, 0x79,
-	0x41, 0xe0, 0x41, 0x27, 0x70, 0x23, 0xac, 0x36, 0xb4, 0xb9, 0xff, 0x78, 0x7c, 0xb4, 0x60, 0x6b,
-	0x95, 0xb2, 0x4f, 0x2e, 0xf5, 0x6f, 0xfa, 0x73, 0x34, 0x4f, 0xbc, 0x07, 0x77, 0x5e, 0x12, 0x75,
-	0xc4, 0x2c, 0xac, 0xf0, 0xb5, 0x2c, 0x3e, 0x86, 0x7c, 0x74, 0x01, 0xa2, 0xb7, 0x66, 0x17, 0x34,
-	0x31, 0x6f, 0x7a, 0x7a, 0x55, 0x8a, 0x2b, 0x90, 0xa6, 0xab, 0x4b, 0x5c, 0x85, 0x0c, 0x0b, 0x31,
-	0xe2, 0x9f, 0xc4, 0x21, 0x1b, 0x2e, 0x11, 0xf4, 0x04, 0x52, 0x61, 0xe3, 0x78, 0x39, 0x5b, 0x52,
-	0x3e, 0x92, 0xc6, 0x07, 0x2b, 0x75, 0xf9, 0xe9, 0x08, 0x58, 0xc5, 0x0e, 0x64, 0xd8, 0x12, 0x43,
-	0x4f, 0x01, 0x26, 0x8e, 0x75, 0x05, 0xad, 0x22, 0xdc, 0xbb, 0xd9, 0xb0, 0xc4, 0x90, 0xfe, 0x29,
-	0x11, 0xe9, 0xf5, 0x4c, 0xee, 0x9c, 0xb6, 0x21, 0x6d, 0x60, 0x4b, 0x1f, 0x73, 0x41, 0x9f, 0x5f,
-	0x69, 0x72, 0x8b, 0x65, 0x02, 0x41, 0xe2, 0x17, 0xc5, 0x42, 0x5f, 0xc3, 0xaa, 0x6e, 0x99, 0x7d,
-	0x5b, 0xf3, 0x1d, 0x6e, 0x93, 0x1f, 0x5f, 0x0d, 0xb7, 0x44, 0x50, 0x3a, 0x0e, 0x89, 0xe2, 0x3a,
-	0xfb, 0x29, 0xbe, 0x07, 0x69, 0x2a, 0x0d, 0xdd, 0x83, 0x3c, 0x95, 0xa6, 0x0d, 0x4c, 0xcb, 0x32,
-	0x3d, 0xde, 0x5f, 0xcb, 0x51, 0x5a, 0x9d, 0x92, 0xc4, 0xcf, 0x60, 0x85, 0x23, 0xa0, 0x5b, 0x90,
-	0x19, 0x62, 0xd7, 0x74, 0x58, 0x2d, 0x96, 0x54, 0xf9, 0x13, 0xa1, 0x3b, 0xbd, 0x9e, 0x87, 0x7d,
-	0x9a, 0x24, 0x24, 0x55, 0xfe, 0xb4, 0x7b, 0x13, 0x36, 0x2f, 0x58, 0x03, 0xd2, 0x1f, 0x27, 0x20,
-	0x1b, 0xb6, 0x3d, 0xd0, 0x21, 0x14, 0xf4, 0x2e, 0xbd, 0x78, 0x33, 0xd4, 0x7d, 0x1f, 0xbb, 0xf6,
-	0x55, 0x6f, 0xe6, 0xae, 0x31, 0x98, 0x16, 0x43, 0x41, 0xcf, 0x60, 0xe5, 0xcc, 0xc4, 0xe7, 0xd7,
-	0x3b, 0xc1, 0xc9, 0x10, 0x88, 0x8a, 0x8d, 0xbe, 0x05, 0x7e, 0x6d, 0x5b, 0x1b, 0xe8, 0xc3, 0x21,
-	0xc9, 0x0f, 0x7a, 0x36, 0xcf, 0xb8, 0xae, 0x02, 0xcb, 0x6b, 0xdb, 0x3a, 0xc3, 0xaa, 0xd8, 0xd2,
-	0x1d, 0xc8, 0x45, 0xae, 0x99, 0x20, 0x01, 0x92, 0x23, 0xd7, 0x0a, 0x3a, 0x4b, 0x23, 0xd7, 0x92,
-	0x7e, 0x06, 0xeb, 0x33, 0x20, 0xaf, 0xe6, 0x22, 0xf3, 0x6f, 0x42, 0x21, 0x72, 0xa5, 0x65, 0xd2,
-	0xe1, 0x5f, 0x8b, 0x50, 0x15, 0x43, 0xda, 0x81, 0xfc, 0x94, 0x6c, 0xaa, 0xa0, 0x3d, 0x51, 0xd0,
-	0x46, 0x5b, 0xb0, 0x12, 0xfd, 0xb3, 0x47, 0x5e, 0x0d, 0x1e, 0xa5, 0xff, 0x4a, 0x41, 0x2e, 0x72,
-	0x61, 0x07, 0x29, 0x90, 0x36, 0x7d, 0x1c, 0x86, 0xc2, 0xc7, 0xcb, 0xdd, 0xf7, 0x29, 0x2a, 0x3e,
-	0x1e, 0xa8, 0x0c, 0x41, 0xec, 0x01, 0x28, 0x06, 0xb6, 0x7d, 0xb3, 0x67, 0x62, 0x77, 0x91, 0x9b,
-	0x27, 0x6f, 0xc1, 0xda, 0x64, 0x08, 0xf9, 0x02, 0x7e, 0xa6, 0x16, 0x12, 0x0f, 0x5c, 0x3b, 0xe8,
-	0xeb, 0x25, 0xc3, 0xbe, 0x9e, 0xf8, 0xab, 0x04, 0xa4, 0x88, 0x5c, 0xa4, 0x40, 0x82, 0x03, 0x2f,
-	0x76, 0x9b, 0x7b, 0x4a, 0xf1, 0x50, 0x53, 0x35, 0x61, 0x1a, 0xa8, 0xc6, 0x0f, 0xdc, 0x13, 0x0b,
-	0xb7, 0x19, 0xa2, 0x60, 0x33, 0x47, 0xee, 0xe8, 0xbd, 0xa0, 0xf3, 0xc8, 0x9c, 0xf2, 0x46, 0x91,
-	0xfd, 0x13, 0xb1, 0x18, 0xfc, 0x13, 0xb1, 0x58, 0xb2, 0x83, 0x3f, 0x49, 0xa1, 0x8f, 0x20, 0xe7,
-	0x9d, 0x38, 0xae, 0xcf, 0x9a, 0x45, 0x3c, 0xb1, 0xbf, 0x98, 0x03, 0xe8, 0x40, 0x7a, 0x78, 0x8b,
-	0x6e, 0x40, 0xda, 0xd2, 0x8f, 0xb1, 0xc5, 0xef, 0xd4, 0xb3, 0x07, 0x74, 0x1b, 0x56, 0x2d, 0xd3,
-	0x3e, 0xd5, 0x88, 0xbf, 0x66, 0xd8, 0x01, 0x0f, 0x79, 0x3e, 0x70, 0x2d, 0xf1, 0x67, 0xfc, 0x1a,
-	0xc0, 0xe8, 0x05, 0xd7, 0x00, 0xda, 0x1d, 0x95, 0x55, 0xc2, 0x39, 0x58, 0x51, 0x1a, 0x1d, 0x79,
-	0x5f, 0x56, 0x85, 0x04, 0xca, 0x42, 0xba, 0x52, 0x6b, 0x96, 0x3a, 0x42, 0x92, 0x9d, 0x07, 0x36,
-	0x6b, 0x72, 0xa9, 0x21, 0xa4, 0xd0, 0x5a, 0xb4, 0xc1, 0x97, 0x46, 0x79, 0x58, 0x2d, 0x1f, 0xa8,
-	0x25, 0x7a, 0x49, 0x30, 0x43, 0x2a, 0xc1, 0xa7, 0xa5, 0xc3, 0x92, 0xb6, 0x57, 0x2b, 0xb5, 0xdb,
-	0xc2, 0xca, 0xce, 0x17, 0xb0, 0x4a, 0xff, 0xe3, 0x49, 0xbc, 0xf3, 0xde, 0xdc, 0xd7, 0x11, 0x8d,
-	0xe8, 0x77, 0x35, 0x87, 0xec, 0xbf, 0x91, 0xff, 0xfc, 0xe7, 0x7f, 0xf5, 0x15, 0xd3, 0x9f, 0x70,
-	0x1d, 0xb8, 0xf6, 0xee, 0xe7, 0xf0, 0xf2, 0x7f, 0x83, 0xee, 0x66, 0x55, 0xda, 0x98, 0x2e, 0x0d,
-	0xcd, 0xaf, 0x73, 0x01, 0x5d, 0x3b, 0xdb, 0x3e, 0xce, 0x50, 0x49, 0x8f, 0xff, 0x2f, 0x00, 0x00,
-	0xff, 0xff, 0xef, 0x2d, 0xc1, 0x29, 0x68, 0x3a, 0x00, 0x00,
+	0x72, 0xd7, 0xb7, 0xa5, 0x92, 0x2c, 0xb3, 0x5f, 0x7f, 0xac, 0x9b, 0x99, 0x9d, 0xe9, 0xe6, 0x74,
+	0x7a, 0x7a, 0x67, 0x7b, 0xd4, 0xe3, 0xee, 0xf9, 0xf4, 0xec, 0xce, 0xac, 0x6c, 0xd1, 0x6d, 0x76,
+	0xcb, 0x92, 0x86, 0x92, 0xdd, 0xd3, 0xb3, 0x93, 0xe1, 0x3c, 0x8b, 0x4f, 0x32, 0x61, 0x8a, 0x54,
+	0x48, 0x4a, 0x1e, 0x05, 0x59, 0x2c, 0x90, 0xc3, 0x22, 0x40, 0x2e, 0xc9, 0x21, 0x01, 0xf6, 0x14,
+	0x60, 0x37, 0xc8, 0x25, 0x87, 0x20, 0xd9, 0x20, 0x41, 0xae, 0xd9, 0xfc, 0x07, 0x9b, 0x53, 0xce,
+	0xf9, 0x07, 0x92, 0x20, 0x08, 0x90, 0x53, 0xf0, 0x3e, 0x48, 0x51, 0xb2, 0xdd, 0x2d, 0xd9, 0x8d,
+	0xbd, 0x89, 0xa5, 0x57, 0xbf, 0x2a, 0xbe, 0x57, 0x55, 0xaf, 0xaa, 0xde, 0x23, 0x5c, 0x3f, 0x24,
+	0x78, 0x60, 0x78, 0x23, 0xc7, 0x21, 0x9e, 0x81, 0x87, 0x56, 0x65, 0xe8, 0xb9, 0x81, 0x8b, 0x6e,
+	0xbb, 0x5e, 0xbf, 0x82, 0x87, 0xb8, 0x7b, 0x44, 0x2a, 0x74, 0x44, 0x65, 0xe0, 0x9a, 0xc4, 0xae,
+	0x0c, 0xad, 0x21, 0xb1, 0x2d, 0x87, 0x54, 0xc6, 0x1b, 0xf2, 0xcd, 0xbe, 0xeb, 0xf6, 0x6d, 0xf2,
+	0x80, 0x31, 0x1c, 0x8e, 0x7a, 0x0f, 0xb0, 0x33, 0xe1, 0xdc, 0xf2, 0xad, 0xf9, 0xbf, 0x4c, 0xe2,
+	0x77, 0x3d, 0x6b, 0x18, 0xb8, 0x1e, 0x1f, 0xa1, 0xfc, 0x63, 0x1e, 0x60, 0xdb, 0x1d, 0x0c, 0x5d,
+	0x87, 0x38, 0x81, 0x8f, 0x7e, 0x0f, 0x20, 0xf0, 0xb0, 0xe3, 0xf7, 0x5c, 0x6f, 0xe0, 0xaf, 0x27,
+	0x6f, 0xa5, 0xef, 0x15, 0x1f, 0xfe, 0xb0, 0xf2, 0x52, 0x1d, 0x2a, 0x53, 0x88, 0x4a, 0x27, 0xe2,
+	0x57, 0x9d, 0xc0, 0x9b, 0xe8, 0x31, 0x40, 0xd4, 0x85, 0xd2, 0xb0, 0xeb, 0xda, 0x36, 0xe9, 0x06,
+	0x96, 0xeb, 0xf8, 0xeb, 0x29, 0x26, 0xe0, 0xb3, 0xe5, 0x04, 0xb4, 0x62, 0x08, 0x5c, 0xc4, 0x0c,
+	0x28, 0x9a, 0xc0, 0xb5, 0x13, 0xcb, 0x31, 0xdd, 0x13, 0xcb, 0xe9, 0x1b, 0x7e, 0xe0, 0xe1, 0x80,
+	0xf4, 0x2d, 0xe2, 0xaf, 0xa7, 0x99, 0xb0, 0x9d, 0xe5, 0x84, 0x3d, 0x0b, 0x91, 0xda, 0x11, 0x10,
+	0x97, 0x79, 0xf5, 0xe4, 0xf4, 0x3f, 0xe8, 0x73, 0xc8, 0x75, 0x5d, 0x93, 0x78, 0xfe, 0x7a, 0x86,
+	0x09, 0xfb, 0x78, 0x39, 0x61, 0xdb, 0x8c, 0x97, 0xe3, 0x0b, 0x20, 0x3a, 0x65, 0xc4, 0x19, 0x5b,
+	0x9e, 0xeb, 0x0c, 0xe8, 0x98, 0xf5, 0xec, 0x45, 0xa6, 0x4c, 0x8d, 0x21, 0x88, 0x29, 0x8b, 0x83,
+	0xca, 0x36, 0xac, 0xcd, 0x2d, 0x1b, 0x92, 0x20, 0x7d, 0x4c, 0x26, 0xeb, 0xc9, 0x5b, 0xc9, 0x7b,
+	0x05, 0x9d, 0xfe, 0x44, 0xdb, 0x90, 0x1d, 0x63, 0x7b, 0x44, 0xd6, 0x53, 0xb7, 0x92, 0xf7, 0x8a,
+	0x0f, 0xdf, 0x59, 0x40, 0x85, 0x56, 0x84, 0xaa, 0x73, 0xde, 0xcd, 0xd4, 0x47, 0x49, 0xd9, 0x85,
+	0x2b, 0xa7, 0xd6, 0xf0, 0x0c, 0x79, 0xb5, 0x59, 0x79, 0x95, 0x45, 0xe4, 0x6d, 0x47, 0xb0, 0x71,
+	0x81, 0x7f, 0x08, 0xeb, 0xe7, 0xad, 0xe3, 0x19, 0x72, 0x9f, 0xcc, 0xca, 0x7d, 0x6f, 0x01, 0xb9,
+	0xf3, 0xe8, 0x93, 0xb8, 0xf4, 0x2e, 0x14, 0x63, 0x0b, 0x7b, 0x86, 0xc0, 0x4f, 0x67, 0x05, 0xde,
+	0x5b, 0x68, 0x6d, 0x4d, 0xe2, 0xcd, 0xcd, 0xe9, 0xa9, 0x45, 0x7e, 0x35, 0x73, 0x1a, 0x83, 0x8d,
+	0x09, 0x54, 0xfe, 0x39, 0x0f, 0xd7, 0xf7, 0x88, 0xef, 0xe3, 0x3e, 0x79, 0x66, 0x05, 0x47, 0xb1,
+	0x18, 0xb2, 0x07, 0xd0, 0x8d, 0x9e, 0x98, 0xf0, 0xc5, 0x8c, 0x65, 0x0a, 0xa1, 0xc7, 0x00, 0xd0,
+	0x8f, 0x20, 0xcb, 0x5c, 0x61, 0xd9, 0xd9, 0xd9, 0x4d, 0xe8, 0x9c, 0x11, 0x7d, 0x05, 0x6b, 0x5d,
+	0x77, 0x70, 0x68, 0x39, 0xc4, 0x18, 0xe2, 0x89, 0xed, 0x62, 0x73, 0x3d, 0xcd, 0xb0, 0x36, 0x16,
+	0xd3, 0x8a, 0x72, 0xb6, 0x38, 0xe3, 0x6e, 0x42, 0x2f, 0x77, 0x67, 0x28, 0xe8, 0x1b, 0xb8, 0xe2,
+	0x9b, 0xc7, 0x46, 0x6f, 0xe4, 0x30, 0xbb, 0x33, 0xfc, 0x21, 0xe9, 0xae, 0x67, 0x18, 0xfe, 0xc3,
+	0x05, 0xf0, 0xdb, 0xe6, 0xf1, 0x8e, 0x60, 0x6d, 0x0f, 0x49, 0x77, 0x37, 0xa1, 0xaf, 0xf9, 0xb3,
+	0x24, 0xf4, 0x0c, 0xca, 0x43, 0xec, 0x19, 0xa6, 0x1b, 0xa9, 0x9f, 0x63, 0xf0, 0x0f, 0x16, 0xf1,
+	0x08, 0xec, 0xd5, 0xdc, 0xa9, 0xf2, 0xa5, 0x61, 0xec, 0x19, 0x35, 0x01, 0x86, 0x51, 0x74, 0x5e,
+	0x5f, 0xb9, 0x80, 0x5b, 0xef, 0x26, 0xf4, 0x18, 0x04, 0xd2, 0xa1, 0x18, 0x0b, 0xc5, 0xeb, 0xf9,
+	0x8b, 0x38, 0xee, 0x6e, 0x42, 0x8f, 0x83, 0xa0, 0x36, 0x94, 0x3c, 0x82, 0xcd, 0xe8, 0xdd, 0x0b,
+	0x0b, 0x83, 0xea, 0x04, 0x9b, 0xd3, 0x57, 0x2f, 0x7a, 0xd3, 0x47, 0x6a, 0xa3, 0xbe, 0x65, 0x12,
+	0xc3, 0x72, 0x86, 0xa3, 0x60, 0xbd, 0xc8, 0x20, 0xef, 0x2f, 0xb2, 0x5a, 0x96, 0x49, 0x34, 0xca,
+	0xb3, 0x9b, 0xd0, 0x0b, 0x7e, 0xf8, 0x80, 0x7a, 0x20, 0xb6, 0x03, 0xc3, 0x72, 0x82, 0xe9, 0x32,
+	0x95, 0x96, 0x0c, 0x20, 0x9a, 0x13, 0xc4, 0xd6, 0xea, 0xca, 0xc9, 0x3c, 0x11, 0x11, 0x40, 0xa7,
+	0xb6, 0xb6, 0xc9, 0xfa, 0xea, 0xc5, 0xe3, 0xd4, 0x54, 0x4c, 0x8c, 0x88, 0x0e, 0x60, 0x75, 0xd6,
+	0x9c, 0xcb, 0x0b, 0xdb, 0xdb, 0x9c, 0x2d, 0x97, 0x7a, 0xb1, 0xe7, 0xad, 0x1c, 0x64, 0x3c, 0xd7,
+	0x0d, 0x94, 0x7f, 0x4f, 0x42, 0xbe, 0x25, 0x98, 0x5e, 0x75, 0xb8, 0xb8, 0x0f, 0x88, 0xca, 0x30,
+	0x22, 0xa3, 0x34, 0x2c, 0x93, 0x27, 0x1a, 0x05, 0x5d, 0xa2, 0xff, 0x44, 0xb6, 0xab, 0x99, 0x74,
+	0xc3, 0x2e, 0x99, 0x96, 0x3f, 0xb4, 0xf1, 0xc4, 0x30, 0x71, 0x80, 0x45, 0x5c, 0x58, 0xc4, 0xb8,
+	0x6a, 0x9c, 0xad, 0x86, 0x03, 0xac, 0x17, 0xcd, 0xe9, 0x83, 0xf2, 0x27, 0x19, 0x80, 0xa9, 0x83,
+	0xa0, 0x37, 0xa0, 0x38, 0x72, 0xac, 0xdf, 0x1f, 0x11, 0xc3, 0xc1, 0x03, 0xb2, 0x9e, 0x65, 0xb1,
+	0x18, 0x38, 0xa9, 0x81, 0x07, 0x04, 0x6d, 0x43, 0x86, 0xcd, 0x71, 0xf2, 0x42, 0x73, 0xac, 0x33,
+	0x66, 0x74, 0x07, 0x56, 0xfd, 0xd1, 0x61, 0x2c, 0x75, 0xe3, 0x2f, 0x3c, 0x4b, 0xa4, 0xe9, 0x09,
+	0x33, 0xf8, 0x30, 0x17, 0xfa, 0x78, 0x29, 0x5f, 0xaf, 0x30, 0x5b, 0x0f, 0xd3, 0x13, 0x0e, 0x84,
+	0x3a, 0xb0, 0xe2, 0x8e, 0x02, 0x86, 0xc9, 0x53, 0x9e, 0xcd, 0xe5, 0x30, 0x9b, 0x9c, 0x99, 0x83,
+	0x86, 0x50, 0xa7, 0x96, 0x25, 0x77, 0xe9, 0x65, 0x91, 0x3f, 0x86, 0x62, 0x4c, 0xff, 0x33, 0xb6,
+	0xc6, 0x6b, 0xf1, 0xad, 0xb1, 0x10, 0xdf, 0x5b, 0x37, 0xa1, 0x14, 0x57, 0x73, 0x19, 0x5e, 0xe5,
+	0x2f, 0x56, 0xe1, 0x6a, 0x3b, 0xc0, 0x8e, 0x89, 0x3d, 0x73, 0xfa, 0xda, 0xbe, 0xf2, 0xbf, 0x29,
+	0x80, 0x96, 0x67, 0x0d, 0xac, 0xc0, 0x1a, 0x13, 0x1f, 0xbd, 0x03, 0xb9, 0x56, 0x55, 0x37, 0x6a,
+	0x4d, 0x29, 0x21, 0xdf, 0xfe, 0xc5, 0x3f, 0xfc, 0xdf, 0xbf, 0x64, 0x7f, 0x67, 0xe4, 0x39, 0x9b,
+	0xf4, 0x25, 0x37, 0xa3, 0x05, 0xdc, 0x1c, 0x62, 0xcf, 0x74, 0x37, 0xc7, 0x1b, 0xe8, 0x3e, 0xac,
+	0xec, 0xd4, 0xab, 0x9d, 0x8e, 0xda, 0x90, 0x92, 0xf2, 0x1b, 0x6c, 0xfc, 0xcd, 0xb9, 0xb1, 0x3d,
+	0x1b, 0x07, 0x01, 0x71, 0xe8, 0xe8, 0x0f, 0xa0, 0xf4, 0x58, 0x6f, 0xee, 0xb7, 0x8c, 0xad, 0xe7,
+	0xc6, 0x53, 0xf5, 0xb9, 0x94, 0x92, 0xef, 0x30, 0x96, 0xd7, 0xe7, 0x58, 0xfa, 0x9e, 0x3b, 0x1a,
+	0x1a, 0x87, 0x13, 0xe3, 0x98, 0x4c, 0x84, 0x14, 0x6d, 0xaf, 0xb5, 0x5f, 0x6f, 0xab, 0x52, 0xfa,
+	0x1c, 0x29, 0xd6, 0x60, 0x38, 0xb2, 0x7d, 0x42, 0x47, 0x7f, 0x08, 0xe5, 0x6a, 0xbb, 0xad, 0x3d,
+	0x6e, 0x18, 0xcf, 0xb4, 0x46, 0xad, 0xf9, 0xac, 0x2d, 0x65, 0xe4, 0x37, 0x19, 0xd3, 0x77, 0xe7,
+	0x98, 0x62, 0xa1, 0x92, 0x33, 0x16, 0x3b, 0x6a, 0xbb, 0x63, 0xb4, 0x3b, 0xba, 0x5a, 0xdd, 0x93,
+	0xb2, 0xf2, 0x5d, 0xc6, 0x75, 0xeb, 0x8c, 0x09, 0x08, 0x88, 0x1f, 0xf8, 0x81, 0x47, 0x89, 0xe3,
+	0x0d, 0xf4, 0x1e, 0x14, 0xf7, 0xaa, 0xad, 0x48, 0x5c, 0xee, 0x1c, 0x71, 0x03, 0x3c, 0x34, 0xb8,
+	0x48, 0x7f, 0x73, 0xbc, 0xa1, 0x04, 0x70, 0xad, 0x46, 0x86, 0x1e, 0xe9, 0xe2, 0x80, 0x98, 0xb1,
+	0x25, 0xb8, 0x0b, 0x19, 0x5d, 0xad, 0xd6, 0xa4, 0x84, 0xfc, 0x1a, 0x83, 0xb9, 0x31, 0x07, 0x43,
+	0xb7, 0x0f, 0x21, 0x75, 0x5b, 0x57, 0xab, 0x1d, 0xd5, 0x38, 0xd0, 0xd4, 0x67, 0x52, 0xf2, 0x1c,
+	0xa9, 0x5d, 0x8f, 0xe0, 0x80, 0x18, 0x63, 0x8b, 0x9c, 0x50, 0xa9, 0xff, 0x95, 0x14, 0x75, 0x96,
+	0x6f, 0x05, 0xc4, 0x47, 0x3f, 0x80, 0xb5, 0xed, 0xe6, 0xde, 0x96, 0xd6, 0x50, 0x8d, 0x96, 0xaa,
+	0xb3, 0x55, 0x49, 0xc8, 0x6f, 0x31, 0xa0, 0xdb, 0xf3, 0x40, 0x61, 0xea, 0x42, 0xbc, 0x70, 0x61,
+	0x3e, 0x05, 0x29, 0xe4, 0x7e, 0x5c, 0x6f, 0x6e, 0x55, 0xeb, 0xf5, 0xe7, 0x52, 0x52, 0xbe, 0xc7,
+	0xd8, 0x95, 0x73, 0xd8, 0xfb, 0xb6, 0x7b, 0x88, 0x6d, 0x9b, 0xf1, 0xbf, 0x0b, 0x05, 0x5d, 0x6d,
+	0xef, 0xee, 0xef, 0xec, 0xd4, 0x55, 0x29, 0x15, 0x1a, 0xdc, 0xa9, 0xf7, 0xf5, 0x8f, 0x46, 0xbd,
+	0x9e, 0x4d, 0xc4, 0x4b, 0x3f, 0xd3, 0xb5, 0x8e, 0x6a, 0xec, 0x68, 0x75, 0xb5, 0x2d, 0xa5, 0xcf,
+	0x5b, 0x59, 0xcf, 0x0a, 0x88, 0xd1, 0xb3, 0x6c, 0xc2, 0xa6, 0xfa, 0x37, 0x19, 0xb8, 0x22, 0xf2,
+	0xa7, 0x58, 0x7e, 0xb8, 0x09, 0xe5, 0xe8, 0xdd, 0x1f, 0x6f, 0x3d, 0xdd, 0x3e, 0x90, 0x12, 0xe1,
+	0x92, 0x9f, 0xf7, 0xea, 0xfd, 0xc3, 0xe3, 0xee, 0x98, 0xea, 0xa1, 0x83, 0x1c, 0xf2, 0xee, 0xa9,
+	0xfa, 0x63, 0xd5, 0xa8, 0x6e, 0x6f, 0xef, 0xef, 0xed, 0xd7, 0xab, 0x9d, 0xa6, 0xde, 0x96, 0x92,
+	0xf2, 0x43, 0x86, 0x73, 0xff, 0x1c, 0x9c, 0x01, 0xf1, 0xfa, 0xc4, 0xc0, 0xdd, 0xee, 0x68, 0x30,
+	0xb2, 0x71, 0xe0, 0x7a, 0x54, 0x4b, 0x54, 0x87, 0xef, 0x84, 0x98, 0xea, 0x17, 0x1d, 0xbd, 0xba,
+	0xdd, 0x31, 0x9a, 0xfb, 0x9d, 0xd6, 0x7e, 0xa7, 0x2d, 0xa5, 0xe4, 0x07, 0x0c, 0xf0, 0x7b, 0xe7,
+	0x00, 0x92, 0x6f, 0x03, 0x0f, 0x77, 0x03, 0x43, 0xc4, 0xad, 0x39, 0x0d, 0xc5, 0xca, 0x1a, 0x2d,
+	0x5d, 0x15, 0x24, 0x29, 0xfd, 0x12, 0x0d, 0xc5, 0x22, 0x1b, 0xd4, 0x3e, 0x39, 0x89, 0x62, 0x1e,
+	0x82, 0x32, 0x8f, 0x79, 0xc6, 0xdb, 0x67, 0xe4, 0x4d, 0x86, 0xfd, 0xde, 0x4b, 0xb0, 0xcf, 0x9e,
+	0x85, 0xaf, 0xe0, 0x8d, 0x79, 0x19, 0xf3, 0xb3, 0x91, 0x95, 0x3f, 0x64, 0x02, 0x36, 0x5e, 0x22,
+	0xe0, 0x8c, 0x59, 0x79, 0x02, 0x37, 0x22, 0x8b, 0xa5, 0xa1, 0x48, 0xad, 0x19, 0x07, 0xd5, 0xfa,
+	0xbe, 0x4a, 0xbd, 0xb6, 0xc2, 0x40, 0xef, 0x9d, 0x67, 0xb7, 0x34, 0x28, 0x11, 0xd3, 0x60, 0x21,
+	0x95, 0x59, 0xd5, 0xaf, 0x53, 0x70, 0xb3, 0x3d, 0xb4, 0xad, 0x20, 0xc0, 0x87, 0x36, 0x61, 0x09,
+	0x6e, 0xcc, 0xba, 0xea, 0x70, 0xbd, 0x55, 0xd5, 0x74, 0xe3, 0x99, 0xd6, 0xd9, 0x35, 0x74, 0xb5,
+	0xdd, 0xd1, 0xb5, 0xed, 0x8e, 0xd6, 0x6c, 0x48, 0x09, 0x79, 0x83, 0x09, 0xfa, 0xfe, 0x9c, 0x20,
+	0xdf, 0xec, 0x19, 0x43, 0x6c, 0x79, 0xc6, 0x89, 0x15, 0x1c, 0x19, 0x1e, 0xf1, 0x03, 0xcf, 0x62,
+	0xdb, 0x2b, 0xd5, 0xbb, 0x06, 0x57, 0xda, 0xad, 0xba, 0xd6, 0x99, 0x41, 0x4a, 0xca, 0xef, 0x30,
+	0xa4, 0xb7, 0xce, 0x40, 0xf2, 0xa9, 0x62, 0xf3, 0x28, 0x0d, 0xb8, 0xd1, 0xd2, 0x9b, 0xdb, 0x6a,
+	0xbb, 0x4d, 0xe7, 0x55, 0xad, 0x19, 0x6a, 0x5d, 0xdd, 0x53, 0x1b, 0xcc, 0xc0, 0xce, 0xb6, 0x07,
+	0xa6, 0x94, 0xe7, 0x76, 0x89, 0xef, 0xd3, 0x29, 0x25, 0xa6, 0x41, 0x6c, 0xc2, 0x2a, 0x3b, 0x8a,
+	0xb7, 0x05, 0x52, 0x88, 0x17, 0x21, 0xa5, 0xe5, 0xfb, 0x0c, 0xe9, 0xee, 0x0b, 0x90, 0x62, 0x18,
+	0xca, 0x1f, 0x25, 0xe1, 0x46, 0xb8, 0x31, 0x45, 0x59, 0x6d, 0x67, 0x32, 0x24, 0xbe, 0x72, 0x04,
+	0x19, 0xd5, 0x19, 0x0d, 0xd0, 0x03, 0xc8, 0x6b, 0x1d, 0x55, 0xaf, 0x6e, 0xd5, 0xd5, 0xe9, 0xb6,
+	0xc4, 0xe0, 0xa7, 0xb9, 0xf3, 0xa6, 0x15, 0x10, 0x8f, 0xae, 0x03, 0xd5, 0xeb, 0x01, 0xe4, 0xf7,
+	0xf6, 0xeb, 0x1d, 0x6d, 0xaf, 0xda, 0x92, 0x92, 0xe7, 0x31, 0x0c, 0x46, 0x76, 0x60, 0x0d, 0xf0,
+	0x90, 0x2a, 0xf1, 0x8b, 0x14, 0x14, 0x63, 0xa9, 0xff, 0x7c, 0xb2, 0x94, 0x3c, 0x95, 0x2c, 0xdd,
+	0x84, 0x3c, 0xab, 0xe9, 0x0c, 0xcb, 0x14, 0x7b, 0xed, 0x0a, 0x7b, 0xd6, 0x4c, 0xd4, 0x02, 0xb0,
+	0x7c, 0xe3, 0xd0, 0x1d, 0x39, 0x26, 0xe1, 0x05, 0x5e, 0x79, 0xa1, 0x02, 0x4f, 0xf3, 0xb7, 0x38,
+	0x4f, 0x85, 0xbe, 0xb4, 0x5e, 0xb0, 0xc2, 0x67, 0xf4, 0x10, 0xae, 0x9f, 0xce, 0xb6, 0xa9, 0xe4,
+	0x0c, 0x93, 0x7c, 0xaa, 0x03, 0x34, 0xd1, 0xcc, 0x53, 0x99, 0x4b, 0xf6, 0xf2, 0x09, 0xe5, 0x2f,
+	0x73, 0x70, 0x43, 0xfd, 0x96, 0x74, 0x47, 0xcc, 0xde, 0xdb, 0x01, 0xee, 0x47, 0xb5, 0x67, 0x0b,
+	0x8a, 0xb1, 0x3e, 0x8e, 0x48, 0x21, 0x97, 0x2d, 0xea, 0xe3, 0x10, 0x34, 0x93, 0xe1, 0x35, 0x91,
+	0xc8, 0x64, 0x2c, 0x51, 0xdf, 0x14, 0xa7, 0x2b, 0x18, 0x66, 0x8f, 0xea, 0x22, 0x72, 0xce, 0xd4,
+	0x7b, 0x5a, 0x46, 0x69, 0xa6, 0x0e, 0x51, 0x19, 0xe5, 0xa3, 0xd7, 0x67, 0xda, 0x8f, 0x19, 0x96,
+	0xc3, 0xc6, 0xfb, 0x87, 0xeb, 0xd3, 0x6c, 0x33, 0xcb, 0xfe, 0x8c, 0x32, 0xc6, 0xd9, 0x2a, 0x22,
+	0x77, 0xd9, 0x2a, 0xa2, 0x07, 0xc5, 0x91, 0x4f, 0x3c, 0xc3, 0x0f, 0x70, 0x40, 0xfc, 0xf5, 0x95,
+	0xcb, 0xbe, 0xf0, 0xbe, 0x4f, 0xbc, 0x36, 0xc5, 0xa2, 0x2f, 0x3c, 0x0a, 0x1f, 0x7c, 0xf4, 0x1c,
+	0x72, 0x81, 0x35, 0x20, 0x9e, 0xbf, 0x9e, 0x67, 0x22, 0xaa, 0x17, 0x17, 0xd1, 0xa1, 0x38, 0x9a,
+	0xa9, 0x0b, 0x40, 0xb9, 0x09, 0xc5, 0xd8, 0x34, 0xa3, 0xdb, 0x50, 0x8a, 0x97, 0x44, 0xc2, 0xb7,
+	0x8a, 0xc1, 0xb4, 0x1a, 0x42, 0xdf, 0x05, 0xb0, 0xdd, 0x2e, 0xb6, 0xb9, 0xf3, 0x71, 0x03, 0x28,
+	0x30, 0x0a, 0xf5, 0x3d, 0x0a, 0x18, 0x7b, 0x8d, 0x57, 0x00, 0xf8, 0x14, 0x56, 0x84, 0xd2, 0x97,
+	0x07, 0x53, 0xfe, 0x23, 0x07, 0xa5, 0x78, 0xb3, 0x03, 0x3d, 0x86, 0xac, 0xe9, 0x1a, 0x3d, 0x47,
+	0x78, 0xc5, 0x05, 0x7a, 0x31, 0x7a, 0xc6, 0x74, 0x77, 0x1c, 0x54, 0x07, 0x18, 0x62, 0x0f, 0x0f,
+	0x48, 0x40, 0xd7, 0x89, 0xb7, 0xac, 0xef, 0x2f, 0xd6, 0x7a, 0xe1, 0x4c, 0x7a, 0x8c, 0x1f, 0x7d,
+	0x73, 0x96, 0x2b, 0x7d, 0xb6, 0x64, 0x27, 0x67, 0xea, 0x40, 0x61, 0x93, 0x3d, 0xe6, 0x44, 0x54,
+	0x02, 0x5d, 0x23, 0x56, 0xba, 0x87, 0x65, 0xd9, 0xf2, 0x12, 0x28, 0x04, 0x9d, 0x85, 0x48, 0x42,
+	0x44, 0xa0, 0x12, 0x98, 0x91, 0x09, 0x09, 0xd9, 0x8b, 0x49, 0x60, 0x6b, 0x1f, 0x97, 0x10, 0x44,
+	0x04, 0x1a, 0x08, 0xfc, 0x68, 0x8b, 0x67, 0xee, 0x9c, 0xd7, 0x63, 0x14, 0xf4, 0x2e, 0x5c, 0x8b,
+	0xed, 0xb1, 0x46, 0xb4, 0x27, 0xac, 0x30, 0xb3, 0x40, 0xb1, 0xff, 0xb6, 0xf9, 0xf6, 0x20, 0x1f,
+	0xc3, 0xda, 0xdc, 0xa4, 0x9d, 0x51, 0xc7, 0x6d, 0xcd, 0xb6, 0x47, 0x97, 0xea, 0x08, 0xc5, 0x2b,
+	0x46, 0x2a, 0x6c, 0x76, 0xfe, 0x5e, 0x91, 0xb0, 0x10, 0x74, 0x4e, 0xd8, 0xdc, 0x54, 0xbe, 0x1a,
+	0x61, 0x11, 0x68, 0xbc, 0x9e, 0xfd, 0x55, 0x12, 0x0a, 0x91, 0x61, 0xa3, 0x27, 0x90, 0x09, 0x26,
+	0x43, 0xbe, 0x51, 0x97, 0x1f, 0x7e, 0xb0, 0x8c, 0x53, 0x54, 0x68, 0xae, 0xc1, 0xb7, 0x5c, 0x86,
+	0x21, 0x7f, 0x09, 0x19, 0x4a, 0x52, 0x74, 0x91, 0x7d, 0xac, 0x41, 0x71, 0xbf, 0xd1, 0x6e, 0xa9,
+	0xdb, 0xda, 0x8e, 0xa6, 0xd6, 0xa4, 0x04, 0x02, 0xc8, 0xf1, 0x52, 0x4f, 0x4a, 0xa2, 0x6b, 0x20,
+	0xb5, 0xb4, 0x96, 0x5a, 0xa7, 0x09, 0x65, 0xb3, 0x45, 0xb3, 0xb2, 0xb6, 0x94, 0x42, 0xdf, 0x81,
+	0xab, 0xb1, 0x3c, 0xcd, 0xa0, 0xd9, 0xeb, 0x53, 0x55, 0x97, 0xd2, 0xca, 0xdf, 0xa5, 0xa1, 0x10,
+	0xcd, 0x1d, 0xd2, 0x01, 0xd8, 0x0b, 0x19, 0xb1, 0xbe, 0xcb, 0x22, 0x99, 0xc2, 0x01, 0x65, 0x8a,
+	0x60, 0x76, 0x13, 0x7a, 0x81, 0xc1, 0x30, 0xcc, 0x3a, 0xe4, 0x0f, 0x71, 0x9f, 0x23, 0xa6, 0x16,
+	0xee, 0xe4, 0x6c, 0xe1, 0x7e, 0x1c, 0x6f, 0xe5, 0x10, 0xf7, 0x19, 0xda, 0xd7, 0x20, 0xba, 0xcc,
+	0x2c, 0xf3, 0xa0, 0x98, 0xbc, 0x31, 0xf5, 0xfe, 0xc2, 0x0d, 0x6b, 0x96, 0x95, 0x4c, 0x91, 0x57,
+	0x23, 0xb8, 0x50, 0x5b, 0x5a, 0x15, 0xc7, 0x5a, 0xd5, 0x8b, 0x68, 0xbb, 0x87, 0x87, 0x33, 0xda,
+	0x0e, 0xf0, 0x30, 0x44, 0xf3, 0x49, 0xc0, 0xd1, 0xb2, 0x0b, 0xa3, 0xb5, 0x49, 0x30, 0x83, 0xe6,
+	0x93, 0x20, 0x6c, 0x12, 0x52, 0x24, 0xe5, 0xfb, 0x50, 0x9e, 0x9d, 0xf0, 0x99, 0xe4, 0x2f, 0x39,
+	0x93, 0xfc, 0x29, 0x1f, 0x41, 0x29, 0x3e, 0x97, 0xe8, 0x1e, 0x48, 0x22, 0xd9, 0x35, 0xe6, 0x58,
+	0xca, 0x82, 0x2e, 0xe2, 0x82, 0xf2, 0xf3, 0x24, 0xa0, 0xd3, 0x53, 0x46, 0x03, 0x4c, 0xac, 0x42,
+	0x9a, 0x07, 0x41, 0xb1, 0xff, 0x04, 0x10, 0xfa, 0x9c, 0x65, 0x20, 0xac, 0x66, 0xe9, 0x39, 0xc2,
+	0x06, 0x2e, 0xb2, 0xe9, 0x14, 0x04, 0xca, 0x8e, 0xa3, 0x1c, 0x40, 0x29, 0x3e, 0xe7, 0xe8, 0x16,
+	0x94, 0x68, 0x7d, 0x35, 0xa7, 0x0c, 0x1c, 0x93, 0x49, 0xa8, 0xc4, 0x1d, 0x28, 0x73, 0xd3, 0x9e,
+	0xcb, 0x92, 0x4b, 0x8c, 0xba, 0x3d, 0x9d, 0xad, 0xf8, 0xec, 0x2f, 0x31, 0x5b, 0xdf, 0x40, 0x21,
+	0x0a, 0x0b, 0xa8, 0xcd, 0xb7, 0x01, 0xc3, 0x74, 0x07, 0xd8, 0x72, 0x44, 0x10, 0x78, 0xb8, 0x60,
+	0x64, 0xa9, 0x31, 0x26, 0x1e, 0x00, 0x58, 0xe4, 0xe7, 0x04, 0xe5, 0x47, 0x50, 0x88, 0x32, 0x72,
+	0xe5, 0xd1, 0x79, 0xb1, 0x60, 0x15, 0x0a, 0xfb, 0x8d, 0xad, 0xe6, 0x7e, 0xa3, 0xa6, 0xd6, 0xa4,
+	0x24, 0x2a, 0xc2, 0x4a, 0xf8, 0x90, 0x52, 0xfe, 0x26, 0x09, 0xc5, 0x58, 0xeb, 0x1f, 0x3d, 0x81,
+	0x9c, 0xef, 0x8e, 0xbc, 0x2e, 0xb9, 0x44, 0x26, 0x20, 0x10, 0xe6, 0x8a, 0x8c, 0xd4, 0xe5, 0x8b,
+	0x0c, 0xc5, 0x84, 0x2b, 0xa7, 0x9a, 0xff, 0xa8, 0x09, 0x05, 0xd1, 0x24, 0xbb, 0x54, 0xfe, 0x92,
+	0xe7, 0x20, 0x3b, 0x8e, 0xf2, 0xe7, 0x49, 0x28, 0xcf, 0x9e, 0x64, 0xcd, 0xd9, 0x6b, 0xf2, 0x15,
+	0xd8, 0xeb, 0xb9, 0x4e, 0x93, 0x3a, 0xcf, 0x69, 0x94, 0xdf, 0xe4, 0xe0, 0x4a, 0x87, 0xf8, 0x41,
+	0x9b, 0x35, 0xf5, 0x42, 0xd5, 0xce, 0x77, 0x74, 0xa4, 0x43, 0x8e, 0x8c, 0x59, 0x8e, 0x9f, 0x5a,
+	0xb8, 0xdd, 0x7c, 0x4a, 0x40, 0x45, 0xa5, 0x10, 0xba, 0x40, 0x92, 0xff, 0x33, 0x03, 0x59, 0x46,
+	0x41, 0x63, 0x58, 0x3b, 0xc1, 0x01, 0xf1, 0x06, 0xd8, 0x3b, 0x36, 0xd8, 0xbf, 0x62, 0x62, 0x9e,
+	0x5e, 0x5c, 0x4c, 0xa5, 0x6a, 0x8e, 0xb1, 0xd3, 0x25, 0xcf, 0x42, 0xe0, 0xdd, 0x84, 0x5e, 0x8e,
+	0xa4, 0x70, 0xb9, 0x3f, 0x4b, 0xc2, 0x75, 0x51, 0xa4, 0xd3, 0x88, 0xcf, 0xbc, 0x8a, 0x8b, 0xe7,
+	0x71, 0xa4, 0x75, 0x79, 0xf1, 0xad, 0x08, 0x9e, 0x7a, 0xdf, 0x6e, 0x42, 0xbf, 0x3a, 0x9c, 0xa1,
+	0x70, 0x45, 0x06, 0xb0, 0x1a, 0x46, 0x02, 0x2e, 0x9f, 0xef, 0x3b, 0x3b, 0x97, 0x92, 0x6f, 0xaa,
+	0xa2, 0xef, 0xb0, 0x9b, 0xd0, 0x4b, 0x02, 0x9e, 0xfd, 0x27, 0x7f, 0x08, 0xd2, 0xfc, 0xec, 0xa0,
+	0x37, 0x61, 0xd5, 0x21, 0x27, 0x46, 0x34, 0x43, 0x6c, 0x05, 0xd2, 0x7a, 0xc9, 0x21, 0x27, 0xd1,
+	0x20, 0x79, 0x0b, 0xae, 0x9f, 0xf9, 0x5e, 0xe8, 0x7b, 0x20, 0x61, 0xfe, 0x87, 0x61, 0x8e, 0x3c,
+	0xcc, 0x8e, 0x21, 0x39, 0xc0, 0x9a, 0xa0, 0xd7, 0x04, 0x59, 0xf6, 0xa0, 0x18, 0xd3, 0x0d, 0x75,
+	0x21, 0x1f, 0xf6, 0x47, 0xc4, 0xc5, 0x97, 0xc7, 0x17, 0x7a, 0x6b, 0xaa, 0x86, 0x1f, 0xe0, 0xc1,
+	0x90, 0x84, 0xd8, 0x7a, 0x04, 0xbc, 0xb5, 0x02, 0x59, 0x36, 0xaf, 0xf2, 0x8f, 0x01, 0x9d, 0x1e,
+	0x88, 0xde, 0x82, 0x35, 0xe2, 0x50, 0x53, 0x8f, 0xfa, 0x3d, 0x4c, 0xf9, 0x92, 0x5e, 0x16, 0xe4,
+	0x70, 0xe0, 0x6b, 0x50, 0x08, 0x42, 0x76, 0x66, 0x23, 0x69, 0x7d, 0x4a, 0x50, 0xfe, 0x3b, 0x0d,
+	0x57, 0x9e, 0x79, 0x56, 0x40, 0x76, 0x2c, 0x9b, 0xf8, 0xa1, 0x57, 0xed, 0x40, 0xc6, 0xb7, 0x9c,
+	0xe3, 0xcb, 0xd4, 0x43, 0x94, 0x1f, 0xfd, 0x18, 0xd6, 0x68, 0x49, 0x86, 0x83, 0xe8, 0xcc, 0xfb,
+	0x12, 0xbb, 0x5d, 0x99, 0x43, 0x85, 0x34, 0x3a, 0x03, 0x3c, 0x68, 0x11, 0xd3, 0x60, 0xdd, 0x64,
+	0x9f, 0x99, 0x60, 0x5e, 0x2f, 0x87, 0x64, 0xf6, 0x62, 0x3e, 0xfa, 0x01, 0xc8, 0xe2, 0xb2, 0x94,
+	0x49, 0xd3, 0xc9, 0x81, 0xe5, 0x10, 0xd3, 0xf0, 0x8f, 0xb0, 0x67, 0x5a, 0x4e, 0x9f, 0x25, 0x35,
+	0x79, 0x7d, 0x9d, 0x8f, 0xa8, 0x45, 0x03, 0xda, 0xe2, 0x7f, 0x44, 0x66, 0xab, 0x30, 0x5e, 0xc1,
+	0xd4, 0x16, 0x39, 0x41, 0x9d, 0x9f, 0xd6, 0x17, 0x95, 0x62, 0xbf, 0xd5, 0xa2, 0x43, 0xf9, 0x29,
+	0x64, 0x59, 0x58, 0x65, 0x0b, 0x3d, 0xcd, 0x6c, 0x2f, 0xb6, 0xd0, 0x74, 0x7f, 0xaf, 0xc0, 0xd5,
+	0xa8, 0x25, 0x12, 0x05, 0xf3, 0xf0, 0x68, 0xf1, 0x4a, 0xf4, 0x97, 0x88, 0xe5, 0xbe, 0xf2, 0x4f,
+	0x69, 0x28, 0x87, 0x2d, 0x45, 0x7e, 0xe3, 0x45, 0xf9, 0xab, 0xb4, 0xd8, 0xc1, 0xef, 0x40, 0x76,
+	0xeb, 0x79, 0x47, 0x6d, 0x4b, 0x09, 0xf9, 0x26, 0xeb, 0x0b, 0x5e, 0x65, 0x7d, 0x41, 0x86, 0xba,
+	0x79, 0x38, 0x09, 0x58, 0x6b, 0x17, 0xbd, 0x0e, 0xa9, 0xa7, 0x07, 0x52, 0x52, 0xbe, 0xc1, 0x86,
+	0x48, 0xb1, 0x21, 0xc7, 0xac, 0xfd, 0x7f, 0x17, 0x72, 0x07, 0x55, 0x5d, 0x6b, 0x74, 0xa4, 0x94,
+	0x2c, 0xb3, 0x31, 0xd7, 0x62, 0x63, 0xc6, 0xd8, 0xb3, 0x9c, 0x80, 0x8e, 0x7b, 0x3b, 0xd6, 0xb9,
+	0x4c, 0xcf, 0x9c, 0xe7, 0xf0, 0x91, 0xf1, 0xa6, 0xe5, 0x1d, 0xc8, 0x76, 0xb4, 0x3d, 0x55, 0x97,
+	0x32, 0x67, 0x68, 0xc6, 0x4a, 0x52, 0x3a, 0xea, 0x23, 0x58, 0xd3, 0x1a, 0x1d, 0x55, 0x3f, 0xa8,
+	0xd6, 0xc5, 0x81, 0x93, 0x94, 0x9d, 0x39, 0x04, 0x11, 0xc0, 0x4e, 0x40, 0xbc, 0x31, 0xb6, 0xc5,
+	0x81, 0x13, 0x3f, 0x3a, 0x59, 0xad, 0xab, 0x8d, 0xc7, 0x9d, 0x5d, 0xa3, 0xa5, 0xab, 0x3b, 0xda,
+	0x17, 0x52, 0x6e, 0xa6, 0x33, 0xca, 0xf9, 0x6c, 0xe2, 0xf4, 0x83, 0x23, 0x63, 0xe8, 0x91, 0x9e,
+	0xf5, 0xad, 0xe0, 0xe2, 0x47, 0x3b, 0xa1, 0xb4, 0x95, 0x33, 0xb8, 0xf8, 0x99, 0x4e, 0x4c, 0xd6,
+	0x07, 0x50, 0xe6, 0xc3, 0xc3, 0xfe, 0xba, 0x94, 0x97, 0x15, 0xc6, 0xf6, 0x5a, 0x8c, 0x2d, 0x72,
+	0x30, 0x6e, 0x3b, 0xe3, 0x0d, 0xe5, 0x8f, 0x73, 0x61, 0x16, 0x12, 0xbf, 0x06, 0xf0, 0xaa, 0xb3,
+	0x10, 0x74, 0x00, 0x25, 0x7e, 0xf8, 0xe0, 0x07, 0x38, 0x18, 0xf9, 0x22, 0x7f, 0x7a, 0xb4, 0x48,
+	0xe9, 0x41, 0xd9, 0xda, 0x8c, 0x8b, 0x67, 0x50, 0xc5, 0xc1, 0x94, 0x82, 0xee, 0x86, 0x41, 0x63,
+	0x9a, 0x72, 0xa4, 0x99, 0x7f, 0xad, 0x72, 0x72, 0x98, 0x1d, 0xd7, 0x60, 0x25, 0xf0, 0xac, 0x7e,
+	0x9f, 0x78, 0xa2, 0xea, 0x79, 0x7b, 0x91, 0x08, 0xcf, 0x39, 0xf4, 0x90, 0x15, 0x11, 0xb8, 0x12,
+	0x65, 0x32, 0x96, 0xeb, 0x18, 0x94, 0x85, 0xd5, 0x3d, 0xe5, 0x87, 0x1f, 0x2d, 0x80, 0x57, 0x8d,
+	0xf1, 0xee, 0xb9, 0xa6, 0xa8, 0x81, 0x25, 0x3c, 0x47, 0xa6, 0xd9, 0x35, 0x6f, 0x6e, 0xb2, 0x74,
+	0x80, 0xf5, 0x40, 0x16, 0xcb, 0xae, 0xf9, 0x59, 0x35, 0xdd, 0x5d, 0x44, 0x76, 0xed, 0x46, 0x04,
+	0x74, 0x08, 0x52, 0xd7, 0x76, 0x59, 0x92, 0x71, 0x48, 0x8e, 0xf0, 0xd8, 0x72, 0x3d, 0xd6, 0x33,
+	0x29, 0x3f, 0xfc, 0x70, 0x91, 0xd2, 0x92, 0xb3, 0x6e, 0x09, 0x4e, 0x0e, 0xbf, 0xd6, 0x9d, 0xa5,
+	0xb2, 0x2d, 0xd8, 0xb6, 0x99, 0x81, 0xd9, 0x38, 0x20, 0x0e, 0xf1, 0x7d, 0x76, 0x13, 0x88, 0x6e,
+	0xc1, 0x9c, 0x5e, 0x17, 0x64, 0x5a, 0xe7, 0x36, 0x1d, 0xaa, 0x58, 0xc8, 0xcc, 0x6e, 0xf7, 0x2c,
+	0xd6, 0x49, 0x98, 0x65, 0xe4, 0xba, 0xcc, 0xa1, 0xa1, 0x0d, 0xb8, 0x8e, 0x7d, 0xdf, 0xea, 0x3b,
+	0xbe, 0x11, 0xb8, 0x86, 0xeb, 0x10, 0xe1, 0x2b, 0xeb, 0xc0, 0xf6, 0x07, 0x24, 0xfe, 0xec, 0xb8,
+	0x4d, 0x87, 0x70, 0xfb, 0x57, 0xbe, 0x82, 0x62, 0xcc, 0xd8, 0x94, 0xbd, 0xf3, 0x2a, 0x90, 0x35,
+	0x28, 0x36, 0x9a, 0x0d, 0x76, 0x0e, 0xa7, 0x35, 0x1e, 0x4b, 0x49, 0x46, 0x50, 0xd5, 0x5a, 0x9b,
+	0x1f, 0xcd, 0x49, 0x29, 0x84, 0xa0, 0x5c, 0xad, 0xeb, 0x6a, 0xb5, 0x26, 0x4e, 0xeb, 0x6a, 0x52,
+	0x5a, 0xd9, 0x03, 0x69, 0x7e, 0xfd, 0x95, 0x8f, 0xcf, 0x13, 0x51, 0x06, 0xa8, 0x69, 0xed, 0xed,
+	0xaa, 0x5e, 0xe3, 0x12, 0x24, 0x28, 0x45, 0x07, 0x7e, 0x94, 0x92, 0x52, 0x3e, 0x87, 0xb5, 0xb9,
+	0x35, 0x51, 0x3e, 0x7d, 0x81, 0xc2, 0xea, 0x9e, 0xd6, 0x31, 0xaa, 0xf5, 0x67, 0xd5, 0xe7, 0x6d,
+	0xde, 0x43, 0x61, 0x04, 0x6d, 0xc7, 0x68, 0x34, 0x1b, 0xea, 0x5e, 0xab, 0xf3, 0x5c, 0x4a, 0x29,
+	0xad, 0xf9, 0x25, 0x79, 0x21, 0xe2, 0x8e, 0xa6, 0xab, 0x33, 0x88, 0x8c, 0x30, 0x8b, 0x78, 0x08,
+	0x30, 0x35, 0x49, 0xa5, 0x73, 0x1e, 0xda, 0x15, 0x58, 0x55, 0x1b, 0x35, 0xa3, 0xb9, 0x63, 0x44,
+	0x5d, 0x1e, 0x04, 0xe5, 0x7a, 0x95, 0xdd, 0x0d, 0xd0, 0x1a, 0x46, 0xab, 0xda, 0xa0, 0xb3, 0x4a,
+	0xb5, 0xae, 0xea, 0x75, 0x2d, 0x4e, 0x4d, 0x2b, 0x36, 0xc0, 0xb4, 0xa8, 0x54, 0xbe, 0x7e, 0xc1,
+	0x8c, 0xaa, 0x07, 0x6a, 0xa3, 0x63, 0xd0, 0x88, 0x2f, 0x25, 0xd1, 0x55, 0x58, 0x13, 0x07, 0x69,
+	0x5a, 0xe3, 0x31, 0x27, 0xa6, 0xd0, 0x2d, 0x78, 0xad, 0xfd, 0xbc, 0xb1, 0xbd, 0xab, 0x37, 0x1b,
+	0xda, 0x97, 0x6a, 0xcd, 0x98, 0x1f, 0x91, 0x56, 0x7e, 0x29, 0xc1, 0x8a, 0x08, 0x0b, 0x48, 0x87,
+	0x02, 0xee, 0x05, 0xc4, 0x33, 0xb0, 0x6d, 0x8b, 0x20, 0xf9, 0x68, 0xf1, 0xa8, 0x52, 0xa9, 0x52,
+	0xde, 0xaa, 0x6d, 0xef, 0x26, 0xf4, 0x3c, 0x16, 0xbf, 0x63, 0x98, 0xce, 0x44, 0x64, 0x05, 0xcb,
+	0x63, 0x3a, 0x93, 0x29, 0xa6, 0x33, 0x41, 0xfb, 0x00, 0x1c, 0x93, 0xe0, 0xee, 0x91, 0x48, 0xeb,
+	0xdf, 0x5b, 0x16, 0x54, 0xc5, 0xdd, 0xa3, 0xdd, 0x84, 0xce, 0xb5, 0xa3, 0x0f, 0xc8, 0x86, 0xab,
+	0x02, 0xd6, 0x31, 0x0d, 0xb7, 0x17, 0xfa, 0x17, 0x0f, 0xaf, 0x9f, 0x2c, 0x8d, 0xef, 0x98, 0xcd,
+	0x1e, 0x77, 0xc4, 0xdd, 0x84, 0x2e, 0xe1, 0x39, 0x1a, 0x0a, 0xe0, 0x3a, 0x97, 0x36, 0x57, 0x2c,
+	0x89, 0xb6, 0xd3, 0xa7, 0xcb, 0xca, 0x3b, 0x5d, 0x14, 0xe1, 0xd3, 0x64, 0xf4, 0xf3, 0x24, 0x28,
+	0x5c, 0xac, 0x3f, 0x71, 0xba, 0x47, 0x9e, 0xeb, 0x58, 0x7f, 0x40, 0xcc, 0x53, 0x3a, 0xf0, 0x43,
+	0xa7, 0x27, 0xcb, 0xea, 0xd0, 0x8e, 0x61, 0x9e, 0xd2, 0xe7, 0x0d, 0xfc, 0xe2, 0x21, 0xe8, 0x29,
+	0xe4, 0xb0, 0x7d, 0x82, 0x27, 0xbe, 0xb8, 0x6c, 0xb8, 0xb1, 0x8c, 0x78, 0xc6, 0xb8, 0x9b, 0xd0,
+	0x05, 0x04, 0x6a, 0xc0, 0x8a, 0x49, 0x7a, 0x78, 0x64, 0x07, 0xe2, 0x32, 0xe8, 0xc3, 0x25, 0xd0,
+	0x6a, 0x9c, 0x73, 0x37, 0xa1, 0x87, 0x20, 0xe8, 0xeb, 0x69, 0x35, 0xd9, 0x75, 0x47, 0x4e, 0x20,
+	0x2e, 0x84, 0x7e, 0xb8, 0x04, 0xaa, 0x1a, 0xf6, 0x9f, 0x46, 0x4e, 0x10, 0x2b, 0x1f, 0xd9, 0x33,
+	0xda, 0x85, 0xac, 0x43, 0xc6, 0xc4, 0x13, 0x77, 0x42, 0xdf, 0x5d, 0x02, 0xb7, 0x41, 0xf9, 0x76,
+	0x13, 0x3a, 0x07, 0xa0, 0xde, 0xe1, 0x7a, 0x46, 0xcf, 0x72, 0xb0, 0x6d, 0x4f, 0xd8, 0xee, 0xb0,
+	0x9c, 0x77, 0x34, 0xbd, 0x1d, 0xce, 0x4b, 0xbd, 0xc3, 0x0d, 0x1f, 0xe8, 0xea, 0x78, 0x64, 0x48,
+	0x70, 0x78, 0xc5, 0x74, 0x99, 0xd5, 0xd1, 0x19, 0x23, 0x5d, 0x1d, 0x0e, 0x21, 0x7f, 0x01, 0xf9,
+	0x30, 0x5a, 0xa0, 0x3a, 0x14, 0xd9, 0xd5, 0x3e, 0x36, 0x34, 0xac, 0x57, 0x97, 0xc9, 0x66, 0xe2,
+	0xec, 0x53, 0x64, 0x67, 0xf2, 0x8a, 0x91, 0x9f, 0x43, 0x21, 0x0a, 0x1c, 0xaf, 0x18, 0xfa, 0xef,
+	0x93, 0x20, 0xcd, 0x07, 0x0d, 0xd4, 0x84, 0x55, 0x82, 0x3d, 0x7b, 0x62, 0xf4, 0x2c, 0xcf, 0x72,
+	0xfa, 0xe1, 0x7d, 0xd2, 0x65, 0x84, 0x94, 0x18, 0xc0, 0x0e, 0xe7, 0x47, 0x7b, 0x50, 0xa2, 0x49,
+	0x4c, 0x84, 0x97, 0x5a, 0x1a, 0xaf, 0x48, 0xf9, 0x05, 0x9c, 0xfc, 0x53, 0xb8, 0x7a, 0x46, 0xe0,
+	0x41, 0x47, 0x70, 0x2d, 0xaa, 0xde, 0x8d, 0x53, 0x1f, 0xe0, 0xbc, 0xbf, 0x60, 0x4b, 0x95, 0xb1,
+	0x4f, 0xbf, 0xb8, 0xb8, 0x1a, 0x9c, 0xa2, 0xf9, 0xf2, 0x6d, 0x78, 0xe3, 0x25, 0x51, 0x47, 0x2e,
+	0xc0, 0x8a, 0xf0, 0x65, 0xf9, 0x11, 0x94, 0xe2, 0x0e, 0x88, 0xde, 0x9c, 0x77, 0x68, 0x3a, 0xbd,
+	0xd9, 0x59, 0xaf, 0x94, 0x57, 0x20, 0xcb, 0xbc, 0x4b, 0xce, 0x43, 0x8e, 0x87, 0x18, 0xf9, 0xcf,
+	0x92, 0x50, 0x88, 0x5c, 0x04, 0x7d, 0x0a, 0x99, 0xa8, 0x61, 0xbc, 0xdc, 0x5c, 0x32, 0x3e, 0x9a,
+	0xc6, 0x87, 0x9e, 0xba, 0xfc, 0x72, 0x84, 0xac, 0x72, 0x07, 0x72, 0xdc, 0xc5, 0xd0, 0x13, 0x80,
+	0xa9, 0x61, 0x5d, 0x40, 0xab, 0x18, 0xf7, 0x56, 0x21, 0x2a, 0x31, 0x94, 0x5f, 0xa7, 0x62, 0x3d,
+	0x9e, 0xe9, 0x85, 0xe0, 0x36, 0x64, 0x4d, 0x62, 0xe3, 0x89, 0x10, 0xf4, 0xc9, 0x85, 0x16, 0xb7,
+	0x52, 0xa3, 0x10, 0x34, 0x7e, 0x31, 0x2c, 0xf4, 0x25, 0xe4, 0xb1, 0x6d, 0xf5, 0x1d, 0x23, 0x70,
+	0xc5, 0x9c, 0xfc, 0xf0, 0x62, 0xb8, 0x55, 0x8a, 0xd2, 0x71, 0x69, 0x14, 0xc7, 0xfc, 0xa7, 0xfc,
+	0x36, 0x64, 0x99, 0x34, 0x74, 0x1b, 0x4a, 0x4c, 0x9a, 0x31, 0xb0, 0x6c, 0xdb, 0xf2, 0x45, 0x5f,
+	0xad, 0xc8, 0x68, 0x7b, 0x8c, 0x24, 0x7f, 0x0c, 0x2b, 0x02, 0x01, 0xdd, 0x80, 0xdc, 0x90, 0x78,
+	0x96, 0xcb, 0x6b, 0xb1, 0xb4, 0x2e, 0x9e, 0x28, 0xdd, 0xed, 0xf5, 0x7c, 0x12, 0xb0, 0x24, 0x21,
+	0xad, 0x8b, 0xa7, 0xad, 0xeb, 0x70, 0xf5, 0x0c, 0x1f, 0x50, 0xfe, 0x34, 0x05, 0x85, 0xa8, 0xdd,
+	0x81, 0x0e, 0xa0, 0x8c, 0xbb, 0xec, 0x2e, 0xd3, 0x10, 0x07, 0x01, 0xf1, 0x9c, 0x8b, 0x5e, 0x9b,
+	0x5e, 0xe5, 0x30, 0x2d, 0x8e, 0x82, 0x9e, 0xc2, 0xca, 0xd8, 0x22, 0x27, 0x97, 0x3b, 0xb9, 0xc9,
+	0x51, 0x88, 0x1d, 0x07, 0x7d, 0x0d, 0xe2, 0x4e, 0xbd, 0x31, 0xc0, 0xc3, 0x21, 0xcd, 0x0f, 0x7a,
+	0x8e, 0xc8, 0xb8, 0x2e, 0x02, 0x2b, 0x6a, 0xdb, 0x3d, 0x8e, 0xb5, 0xe3, 0x28, 0x4f, 0xa1, 0x18,
+	0xbb, 0xbf, 0x83, 0x24, 0x48, 0x8f, 0x3c, 0x3b, 0xec, 0x28, 0x8d, 0x3c, 0x9b, 0x53, 0x1c, 0xd1,
+	0x76, 0xa7, 0x3f, 0xd1, 0x3a, 0xac, 0xc4, 0x3f, 0x7d, 0x29, 0xe9, 0xe1, 0xa3, 0xf2, 0x97, 0x49,
+	0xb8, 0x16, 0x36, 0x6d, 0xe2, 0x5f, 0x10, 0x29, 0x3f, 0x4b, 0x42, 0x29, 0x4e, 0x40, 0x77, 0x20,
+	0x57, 0x6b, 0xb2, 0x03, 0xd5, 0x84, 0xbc, 0xce, 0x9a, 0x0a, 0x88, 0x35, 0x15, 0x88, 0x33, 0xde,
+	0x34, 0xdd, 0xee, 0x31, 0x6f, 0x94, 0xdc, 0x85, 0x15, 0x91, 0x30, 0x4b, 0xc9, 0x99, 0x86, 0x0a,
+	0x1d, 0x26, 0x52, 0x26, 0x3a, 0xee, 0x1e, 0xe4, 0xd5, 0x2f, 0x3a, 0xaa, 0xde, 0xa8, 0xd6, 0xe7,
+	0x9a, 0x39, 0x74, 0x20, 0xf9, 0x96, 0x2e, 0x0b, 0xb6, 0x37, 0xc7, 0x1b, 0xca, 0x47, 0xb0, 0x5a,
+	0x63, 0xf0, 0x61, 0x23, 0xf3, 0x2d, 0x58, 0xeb, 0xba, 0x4e, 0x80, 0x2d, 0x87, 0xd6, 0xfa, 0x03,
+	0xdc, 0x0f, 0x6f, 0x8a, 0x95, 0x23, 0xb2, 0x46, 0xa9, 0xca, 0xbf, 0x25, 0xa1, 0x2c, 0x82, 0x5b,
+	0xc8, 0x5b, 0x86, 0x94, 0xeb, 0x8b, 0xe1, 0x29, 0xd7, 0x47, 0x08, 0x32, 0xd8, 0xeb, 0x1e, 0x89,
+	0xb9, 0x62, 0xbf, 0xe9, 0x64, 0x75, 0xdd, 0xc1, 0x00, 0x3b, 0x61, 0x1b, 0x21, 0x7c, 0x44, 0x75,
+	0x48, 0x13, 0x67, 0xbc, 0xcc, 0x4d, 0xf7, 0x19, 0xe9, 0x15, 0xd5, 0x19, 0xf3, 0x26, 0x21, 0x85,
+	0x91, 0x3f, 0x80, 0x7c, 0x48, 0x58, 0xea, 0x4e, 0xf9, 0x4f, 0x60, 0x6d, 0xce, 0x46, 0x5e, 0xcd,
+	0x47, 0x04, 0xbf, 0x0b, 0xe5, 0xd8, 0x55, 0xb0, 0xe9, 0xc1, 0xcd, 0x6a, 0x8c, 0xaa, 0x99, 0xca,
+	0x26, 0x94, 0x66, 0x64, 0x0b, 0x6b, 0x4b, 0x2e, 0x62, 0x6d, 0xff, 0x93, 0x81, 0x62, 0xec, 0xa2,
+	0x1b, 0xd2, 0x20, 0x6b, 0x05, 0x24, 0xda, 0xe9, 0x1e, 0x2d, 0x77, 0x4f, 0xae, 0xa2, 0x05, 0x64,
+	0xa0, 0x73, 0x04, 0xb9, 0x07, 0xa0, 0x99, 0xc4, 0x09, 0xac, 0x9e, 0x45, 0xbc, 0x45, 0x2e, 0x14,
+	0xbd, 0x09, 0xab, 0xd3, 0x21, 0x53, 0x7f, 0x99, 0xf2, 0xed, 0x7b, 0x4e, 0xb8, 0x2e, 0xe9, 0x68,
+	0x5d, 0xe4, 0x5f, 0xa5, 0x20, 0x43, 0xe5, 0x22, 0x0d, 0x52, 0x02, 0x78, 0xb1, 0x2f, 0x29, 0x66,
+	0x14, 0x8f, 0x34, 0xd5, 0x53, 0x16, 0xb5, 0x2b, 0x7e, 0x8f, 0x22, 0xb5, 0x70, 0x17, 0x29, 0x0e,
+	0x36, 0x77, 0x93, 0x02, 0xbd, 0x1d, 0x5a, 0x0e, 0x8f, 0x39, 0xd7, 0x2a, 0xfc, 0x2b, 0xe0, 0x4a,
+	0xf8, 0x15, 0x70, 0xa5, 0xea, 0x84, 0x1f, 0x28, 0xa2, 0xf7, 0xa1, 0xe8, 0x1f, 0xb9, 0x5e, 0xc0,
+	0x7b, 0x81, 0xa2, 0x6e, 0x3b, 0x9b, 0x03, 0xd8, 0x40, 0x76, 0x26, 0x4f, 0x8d, 0xd3, 0xc6, 0x87,
+	0xc4, 0x16, 0xdf, 0xb3, 0xf0, 0x07, 0x74, 0x13, 0xf2, 0xb6, 0xe5, 0x1c, 0x1b, 0x34, 0x1c, 0xe5,
+	0xb8, 0xe7, 0xd0, 0xe7, 0x7d, 0xcf, 0x96, 0x7f, 0x22, 0x6e, 0x77, 0x8c, 0x5e, 0x70, 0xbb, 0xa3,
+	0xdd, 0xd1, 0x79, 0xa3, 0xa3, 0x08, 0x2b, 0x5a, 0xa3, 0xa3, 0x3e, 0x56, 0x75, 0x29, 0x85, 0x0a,
+	0x90, 0xdd, 0xa9, 0x37, 0xab, 0x1d, 0x29, 0xcd, 0x8f, 0x79, 0x9b, 0x75, 0xb5, 0xda, 0x90, 0x32,
+	0x68, 0x15, 0x0a, 0xb4, 0x1c, 0x6f, 0x77, 0xaa, 0x7b, 0x2d, 0x29, 0x8b, 0x4a, 0x90, 0xaf, 0xed,
+	0xeb, 0x55, 0x76, 0x41, 0x37, 0x47, 0x0b, 0xfd, 0x27, 0xd5, 0x83, 0xaa, 0xb1, 0x5d, 0xaf, 0xb6,
+	0xdb, 0xd2, 0xca, 0xe6, 0x67, 0x90, 0x67, 0xdf, 0x57, 0x53, 0xeb, 0xbc, 0x7d, 0xea, 0xed, 0xa8,
+	0x46, 0xec, 0xbd, 0x9a, 0x43, 0xfe, 0x5d, 0xf2, 0xbf, 0xfe, 0xf5, 0xdf, 0x7e, 0xc1, 0xf5, 0xa7,
+	0x5c, 0xfb, 0x9e, 0xb3, 0xf5, 0x09, 0xbc, 0xfc, 0x4b, 0xec, 0xad, 0x82, 0xce, 0xce, 0x1b, 0xaa,
+	0x43, 0xeb, 0xcb, 0x62, 0x48, 0x37, 0xc6, 0x1b, 0x87, 0x39, 0x26, 0xe9, 0xd1, 0xff, 0x07, 0x00,
+	0x00, 0xff, 0xff, 0x12, 0x34, 0xea, 0xe3, 0xe4, 0x3d, 0x00, 0x00,
 }
diff --git a/sdks/go/pkg/beam/model/pipeline_v1/endpoints.pb.go b/sdks/go/pkg/beam/model/pipeline_v1/endpoints.pb.go
index 0964986d3c5..0b4bea22e10 100644
--- a/sdks/go/pkg/beam/model/pipeline_v1/endpoints.pb.go
+++ b/sdks/go/pkg/beam/model/pipeline_v1/endpoints.pb.go
@@ -3,9 +3,11 @@
 
 package pipeline_v1
 
-import proto "github.com/golang/protobuf/proto"
-import fmt "fmt"
-import math "math"
+import (
+	fmt "fmt"
+	proto "github.com/golang/protobuf/proto"
+	math "math"
+)
 
 // Reference imports to suppress errors if they are not otherwise used.
 var _ = proto.Marshal
@@ -37,7 +39,7 @@ func (m *ApiServiceDescriptor) Reset()         { *m = ApiServiceDescriptor{} }
 func (m *ApiServiceDescriptor) String() string { return proto.CompactTextString(m) }
 func (*ApiServiceDescriptor) ProtoMessage()    {}
 func (*ApiServiceDescriptor) Descriptor() ([]byte, []int) {
-	return fileDescriptor_endpoints_d652457ab0ad8b4f, []int{0}
+	return fileDescriptor_6445e0c85107719d, []int{0}
 }
 func (m *ApiServiceDescriptor) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_ApiServiceDescriptor.Unmarshal(m, b)
@@ -45,8 +47,8 @@ func (m *ApiServiceDescriptor) XXX_Unmarshal(b []byte) error {
 func (m *ApiServiceDescriptor) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_ApiServiceDescriptor.Marshal(b, m, deterministic)
 }
-func (dst *ApiServiceDescriptor) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ApiServiceDescriptor.Merge(dst, src)
+func (m *ApiServiceDescriptor) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_ApiServiceDescriptor.Merge(m, src)
 }
 func (m *ApiServiceDescriptor) XXX_Size() int {
 	return xxx_messageInfo_ApiServiceDescriptor.Size(m)
@@ -57,6 +59,13 @@ func (m *ApiServiceDescriptor) XXX_DiscardUnknown() {
 
 var xxx_messageInfo_ApiServiceDescriptor proto.InternalMessageInfo
 
+func (m *ApiServiceDescriptor) GetUrl() string {
+	if m != nil {
+		return m.Url
+	}
+	return ""
+}
+
 type isApiServiceDescriptor_Authentication interface {
 	isApiServiceDescriptor_Authentication()
 }
@@ -74,13 +83,6 @@ func (m *ApiServiceDescriptor) GetAuthentication() isApiServiceDescriptor_Authen
 	return nil
 }
 
-func (m *ApiServiceDescriptor) GetUrl() string {
-	if m != nil {
-		return m.Url
-	}
-	return ""
-}
-
 func (m *ApiServiceDescriptor) GetOauth2ClientCredentialsGrant() *OAuth2ClientCredentialsGrant {
 	if x, ok := m.GetAuthentication().(*ApiServiceDescriptor_Oauth2ClientCredentialsGrant); ok {
 		return x.Oauth2ClientCredentialsGrant
@@ -156,7 +158,7 @@ func (m *OAuth2ClientCredentialsGrant) Reset()         { *m = OAuth2ClientCreden
 func (m *OAuth2ClientCredentialsGrant) String() string { return proto.CompactTextString(m) }
 func (*OAuth2ClientCredentialsGrant) ProtoMessage()    {}
 func (*OAuth2ClientCredentialsGrant) Descriptor() ([]byte, []int) {
-	return fileDescriptor_endpoints_d652457ab0ad8b4f, []int{1}
+	return fileDescriptor_6445e0c85107719d, []int{1}
 }
 func (m *OAuth2ClientCredentialsGrant) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_OAuth2ClientCredentialsGrant.Unmarshal(m, b)
@@ -164,8 +166,8 @@ func (m *OAuth2ClientCredentialsGrant) XXX_Unmarshal(b []byte) error {
 func (m *OAuth2ClientCredentialsGrant) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_OAuth2ClientCredentialsGrant.Marshal(b, m, deterministic)
 }
-func (dst *OAuth2ClientCredentialsGrant) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OAuth2ClientCredentialsGrant.Merge(dst, src)
+func (m *OAuth2ClientCredentialsGrant) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_OAuth2ClientCredentialsGrant.Merge(m, src)
 }
 func (m *OAuth2ClientCredentialsGrant) XXX_Size() int {
 	return xxx_messageInfo_OAuth2ClientCredentialsGrant.Size(m)
@@ -188,9 +190,9 @@ func init() {
 	proto.RegisterType((*OAuth2ClientCredentialsGrant)(nil), "org.apache.beam.model.pipeline.v1.OAuth2ClientCredentialsGrant")
 }
 
-func init() { proto.RegisterFile("endpoints.proto", fileDescriptor_endpoints_d652457ab0ad8b4f) }
+func init() { proto.RegisterFile("endpoints.proto", fileDescriptor_6445e0c85107719d) }
 
-var fileDescriptor_endpoints_d652457ab0ad8b4f = []byte{
+var fileDescriptor_6445e0c85107719d = []byte{
 	// 235 bytes of a gzipped FileDescriptorProto
 	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x84, 0x90, 0xb1, 0x4a, 0x03, 0x41,
 	0x10, 0x86, 0x5d, 0x03, 0x42, 0x36, 0xa0, 0xe1, 0xb0, 0x48, 0x11, 0x30, 0xa6, 0x4a, 0xb5, 0x98,
diff --git a/sdks/go/pkg/beam/model/pipeline_v1/standard_window_fns.pb.go b/sdks/go/pkg/beam/model/pipeline_v1/standard_window_fns.pb.go
index c8e53237eec..424dba9d9c7 100644
--- a/sdks/go/pkg/beam/model/pipeline_v1/standard_window_fns.pb.go
+++ b/sdks/go/pkg/beam/model/pipeline_v1/standard_window_fns.pb.go
@@ -3,11 +3,13 @@
 
 package pipeline_v1
 
-import proto "github.com/golang/protobuf/proto"
-import fmt "fmt"
-import math "math"
-import duration "github.com/golang/protobuf/ptypes/duration"
-import timestamp "github.com/golang/protobuf/ptypes/timestamp"
+import (
+	fmt "fmt"
+	proto "github.com/golang/protobuf/proto"
+	duration "github.com/golang/protobuf/ptypes/duration"
+	timestamp "github.com/golang/protobuf/ptypes/timestamp"
+	math "math"
+)
 
 // Reference imports to suppress errors if they are not otherwise used.
 var _ = proto.Marshal
@@ -30,6 +32,7 @@ const (
 var GlobalWindowsPayload_Enum_name = map[int32]string{
 	0: "PROPERTIES",
 }
+
 var GlobalWindowsPayload_Enum_value = map[string]int32{
 	"PROPERTIES": 0,
 }
@@ -37,8 +40,9 @@ var GlobalWindowsPayload_Enum_value = map[string]int32{
 func (x GlobalWindowsPayload_Enum) String() string {
 	return proto.EnumName(GlobalWindowsPayload_Enum_name, int32(x))
 }
+
 func (GlobalWindowsPayload_Enum) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_standard_window_fns_da1ada993b818d10, []int{0, 0}
+	return fileDescriptor_fab9dd76b0d0d680, []int{0, 0}
 }
 
 type FixedWindowsPayload_Enum int32
@@ -51,6 +55,7 @@ const (
 var FixedWindowsPayload_Enum_name = map[int32]string{
 	0: "PROPERTIES",
 }
+
 var FixedWindowsPayload_Enum_value = map[string]int32{
 	"PROPERTIES": 0,
 }
@@ -58,8 +63,9 @@ var FixedWindowsPayload_Enum_value = map[string]int32{
 func (x FixedWindowsPayload_Enum) String() string {
 	return proto.EnumName(FixedWindowsPayload_Enum_name, int32(x))
 }
+
 func (FixedWindowsPayload_Enum) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_standard_window_fns_da1ada993b818d10, []int{1, 0}
+	return fileDescriptor_fab9dd76b0d0d680, []int{1, 0}
 }
 
 type SlidingWindowsPayload_Enum int32
@@ -72,6 +78,7 @@ const (
 var SlidingWindowsPayload_Enum_name = map[int32]string{
 	0: "PROPERTIES",
 }
+
 var SlidingWindowsPayload_Enum_value = map[string]int32{
 	"PROPERTIES": 0,
 }
@@ -79,8 +86,9 @@ var SlidingWindowsPayload_Enum_value = map[string]int32{
 func (x SlidingWindowsPayload_Enum) String() string {
 	return proto.EnumName(SlidingWindowsPayload_Enum_name, int32(x))
 }
+
 func (SlidingWindowsPayload_Enum) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_standard_window_fns_da1ada993b818d10, []int{2, 0}
+	return fileDescriptor_fab9dd76b0d0d680, []int{2, 0}
 }
 
 type SessionsPayload_Enum int32
@@ -93,6 +101,7 @@ const (
 var SessionsPayload_Enum_name = map[int32]string{
 	0: "PROPERTIES",
 }
+
 var SessionsPayload_Enum_value = map[string]int32{
 	"PROPERTIES": 0,
 }
@@ -100,8 +109,9 @@ var SessionsPayload_Enum_value = map[string]int32{
 func (x SessionsPayload_Enum) String() string {
 	return proto.EnumName(SessionsPayload_Enum_name, int32(x))
 }
+
 func (SessionsPayload_Enum) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_standard_window_fns_da1ada993b818d10, []int{3, 0}
+	return fileDescriptor_fab9dd76b0d0d680, []int{3, 0}
 }
 
 type GlobalWindowsPayload struct {
@@ -114,7 +124,7 @@ func (m *GlobalWindowsPayload) Reset()         { *m = GlobalWindowsPayload{} }
 func (m *GlobalWindowsPayload) String() string { return proto.CompactTextString(m) }
 func (*GlobalWindowsPayload) ProtoMessage()    {}
 func (*GlobalWindowsPayload) Descriptor() ([]byte, []int) {
-	return fileDescriptor_standard_window_fns_da1ada993b818d10, []int{0}
+	return fileDescriptor_fab9dd76b0d0d680, []int{0}
 }
 func (m *GlobalWindowsPayload) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_GlobalWindowsPayload.Unmarshal(m, b)
@@ -122,8 +132,8 @@ func (m *GlobalWindowsPayload) XXX_Unmarshal(b []byte) error {
 func (m *GlobalWindowsPayload) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_GlobalWindowsPayload.Marshal(b, m, deterministic)
 }
-func (dst *GlobalWindowsPayload) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_GlobalWindowsPayload.Merge(dst, src)
+func (m *GlobalWindowsPayload) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_GlobalWindowsPayload.Merge(m, src)
 }
 func (m *GlobalWindowsPayload) XXX_Size() int {
 	return xxx_messageInfo_GlobalWindowsPayload.Size(m)
@@ -146,7 +156,7 @@ func (m *FixedWindowsPayload) Reset()         { *m = FixedWindowsPayload{} }
 func (m *FixedWindowsPayload) String() string { return proto.CompactTextString(m) }
 func (*FixedWindowsPayload) ProtoMessage()    {}
 func (*FixedWindowsPayload) Descriptor() ([]byte, []int) {
-	return fileDescriptor_standard_window_fns_da1ada993b818d10, []int{1}
+	return fileDescriptor_fab9dd76b0d0d680, []int{1}
 }
 func (m *FixedWindowsPayload) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_FixedWindowsPayload.Unmarshal(m, b)
@@ -154,8 +164,8 @@ func (m *FixedWindowsPayload) XXX_Unmarshal(b []byte) error {
 func (m *FixedWindowsPayload) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_FixedWindowsPayload.Marshal(b, m, deterministic)
 }
-func (dst *FixedWindowsPayload) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_FixedWindowsPayload.Merge(dst, src)
+func (m *FixedWindowsPayload) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_FixedWindowsPayload.Merge(m, src)
 }
 func (m *FixedWindowsPayload) XXX_Size() int {
 	return xxx_messageInfo_FixedWindowsPayload.Size(m)
@@ -193,7 +203,7 @@ func (m *SlidingWindowsPayload) Reset()         { *m = SlidingWindowsPayload{} }
 func (m *SlidingWindowsPayload) String() string { return proto.CompactTextString(m) }
 func (*SlidingWindowsPayload) ProtoMessage()    {}
 func (*SlidingWindowsPayload) Descriptor() ([]byte, []int) {
-	return fileDescriptor_standard_window_fns_da1ada993b818d10, []int{2}
+	return fileDescriptor_fab9dd76b0d0d680, []int{2}
 }
 func (m *SlidingWindowsPayload) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_SlidingWindowsPayload.Unmarshal(m, b)
@@ -201,8 +211,8 @@ func (m *SlidingWindowsPayload) XXX_Unmarshal(b []byte) error {
 func (m *SlidingWindowsPayload) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_SlidingWindowsPayload.Marshal(b, m, deterministic)
 }
-func (dst *SlidingWindowsPayload) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_SlidingWindowsPayload.Merge(dst, src)
+func (m *SlidingWindowsPayload) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_SlidingWindowsPayload.Merge(m, src)
 }
 func (m *SlidingWindowsPayload) XXX_Size() int {
 	return xxx_messageInfo_SlidingWindowsPayload.Size(m)
@@ -245,7 +255,7 @@ func (m *SessionsPayload) Reset()         { *m = SessionsPayload{} }
 func (m *SessionsPayload) String() string { return proto.CompactTextString(m) }
 func (*SessionsPayload) ProtoMessage()    {}
 func (*SessionsPayload) Descriptor() ([]byte, []int) {
-	return fileDescriptor_standard_window_fns_da1ada993b818d10, []int{3}
+	return fileDescriptor_fab9dd76b0d0d680, []int{3}
 }
 func (m *SessionsPayload) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_SessionsPayload.Unmarshal(m, b)
@@ -253,8 +263,8 @@ func (m *SessionsPayload) XXX_Unmarshal(b []byte) error {
 func (m *SessionsPayload) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
 	return xxx_messageInfo_SessionsPayload.Marshal(b, m, deterministic)
 }
-func (dst *SessionsPayload) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_SessionsPayload.Merge(dst, src)
+func (m *SessionsPayload) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_SessionsPayload.Merge(m, src)
 }
 func (m *SessionsPayload) XXX_Size() int {
 	return xxx_messageInfo_SessionsPayload.Size(m)
@@ -283,11 +293,9 @@ func init() {
 	proto.RegisterEnum("org.apache.beam.model.pipeline.v1.SessionsPayload_Enum", SessionsPayload_Enum_name, SessionsPayload_Enum_value)
 }
 
-func init() {
-	proto.RegisterFile("standard_window_fns.proto", fileDescriptor_standard_window_fns_da1ada993b818d10)
-}
+func init() { proto.RegisterFile("standard_window_fns.proto", fileDescriptor_fab9dd76b0d0d680) }
 
-var fileDescriptor_standard_window_fns_da1ada993b818d10 = []byte{
+var fileDescriptor_fab9dd76b0d0d680 = []byte{
 	// 407 bytes of a gzipped FileDescriptorProto
 	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xbc, 0x93, 0x31, 0x4f, 0xdb, 0x40,
 	0x14, 0xc7, 0xeb, 0x36, 0x4d, 0xab, 0xcb, 0xd0, 0xd6, 0x6d, 0xa4, 0xc4, 0x43, 0x9b, 0x78, 0x68,
diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/AssignWindowsRunnerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/AssignWindowsRunnerTest.java
index 54ce41cd758..0d3126ab837 100644
--- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/AssignWindowsRunnerTest.java
+++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/AssignWindowsRunnerTest.java
@@ -31,10 +31,10 @@
 import java.util.Collection;
 import java.util.Collections;
 import org.apache.beam.fn.harness.AssignWindowsRunner.AssignWindowsMapFnFactory;
-import org.apache.beam.model.pipeline.v1.RunnerApi.Environment;
 import org.apache.beam.model.pipeline.v1.RunnerApi.FunctionSpec;
 import org.apache.beam.model.pipeline.v1.RunnerApi.PTransform;
 import org.apache.beam.model.pipeline.v1.RunnerApi.WindowIntoPayload;
+import org.apache.beam.runners.core.construction.Environments;
 import org.apache.beam.runners.core.construction.PTransformTranslation;
 import org.apache.beam.runners.core.construction.SdkComponents;
 import org.apache.beam.runners.core.construction.WindowingStrategyTranslation;
@@ -175,7 +175,7 @@ public boolean isCompatible(WindowFn<?, ?> other) {
     ListMultimap<String, FnDataReceiver<WindowedValue<?>>> receivers = ArrayListMultimap.create();
     receivers.put("output", outputs::add);
     SdkComponents components = SdkComponents.create();
-    components.registerEnvironment(Environment.newBuilder().setUrl("java").build());
+    components.registerEnvironment(Environments.createDockerEnvironment("java"));
     MapFnRunners.forWindowedValueMapFnFactory(new AssignWindowsMapFnFactory<>())
         .createRunnerForPTransform(
             null /* pipelineOptions */,
@@ -277,7 +277,7 @@ public boolean isCompatible(WindowFn<?, ?> other) {
   @Test
   public void factoryCreatesFromJavaWindowFn() throws Exception {
     SdkComponents components = SdkComponents.create();
-    components.registerEnvironment(Environment.newBuilder().setUrl("java").build());
+    components.registerEnvironment(Environments.createDockerEnvironment("java"));
     PTransform windowPTransform =
         PTransform.newBuilder()
             .putInputs("in", "input")
@@ -316,7 +316,7 @@ public void factoryCreatesFromJavaWindowFn() throws Exception {
   @Test
   public void factoryCreatesFromKnownWindowFn() throws Exception {
     SdkComponents components = SdkComponents.create();
-    components.registerEnvironment(Environment.newBuilder().setUrl("java").build());
+    components.registerEnvironment(Environments.createDockerEnvironment("java"));
     PTransform windowPTransform =
         PTransform.newBuilder()
             .putInputs("in", "input")
diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/WindowMappingFnRunnerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/WindowMappingFnRunnerTest.java
index 20af6528b37..702a28b8030 100644
--- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/WindowMappingFnRunnerTest.java
+++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/WindowMappingFnRunnerTest.java
@@ -20,7 +20,7 @@
 import static org.junit.Assert.assertEquals;
 
 import org.apache.beam.model.pipeline.v1.RunnerApi;
-import org.apache.beam.model.pipeline.v1.RunnerApi.Environment;
+import org.apache.beam.runners.core.construction.Environments;
 import org.apache.beam.runners.core.construction.ParDoTranslation;
 import org.apache.beam.runners.core.construction.SdkComponents;
 import org.apache.beam.sdk.fn.function.ThrowingFunction;
@@ -43,7 +43,7 @@ public void testWindowMapping() throws Exception {
     String pTransformId = "pTransformId";
 
     SdkComponents components = SdkComponents.create();
-    components.registerEnvironment(Environment.newBuilder().setUrl("java").build());
+    components.registerEnvironment(Environments.createDockerEnvironment("java"));
     RunnerApi.FunctionSpec functionSpec =
         RunnerApi.FunctionSpec.newBuilder()
             .setUrn(WindowMappingFnRunner.URN)
diff --git a/sdks/python/apache_beam/portability/common_urns.py b/sdks/python/apache_beam/portability/common_urns.py
index c2a79cce12a..30d499e2bde 100644
--- a/sdks/python/apache_beam/portability/common_urns.py
+++ b/sdks/python/apache_beam/portability/common_urns.py
@@ -51,6 +51,9 @@ def __init__(self, enum_type):
 
 coders = PropertiesFromEnumType(beam_runner_api_pb2.StandardCoders.Enum)
 
+environments = PropertiesFromEnumType(
+    beam_runner_api_pb2.StandardEnvironments.Environments)
+
 
 def PropertiesFromPayloadType(payload_type):
   return PropertiesFromEnumType(payload_type.Enum).PROPERTIES
diff --git a/sdks/python/apache_beam/runners/pipeline_context.py b/sdks/python/apache_beam/runners/pipeline_context.py
index 511dbd915f5..78048bd4d90 100644
--- a/sdks/python/apache_beam/runners/pipeline_context.py
+++ b/sdks/python/apache_beam/runners/pipeline_context.py
@@ -27,6 +27,7 @@
 from apache_beam import coders
 from apache_beam import pipeline
 from apache_beam import pvalue
+from apache_beam.portability import common_urns
 from apache_beam.portability.api import beam_fn_api_pb2
 from apache_beam.portability.api import beam_runner_api_pb2
 from apache_beam.transforms import core
@@ -123,7 +124,11 @@ def __init__(self, proto=None, default_environment_url=None):
       self._default_environment_id = self.environments.get_id(
           Environment(
               beam_runner_api_pb2.Environment(
-                  url=default_environment_url)))
+                  url=default_environment_url,
+                  urn=common_urns.environments.DOCKER.urn,
+                  payload=beam_runner_api_pb2.DockerPayload(
+                      container_image=default_environment_url
+                  ).SerializeToString())))
     else:
       self._default_environment_id = None
 


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Issue Time Tracking
-------------------

    Worklog Id:     (was: 145910)
    Time Spent: 4h 40m  (was: 4.5h)

> Modify Environment to support non-dockerized SDK harness deployments 
> ---------------------------------------------------------------------
>
>                 Key: BEAM-5288
>                 URL: https://issues.apache.org/jira/browse/BEAM-5288
>             Project: Beam
>          Issue Type: New Feature
>          Components: beam-model
>            Reporter: Maximilian Michels
>            Assignee: Ankur Goenka
>            Priority: Major
>          Time Spent: 4h 40m
>  Remaining Estimate: 0h
>
> As of mailing discussions and BEAM-5187, it has become clear that we need to extend the Environment information. In addition to the Docker environment, the extended environment holds deployment options for 1) a process-based environment, 2) an externally managed environment.
> The proto definition, as of now, looks as follows:
> {noformat}
>  message Environment {
>    // (Required) The URN of the payload
>    string urn = 1;
>    // (Optional) The data specifying any parameters to the URN. If
>    // the URN does not require any arguments, this may be omitted.
>    bytes payload = 2;
>  }
>  message StandardEnvironments {
>    enum Environments {
>      DOCKER = 0 [(beam_urn) = "beam:env:docker:v1"];
>      PROCESS = 1 [(beam_urn) = "beam:env:process:v1"];
>      EXTERNAL = 2 [(beam_urn) = "beam:env:external:v1"];
>    }
>  }
>  // The payload of a Docker image
>  message DockerPayload {
>    string container_image = 1;  // implicitly linux_amd64.
>  }
>  message ProcessPayload {
>    string os = 1;  // "linux", "darwin", ..
>    string arch = 2;  // "amd64", ..
>    string command = 3; // process to execute
>    map<string, string> env = 4; // environment variables
>  }
> {noformat}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Mime
View raw message