Merge pull request #10108 from ibzib/artifact-endpoint

[BEAM-8660] Override returned artifact staging endpoint
diff --git a/.test-infra/jenkins/job_Dependency_Check.groovy b/.test-infra/jenkins/job_Dependency_Check.groovy
index ac66881..dddd2f7 100644
--- a/.test-infra/jenkins/job_Dependency_Check.groovy
+++ b/.test-infra/jenkins/job_Dependency_Check.groovy
@@ -38,7 +38,7 @@
   steps {
     gradle {
       rootBuildScriptDir(commonJobProperties.checkoutDir)
-      tasks(':runBeamDependencyCheck')
+      tasks('runBeamDependencyCheck')
       commonJobProperties.setGradleSwitches(delegate)
       switches('-Drevision=release')
     }
diff --git a/model/fn-execution/src/main/proto/beam_fn_api.proto b/model/fn-execution/src/main/proto/beam_fn_api.proto
index ed2f013..07ee590 100644
--- a/model/fn-execution/src/main/proto/beam_fn_api.proto
+++ b/model/fn-execution/src/main/proto/beam_fn_api.proto
@@ -42,6 +42,7 @@
 import "endpoints.proto";
 import "google/protobuf/descriptor.proto";
 import "google/protobuf/timestamp.proto";
+import "google/protobuf/duration.proto";
 import "google/protobuf/wrappers.proto";
 import "metrics.proto";
 
@@ -203,13 +204,21 @@
 }
 
 // An Application should be scheduled for execution after a delay.
+// Either an absolute timestamp or a relative timestamp can represent a
+// scheduled execution time.
 message DelayedBundleApplication {
   // Recommended time at which the application should be scheduled to execute
   // by the runner. Times in the past may be scheduled to execute immediately.
+  // TODO(BEAM-8536): Migrate usage of absolute time to requested_time_delay.
   google.protobuf.Timestamp requested_execution_time = 1;
 
   // (Required) The application that should be scheduled.
   BundleApplication application = 2;
+
+  // Recommended time delay at which the application should be scheduled to
+  // execute by the runner. Time delay that equals 0 may be scheduled to execute
+  // immediately. The unit of time delay should be microsecond.
+  google.protobuf.Duration requested_time_delay = 3;
 }
 
 // A request to process a given bundle.
@@ -628,6 +637,18 @@
     bytes key = 1;
   }
 
+  // Represents a request for the values associated with a specified user key
+  // and window in a PCollection. See
+  // https://s.apache.org/beam-fn-state-api-and-bundle-processing for further
+  // details.
+  //
+  // Can only be used to perform StateGetRequests on side inputs of the URN
+  // beam:side_input:multimap:v1.
+  //
+  // For a PCollection<KV<K, V>>, the response data stream will be a
+  // concatenation of all V's associated with the specified key K. See
+  // https://s.apache.org/beam-fn-api-send-and-receive-data for further
+  // details.
   message MultimapSideInput {
     // (Required) The id of the PTransform containing a side input.
     string transform_id = 1;
@@ -652,11 +673,33 @@
     bytes key = 4;
   }
 
+  // Represents a request for the values associated with a specified window
+  // in a PCollection. See
+  // https://s.apache.org/beam-fn-state-api-and-bundle-processing for further
+  // details.
+  //
+  // Can only be used to perform StateGetRequests on side inputs of the URN
+  // beam:side_input:iterable:v1 and beam:side_input:multimap:v1.
+  //
+  // For a PCollection<V>, the response data stream will be a concatenation
+  // of all V's. See https://s.apache.org/beam-fn-api-send-and-receive-data
+  // for further details.
+  message IterableSideInput {
+    // (Required) The id of the PTransform containing a side input.
+    string transform_id = 1;
+    // (Required) The id of the side input.
+    string side_input_id = 2;
+    // (Required) The window (after mapping the currently executing elements
+    // window into the side input windows domain) encoded in a nested context.
+    bytes window = 3;
+  }
+
   // (Required) One of the following state keys must be set.
   oneof type {
     Runner runner = 1;
     MultimapSideInput multimap_side_input = 2;
     BagUserState bag_user_state = 3;
+    IterableSideInput iterable_side_input = 4;
     // TODO: represent a state key for user map state
   }
 }
diff --git a/model/pipeline/src/main/proto/beam_runner_api.proto b/model/pipeline/src/main/proto/beam_runner_api.proto
index 90f52fc..a1d0164 100644
--- a/model/pipeline/src/main/proto/beam_runner_api.proto
+++ b/model/pipeline/src/main/proto/beam_runner_api.proto
@@ -318,7 +318,16 @@
 
 message StandardSideInputTypes {
   enum Enum {
+    // Represents a view over a PCollection<V>.
+    //
+    // StateGetRequests performed on this side input must use
+    // StateKey.IterableSideInput.
     ITERABLE = 0 [(beam_urn) = "beam:side_input:iterable:v1"];
+
+    // Represents a view over a PCollection<KV<K, V>>.
+    //
+    // StateGetRequests performed on this side input must use
+    // StateKey.IterableSideInput or StateKey.MultimapSideInput.
     MULTIMAP = 1 [(beam_urn) = "beam:side_input:multimap:v1"];
   }
 }
diff --git a/ownership/JAVA_DEPENDENCY_OWNERS.yaml b/ownership/JAVA_DEPENDENCY_OWNERS.yaml
index 6602e4d..c0a62d8 100644
--- a/ownership/JAVA_DEPENDENCY_OWNERS.yaml
+++ b/ownership/JAVA_DEPENDENCY_OWNERS.yaml
@@ -47,12 +47,12 @@
   com.amazonaws:amazon-kinesis-client:
     group: com.amazonaws
     artifact: amazon-kinesis-client
-    owners:
+    owners: aromanenko-dev
 
   com.amazonaws:amazon-kinesis-producer:
     group: com.amazonaws
     artifact: amazon-kinesis-producer
-    owners:
+    owners: aromanenko-dev
 
   com.amazonaws:aws-java-sdk-cloudwatch:
     group: com.amazonaws
diff --git a/runners/flink/job-server/flink_job_server.gradle b/runners/flink/job-server/flink_job_server.gradle
index bf87aca..4e5b13e 100644
--- a/runners/flink/job-server/flink_job_server.gradle
+++ b/runners/flink/job-server/flink_job_server.gradle
@@ -153,6 +153,7 @@
       excludeCategories 'org.apache.beam.sdk.testing.UsesStrictTimerOrdering'
       if (streaming) {
         excludeCategories 'org.apache.beam.sdk.testing.UsesTestStreamWithProcessingTime'
+        excludeCategories 'org.apache.beam.sdk.testing.UsesTestStreamWithMultipleStages'
       } else {
         excludeCategories 'org.apache.beam.sdk.testing.UsesTestStream'
       }
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/ExecutableStageDoFnOperator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/ExecutableStageDoFnOperator.java
index 915acfb..11b2af2 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/ExecutableStageDoFnOperator.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/ExecutableStageDoFnOperator.java
@@ -94,6 +94,8 @@
 import org.apache.flink.api.common.state.ListStateDescriptor;
 import org.apache.flink.api.common.typeutils.base.StringSerializer;
 import org.apache.flink.api.java.functions.KeySelector;
+import org.apache.flink.runtime.state.AbstractKeyedStateBackend;
+import org.apache.flink.runtime.state.KeyGroupRange;
 import org.apache.flink.runtime.state.KeyedStateBackend;
 import org.apache.flink.streaming.api.operators.InternalTimer;
 import org.apache.flink.streaming.api.watermark.Watermark;
@@ -246,7 +248,8 @@
                   () -> UUID.randomUUID().toString(),
                   keyedStateInternals,
                   getKeyedStateBackend(),
-                  stateBackendLock));
+                  stateBackendLock,
+                  keyCoder));
     } else {
       userStateRequestHandler = StateRequestHandler.unsupported();
     }
@@ -258,12 +261,17 @@
     return StateRequestHandlers.delegateBasedUponType(handlerMap);
   }
 
-  static class BagUserStateFactory<K extends ByteString, V, W extends BoundedWindow>
-      implements StateRequestHandlers.BagUserStateHandlerFactory<K, V, W> {
+  static class BagUserStateFactory<V, W extends BoundedWindow>
+      implements StateRequestHandlers.BagUserStateHandlerFactory<ByteString, V, W> {
 
     private final StateInternals stateInternals;
     private final KeyedStateBackend<ByteBuffer> keyedStateBackend;
+    /** Lock to hold whenever accessing the state backend. */
     private final Lock stateBackendLock;
+    /** For debugging: The key coder used by the Runner. */
+    @Nullable private final Coder runnerKeyCoder;
+    /** For debugging: Same as keyedStateBackend but upcasted, to access key group meta info. */
+    @Nullable private final AbstractKeyedStateBackend<ByteBuffer> keyStateBackendWithKeyGroupInfo;
     /** Holds the valid cache token for user state for this operator. */
     private final ByteString cacheToken;
 
@@ -271,26 +279,36 @@
         IdGenerator cacheTokenGenerator,
         StateInternals stateInternals,
         KeyedStateBackend<ByteBuffer> keyedStateBackend,
-        Lock stateBackendLock) {
+        Lock stateBackendLock,
+        @Nullable Coder runnerKeyCoder) {
       this.stateInternals = stateInternals;
       this.keyedStateBackend = keyedStateBackend;
       this.stateBackendLock = stateBackendLock;
+      if (keyedStateBackend instanceof AbstractKeyedStateBackend) {
+        // This will always succeed, unless a custom state backend is used which does not extend
+        // AbstractKeyedStateBackend. This is unlikely but we should still consider this case.
+        this.keyStateBackendWithKeyGroupInfo =
+            (AbstractKeyedStateBackend<ByteBuffer>) keyedStateBackend;
+      } else {
+        this.keyStateBackendWithKeyGroupInfo = null;
+      }
+      this.runnerKeyCoder = runnerKeyCoder;
       this.cacheToken = ByteString.copyFrom(cacheTokenGenerator.getId().getBytes(Charsets.UTF_8));
     }
 
     @Override
-    public StateRequestHandlers.BagUserStateHandler<K, V, W> forUserState(
+    public StateRequestHandlers.BagUserStateHandler<ByteString, V, W> forUserState(
         // Transform id not used because multiple operators with state will not
         // be fused together. See GreedyPCollectionFusers
         String pTransformId,
         String userStateId,
-        Coder<K> keyCoder,
+        Coder<ByteString> keyCoder,
         Coder<V> valueCoder,
         Coder<W> windowCoder) {
-      return new StateRequestHandlers.BagUserStateHandler<K, V, W>() {
+      return new StateRequestHandlers.BagUserStateHandler<ByteString, V, W>() {
 
         @Override
-        public Iterable<V> get(K key, W window) {
+        public Iterable<V> get(ByteString key, W window) {
           try {
             stateBackendLock.lock();
             prepareStateBackend(key);
@@ -313,7 +331,7 @@
         }
 
         @Override
-        public void append(K key, W window, Iterator<V> values) {
+        public void append(ByteString key, W window, Iterator<V> values) {
           try {
             stateBackendLock.lock();
             prepareStateBackend(key);
@@ -337,7 +355,7 @@
         }
 
         @Override
-        public void clear(K key, W window) {
+        public void clear(ByteString key, W window) {
           try {
             stateBackendLock.lock();
             prepareStateBackend(key);
@@ -364,10 +382,23 @@
           return Optional.of(cacheToken);
         }
 
-        private void prepareStateBackend(K key) {
+        private void prepareStateBackend(ByteString key) {
           // Key for state request is shipped encoded with NESTED context.
           ByteBuffer encodedKey = FlinkKeyUtils.fromEncodedKey(key);
           keyedStateBackend.setCurrentKey(encodedKey);
+          if (keyStateBackendWithKeyGroupInfo != null) {
+            int currentKeyGroupIndex = keyStateBackendWithKeyGroupInfo.getCurrentKeyGroupIndex();
+            KeyGroupRange keyGroupRange = keyStateBackendWithKeyGroupInfo.getKeyGroupRange();
+            Preconditions.checkState(
+                keyGroupRange.contains(currentKeyGroupIndex),
+                "The current key '%s' with key group index '%s' does not belong to the key group range '%s'. Runner keyCoder: %s. Ptransformid: %s Userstateid: %s",
+                Arrays.toString(key.toByteArray()),
+                currentKeyGroupIndex,
+                keyGroupRange,
+                runnerKeyCoder,
+                pTransformId,
+                userStateId);
+          }
         }
       };
     }
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/ExecutableStageDoFnOperatorTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/ExecutableStageDoFnOperatorTest.java
index 0d7c99f..9f7eff4 100644
--- a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/ExecutableStageDoFnOperatorTest.java
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/ExecutableStageDoFnOperatorTest.java
@@ -656,10 +656,9 @@
     // User state the cache token is valid for the lifetime of the operator
     for (String expectedToken : new String[] {"first token", "second token"}) {
       final IdGenerator cacheTokenGenerator = () -> expectedToken;
-      ExecutableStageDoFnOperator.BagUserStateFactory<ByteString, Integer, GlobalWindow>
-          bagUserStateFactory =
-              new ExecutableStageDoFnOperator.BagUserStateFactory<>(
-                  cacheTokenGenerator, test, stateBackend, NoopLock.get());
+      ExecutableStageDoFnOperator.BagUserStateFactory<Integer, GlobalWindow> bagUserStateFactory =
+          new ExecutableStageDoFnOperator.BagUserStateFactory<>(
+              cacheTokenGenerator, test, stateBackend, NoopLock.get(), null);
 
       ByteString key1 = ByteString.copyFrom("key1", Charsets.UTF_8);
       ByteString key2 = ByteString.copyFrom("key2", Charsets.UTF_8);
diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/BundleFinalizationHandler.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/BundleFinalizationHandler.java
index 849663b..646cdf9 100644
--- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/BundleFinalizationHandler.java
+++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/BundleFinalizationHandler.java
@@ -17,8 +17,6 @@
  */
 package org.apache.beam.runners.fnexecution.control;
 
-import org.apache.beam.model.fnexecution.v1.BeamFnApi.ProcessBundleResponse;
-
 /**
  * A handler for the runner when a finalization request has been received.
  *
@@ -29,5 +27,12 @@
  * details.
  */
 public interface BundleFinalizationHandler {
-  void requestsFinalization(ProcessBundleResponse response);
+  /**
+   * This callback is invoked whenever an inflight bundle that is being processed requests
+   * finalization.
+   *
+   * <p>The runner is responsible for invoking bundle finalization when the output of the bundle has
+   * been durably persisted.
+   */
+  void requestsFinalization(String bundleId);
 }
diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/BundleFinalizationHandlers.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/BundleFinalizationHandlers.java
new file mode 100644
index 0000000..8a7a4e8
--- /dev/null
+++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/BundleFinalizationHandlers.java
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.fnexecution.control;
+
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.beam.model.fnexecution.v1.BeamFnApi.FinalizeBundleRequest;
+import org.apache.beam.model.fnexecution.v1.BeamFnApi.InstructionRequest;
+
+/** Utility methods for creating {@link BundleFinalizationHandler}s. */
+public class BundleFinalizationHandlers {
+
+  /**
+   * A bundle finalizer that stores all bundle finalization requests in memory. After the runner
+   * durably persists the output, the runner is responsible for invoking {@link
+   * InMemoryFinalizer#finalizeAllOutstandingBundles()}.
+   */
+  public static InMemoryFinalizer inMemoryFinalizer(InstructionRequestHandler fnApiControlClient) {
+    return new InMemoryFinalizer(fnApiControlClient);
+  }
+
+  /** See {@link #inMemoryFinalizer(InstructionRequestHandler)} for details. */
+  public static class InMemoryFinalizer implements BundleFinalizationHandler {
+    private final InstructionRequestHandler fnApiControlClient;
+    private final List<String> bundleIds;
+
+    private InMemoryFinalizer(InstructionRequestHandler fnApiControlClient) {
+      this.fnApiControlClient = fnApiControlClient;
+      this.bundleIds = new ArrayList<>();
+    }
+
+    /** All finalization requests will be sent without waiting for the responses. */
+    public synchronized void finalizeAllOutstandingBundles() {
+      for (String bundleId : bundleIds) {
+        InstructionRequest request =
+            InstructionRequest.newBuilder()
+                .setFinalizeBundle(
+                    FinalizeBundleRequest.newBuilder().setInstructionId(bundleId).build())
+                .build();
+        fnApiControlClient.handle(request);
+      }
+      bundleIds.clear();
+    }
+
+    @Override
+    public synchronized void requestsFinalization(String bundleId) {
+      bundleIds.add(bundleId);
+    }
+  }
+}
diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/ProcessBundleDescriptors.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/ProcessBundleDescriptors.java
index b324cb1..f2d374a 100644
--- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/ProcessBundleDescriptors.java
+++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/ProcessBundleDescriptors.java
@@ -57,6 +57,7 @@
 import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder;
 import org.apache.beam.sdk.values.KV;
 import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableTable;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables;
@@ -135,6 +136,10 @@
         forTimerSpecs(
             dataEndpoint, stage, components, inputDestinationsBuilder, remoteOutputCodersBuilder);
 
+    if (bagUserStateSpecs.size() > 0 || timerSpecs.size() > 0) {
+      lengthPrefixKeyCoder(stage.getInputPCollection().getId(), components);
+    }
+
     // Copy data from components to ProcessBundleDescriptor.
     ProcessBundleDescriptor.Builder bundleDescriptorBuilder =
         ProcessBundleDescriptor.newBuilder().setId(id);
@@ -158,6 +163,29 @@
         timerSpecs);
   }
 
+  /**
+   * Patches the input coder of a stateful transform to ensure that the byte representation of a key
+   * used to partition the input element at the Runner, matches the key byte representation received
+   * for state requests and timers from the SDK Harness. Stateful transforms always have a KvCoder
+   * as input.
+   */
+  private static void lengthPrefixKeyCoder(
+      String inputColId, Components.Builder componentsBuilder) {
+    RunnerApi.PCollection pcollection = componentsBuilder.getPcollectionsOrThrow(inputColId);
+    RunnerApi.Coder kvCoder = componentsBuilder.getCodersOrThrow(pcollection.getCoderId());
+    Preconditions.checkState(
+        ModelCoders.KV_CODER_URN.equals(kvCoder.getSpec().getUrn()),
+        "Stateful executable stages must use a KV coder, but is: %s",
+        kvCoder.getSpec().getUrn());
+    String keyCoderId = ModelCoders.getKvCoderComponents(kvCoder).keyCoderId();
+    // Retain the original coder, but wrap in LengthPrefixCoder
+    String newKeyCoderId =
+        LengthPrefixUnknownCoders.addLengthPrefixedCoder(keyCoderId, componentsBuilder, false);
+    // Replace old key coder with LengthPrefixCoder<old_key_coder>
+    kvCoder = kvCoder.toBuilder().setComponentCoderIds(0, newKeyCoderId).build();
+    componentsBuilder.putCoders(pcollection.getCoderId(), kvCoder);
+  }
+
   private static Map<String, Coder<WindowedValue<?>>> addStageOutputs(
       ApiServiceDescriptor dataEndpoint,
       Collection<PCollectionNode> outputPCollections,
diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/SdkHarnessClient.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/SdkHarnessClient.java
index 86232dd..2799e58 100644
--- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/SdkHarnessClient.java
+++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/SdkHarnessClient.java
@@ -146,7 +146,7 @@
                     "The %s does not have a registered bundle checkpoint handler.",
                     ActiveBundle.class.getSimpleName()));
           },
-          request -> {
+          bundleId -> {
             throw new UnsupportedOperationException(
                 String.format(
                     "The %s does not have a registered bundle finalization handler.",
@@ -313,7 +313,7 @@
             checkpointHandler.onCheckpoint(completedResponse);
           }
           if (completedResponse.getRequiresFinalization()) {
-            finalizationHandler.requestsFinalization(completedResponse);
+            finalizationHandler.requestsFinalization(bundleId);
           }
         } else {
           // TODO: [BEAM-3962] Handle aborting the bundle being processed.
diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/BundleFinalizationHandlersTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/BundleFinalizationHandlersTest.java
new file mode 100644
index 0000000..9494104
--- /dev/null
+++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/BundleFinalizationHandlersTest.java
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.fnexecution.control;
+
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.verifyZeroInteractions;
+
+import org.apache.beam.model.fnexecution.v1.BeamFnApi.FinalizeBundleRequest;
+import org.apache.beam.model.fnexecution.v1.BeamFnApi.InstructionRequest;
+import org.apache.beam.runners.fnexecution.control.BundleFinalizationHandlers.InMemoryFinalizer;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/** Tests for {@link BundleFinalizationHandlers}. */
+@RunWith(JUnit4.class)
+public class BundleFinalizationHandlersTest {
+  @Test
+  public void testInMemoryFinalizer() {
+    InstructionRequestHandler mockHandler = mock(InstructionRequestHandler.class);
+    InMemoryFinalizer finalizer = BundleFinalizationHandlers.inMemoryFinalizer(mockHandler);
+
+    finalizer.finalizeAllOutstandingBundles();
+    verifyZeroInteractions(mockHandler);
+
+    finalizer.requestsFinalization("A");
+    finalizer.requestsFinalization("B");
+    verifyZeroInteractions(mockHandler);
+
+    finalizer.finalizeAllOutstandingBundles();
+    verify(mockHandler).handle(requestFor("A"));
+    verify(mockHandler).handle(requestFor("B"));
+  }
+
+  private static InstructionRequest requestFor(String bundleId) {
+    return InstructionRequest.newBuilder()
+        .setFinalizeBundle(FinalizeBundleRequest.newBuilder().setInstructionId(bundleId).build())
+        .build();
+  }
+}
diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/ProcessBundleDescriptorsTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/ProcessBundleDescriptorsTest.java
new file mode 100644
index 0000000..ccabb2e
--- /dev/null
+++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/ProcessBundleDescriptorsTest.java
@@ -0,0 +1,144 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.fnexecution.control;
+
+import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkState;
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.MatcherAssert.assertThat;
+
+import java.io.Serializable;
+import java.util.Map;
+import org.apache.beam.model.fnexecution.v1.BeamFnApi;
+import org.apache.beam.model.pipeline.v1.Endpoints;
+import org.apache.beam.model.pipeline.v1.RunnerApi;
+import org.apache.beam.runners.core.construction.CoderTranslation;
+import org.apache.beam.runners.core.construction.ModelCoderRegistrar;
+import org.apache.beam.runners.core.construction.ModelCoders;
+import org.apache.beam.runners.core.construction.PipelineTranslation;
+import org.apache.beam.runners.core.construction.graph.ExecutableStage;
+import org.apache.beam.runners.core.construction.graph.FusedPipeline;
+import org.apache.beam.runners.core.construction.graph.GreedyPipelineFuser;
+import org.apache.beam.runners.core.construction.graph.PipelineNode;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.KvCoder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.coders.VoidCoder;
+import org.apache.beam.sdk.state.BagState;
+import org.apache.beam.sdk.state.StateSpec;
+import org.apache.beam.sdk.state.StateSpecs;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.GroupByKey;
+import org.apache.beam.sdk.transforms.Impulse;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Optional;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables;
+import org.junit.Test;
+
+/** Tests for {@link ProcessBundleDescriptors}. */
+public class ProcessBundleDescriptorsTest implements Serializable {
+
+  /**
+   * Tests that a stateful stage will wrap the key coder of a stateful transform in a
+   * LengthPrefixCoder.
+   */
+  @Test
+  public void testWrapKeyCoderOfStatefulExecutableStageInLengthPrefixCoder() throws Exception {
+    // Add another stateful stage with a non-standard key coder
+    Pipeline p = Pipeline.create();
+    Coder<Void> keycoder = VoidCoder.of();
+    assertThat(ModelCoderRegistrar.isKnownCoder(keycoder), is(false));
+    p.apply("impulse", Impulse.create())
+        .apply(
+            "create",
+            ParDo.of(
+                new DoFn<byte[], KV<Void, String>>() {
+                  @ProcessElement
+                  public void process(ProcessContext ctxt) {}
+                }))
+        .setCoder(KvCoder.of(keycoder, StringUtf8Coder.of()))
+        .apply(
+            "userState",
+            ParDo.of(
+                new DoFn<KV<Void, String>, KV<Void, String>>() {
+
+                  @StateId("stateId")
+                  private final StateSpec<BagState<String>> bufferState =
+                      StateSpecs.bag(StringUtf8Coder.of());
+
+                  @ProcessElement
+                  public void processElement(
+                      @Element KV<Void, String> element,
+                      @StateId("stateId") BagState<String> state,
+                      OutputReceiver<KV<Void, String>> r) {
+                    for (String value : state.read()) {
+                      r.output(KV.of(element.getKey(), value));
+                    }
+                    state.add(element.getValue());
+                  }
+                }))
+        // Force the output to be materialized
+        .apply("gbk", GroupByKey.create());
+
+    RunnerApi.Pipeline pipelineProto = PipelineTranslation.toProto(p);
+    FusedPipeline fused = GreedyPipelineFuser.fuse(pipelineProto);
+    Optional<ExecutableStage> optionalStage =
+        Iterables.tryFind(
+            fused.getFusedStages(),
+            (ExecutableStage stage) ->
+                stage.getUserStates().stream()
+                    .anyMatch(spec -> spec.localName().equals("stateId")));
+    checkState(optionalStage.isPresent(), "Expected a stage with user state.");
+
+    ExecutableStage stage = optionalStage.get();
+    PipelineNode.PCollectionNode inputPCollection = stage.getInputPCollection();
+
+    // Ensure original key coder is not a LengthPrefixCoder
+    Map<String, RunnerApi.Coder> stageCoderMap = stage.getComponents().getCodersMap();
+    RunnerApi.Coder originalCoder =
+        stageCoderMap.get(inputPCollection.getPCollection().getCoderId());
+    String originalKeyCoderId = ModelCoders.getKvCoderComponents(originalCoder).keyCoderId();
+    assertThat(
+        stageCoderMap.get(originalKeyCoderId).getSpec().getUrn(),
+        is(CoderTranslation.JAVA_SERIALIZED_CODER_URN));
+
+    // Now create ProcessBundleDescriptor and check for the LengthPrefixCoder around the key coder
+    BeamFnApi.ProcessBundleDescriptor pbDescriptor =
+        ProcessBundleDescriptors.fromExecutableStage(
+                "test_stage", stage, Endpoints.ApiServiceDescriptor.getDefaultInstance())
+            .getProcessBundleDescriptor();
+
+    String inputPCollectionId = inputPCollection.getId();
+    String inputCoderId = pbDescriptor.getPcollectionsMap().get(inputPCollectionId).getCoderId();
+
+    Map<String, RunnerApi.Coder> pbCoderMap = pbDescriptor.getCodersMap();
+    RunnerApi.Coder coder = pbCoderMap.get(inputCoderId);
+    String keyCoderId = ModelCoders.getKvCoderComponents(coder).keyCoderId();
+
+    RunnerApi.Coder keyCoder = pbCoderMap.get(keyCoderId);
+    // Ensure length prefix
+    assertThat(keyCoder.getSpec().getUrn(), is(ModelCoders.LENGTH_PREFIX_CODER_URN));
+    String lengthPrefixWrappedCoderId = keyCoder.getComponentCoderIds(0);
+
+    // Check that the wrapped coder is unchanged
+    assertThat(lengthPrefixWrappedCoderId, is(originalKeyCoderId));
+    assertThat(
+        pbCoderMap.get(lengthPrefixWrappedCoderId), is(stageCoderMap.get(originalKeyCoderId)));
+  }
+}
diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/RemoteExecutionTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/RemoteExecutionTest.java
index 0925767..5d4c8f0 100644
--- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/RemoteExecutionTest.java
+++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/RemoteExecutionTest.java
@@ -18,10 +18,10 @@
 package org.apache.beam.runners.fnexecution.control;
 
 import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkState;
+import static org.hamcrest.MatcherAssert.assertThat;
 import static org.hamcrest.Matchers.allOf;
 import static org.hamcrest.Matchers.containsInAnyOrder;
 import static org.hamcrest.Matchers.equalTo;
-import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
@@ -946,8 +946,6 @@
   @Test
   public void testExecutionWithTimer() throws Exception {
     Pipeline p = Pipeline.create();
-    final String timerId = "foo";
-    final String timerId2 = "foo2";
 
     p.apply("impulse", Impulse.create())
         .apply(
diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/SdkHarnessClientTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/SdkHarnessClientTest.java
index 089c8d1..ef0b2ac 100644
--- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/SdkHarnessClientTest.java
+++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/SdkHarnessClientTest.java
@@ -627,8 +627,6 @@
 
   @Test
   public void testBundleCheckpointCallback() throws Exception {
-    Exception testException = new Exception();
-
     InboundDataClient mockOutputReceiver = mock(InboundDataClient.class);
     CloseableFnDataReceiver mockInputSender = mock(CloseableFnDataReceiver.class);
 
@@ -658,8 +656,6 @@
         ProcessBundleResponse.newBuilder()
             .addResidualRoots(DelayedBundleApplication.getDefaultInstance())
             .build();
-    ArrayList<ProcessBundleResponse> checkpoints = new ArrayList<>();
-
     try (ActiveBundle activeBundle =
         processor.newBundle(
             ImmutableMap.of(SDK_GRPC_WRITE_TRANSFORM, mockRemoteOutputReceiver),
@@ -680,8 +676,6 @@
 
   @Test
   public void testBundleFinalizationCallback() throws Exception {
-    Exception testException = new Exception();
-
     InboundDataClient mockOutputReceiver = mock(InboundDataClient.class);
     CloseableFnDataReceiver mockInputSender = mock(CloseableFnDataReceiver.class);
 
@@ -709,8 +703,7 @@
 
     ProcessBundleResponse response =
         ProcessBundleResponse.newBuilder().setRequiresFinalization(true).build();
-    ArrayList<ProcessBundleResponse> checkpoints = new ArrayList<>();
-
+    String bundleId;
     try (ActiveBundle activeBundle =
         processor.newBundle(
             ImmutableMap.of(SDK_GRPC_WRITE_TRANSFORM, mockRemoteOutputReceiver),
@@ -720,12 +713,13 @@
             mockProgressHandler,
             mockCheckpointHandler,
             mockFinalizationHandler)) {
+      bundleId = activeBundle.getId();
       processBundleResponseFuture.complete(
           InstructionResponse.newBuilder().setProcessBundle(response).build());
     }
 
     verify(mockProgressHandler).onCompleted(response);
-    verify(mockFinalizationHandler).requestsFinalization(response);
+    verify(mockFinalizationHandler).requestsFinalization(bundleId);
     verifyZeroInteractions(mockCheckpointHandler);
   }
 
diff --git a/runners/spark/job-server/build.gradle b/runners/spark/job-server/build.gradle
index 2b27a88..6fb7581 100644
--- a/runners/spark/job-server/build.gradle
+++ b/runners/spark/job-server/build.gradle
@@ -114,6 +114,7 @@
       excludeCategories 'org.apache.beam.sdk.testing.UsesBoundedSplittableParDo'
       excludeCategories 'org.apache.beam.sdk.testing.UsesSplittableParDoWithWindowedSideInputs'
       excludeCategories 'org.apache.beam.sdk.testing.UsesUnboundedSplittableParDo'
+      excludeCategories 'org.apache.beam.sdk.testing.UsesStrictTimerOrdering'
     },
   )
 }
diff --git a/sdks/go/pkg/beam/core/runtime/exec/fullvalue.go b/sdks/go/pkg/beam/core/runtime/exec/fullvalue.go
index 538eb9a..3291615 100644
--- a/sdks/go/pkg/beam/core/runtime/exec/fullvalue.go
+++ b/sdks/go/pkg/beam/core/runtime/exec/fullvalue.go
@@ -95,36 +95,7 @@
 // to drop the universal type and convert Aggregate types.
 func Convert(v interface{}, to reflect.Type) interface{} {
 	from := reflect.TypeOf(v)
-
-	switch {
-	case from == to:
-		return v
-
-	case typex.IsUniversal(from):
-		// We need to drop T to obtain the underlying type of the value.
-		return reflectx.UnderlyingType(reflect.ValueOf(v)).Interface()
-		// TODO(herohde) 1/19/2018: reflect.ValueOf(v).Convert(to).Interface() instead?
-
-	case typex.IsList(from) && typex.IsList(to):
-		// Convert []A to []B.
-
-		value := reflect.ValueOf(v)
-
-		ret := reflect.New(to).Elem()
-		for i := 0; i < value.Len(); i++ {
-			ret = reflect.Append(ret, reflect.ValueOf(Convert(value.Index(i).Interface(), to.Elem())))
-		}
-		return ret.Interface()
-
-	default:
-		// Arguably this should be:
-		//   reflect.ValueOf(v).Convert(to).Interface()
-		// but this isn't desirable as it would add avoidable overhead to
-		// functions where it applies. A user will have better performance
-		// by explicitly doing the type conversion in their code, which
-		// the error will indicate. Slow Magic vs Fast & Explicit.
-		return v
-	}
+	return ConvertFn(from, to)(v)
 }
 
 // ConvertFn returns a function that converts type of the runtime value to the desired one. It is needed
@@ -151,7 +122,35 @@
 			}
 			return ret.Interface()
 		}
+
+	case typex.IsList(from) && typex.IsUniversal(from.Elem()) && typex.IsUniversal(to):
+		fromE := from.Elem()
+		return func(v interface{}) interface{} {
+			// Convert []typex.T to the underlying type []T.
+
+			value := reflect.ValueOf(v)
+			// We don't know the underlying element type of a nil/empty universal-typed slice.
+			// So the best we could do is to return it as is.
+			if value.Len() == 0 {
+				return v
+			}
+
+			toE := reflectx.UnderlyingType(value.Index(0)).Type()
+			cvtFn := ConvertFn(fromE, toE)
+			ret := reflect.New(reflect.SliceOf(toE)).Elem()
+			for i := 0; i < value.Len(); i++ {
+				ret = reflect.Append(ret, reflect.ValueOf(cvtFn(value.Index(i).Interface())))
+			}
+			return ret.Interface()
+		}
+
 	default:
+		// Arguably this should be:
+		//   reflect.ValueOf(v).Convert(to).Interface()
+		// but this isn't desirable as it would add avoidable overhead to
+		// functions where it applies. A user will have better performance
+		// by explicitly doing the type conversion in their code, which
+		// the error will indicate. Slow Magic vs Fast & Explicit.
 		return identity
 	}
 }
diff --git a/sdks/go/pkg/beam/core/runtime/exec/fullvalue_test.go b/sdks/go/pkg/beam/core/runtime/exec/fullvalue_test.go
index 9cdc67d..d760a44 100644
--- a/sdks/go/pkg/beam/core/runtime/exec/fullvalue_test.go
+++ b/sdks/go/pkg/beam/core/runtime/exec/fullvalue_test.go
@@ -194,17 +194,35 @@
 			v:    []typex.T{1, 2, 3},
 			want: []int{1, 2, 3},
 		},
+		{
+			name: "[]typexT_to_typexX",
+			to:   typex.XType,
+			v:    []typex.T{1, 2, 3},
+			want: []int{1, 2, 3},
+		},
+		{
+			name: "empty_[]typexT_to_typexX",
+			to:   typex.XType,
+			v:    []typex.T{},
+			want: []typex.T{},
+		},
+		{
+			name: "nil_[]typexT_to_typexX",
+			to:   typex.XType,
+			v:    []typex.T(nil),
+			want: []typex.T(nil),
+		},
 	}
 	for _, test := range tests {
 		test := test
 		t.Run(test.name, func(t *testing.T) {
-			if got := Convert(test.v, test.to); reflect.TypeOf(got) != test.to {
+			if got := Convert(test.v, test.to); !reflect.DeepEqual(got, test.want) {
 				t.Errorf("Convert(%v,%v) = %v,  want %v", test.v, test.to, got, test.want)
 			}
 		})
 		t.Run("Fn_"+test.name, func(t *testing.T) {
 			fn := ConvertFn(reflect.TypeOf(test.v), test.to)
-			if got := fn(test.v); reflect.TypeOf(got) != test.to {
+			if got := fn(test.v); !reflect.DeepEqual(got, test.want) {
 				t.Errorf("ConvertFn(%T, %v)(%v) = %v,  want %v", test.v, test.to, test.v, got, test.want)
 			}
 		})
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 55de402..32abbe6 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
@@ -8,6 +8,7 @@
 import math "math"
 import pipeline_v1 "github.com/apache/beam/sdks/go/pkg/beam/model/pipeline_v1"
 import _ "github.com/golang/protobuf/protoc-gen-go/descriptor"
+import duration "github.com/golang/protobuf/ptypes/duration"
 import timestamp "github.com/golang/protobuf/ptypes/timestamp"
 import _ "github.com/golang/protobuf/ptypes/wrappers"
 
@@ -74,7 +75,7 @@
 	return proto.EnumName(LogEntry_Severity_Enum_name, int32(x))
 }
 func (LogEntry_Severity_Enum) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_d24d1635dfa071c8, []int{27, 1, 0}
+	return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{27, 1, 0}
 }
 
 // A descriptor for connecting to a remote port using the Beam Fn Data API.
@@ -97,7 +98,7 @@
 func (m *RemoteGrpcPort) String() string { return proto.CompactTextString(m) }
 func (*RemoteGrpcPort) ProtoMessage()    {}
 func (*RemoteGrpcPort) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_d24d1635dfa071c8, []int{0}
+	return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{0}
 }
 func (m *RemoteGrpcPort) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_RemoteGrpcPort.Unmarshal(m, b)
@@ -157,7 +158,7 @@
 func (m *InstructionRequest) String() string { return proto.CompactTextString(m) }
 func (*InstructionRequest) ProtoMessage()    {}
 func (*InstructionRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_d24d1635dfa071c8, []int{1}
+	return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{1}
 }
 func (m *InstructionRequest) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_InstructionRequest.Unmarshal(m, b)
@@ -413,7 +414,7 @@
 func (m *InstructionResponse) String() string { return proto.CompactTextString(m) }
 func (*InstructionResponse) ProtoMessage()    {}
 func (*InstructionResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_d24d1635dfa071c8, []int{2}
+	return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{2}
 }
 func (m *InstructionResponse) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_InstructionResponse.Unmarshal(m, b)
@@ -661,7 +662,7 @@
 func (m *RegisterRequest) String() string { return proto.CompactTextString(m) }
 func (*RegisterRequest) ProtoMessage()    {}
 func (*RegisterRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_d24d1635dfa071c8, []int{3}
+	return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{3}
 }
 func (m *RegisterRequest) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_RegisterRequest.Unmarshal(m, b)
@@ -699,7 +700,7 @@
 func (m *RegisterResponse) String() string { return proto.CompactTextString(m) }
 func (*RegisterResponse) ProtoMessage()    {}
 func (*RegisterResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_d24d1635dfa071c8, []int{4}
+	return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{4}
 }
 func (m *RegisterResponse) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_RegisterResponse.Unmarshal(m, b)
@@ -747,7 +748,7 @@
 func (m *ProcessBundleDescriptor) String() string { return proto.CompactTextString(m) }
 func (*ProcessBundleDescriptor) ProtoMessage()    {}
 func (*ProcessBundleDescriptor) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_d24d1635dfa071c8, []int{5}
+	return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{5}
 }
 func (m *ProcessBundleDescriptor) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_ProcessBundleDescriptor.Unmarshal(m, b)
@@ -855,7 +856,7 @@
 func (m *BundleApplication) String() string { return proto.CompactTextString(m) }
 func (*BundleApplication) ProtoMessage()    {}
 func (*BundleApplication) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_d24d1635dfa071c8, []int{6}
+	return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{6}
 }
 func (m *BundleApplication) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_BundleApplication.Unmarshal(m, b)
@@ -918,12 +919,19 @@
 }
 
 // An Application should be scheduled for execution after a delay.
+// Either an absolute timestamp or a relative timestamp can represent a
+// scheduled execution time.
 type DelayedBundleApplication struct {
 	// Recommended time at which the application should be scheduled to execute
 	// by the runner. Times in the past may be scheduled to execute immediately.
+	// TODO(BEAM-8536): Migrate usage of absolute time to requested_time_delay.
 	RequestedExecutionTime *timestamp.Timestamp `protobuf:"bytes,1,opt,name=requested_execution_time,json=requestedExecutionTime,proto3" json:"requested_execution_time,omitempty"`
 	// (Required) The application that should be scheduled.
-	Application          *BundleApplication `protobuf:"bytes,2,opt,name=application,proto3" json:"application,omitempty"`
+	Application *BundleApplication `protobuf:"bytes,2,opt,name=application,proto3" json:"application,omitempty"`
+	// Recommended time delay at which the application should be scheduled to
+	// execute by the runner. Time delay that equals 0 may be scheduled to execute
+	// immediately. The unit of time delay should be microsecond.
+	RequestedTimeDelay   *duration.Duration `protobuf:"bytes,3,opt,name=requested_time_delay,json=requestedTimeDelay,proto3" json:"requested_time_delay,omitempty"`
 	XXX_NoUnkeyedLiteral struct{}           `json:"-"`
 	XXX_unrecognized     []byte             `json:"-"`
 	XXX_sizecache        int32              `json:"-"`
@@ -933,7 +941,7 @@
 func (m *DelayedBundleApplication) String() string { return proto.CompactTextString(m) }
 func (*DelayedBundleApplication) ProtoMessage()    {}
 func (*DelayedBundleApplication) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_d24d1635dfa071c8, []int{7}
+	return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{7}
 }
 func (m *DelayedBundleApplication) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_DelayedBundleApplication.Unmarshal(m, b)
@@ -967,6 +975,13 @@
 	return nil
 }
 
+func (m *DelayedBundleApplication) GetRequestedTimeDelay() *duration.Duration {
+	if m != nil {
+		return m.RequestedTimeDelay
+	}
+	return nil
+}
+
 // A request to process a given bundle.
 // Stable
 type ProcessBundleRequest struct {
@@ -985,7 +1000,7 @@
 func (m *ProcessBundleRequest) String() string { return proto.CompactTextString(m) }
 func (*ProcessBundleRequest) ProtoMessage()    {}
 func (*ProcessBundleRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_d24d1635dfa071c8, []int{8}
+	return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{8}
 }
 func (m *ProcessBundleRequest) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_ProcessBundleRequest.Unmarshal(m, b)
@@ -1039,7 +1054,7 @@
 func (m *ProcessBundleRequest_CacheToken) String() string { return proto.CompactTextString(m) }
 func (*ProcessBundleRequest_CacheToken) ProtoMessage()    {}
 func (*ProcessBundleRequest_CacheToken) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_d24d1635dfa071c8, []int{8, 0}
+	return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{8, 0}
 }
 func (m *ProcessBundleRequest_CacheToken) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_ProcessBundleRequest_CacheToken.Unmarshal(m, b)
@@ -1188,7 +1203,7 @@
 func (m *ProcessBundleRequest_CacheToken_UserState) String() string { return proto.CompactTextString(m) }
 func (*ProcessBundleRequest_CacheToken_UserState) ProtoMessage()    {}
 func (*ProcessBundleRequest_CacheToken_UserState) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_d24d1635dfa071c8, []int{8, 0, 0}
+	return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{8, 0, 0}
 }
 func (m *ProcessBundleRequest_CacheToken_UserState) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_ProcessBundleRequest_CacheToken_UserState.Unmarshal(m, b)
@@ -1223,7 +1238,7 @@
 func (m *ProcessBundleRequest_CacheToken_SideInput) String() string { return proto.CompactTextString(m) }
 func (*ProcessBundleRequest_CacheToken_SideInput) ProtoMessage()    {}
 func (*ProcessBundleRequest_CacheToken_SideInput) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_d24d1635dfa071c8, []int{8, 0, 1}
+	return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{8, 0, 1}
 }
 func (m *ProcessBundleRequest_CacheToken_SideInput) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_ProcessBundleRequest_CacheToken_SideInput.Unmarshal(m, b)
@@ -1277,7 +1292,7 @@
 func (m *ProcessBundleResponse) String() string { return proto.CompactTextString(m) }
 func (*ProcessBundleResponse) ProtoMessage()    {}
 func (*ProcessBundleResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_d24d1635dfa071c8, []int{9}
+	return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{9}
 }
 func (m *ProcessBundleResponse) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_ProcessBundleResponse.Unmarshal(m, b)
@@ -1341,7 +1356,7 @@
 func (m *ProcessBundleProgressRequest) String() string { return proto.CompactTextString(m) }
 func (*ProcessBundleProgressRequest) ProtoMessage()    {}
 func (*ProcessBundleProgressRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_d24d1635dfa071c8, []int{10}
+	return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{10}
 }
 func (m *ProcessBundleProgressRequest) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_ProcessBundleProgressRequest.Unmarshal(m, b)
@@ -1380,7 +1395,7 @@
 func (m *Metrics) String() string { return proto.CompactTextString(m) }
 func (*Metrics) ProtoMessage()    {}
 func (*Metrics) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_d24d1635dfa071c8, []int{11}
+	return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{11}
 }
 func (m *Metrics) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Metrics.Unmarshal(m, b)
@@ -1433,7 +1448,7 @@
 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_d24d1635dfa071c8, []int{11, 0}
+	return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{11, 0}
 }
 func (m *Metrics_PTransform) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Metrics_PTransform.Unmarshal(m, b)
@@ -1503,7 +1518,7 @@
 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_d24d1635dfa071c8, []int{11, 0, 0}
+	return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{11, 0, 0}
 }
 func (m *Metrics_PTransform_Measured) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Metrics_PTransform_Measured.Unmarshal(m, b)
@@ -1557,7 +1572,7 @@
 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_d24d1635dfa071c8, []int{11, 0, 1}
+	return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{11, 0, 1}
 }
 func (m *Metrics_PTransform_ProcessedElements) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Metrics_PTransform_ProcessedElements.Unmarshal(m, b)
@@ -1611,7 +1626,7 @@
 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_d24d1635dfa071c8, []int{11, 0, 2}
+	return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{11, 0, 2}
 }
 func (m *Metrics_PTransform_ActiveElements) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Metrics_PTransform_ActiveElements.Unmarshal(m, b)
@@ -1672,7 +1687,7 @@
 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_d24d1635dfa071c8, []int{11, 1}
+	return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{11, 1}
 }
 func (m *Metrics_User) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Metrics_User.Unmarshal(m, b)
@@ -1853,7 +1868,7 @@
 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_d24d1635dfa071c8, []int{11, 1, 0}
+	return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{11, 1, 0}
 }
 func (m *Metrics_User_MetricName) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Metrics_User_MetricName.Unmarshal(m, b)
@@ -1899,7 +1914,7 @@
 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_d24d1635dfa071c8, []int{11, 1, 1}
+	return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{11, 1, 1}
 }
 func (m *Metrics_User_CounterData) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Metrics_User_CounterData.Unmarshal(m, b)
@@ -1941,7 +1956,7 @@
 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_d24d1635dfa071c8, []int{11, 1, 2}
+	return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{11, 1, 2}
 }
 func (m *Metrics_User_DistributionData) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Metrics_User_DistributionData.Unmarshal(m, b)
@@ -2002,7 +2017,7 @@
 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_d24d1635dfa071c8, []int{11, 1, 3}
+	return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{11, 1, 3}
 }
 func (m *Metrics_User_GaugeData) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Metrics_User_GaugeData.Unmarshal(m, b)
@@ -2054,7 +2069,7 @@
 func (m *ProcessBundleProgressResponse) String() string { return proto.CompactTextString(m) }
 func (*ProcessBundleProgressResponse) ProtoMessage()    {}
 func (*ProcessBundleProgressResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_d24d1635dfa071c8, []int{12}
+	return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{12}
 }
 func (m *ProcessBundleProgressResponse) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_ProcessBundleProgressResponse.Unmarshal(m, b)
@@ -2115,7 +2130,7 @@
 func (m *ProcessBundleSplitRequest) String() string { return proto.CompactTextString(m) }
 func (*ProcessBundleSplitRequest) ProtoMessage()    {}
 func (*ProcessBundleSplitRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_d24d1635dfa071c8, []int{13}
+	return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{13}
 }
 func (m *ProcessBundleSplitRequest) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_ProcessBundleSplitRequest.Unmarshal(m, b)
@@ -2175,7 +2190,7 @@
 func (m *ProcessBundleSplitRequest_DesiredSplit) String() string { return proto.CompactTextString(m) }
 func (*ProcessBundleSplitRequest_DesiredSplit) ProtoMessage()    {}
 func (*ProcessBundleSplitRequest_DesiredSplit) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_d24d1635dfa071c8, []int{13, 0}
+	return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{13, 0}
 }
 func (m *ProcessBundleSplitRequest_DesiredSplit) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_ProcessBundleSplitRequest_DesiredSplit.Unmarshal(m, b)
@@ -2245,7 +2260,7 @@
 func (m *ProcessBundleSplitResponse) String() string { return proto.CompactTextString(m) }
 func (*ProcessBundleSplitResponse) ProtoMessage()    {}
 func (*ProcessBundleSplitResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_d24d1635dfa071c8, []int{14}
+	return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{14}
 }
 func (m *ProcessBundleSplitResponse) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_ProcessBundleSplitResponse.Unmarshal(m, b)
@@ -2316,7 +2331,7 @@
 func (m *ProcessBundleSplitResponse_ChannelSplit) String() string { return proto.CompactTextString(m) }
 func (*ProcessBundleSplitResponse_ChannelSplit) ProtoMessage()    {}
 func (*ProcessBundleSplitResponse_ChannelSplit) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_d24d1635dfa071c8, []int{14, 0}
+	return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{14, 0}
 }
 func (m *ProcessBundleSplitResponse_ChannelSplit) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_ProcessBundleSplitResponse_ChannelSplit.Unmarshal(m, b)
@@ -2370,7 +2385,7 @@
 func (m *FinalizeBundleRequest) String() string { return proto.CompactTextString(m) }
 func (*FinalizeBundleRequest) ProtoMessage()    {}
 func (*FinalizeBundleRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_d24d1635dfa071c8, []int{15}
+	return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{15}
 }
 func (m *FinalizeBundleRequest) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_FinalizeBundleRequest.Unmarshal(m, b)
@@ -2407,7 +2422,7 @@
 func (m *FinalizeBundleResponse) String() string { return proto.CompactTextString(m) }
 func (*FinalizeBundleResponse) ProtoMessage()    {}
 func (*FinalizeBundleResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_d24d1635dfa071c8, []int{16}
+	return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{16}
 }
 func (m *FinalizeBundleResponse) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_FinalizeBundleResponse.Unmarshal(m, b)
@@ -2441,7 +2456,7 @@
 func (m *Elements) String() string { return proto.CompactTextString(m) }
 func (*Elements) ProtoMessage()    {}
 func (*Elements) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_d24d1635dfa071c8, []int{17}
+	return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{17}
 }
 func (m *Elements) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Elements.Unmarshal(m, b)
@@ -2499,7 +2514,7 @@
 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_d24d1635dfa071c8, []int{17, 0}
+	return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{17, 0}
 }
 func (m *Elements_Data) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Elements_Data.Unmarshal(m, b)
@@ -2567,7 +2582,7 @@
 func (m *StateRequest) String() string { return proto.CompactTextString(m) }
 func (*StateRequest) ProtoMessage()    {}
 func (*StateRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_d24d1635dfa071c8, []int{18}
+	return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{18}
 }
 func (m *StateRequest) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_StateRequest.Unmarshal(m, b)
@@ -2772,7 +2787,7 @@
 func (m *StateResponse) String() string { return proto.CompactTextString(m) }
 func (*StateResponse) ProtoMessage()    {}
 func (*StateResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_d24d1635dfa071c8, []int{19}
+	return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{19}
 }
 func (m *StateResponse) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_StateResponse.Unmarshal(m, b)
@@ -2952,6 +2967,7 @@
 	//	*StateKey_Runner_
 	//	*StateKey_MultimapSideInput_
 	//	*StateKey_BagUserState_
+	//	*StateKey_IterableSideInput_
 	Type                 isStateKey_Type `protobuf_oneof:"type"`
 	XXX_NoUnkeyedLiteral struct{}        `json:"-"`
 	XXX_unrecognized     []byte          `json:"-"`
@@ -2962,7 +2978,7 @@
 func (m *StateKey) String() string { return proto.CompactTextString(m) }
 func (*StateKey) ProtoMessage()    {}
 func (*StateKey) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_d24d1635dfa071c8, []int{20}
+	return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{20}
 }
 func (m *StateKey) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_StateKey.Unmarshal(m, b)
@@ -2995,10 +3011,14 @@
 type StateKey_BagUserState_ struct {
 	BagUserState *StateKey_BagUserState `protobuf:"bytes,3,opt,name=bag_user_state,json=bagUserState,proto3,oneof"`
 }
+type StateKey_IterableSideInput_ struct {
+	IterableSideInput *StateKey_IterableSideInput `protobuf:"bytes,4,opt,name=iterable_side_input,json=iterableSideInput,proto3,oneof"`
+}
 
 func (*StateKey_Runner_) isStateKey_Type()            {}
 func (*StateKey_MultimapSideInput_) isStateKey_Type() {}
 func (*StateKey_BagUserState_) isStateKey_Type()      {}
+func (*StateKey_IterableSideInput_) isStateKey_Type() {}
 
 func (m *StateKey) GetType() isStateKey_Type {
 	if m != nil {
@@ -3028,12 +3048,20 @@
 	return nil
 }
 
+func (m *StateKey) GetIterableSideInput() *StateKey_IterableSideInput {
+	if x, ok := m.GetType().(*StateKey_IterableSideInput_); ok {
+		return x.IterableSideInput
+	}
+	return nil
+}
+
 // XXX_OneofFuncs is for the internal use of the proto package.
 func (*StateKey) XXX_OneofFuncs() (func(msg proto.Message, b *proto.Buffer) error, func(msg proto.Message, tag, wire int, b *proto.Buffer) (bool, error), func(msg proto.Message) (n int), []interface{}) {
 	return _StateKey_OneofMarshaler, _StateKey_OneofUnmarshaler, _StateKey_OneofSizer, []interface{}{
 		(*StateKey_Runner_)(nil),
 		(*StateKey_MultimapSideInput_)(nil),
 		(*StateKey_BagUserState_)(nil),
+		(*StateKey_IterableSideInput_)(nil),
 	}
 }
 
@@ -3056,6 +3084,11 @@
 		if err := b.EncodeMessage(x.BagUserState); err != nil {
 			return err
 		}
+	case *StateKey_IterableSideInput_:
+		b.EncodeVarint(4<<3 | proto.WireBytes)
+		if err := b.EncodeMessage(x.IterableSideInput); err != nil {
+			return err
+		}
 	case nil:
 	default:
 		return fmt.Errorf("StateKey.Type has unexpected type %T", x)
@@ -3090,6 +3123,14 @@
 		err := b.DecodeMessage(msg)
 		m.Type = &StateKey_BagUserState_{msg}
 		return true, err
+	case 4: // type.iterable_side_input
+		if wire != proto.WireBytes {
+			return true, proto.ErrInternalBadWireType
+		}
+		msg := new(StateKey_IterableSideInput)
+		err := b.DecodeMessage(msg)
+		m.Type = &StateKey_IterableSideInput_{msg}
+		return true, err
 	default:
 		return false, nil
 	}
@@ -3114,6 +3155,11 @@
 		n += 1 // tag and wire
 		n += proto.SizeVarint(uint64(s))
 		n += s
+	case *StateKey_IterableSideInput_:
+		s := proto.Size(x.IterableSideInput)
+		n += 1 // tag and wire
+		n += proto.SizeVarint(uint64(s))
+		n += s
 	case nil:
 	default:
 		panic(fmt.Sprintf("proto: unexpected type %T in oneof", x))
@@ -3139,7 +3185,7 @@
 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_d24d1635dfa071c8, []int{20, 0}
+	return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{20, 0}
 }
 func (m *StateKey_Runner) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_StateKey_Runner.Unmarshal(m, b)
@@ -3166,6 +3212,13 @@
 	return nil
 }
 
+// Represents a request for the values associated with a specified user key and window
+// in a PCollection.
+//
+// Can only perform StateGetRequests on side inputs with the URN beam:side_input:multimap:v1.
+//
+// For a PCollection<KV<K, V>>, the response data stream will be a concatenation of all V's
+// associated with the specified key K.
 type StateKey_MultimapSideInput struct {
 	// (Required) The id of the PTransform containing a side input.
 	TransformId string `protobuf:"bytes,1,opt,name=transform_id,json=transformId,proto3" json:"transform_id,omitempty"`
@@ -3185,7 +3238,7 @@
 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_d24d1635dfa071c8, []int{20, 1}
+	return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{20, 1}
 }
 func (m *StateKey_MultimapSideInput) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_StateKey_MultimapSideInput.Unmarshal(m, b)
@@ -3252,7 +3305,7 @@
 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_d24d1635dfa071c8, []int{20, 2}
+	return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{20, 2}
 }
 func (m *StateKey_BagUserState) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_StateKey_BagUserState.Unmarshal(m, b)
@@ -3300,6 +3353,70 @@
 	return nil
 }
 
+// Represents a request for the values associated with a specified window in a PCollection.
+//
+// Can only perform StateGetRequests on side inputs with the URN beam:side_input:iterable:v1 and
+// beam:side_input:multimap:v1.
+//
+// For a PCollection<V>, the response data stream will be a concatenation of all V's.
+type StateKey_IterableSideInput struct {
+	// (Required) The id of the PTransform containing a side input.
+	TransformId string `protobuf:"bytes,1,opt,name=transform_id,json=transformId,proto3" json:"transform_id,omitempty"`
+	// (Required) The id of the side input.
+	SideInputId string `protobuf:"bytes,2,opt,name=side_input_id,json=sideInputId,proto3" json:"side_input_id,omitempty"`
+	// (Required) The window (after mapping the currently executing elements
+	// window into the side input windows domain) encoded in a nested context.
+	Window               []byte   `protobuf:"bytes,3,opt,name=window,proto3" json:"window,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *StateKey_IterableSideInput) Reset()         { *m = StateKey_IterableSideInput{} }
+func (m *StateKey_IterableSideInput) String() string { return proto.CompactTextString(m) }
+func (*StateKey_IterableSideInput) ProtoMessage()    {}
+func (*StateKey_IterableSideInput) Descriptor() ([]byte, []int) {
+	return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{20, 3}
+}
+func (m *StateKey_IterableSideInput) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_StateKey_IterableSideInput.Unmarshal(m, b)
+}
+func (m *StateKey_IterableSideInput) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_StateKey_IterableSideInput.Marshal(b, m, deterministic)
+}
+func (dst *StateKey_IterableSideInput) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_StateKey_IterableSideInput.Merge(dst, src)
+}
+func (m *StateKey_IterableSideInput) XXX_Size() int {
+	return xxx_messageInfo_StateKey_IterableSideInput.Size(m)
+}
+func (m *StateKey_IterableSideInput) XXX_DiscardUnknown() {
+	xxx_messageInfo_StateKey_IterableSideInput.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_StateKey_IterableSideInput proto.InternalMessageInfo
+
+func (m *StateKey_IterableSideInput) GetTransformId() string {
+	if m != nil {
+		return m.TransformId
+	}
+	return ""
+}
+
+func (m *StateKey_IterableSideInput) GetSideInputId() string {
+	if m != nil {
+		return m.SideInputId
+	}
+	return ""
+}
+
+func (m *StateKey_IterableSideInput) GetWindow() []byte {
+	if m != nil {
+		return m.Window
+	}
+	return nil
+}
+
 // A request to get state.
 type StateGetRequest struct {
 	// (Optional) If specified, signals to the runner that the response
@@ -3317,7 +3434,7 @@
 func (m *StateGetRequest) String() string { return proto.CompactTextString(m) }
 func (*StateGetRequest) ProtoMessage()    {}
 func (*StateGetRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_d24d1635dfa071c8, []int{21}
+	return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{21}
 }
 func (m *StateGetRequest) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_StateGetRequest.Unmarshal(m, b)
@@ -3364,7 +3481,7 @@
 func (m *StateGetResponse) String() string { return proto.CompactTextString(m) }
 func (*StateGetResponse) ProtoMessage()    {}
 func (*StateGetResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_d24d1635dfa071c8, []int{22}
+	return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{22}
 }
 func (m *StateGetResponse) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_StateGetResponse.Unmarshal(m, b)
@@ -3413,7 +3530,7 @@
 func (m *StateAppendRequest) String() string { return proto.CompactTextString(m) }
 func (*StateAppendRequest) ProtoMessage()    {}
 func (*StateAppendRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_d24d1635dfa071c8, []int{23}
+	return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{23}
 }
 func (m *StateAppendRequest) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_StateAppendRequest.Unmarshal(m, b)
@@ -3451,7 +3568,7 @@
 func (m *StateAppendResponse) String() string { return proto.CompactTextString(m) }
 func (*StateAppendResponse) ProtoMessage()    {}
 func (*StateAppendResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_d24d1635dfa071c8, []int{24}
+	return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{24}
 }
 func (m *StateAppendResponse) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_StateAppendResponse.Unmarshal(m, b)
@@ -3482,7 +3599,7 @@
 func (m *StateClearRequest) String() string { return proto.CompactTextString(m) }
 func (*StateClearRequest) ProtoMessage()    {}
 func (*StateClearRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_d24d1635dfa071c8, []int{25}
+	return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{25}
 }
 func (m *StateClearRequest) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_StateClearRequest.Unmarshal(m, b)
@@ -3513,7 +3630,7 @@
 func (m *StateClearResponse) String() string { return proto.CompactTextString(m) }
 func (*StateClearResponse) ProtoMessage()    {}
 func (*StateClearResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_d24d1635dfa071c8, []int{26}
+	return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{26}
 }
 func (m *StateClearResponse) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_StateClearResponse.Unmarshal(m, b)
@@ -3569,7 +3686,7 @@
 func (m *LogEntry) String() string { return proto.CompactTextString(m) }
 func (*LogEntry) ProtoMessage()    {}
 func (*LogEntry) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_d24d1635dfa071c8, []int{27}
+	return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{27}
 }
 func (m *LogEntry) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_LogEntry.Unmarshal(m, b)
@@ -3659,7 +3776,7 @@
 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_d24d1635dfa071c8, []int{27, 0}
+	return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{27, 0}
 }
 func (m *LogEntry_List) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_LogEntry_List.Unmarshal(m, b)
@@ -3709,7 +3826,7 @@
 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_d24d1635dfa071c8, []int{27, 1}
+	return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{27, 1}
 }
 func (m *LogEntry_Severity) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_LogEntry_Severity.Unmarshal(m, b)
@@ -3739,7 +3856,7 @@
 func (m *LogControl) String() string { return proto.CompactTextString(m) }
 func (*LogControl) ProtoMessage()    {}
 func (*LogControl) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_d24d1635dfa071c8, []int{28}
+	return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{28}
 }
 func (m *LogControl) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_LogControl.Unmarshal(m, b)
@@ -3775,7 +3892,7 @@
 func (m *StartWorkerRequest) String() string { return proto.CompactTextString(m) }
 func (*StartWorkerRequest) ProtoMessage()    {}
 func (*StartWorkerRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_d24d1635dfa071c8, []int{29}
+	return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{29}
 }
 func (m *StartWorkerRequest) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_StartWorkerRequest.Unmarshal(m, b)
@@ -3848,7 +3965,7 @@
 func (m *StartWorkerResponse) String() string { return proto.CompactTextString(m) }
 func (*StartWorkerResponse) ProtoMessage()    {}
 func (*StartWorkerResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_d24d1635dfa071c8, []int{30}
+	return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{30}
 }
 func (m *StartWorkerResponse) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_StartWorkerResponse.Unmarshal(m, b)
@@ -3886,7 +4003,7 @@
 func (m *StopWorkerRequest) String() string { return proto.CompactTextString(m) }
 func (*StopWorkerRequest) ProtoMessage()    {}
 func (*StopWorkerRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_d24d1635dfa071c8, []int{31}
+	return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{31}
 }
 func (m *StopWorkerRequest) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_StopWorkerRequest.Unmarshal(m, b)
@@ -3924,7 +4041,7 @@
 func (m *StopWorkerResponse) String() string { return proto.CompactTextString(m) }
 func (*StopWorkerResponse) ProtoMessage()    {}
 func (*StopWorkerResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_d24d1635dfa071c8, []int{32}
+	return fileDescriptor_beam_fn_api_95f219ade4a36a20, []int{32}
 }
 func (m *StopWorkerResponse) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_StopWorkerResponse.Unmarshal(m, b)
@@ -4003,6 +4120,7 @@
 	proto.RegisterType((*StateKey_Runner)(nil), "org.apache.beam.model.fn_execution.v1.StateKey.Runner")
 	proto.RegisterType((*StateKey_MultimapSideInput)(nil), "org.apache.beam.model.fn_execution.v1.StateKey.MultimapSideInput")
 	proto.RegisterType((*StateKey_BagUserState)(nil), "org.apache.beam.model.fn_execution.v1.StateKey.BagUserState")
+	proto.RegisterType((*StateKey_IterableSideInput)(nil), "org.apache.beam.model.fn_execution.v1.StateKey.IterableSideInput")
 	proto.RegisterType((*StateGetRequest)(nil), "org.apache.beam.model.fn_execution.v1.StateGetRequest")
 	proto.RegisterType((*StateGetResponse)(nil), "org.apache.beam.model.fn_execution.v1.StateGetResponse")
 	proto.RegisterType((*StateAppendRequest)(nil), "org.apache.beam.model.fn_execution.v1.StateAppendRequest")
@@ -4526,205 +4644,209 @@
 	Metadata: "beam_fn_api.proto",
 }
 
-func init() { proto.RegisterFile("beam_fn_api.proto", fileDescriptor_beam_fn_api_d24d1635dfa071c8) }
+func init() { proto.RegisterFile("beam_fn_api.proto", fileDescriptor_beam_fn_api_95f219ade4a36a20) }
 
-var fileDescriptor_beam_fn_api_d24d1635dfa071c8 = []byte{
-	// 3139 bytes of a gzipped FileDescriptorProto
-	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xbc, 0x5a, 0x5b, 0x6f, 0x1b, 0xc7,
-	0xf5, 0xf7, 0x92, 0x94, 0x44, 0x1e, 0x52, 0x12, 0x35, 0x92, 0x6c, 0x7a, 0xff, 0xce, 0xbf, 0x0e,
-	0xdb, 0x00, 0x42, 0x8a, 0xac, 0xaf, 0x48, 0xec, 0x34, 0x71, 0x22, 0x51, 0xb4, 0xcd, 0x44, 0xb6,
-	0xd9, 0x95, 0x5c, 0xb7, 0x49, 0x93, 0xc5, 0x8a, 0x3b, 0xa4, 0x17, 0x5e, 0xee, 0x6e, 0x66, 0x96,
-	0xb2, 0xe4, 0x06, 0x0d, 0x7a, 0x41, 0x8b, 0x16, 0x6d, 0xf3, 0xd2, 0x87, 0xb4, 0x6f, 0x6d, 0x81,
-	0x02, 0x7d, 0xe9, 0x07, 0xc8, 0x37, 0x28, 0x50, 0xa0, 0x5f, 0x20, 0x2f, 0x45, 0x5b, 0xa0, 0x6d,
-	0xfa, 0x5c, 0xa0, 0x6f, 0xc5, 0x5c, 0xf6, 0xc2, 0x25, 0xe9, 0x2c, 0x29, 0xa5, 0x6f, 0x3b, 0x73,
-	0xf6, 0xfc, 0x7e, 0x33, 0x67, 0xcf, 0x9c, 0x39, 0x67, 0x66, 0x61, 0x65, 0x1f, 0x9b, 0x7d, 0xa3,
-	0xeb, 0x1a, 0xa6, 0x6f, 0x6b, 0x3e, 0xf1, 0x02, 0x0f, 0x3d, 0xe7, 0x91, 0x9e, 0x66, 0xfa, 0x66,
-	0xe7, 0x21, 0xd6, 0x98, 0x54, 0xeb, 0x7b, 0x16, 0x76, 0xb4, 0xae, 0x6b, 0xe0, 0x43, 0xdc, 0x19,
-	0x04, 0xb6, 0xe7, 0x6a, 0x07, 0x97, 0xd4, 0x75, 0xae, 0x49, 0x06, 0xae, 0x8b, 0x49, 0xac, 0xad,
-	0x2e, 0x63, 0xd7, 0xf2, 0x3d, 0xdb, 0x0d, 0xa8, 0xec, 0x38, 0xdf, 0xf3, 0xbc, 0x9e, 0x83, 0x2f,
-	0xf0, 0xd6, 0xfe, 0xa0, 0x7b, 0xc1, 0xc2, 0xb4, 0x43, 0x6c, 0x3f, 0xf0, 0x88, 0x7c, 0xe3, 0x0b,
-	0xe9, 0x37, 0x02, 0xbb, 0x8f, 0x69, 0x60, 0xf6, 0x7d, 0xf9, 0xc2, 0xff, 0xa7, 0x5f, 0x78, 0x4c,
-	0x4c, 0xdf, 0xc7, 0x24, 0xa4, 0x58, 0xec, 0xe3, 0x80, 0xd8, 0x1d, 0xd9, 0xac, 0xff, 0x52, 0x81,
-	0x25, 0x1d, 0xf7, 0xbd, 0x00, 0xdf, 0x22, 0x7e, 0xa7, 0xed, 0x91, 0x00, 0xf5, 0xe1, 0xb4, 0xe9,
-	0xdb, 0x06, 0xc5, 0xe4, 0xc0, 0xee, 0x60, 0x23, 0x1e, 0x42, 0x4d, 0x39, 0xaf, 0x6c, 0x94, 0x2f,
-	0xbf, 0xa4, 0x8d, 0x9f, 0xb4, 0x6f, 0xfb, 0xd8, 0xb1, 0x5d, 0xac, 0x1d, 0x5c, 0xd2, 0x36, 0x7d,
-	0x7b, 0x57, 0xe8, 0x6f, 0x47, 0xea, 0xfa, 0x9a, 0x39, 0xa6, 0x17, 0x9d, 0x85, 0x62, 0xc7, 0xb3,
-	0x30, 0x31, 0x6c, 0xab, 0x96, 0x3b, 0xaf, 0x6c, 0x94, 0xf4, 0x05, 0xde, 0x6e, 0x59, 0xf5, 0xbf,
-	0x15, 0x00, 0xb5, 0x5c, 0x1a, 0x90, 0x41, 0x87, 0x59, 0x52, 0xc7, 0xef, 0x0d, 0x30, 0x0d, 0xd0,
-	0x73, 0xb0, 0x64, 0xc7, 0xbd, 0x4c, 0x4f, 0xe1, 0x7a, 0x8b, 0x89, 0xde, 0x96, 0x85, 0xee, 0x43,
-	0x91, 0xe0, 0x9e, 0x4d, 0x03, 0x4c, 0x6a, 0x7f, 0x5f, 0xe0, 0x43, 0x7f, 0x51, 0xcb, 0xf4, 0xbd,
-	0x34, 0x5d, 0xea, 0x49, 0xc6, 0xdb, 0xa7, 0xf4, 0x08, 0x0a, 0x61, 0x58, 0xf2, 0x89, 0xd7, 0xc1,
-	0x94, 0x1a, 0xfb, 0x03, 0xd7, 0x72, 0x70, 0xed, 0x1f, 0x02, 0xfc, 0x2b, 0x19, 0xc1, 0xdb, 0x42,
-	0x7b, 0x8b, 0x2b, 0xc7, 0x0c, 0x8b, 0x7e, 0xb2, 0x1f, 0x7d, 0x1b, 0xce, 0x0c, 0xd3, 0x18, 0x3e,
-	0xf1, 0x7a, 0x04, 0x53, 0x5a, 0xfb, 0xa7, 0xe0, 0x6b, 0xcc, 0xc2, 0xd7, 0x96, 0x20, 0x31, 0xef,
-	0xba, 0x3f, 0x4e, 0x8e, 0x06, 0xb0, 0x96, 0xe2, 0xa7, 0xbe, 0x63, 0x07, 0xb5, 0x4f, 0x05, 0xf9,
-	0xeb, 0xb3, 0x90, 0xef, 0x32, 0x84, 0x98, 0x19, 0xf9, 0x23, 0x42, 0xf4, 0x10, 0x96, 0xbb, 0xb6,
-	0x6b, 0x3a, 0xf6, 0x13, 0x1c, 0x9a, 0xf7, 0x5f, 0x82, 0xf1, 0x95, 0x8c, 0x8c, 0x37, 0xa5, 0x7a,
-	0xda, 0xbe, 0x4b, 0xdd, 0x21, 0xc1, 0x56, 0x09, 0x16, 0x88, 0x10, 0xd6, 0xbf, 0x3b, 0x07, 0xab,
-	0x43, 0x7e, 0x46, 0x7d, 0xcf, 0xa5, 0x38, 0xab, 0xa3, 0xad, 0xc1, 0x1c, 0x26, 0xc4, 0x23, 0xd2,
-	0x7d, 0x45, 0x03, 0x7d, 0x6d, 0xd4, 0xfd, 0x5e, 0x9a, 0xda, 0xfd, 0xc4, 0x40, 0x86, 0xfc, 0xaf,
-	0x3b, 0xc9, 0xff, 0x5e, 0x99, 0xcd, 0xff, 0x22, 0x8a, 0x94, 0x03, 0x7e, 0xf0, 0x99, 0x0e, 0xb8,
-	0x7d, 0x3c, 0x07, 0x8c, 0x88, 0x27, 0x78, 0xe0, 0xc1, 0xd3, 0x3d, 0x70, 0xf3, 0x18, 0x1e, 0x18,
-	0x51, 0x8f, 0x73, 0x41, 0x7b, 0xa2, 0x0b, 0xbe, 0x3a, 0xa3, 0x0b, 0x46, 0x74, 0x69, 0x1f, 0x04,
-	0xe6, 0x23, 0x42, 0x5a, 0xff, 0xa9, 0x02, 0xcb, 0xa9, 0xb8, 0x83, 0x9e, 0xc0, 0xd9, 0x94, 0x09,
-	0x86, 0xa2, 0x71, 0x7e, 0xa3, 0x7c, 0xf9, 0xc6, 0x2c, 0x66, 0x48, 0x04, 0xe5, 0x33, 0xfe, 0x78,
-	0x41, 0x1d, 0x41, 0x35, 0xed, 0x87, 0xf5, 0xdf, 0x00, 0x9c, 0x99, 0x00, 0x84, 0x96, 0x20, 0x17,
-	0x2d, 0x90, 0x9c, 0x6d, 0x21, 0x17, 0x20, 0x20, 0xa6, 0x4b, 0xbb, 0x1e, 0xe9, 0xd3, 0x5a, 0x8e,
-	0x0f, 0xf6, 0xee, 0xf1, 0x06, 0xab, 0xed, 0x45, 0x80, 0x4d, 0x37, 0x20, 0x47, 0x7a, 0x82, 0x01,
-	0x05, 0x50, 0xf1, 0x3b, 0x9e, 0xe3, 0x60, 0xbe, 0x2c, 0x69, 0x2d, 0xcf, 0x19, 0xdb, 0xc7, 0x64,
-	0x6c, 0x27, 0x20, 0x05, 0xe7, 0x10, 0x0b, 0xfa, 0xb1, 0x02, 0x6b, 0x8f, 0x6d, 0xd7, 0xf2, 0x1e,
-	0xdb, 0x6e, 0xcf, 0xa0, 0x01, 0x31, 0x03, 0xdc, 0xb3, 0x31, 0xad, 0x15, 0x38, 0xfd, 0x83, 0x63,
-	0xd2, 0x3f, 0x08, 0xa1, 0x77, 0x23, 0x64, 0x31, 0x8a, 0xd5, 0xc7, 0xa3, 0x12, 0xb4, 0x0f, 0xf3,
-	0x7c, 0xeb, 0xa4, 0xb5, 0x39, 0xce, 0xfe, 0xc6, 0x31, 0xd9, 0x1b, 0x1c, 0x4c, 0x10, 0x4a, 0x64,
-	0x66, 0x66, 0xec, 0x1e, 0xd8, 0xc4, 0x73, 0xfb, 0xd8, 0x0d, 0x68, 0x6d, 0xfe, 0x44, 0xcc, 0xdc,
-	0x4c, 0x40, 0x4a, 0x33, 0x27, 0x59, 0xd0, 0x21, 0x9c, 0xa3, 0x81, 0x19, 0x60, 0x63, 0x42, 0x66,
-	0xb2, 0x70, 0xbc, 0xcc, 0xe4, 0x2c, 0x07, 0x1f, 0x27, 0x52, 0x1d, 0x58, 0x4e, 0x79, 0x1d, 0xaa,
-	0x42, 0xfe, 0x11, 0x3e, 0x92, 0xae, 0xce, 0x1e, 0x51, 0x03, 0xe6, 0x0e, 0x4c, 0x67, 0x80, 0xf9,
-	0x0e, 0x50, 0xbe, 0xfc, 0x42, 0x86, 0x71, 0xb4, 0x23, 0x54, 0x5d, 0xe8, 0xbe, 0x9c, 0xbb, 0xa6,
-	0xa8, 0x1e, 0xac, 0x8c, 0x78, 0xdc, 0x18, 0xbe, 0xed, 0x61, 0x3e, 0x2d, 0x0b, 0x5f, 0x23, 0x82,
-	0x4d, 0x12, 0xbe, 0x0f, 0xb5, 0x49, 0x3e, 0x36, 0x86, 0xf7, 0x8d, 0x61, 0xde, 0xab, 0x19, 0x78,
-	0xd3, 0xe8, 0x47, 0x49, 0xf6, 0x0e, 0x94, 0x13, 0x3e, 0x36, 0x86, 0xf0, 0xc6, 0x30, 0xe1, 0x46,
-	0x06, 0x42, 0x0e, 0x98, 0xb2, 0xe9, 0x88, 0x7b, 0x9d, 0x8c, 0x4d, 0x13, 0xb0, 0x09, 0xc2, 0xfa,
-	0x7f, 0xf2, 0xb0, 0x22, 0x3c, 0x7c, 0xd3, 0xf7, 0x1d, 0xbb, 0x63, 0x32, 0xa3, 0xa3, 0x67, 0xa1,
-	0x12, 0x45, 0xab, 0x38, 0x95, 0x28, 0x47, 0x7d, 0x2d, 0x8b, 0xa5, 0xc2, 0xb6, 0xeb, 0x0f, 0x82,
-	0x44, 0x2a, 0xcc, 0xdb, 0x2d, 0x0b, 0xd5, 0x60, 0x01, 0x3b, 0x98, 0x31, 0xd5, 0xf2, 0xe7, 0x95,
-	0x8d, 0x8a, 0x1e, 0x36, 0xd1, 0xb7, 0x60, 0xc5, 0x1b, 0x04, 0x4c, 0xeb, 0xb1, 0x19, 0x60, 0xd2,
-	0x37, 0xc9, 0xa3, 0x30, 0xfa, 0x64, 0x0d, 0xb7, 0x23, 0x83, 0xd5, 0xee, 0x71, 0xc4, 0x07, 0x11,
-	0xa0, 0x58, 0x93, 0x55, 0x2f, 0xd5, 0x8d, 0xda, 0x00, 0x36, 0x35, 0xf6, 0xbd, 0x81, 0x6b, 0x61,
-	0xab, 0x36, 0x77, 0x5e, 0xd9, 0x58, 0xba, 0x7c, 0x29, 0x83, 0xe5, 0x5a, 0x74, 0x4b, 0xe8, 0x68,
-	0x4d, 0x77, 0xd0, 0xd7, 0x4b, 0x76, 0xd8, 0x46, 0xdf, 0x84, 0x6a, 0xdf, 0x73, 0xed, 0xc0, 0x23,
-	0x2c, 0xa0, 0xda, 0x6e, 0xd7, 0x0b, 0x63, 0x4c, 0x16, 0xdc, 0x3b, 0x91, 0x6a, 0xcb, 0xed, 0x7a,
-	0xfa, 0x72, 0x7f, 0xa8, 0x4d, 0x55, 0x03, 0xd6, 0xc7, 0x4e, 0x6d, 0x8c, 0x3f, 0x5c, 0x1c, 0xf6,
-	0x07, 0x55, 0x13, 0x85, 0x95, 0x16, 0x16, 0x56, 0xda, 0x5e, 0x58, 0x79, 0x25, 0xbf, 0xfd, 0x1f,
-	0x15, 0xa8, 0x6d, 0x63, 0xc7, 0x3c, 0xc2, 0xd6, 0xa8, 0x0b, 0xec, 0x41, 0x4d, 0xa6, 0x9c, 0xd8,
-	0x8a, 0xbf, 0x80, 0xc1, 0x4a, 0x38, 0x59, 0x5b, 0x3d, 0x8d, 0xe5, 0x74, 0xa4, 0xdb, 0x0c, 0x55,
-	0x99, 0x10, 0xbd, 0x05, 0x65, 0x33, 0x26, 0x91, 0xc3, 0xbd, 0x36, 0xeb, 0xa7, 0xd7, 0x93, 0x60,
-	0xf5, 0x9f, 0x15, 0x60, 0x6d, 0x5c, 0xbd, 0x82, 0x5e, 0x83, 0x73, 0x13, 0x33, 0x93, 0xd8, 0xbb,
-	0xcf, 0x4e, 0x48, 0x2e, 0x5a, 0x16, 0xb2, 0xa1, 0xd2, 0x61, 0x83, 0x33, 0x02, 0xef, 0x11, 0x76,
-	0xc3, 0x04, 0xe1, 0xe6, 0x31, 0x6a, 0x28, 0xad, 0xc1, 0xb4, 0xf6, 0x18, 0x9c, 0x5e, 0xee, 0x44,
-	0xcf, 0x54, 0xfd, 0x43, 0x0e, 0x20, 0x96, 0xa1, 0xf7, 0x00, 0x06, 0x14, 0x13, 0x83, 0xc7, 0x7c,
-	0x69, 0xf7, 0xf6, 0xc9, 0xf0, 0x6a, 0xf7, 0x29, 0x26, 0xbb, 0x0c, 0xf7, 0xf6, 0x29, 0xbd, 0x34,
-	0x08, 0x1b, 0x8c, 0x92, 0xda, 0x16, 0x36, 0xf8, 0x6a, 0x96, 0x5f, 0xe8, 0xa4, 0x28, 0x77, 0x6d,
-	0x0b, 0xb7, 0x18, 0x2e, 0xa3, 0xa4, 0x61, 0x83, 0x15, 0x25, 0xdc, 0xb2, 0x35, 0xe0, 0xe1, 0x42,
-	0x34, 0xd4, 0x32, 0x94, 0xa2, 0x21, 0xaa, 0xcf, 0x43, 0x29, 0x52, 0x46, 0xcf, 0x0c, 0x0d, 0x51,
-	0x7c, 0xbe, 0x18, 0x6e, 0x6b, 0x1e, 0x0a, 0xc1, 0x91, 0x8f, 0xeb, 0x9f, 0xe4, 0x60, 0x7d, 0x6c,
-	0x01, 0x81, 0x6e, 0xc3, 0x82, 0x3c, 0x5a, 0x90, 0x36, 0xd5, 0x32, 0x4e, 0xf0, 0x8e, 0xd0, 0xd2,
-	0x43, 0x75, 0x56, 0xe1, 0x10, 0x4c, 0x6d, 0x6b, 0x60, 0x3a, 0x06, 0xf1, 0xbc, 0x20, 0x74, 0x8e,
-	0xd7, 0x32, 0x02, 0x4e, 0x5a, 0x7f, 0xfa, 0x62, 0x08, 0xab, 0x33, 0xd4, 0xb1, 0xa1, 0x26, 0x7f,
-	0x52, 0xa1, 0x06, 0x5d, 0x81, 0x75, 0xb6, 0x60, 0x6d, 0x82, 0xa9, 0x21, 0xd3, 0x7e, 0xb1, 0x40,
-	0x0b, 0xe7, 0x95, 0x8d, 0xa2, 0xbe, 0x16, 0x0a, 0x6f, 0x26, 0x64, 0xf5, 0x26, 0x9c, 0x7b, 0x5a,
-	0xb9, 0x9e, 0xb1, 0x22, 0xad, 0x7f, 0xb4, 0x0a, 0x0b, 0xd2, 0xac, 0xc8, 0x84, 0xb2, 0x9f, 0x48,
-	0xc4, 0x95, 0xa9, 0x4c, 0x29, 0x41, 0xb4, 0x76, 0x90, 0xca, 0xbc, 0x93, 0x98, 0xea, 0x27, 0x65,
-	0x80, 0x38, 0x9f, 0x41, 0x4f, 0x20, 0x2c, 0xab, 0x58, 0x98, 0x13, 0xdb, 0x54, 0xe8, 0x14, 0x6f,
-	0x4e, 0x4b, 0x1c, 0xc1, 0x86, 0x0b, 0x01, 0x5b, 0x4d, 0x09, 0xa9, 0xaf, 0xf8, 0xe9, 0x2e, 0xf4,
-	0x1e, 0x2c, 0x9b, 0x9d, 0xc0, 0x3e, 0xc0, 0x31, 0xb1, 0x58, 0x6e, 0xb7, 0x67, 0x27, 0xde, 0xe4,
-	0x80, 0x11, 0xeb, 0x92, 0x39, 0xd4, 0x46, 0x36, 0x40, 0x62, 0xe7, 0x15, 0x0e, 0xd4, 0x9a, 0x9d,
-	0x2d, 0xbd, 0xe9, 0x26, 0xc0, 0xd1, 0x2d, 0x28, 0xb0, 0xa0, 0x22, 0xb7, 0xf7, 0x2b, 0x53, 0x92,
-	0xb0, 0x95, 0xaf, 0x73, 0x00, 0xf5, 0xaf, 0x79, 0x28, 0xde, 0xc1, 0x26, 0x1d, 0x10, 0x6c, 0xa1,
-	0x9f, 0x28, 0xb0, 0x26, 0xf2, 0x0e, 0x69, 0x33, 0xa3, 0xe3, 0x0d, 0xc4, 0x27, 0x63, 0x34, 0x6f,
-	0xcd, 0x3e, 0x97, 0x90, 0x42, 0xe3, 0x41, 0x44, 0x5a, 0xac, 0xc1, 0xc1, 0xc5, 0xe4, 0x90, 0x3d,
-	0x22, 0x40, 0x1f, 0x2a, 0xb0, 0x2e, 0x33, 0x9a, 0xd4, 0x78, 0x44, 0x18, 0x78, 0xfb, 0x04, 0xc6,
-	0x23, 0x92, 0x80, 0x31, 0x03, 0x5a, 0xf5, 0x46, 0x25, 0x68, 0x03, 0xaa, 0x81, 0x17, 0x98, 0x0e,
-	0xdf, 0xa9, 0x0d, 0xea, 0x87, 0x59, 0x98, 0xa2, 0x2f, 0xf1, 0x7e, 0xb6, 0x0d, 0xef, 0xb2, 0x5e,
-	0xb5, 0x09, 0x67, 0x26, 0x4c, 0x75, 0x4c, 0x86, 0xb1, 0x96, 0xcc, 0x30, 0xf2, 0xc9, 0x94, 0xf5,
-	0x26, 0xd4, 0x26, 0x8d, 0x70, 0x2a, 0x1c, 0x0a, 0x2b, 0x23, 0xab, 0x06, 0xbd, 0x0b, 0xc5, 0xbe,
-	0xb4, 0x83, 0x5c, 0x94, 0x5b, 0xc7, 0xb7, 0xa8, 0x1e, 0x61, 0xaa, 0x1f, 0xe6, 0x61, 0x69, 0x78,
-	0xc9, 0x7c, 0xde, 0x94, 0xe8, 0x05, 0x40, 0x5d, 0x62, 0x8a, 0x98, 0x48, 0x70, 0xdf, 0xb4, 0x5d,
-	0xdb, 0xed, 0x71, 0x73, 0x28, 0xfa, 0x4a, 0x28, 0xd1, 0x43, 0x01, 0xfa, 0x95, 0x02, 0x67, 0x87,
-	0x3d, 0x8c, 0x26, 0xd4, 0xc4, 0x0a, 0xc6, 0x27, 0x15, 0x2f, 0x86, 0x7d, 0x8d, 0x46, 0xa3, 0x10,
-	0xfe, 0x76, 0xc6, 0x1b, 0x2f, 0x55, 0xdf, 0x80, 0x73, 0x4f, 0x53, 0x9c, 0xca, 0x0d, 0x5e, 0x85,
-	0xe5, 0xcf, 0xce, 0x77, 0x27, 0xab, 0xff, 0x69, 0x0e, 0x0a, 0x2c, 0x76, 0x20, 0x03, 0xca, 0x62,
-	0x8f, 0x36, 0x5c, 0x33, 0x4a, 0x59, 0x6f, 0xcc, 0x10, 0x85, 0x64, 0xe3, 0xae, 0xd9, 0xc7, 0x3a,
-	0xf4, 0xa3, 0x67, 0x84, 0xa1, 0xc2, 0x97, 0x3a, 0x26, 0x86, 0x65, 0x06, 0x66, 0x78, 0xb2, 0xf9,
-	0xda, 0x2c, 0x14, 0x0d, 0x01, 0xb4, 0x6d, 0x06, 0xe6, 0xed, 0x53, 0x7a, 0xb9, 0x13, 0x37, 0x51,
-	0x00, 0x2b, 0x96, 0x4d, 0x03, 0x62, 0xef, 0x8b, 0x04, 0x9c, 0x73, 0x4d, 0x79, 0xa8, 0x39, 0xc4,
-	0xb5, 0x9d, 0x40, 0x93, 0x84, 0x55, 0x2b, 0xd5, 0x87, 0x0c, 0x80, 0x9e, 0x39, 0xe8, 0x61, 0x41,
-	0xf7, 0xe9, 0x74, 0x47, 0x8a, 0x43, 0x74, 0xb7, 0x18, 0x8c, 0xe4, 0x29, 0xf5, 0xc2, 0x86, 0x7a,
-	0x03, 0x20, 0xb6, 0x2b, 0x3a, 0x07, 0x25, 0xf6, 0x95, 0xa8, 0x6f, 0x76, 0xb0, 0xac, 0x26, 0xe3,
-	0x0e, 0x84, 0xa0, 0xc0, 0xbf, 0x61, 0x9e, 0x0b, 0xf8, 0xb3, 0xfa, 0x45, 0x56, 0x8d, 0xc7, 0x56,
-	0x8a, 0x1c, 0x42, 0x49, 0x38, 0x84, 0xfa, 0x2e, 0x54, 0xd3, 0xb3, 0x65, 0x6f, 0x72, 0xf3, 0x86,
-	0x6f, 0xf2, 0x06, 0x73, 0x31, 0x3a, 0xe8, 0x4b, 0x77, 0x62, 0x8f, 0xac, 0xa7, 0x6f, 0xbb, 0x9c,
-	0x33, 0xaf, 0xb3, 0x47, 0xde, 0x63, 0x1e, 0xf2, 0x94, 0x88, 0xf5, 0x98, 0x87, 0xea, 0xdb, 0x50,
-	0x8a, 0xa6, 0x37, 0x7e, 0x08, 0xe8, 0x1a, 0x94, 0xa2, 0x5b, 0xaf, 0x0c, 0xd5, 0x59, 0xfc, 0x32,
-	0xcb, 0x62, 0x99, 0xf1, 0xd5, 0x23, 0xa8, 0xa6, 0x33, 0x9a, 0x31, 0x2b, 0xe2, 0xde, 0x70, 0x05,
-	0x78, 0x7d, 0xe6, 0x88, 0x90, 0x2c, 0x10, 0x7f, 0x9b, 0x83, 0x67, 0x9e, 0x7a, 0x20, 0x7e, 0x82,
-	0x89, 0xf4, 0xe7, 0x9b, 0xe0, 0xbe, 0x03, 0x8b, 0x3e, 0xb1, 0xfb, 0x26, 0x39, 0x92, 0x59, 0xba,
-	0xc8, 0x4a, 0x66, 0xaf, 0x3c, 0x2b, 0x12, 0x8e, 0x67, 0xe7, 0xf5, 0xef, 0x14, 0xe0, 0xec, 0xc4,
-	0xdb, 0xa3, 0xac, 0x57, 0x33, 0x4f, 0x60, 0xc9, 0xc2, 0xd4, 0x26, 0xd8, 0x12, 0x97, 0x07, 0xe1,
-	0xfc, 0x77, 0x8f, 0x7b, 0x7d, 0xa5, 0x6d, 0x0b, 0x58, 0xde, 0x27, 0x73, 0x87, 0x45, 0x2b, 0xd9,
-	0xa7, 0xfe, 0x5e, 0x81, 0x4a, 0xf2, 0x2d, 0x74, 0x19, 0xd6, 0xa3, 0x5d, 0xca, 0xeb, 0xca, 0x1d,
-	0xc7, 0xc2, 0xe2, 0x5e, 0x35, 0xa7, 0xaf, 0x86, 0xc2, 0x7b, 0x5d, 0x3d, 0x14, 0xa1, 0x8b, 0xb0,
-	0x66, 0x3a, 0x8e, 0xf7, 0x38, 0x9c, 0x80, 0x21, 0xee, 0x8b, 0xf9, 0x34, 0xf2, 0x3a, 0x92, 0x32,
-	0x8e, 0xdf, 0xe6, 0x12, 0x74, 0x0d, 0x6a, 0x98, 0x06, 0x76, 0xdf, 0x0c, 0xb0, 0x65, 0x0c, 0xa5,
-	0x75, 0x54, 0xae, 0xc5, 0xd3, 0x91, 0x3c, 0x99, 0xab, 0x50, 0xf5, 0x43, 0x05, 0xd0, 0xe8, 0xb4,
-	0xc6, 0x2c, 0x8c, 0xce, 0xf0, 0xc2, 0xb8, 0x73, 0xa2, 0xc6, 0x4c, 0x2e, 0x96, 0x7f, 0xe7, 0x41,
-	0x9d, 0x7c, 0x7f, 0x33, 0xea, 0x81, 0xca, 0x49, 0x7a, 0xe0, 0xff, 0xac, 0x0e, 0x1d, 0xc0, 0x52,
-	0xe7, 0xa1, 0xe9, 0xba, 0xd8, 0x19, 0x76, 0xd2, 0xbb, 0xc7, 0xbe, 0xe1, 0xd2, 0x1a, 0x02, 0x57,
-	0x74, 0x2e, 0x76, 0x12, 0x2d, 0xaa, 0xfe, 0x42, 0x81, 0x4a, 0x52, 0x9e, 0xe5, 0x84, 0xf2, 0x22,
-	0xac, 0x39, 0x26, 0x0d, 0x8c, 0xd0, 0xec, 0xe1, 0x99, 0x24, 0x73, 0x84, 0x39, 0x1d, 0x31, 0x59,
-	0x5b, 0x88, 0xa4, 0x57, 0xa1, 0xab, 0x70, 0xba, 0x6b, 0x13, 0x1a, 0x18, 0x91, 0x29, 0x93, 0xe7,
-	0x98, 0x73, 0xfa, 0x1a, 0x97, 0xea, 0x52, 0x28, 0xb5, 0xea, 0x37, 0x60, 0x7d, 0xec, 0x3d, 0x6e,
-	0xd6, 0x02, 0xb8, 0x06, 0xa7, 0xc7, 0x5f, 0xc2, 0xd5, 0x3f, 0x56, 0xa0, 0x18, 0xe5, 0xa5, 0xb7,
-	0xc5, 0x7e, 0x20, 0xfd, 0xe6, 0x6a, 0x46, 0x7b, 0x47, 0x99, 0x1d, 0xdb, 0xa3, 0x74, 0xb1, 0xa3,
-	0x58, 0x50, 0xe0, 0x3b, 0x56, 0xc6, 0xb8, 0x94, 0x36, 0x75, 0x6e, 0xd4, 0xd4, 0x48, 0x8e, 0x4d,
-	0x1c, 0xf7, 0xf2, 0xe7, 0xfa, 0xcf, 0xf3, 0x50, 0xe1, 0x67, 0x37, 0xa1, 0x39, 0xd2, 0x97, 0x6e,
-	0xa3, 0xf4, 0xb9, 0x71, 0xf4, 0x3b, 0x50, 0x12, 0xd7, 0x29, 0x6c, 0x61, 0xe7, 0xf9, 0x22, 0xbe,
-	0x90, 0x71, 0xf2, 0x9c, 0xfe, 0x4d, 0x7c, 0xa4, 0x17, 0xa9, 0x7c, 0x42, 0x6f, 0x42, 0xbe, 0x87,
-	0x83, 0x69, 0xff, 0xb1, 0xe0, 0x40, 0xb7, 0x70, 0xe2, 0x7f, 0x00, 0x86, 0x82, 0xf6, 0x60, 0xde,
-	0xf4, 0x7d, 0xec, 0x5a, 0x61, 0xf2, 0x77, 0x7d, 0x1a, 0xbc, 0x4d, 0xae, 0x1a, 0x43, 0x4a, 0x2c,
-	0xf4, 0x55, 0x98, 0xeb, 0x38, 0xd8, 0x24, 0x61, 0x96, 0x77, 0x6d, 0x1a, 0xd0, 0x06, 0xd3, 0x8c,
-	0x31, 0x05, 0x52, 0xf2, 0xff, 0x81, 0x8f, 0x73, 0xb0, 0x28, 0x3f, 0x8b, 0x8c, 0x4c, 0xe9, 0xef,
-	0x32, 0xfe, 0x17, 0x81, 0x9d, 0x21, 0xc3, 0xbd, 0x34, 0xb5, 0xe1, 0xa2, 0x7b, 0x65, 0x6e, 0xb9,
-	0xfb, 0x69, 0xcb, 0xbd, 0x3c, 0x8b, 0xe5, 0x22, 0xcc, 0xd0, 0x74, 0x7a, 0xca, 0x74, 0xd7, 0x67,
-	0x30, 0x5d, 0x04, 0x2a, 0x6d, 0x97, 0xbc, 0xf7, 0xfe, 0x4b, 0x01, 0x8a, 0xa1, 0x53, 0xa1, 0x36,
-	0xcc, 0x8b, 0xbf, 0xa4, 0x64, 0xea, 0xf3, 0xe2, 0x94, 0x5e, 0xa9, 0xe9, 0x5c, 0x9b, 0x0d, 0x5f,
-	0xe0, 0x20, 0x0a, 0xab, 0xfd, 0x81, 0xc3, 0xf6, 0x3b, 0xdf, 0x18, 0x39, 0x83, 0xdd, 0x9c, 0x16,
-	0xfe, 0x8e, 0x84, 0x4a, 0x1e, 0xba, 0xae, 0xf4, 0xd3, 0x9d, 0xc8, 0x82, 0xa5, 0x7d, 0xb3, 0x67,
-	0x24, 0x8e, 0x99, 0xf3, 0x53, 0xfd, 0xa2, 0x11, 0xf1, 0x6d, 0x99, 0xbd, 0xe4, 0x91, 0x72, 0x65,
-	0x3f, 0xd1, 0x56, 0x55, 0x98, 0x17, 0xd3, 0x4d, 0x6e, 0xd1, 0x15, 0xbe, 0x45, 0xab, 0xdf, 0x57,
-	0x60, 0x65, 0x64, 0xb0, 0x59, 0x22, 0x7c, 0x1d, 0x16, 0x63, 0x33, 0x25, 0x42, 0x53, 0x74, 0x14,
-	0xdc, 0xb2, 0xd0, 0x69, 0x98, 0x17, 0xd7, 0xcf, 0x32, 0x38, 0xc9, 0x56, 0x38, 0x8c, 0x42, 0x3c,
-	0x8c, 0x0f, 0xa0, 0x92, 0x9c, 0x42, 0xc6, 0x01, 0xc4, 0x76, 0x4b, 0x0c, 0x20, 0x3a, 0x4d, 0x9f,
-	0x66, 0x00, 0xd1, 0xb9, 0xf5, 0xeb, 0xb0, 0x9c, 0x0a, 0x38, 0xe8, 0x05, 0x40, 0x1d, 0xcf, 0x0d,
-	0x6c, 0x77, 0x60, 0x8a, 0x6b, 0x18, 0x7e, 0x5c, 0x2e, 0x6c, 0xb8, 0x92, 0x94, 0xf0, 0x73, 0xf6,
-	0xfa, 0x7d, 0xa8, 0xa6, 0x57, 0xde, 0x94, 0x10, 0x51, 0x48, 0xcf, 0x25, 0x42, 0xfa, 0x06, 0xa0,
-	0xd1, 0xc8, 0x15, 0xbd, 0xa9, 0x24, 0xde, 0x5c, 0x87, 0xd5, 0x31, 0x2b, 0xb5, 0xbe, 0x0a, 0x2b,
-	0x23, 0x51, 0xaa, 0xbe, 0x26, 0x51, 0x87, 0xd6, 0x5f, 0xfd, 0xd7, 0x05, 0x28, 0xee, 0x78, 0xf2,
-	0x00, 0xe1, 0x1b, 0x50, 0xa4, 0xf8, 0x00, 0x13, 0x3b, 0x10, 0x8e, 0xb3, 0x94, 0xb9, 0x16, 0x0d,
-	0x21, 0xb4, 0x5d, 0xa9, 0x2f, 0x2e, 0xf1, 0x22, 0xb8, 0xd9, 0x0b, 0x34, 0x54, 0x63, 0xb5, 0x0f,
-	0xa5, 0x66, 0x2f, 0xac, 0x4c, 0xc3, 0x26, 0xbf, 0xcf, 0x20, 0xac, 0x94, 0x2d, 0x88, 0x08, 0xca,
-	0x1b, 0x63, 0xf6, 0xbb, 0xb9, 0x2c, 0xdb, 0xed, 0xfc, 0xa8, 0xdb, 0x3d, 0x0b, 0x15, 0xc7, 0xeb,
-	0x19, 0x8e, 0x27, 0xaf, 0xd1, 0x16, 0xc4, 0x2b, 0x8e, 0xd7, 0xdb, 0x91, 0x5d, 0xcc, 0xeb, 0x82,
-	0x87, 0x04, 0x9b, 0x56, 0xad, 0xc8, 0x85, 0xb2, 0xa5, 0x7e, 0x1d, 0x0a, 0x3b, 0x36, 0x0d, 0x50,
-	0x1b, 0xd8, 0xeb, 0x06, 0x76, 0x03, 0x62, 0xe3, 0x30, 0x19, 0xbd, 0x30, 0xa5, 0x51, 0x75, 0x70,
-	0xc4, 0x93, 0x8d, 0xa9, 0x4a, 0xa0, 0x18, 0xda, 0xb8, 0xde, 0x85, 0x02, 0x33, 0x33, 0x5a, 0x86,
-	0xf2, 0xfd, 0xbb, 0xbb, 0xed, 0x66, 0xa3, 0x75, 0xb3, 0xd5, 0xdc, 0xae, 0x9e, 0x42, 0x25, 0x98,
-	0xdb, 0xd3, 0x37, 0x1b, 0xcd, 0xaa, 0xc2, 0x1e, 0xb7, 0x9b, 0x5b, 0xf7, 0x6f, 0x55, 0x73, 0xa8,
-	0x08, 0x85, 0xd6, 0xdd, 0x9b, 0xf7, 0xaa, 0x79, 0x04, 0x30, 0x7f, 0xf7, 0xde, 0x5e, 0xab, 0xd1,
-	0xac, 0x16, 0x58, 0xef, 0x83, 0x4d, 0xfd, 0x6e, 0x75, 0x8e, 0xbd, 0xda, 0xd4, 0xf5, 0x7b, 0x7a,
-	0x75, 0x1e, 0x55, 0xa0, 0xd8, 0xd0, 0x5b, 0x7b, 0xad, 0xc6, 0xe6, 0x4e, 0x75, 0xa1, 0x5e, 0x01,
-	0xd8, 0xf1, 0x7a, 0x0d, 0xcf, 0x0d, 0x88, 0xe7, 0xd4, 0xff, 0x5c, 0xe0, 0x9e, 0x44, 0x82, 0x07,
-	0x1e, 0x79, 0x14, 0xff, 0x98, 0xf4, 0x7f, 0x50, 0x7a, 0xcc, 0x3b, 0xe2, 0x45, 0x5c, 0x14, 0x1d,
-	0x2d, 0x0b, 0xed, 0x43, 0xb5, 0x23, 0xd4, 0x8d, 0xf0, 0x07, 0x57, 0xe9, 0x05, 0x33, 0xff, 0xa0,
-	0xb1, 0x2c, 0x01, 0x9b, 0x12, 0x8f, 0x71, 0x38, 0x5e, 0xaf, 0xc7, 0xea, 0xda, 0x88, 0x23, 0x7f,
-	0x4c, 0x0e, 0x09, 0x18, 0x71, 0x58, 0xb0, 0x62, 0x92, 0xc0, 0xee, 0x9a, 0x9d, 0x20, 0x26, 0x29,
-	0x1c, 0x8f, 0xa4, 0x1a, 0x22, 0x46, 0x2c, 0x5d, 0x7e, 0x5b, 0x72, 0x60, 0x53, 0xe6, 0xc0, 0x11,
-	0xcd, 0xdc, 0xf1, 0x68, 0x56, 0x22, 0xc8, 0x88, 0xe7, 0x1d, 0x98, 0xf7, 0x4d, 0x62, 0xf6, 0x69,
-	0x0d, 0xb8, 0x63, 0x36, 0xb3, 0xef, 0x45, 0xa9, 0xaf, 0xaf, 0xb5, 0x39, 0x8e, 0xfc, 0x2f, 0x48,
-	0x80, 0xaa, 0xd7, 0xa1, 0x9c, 0xe8, 0xfe, 0xac, 0xf3, 0xc5, 0x52, 0xb2, 0xca, 0xfb, 0x32, 0x0f,
-	0x6c, 0x31, 0x89, 0x0c, 0xae, 0x51, 0xce, 0xa4, 0x24, 0x72, 0xa6, 0xfa, 0x45, 0x16, 0xee, 0x3c,
-	0x3f, 0xbb, 0x3b, 0xd6, 0x9f, 0x67, 0x1e, 0x1c, 0x6b, 0x3c, 0x0d, 0xfd, 0xf2, 0x47, 0x0a, 0x2c,
-	0x6e, 0x61, 0xb3, 0x7f, 0xd3, 0x95, 0x0b, 0x00, 0xfd, 0x40, 0x81, 0x85, 0xf0, 0x39, 0x6b, 0x42,
-	0x35, 0xe6, 0x5f, 0x52, 0xf5, 0xfa, 0x2c, 0xba, 0x22, 0x98, 0x9f, 0xda, 0x50, 0x2e, 0x2a, 0x97,
-	0xdf, 0x07, 0x10, 0x23, 0xe3, 0x75, 0x86, 0x2b, 0xeb, 0x8d, 0x0b, 0x53, 0xd6, 0x2c, 0xea, 0xb4,
-	0x0a, 0x92, 0xfd, 0x87, 0x0a, 0x94, 0x05, 0xbd, 0xd8, 0xc8, 0x0f, 0x61, 0x4e, 0x3c, 0x5c, 0x99,
-	0x26, 0xa5, 0x91, 0x33, 0x52, 0xaf, 0x4e, 0xa7, 0x24, 0xb7, 0x2f, 0x31, 0x92, 0x1f, 0x45, 0x9f,
-	0x68, 0x47, 0xac, 0x57, 0x74, 0x08, 0x0b, 0xe1, 0xe3, 0xd5, 0x69, 0xb7, 0x30, 0x16, 0xb8, 0xd5,
-	0x4b, 0xd9, 0xb5, 0xc2, 0xb8, 0x28, 0xc6, 0xf2, 0xbb, 0x1c, 0xd4, 0xc4, 0x58, 0x9a, 0x87, 0x01,
-	0x26, 0xae, 0xe9, 0x08, 0x2f, 0x6b, 0x7b, 0xc2, 0x73, 0xca, 0x09, 0xbf, 0x46, 0xd7, 0x67, 0x5e,
-	0x70, 0xea, 0xcb, 0xb3, 0xa8, 0x86, 0x56, 0x43, 0xdf, 0x53, 0x00, 0xe2, 0x15, 0x80, 0xb2, 0xd7,
-	0x3e, 0xa9, 0x65, 0xa6, 0x5e, 0x9f, 0x41, 0x33, 0x1c, 0xc5, 0xd6, 0x26, 0x7c, 0x69, 0x92, 0x76,
-	0x52, 0x79, 0xab, 0x24, 0x0c, 0xba, 0xe9, 0xdb, 0x6f, 0x2d, 0x25, 0x44, 0xc6, 0xc1, 0xa5, 0xfd,
-	0x79, 0x9e, 0x3c, 0x5c, 0xf9, 0x6f, 0x00, 0x00, 0x00, 0xff, 0xff, 0x0f, 0x46, 0x65, 0x7e, 0x89,
-	0x31, 0x00, 0x00,
+var fileDescriptor_beam_fn_api_95f219ade4a36a20 = []byte{
+	// 3204 bytes of a gzipped FileDescriptorProto
+	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xbc, 0x5a, 0x5f, 0x6f, 0x1b, 0xc7,
+	0xb5, 0xf7, 0x92, 0x94, 0x44, 0x1e, 0x52, 0x12, 0x39, 0x92, 0x6c, 0x7a, 0xaf, 0x73, 0xaf, 0xc3,
+	0x7b, 0x03, 0x08, 0xb9, 0x08, 0x6d, 0xcb, 0x46, 0x62, 0xa7, 0x89, 0x13, 0x89, 0xa2, 0x6d, 0xc6,
+	0xb2, 0xcd, 0xae, 0xe4, 0xba, 0x4d, 0x9a, 0x2c, 0x56, 0xdc, 0x21, 0xbd, 0xf0, 0x72, 0x77, 0x33,
+	0xb3, 0x94, 0x2d, 0x37, 0x68, 0xd0, 0x3f, 0x68, 0xd1, 0xa2, 0x6d, 0x5e, 0xfa, 0x90, 0xf4, 0xad,
+	0x2d, 0x50, 0xa0, 0x2f, 0xfd, 0x00, 0xf9, 0x06, 0x05, 0x0a, 0xf4, 0x0b, 0xe4, 0xa5, 0x40, 0x0b,
+	0xb4, 0x4d, 0x9f, 0x0b, 0xf4, 0xad, 0x98, 0x3f, 0xfb, 0x87, 0x4b, 0xd2, 0x59, 0x92, 0x4a, 0xdf,
+	0x76, 0x66, 0xf6, 0xfc, 0x7e, 0x67, 0xce, 0x9e, 0x39, 0x73, 0xce, 0xcc, 0x42, 0xe5, 0x10, 0x1b,
+	0x7d, 0xbd, 0xeb, 0xe8, 0x86, 0x67, 0xd5, 0x3d, 0xe2, 0xfa, 0x2e, 0x7a, 0xc1, 0x25, 0xbd, 0xba,
+	0xe1, 0x19, 0x9d, 0x87, 0xb8, 0xce, 0x46, 0xeb, 0x7d, 0xd7, 0xc4, 0x76, 0xbd, 0xeb, 0xe8, 0xf8,
+	0x09, 0xee, 0x0c, 0x7c, 0xcb, 0x75, 0xea, 0x47, 0x97, 0xd4, 0x0d, 0x2e, 0x49, 0x06, 0x8e, 0x83,
+	0x49, 0x24, 0xad, 0xae, 0x62, 0xc7, 0xf4, 0x5c, 0xcb, 0xf1, 0xa9, 0xec, 0x38, 0xdf, 0x73, 0xdd,
+	0x9e, 0x8d, 0x2f, 0xf0, 0xd6, 0xe1, 0xa0, 0x7b, 0xc1, 0xc4, 0xb4, 0x43, 0x2c, 0xcf, 0x77, 0x89,
+	0x7c, 0xe3, 0x7f, 0x92, 0x6f, 0xf8, 0x56, 0x1f, 0x53, 0xdf, 0xe8, 0x7b, 0xf2, 0x85, 0xff, 0x1e,
+	0x81, 0x18, 0x10, 0x83, 0xeb, 0x31, 0x61, 0xfc, 0x31, 0x31, 0x3c, 0x0f, 0x93, 0x40, 0x85, 0xe5,
+	0x3e, 0xf6, 0x89, 0xd5, 0x91, 0xcd, 0xda, 0x2f, 0x14, 0x58, 0xd1, 0x70, 0xdf, 0xf5, 0xf1, 0x4d,
+	0xe2, 0x75, 0xda, 0x2e, 0xf1, 0x51, 0x1f, 0x4e, 0x1b, 0x9e, 0xa5, 0x53, 0x4c, 0x8e, 0xac, 0x0e,
+	0xd6, 0x23, 0x15, 0xab, 0xca, 0x79, 0x65, 0xb3, 0xb8, 0xf5, 0x4a, 0x7d, 0xbc, 0x51, 0x3c, 0xcb,
+	0xc3, 0xb6, 0xe5, 0xe0, 0xfa, 0xd1, 0xa5, 0xfa, 0xb6, 0x67, 0xed, 0x0b, 0xf9, 0xdd, 0x50, 0x5c,
+	0x5b, 0x37, 0xc6, 0xf4, 0xa2, 0xb3, 0x90, 0xef, 0xb8, 0x26, 0x26, 0xba, 0x65, 0x56, 0x33, 0xe7,
+	0x95, 0xcd, 0x82, 0xb6, 0xc4, 0xdb, 0x2d, 0xb3, 0xf6, 0x97, 0x1c, 0xa0, 0x96, 0x43, 0x7d, 0x32,
+	0xe8, 0xb0, 0x19, 0x6a, 0xf8, 0xfd, 0x01, 0xa6, 0x3e, 0x7a, 0x01, 0x56, 0xac, 0xa8, 0x97, 0xc9,
+	0x29, 0x5c, 0x6e, 0x39, 0xd6, 0xdb, 0x32, 0xd1, 0x7d, 0xc8, 0x13, 0xdc, 0xb3, 0xa8, 0x8f, 0x49,
+	0xf5, 0xaf, 0x4b, 0x5c, 0xf5, 0x97, 0xeb, 0xa9, 0xbe, 0x67, 0x5d, 0x93, 0x72, 0x92, 0xf1, 0xd6,
+	0x29, 0x2d, 0x84, 0x42, 0x18, 0x56, 0x3c, 0xe2, 0x76, 0x30, 0xa5, 0xfa, 0xe1, 0xc0, 0x31, 0x6d,
+	0x5c, 0xfd, 0x9b, 0x00, 0xff, 0x4a, 0x4a, 0xf0, 0xb6, 0x90, 0xde, 0xe1, 0xc2, 0x11, 0xc3, 0xb2,
+	0x17, 0xef, 0x47, 0xdf, 0x86, 0x33, 0xc3, 0x34, 0xba, 0x47, 0xdc, 0x1e, 0xc1, 0x94, 0x56, 0xff,
+	0x2e, 0xf8, 0x1a, 0xb3, 0xf0, 0xb5, 0x25, 0x48, 0xc4, 0xbb, 0xe1, 0x8d, 0x1b, 0x47, 0x03, 0x58,
+	0x4f, 0xf0, 0x53, 0xcf, 0xb6, 0xfc, 0xea, 0xe7, 0x82, 0xfc, 0xcd, 0x59, 0xc8, 0xf7, 0x19, 0x42,
+	0xc4, 0x8c, 0xbc, 0x91, 0x41, 0xf4, 0x10, 0x56, 0xbb, 0x96, 0x63, 0xd8, 0xd6, 0x53, 0x1c, 0x98,
+	0xf7, 0x1f, 0x82, 0xf1, 0xb5, 0x94, 0x8c, 0x37, 0xa4, 0x78, 0xd2, 0xbe, 0x2b, 0xdd, 0xa1, 0x81,
+	0x9d, 0x02, 0x2c, 0x11, 0x31, 0x58, 0xfb, 0xee, 0x02, 0xac, 0x0d, 0xf9, 0x19, 0xf5, 0x5c, 0x87,
+	0xe2, 0xb4, 0x8e, 0xb6, 0x0e, 0x0b, 0x98, 0x10, 0x97, 0x48, 0xf7, 0x15, 0x0d, 0xf4, 0xb5, 0x51,
+	0xf7, 0x7b, 0x65, 0x6a, 0xf7, 0x13, 0x8a, 0x0c, 0xf9, 0x5f, 0x77, 0x92, 0xff, 0xbd, 0x36, 0x9b,
+	0xff, 0x85, 0x14, 0x09, 0x07, 0xfc, 0xf0, 0x0b, 0x1d, 0x70, 0x77, 0x3e, 0x07, 0x0c, 0x89, 0x27,
+	0x78, 0xe0, 0xd1, 0xb3, 0x3d, 0x70, 0x7b, 0x0e, 0x0f, 0x0c, 0xa9, 0xc7, 0xb9, 0xa0, 0x35, 0xd1,
+	0x05, 0x5f, 0x9f, 0xd1, 0x05, 0x43, 0xba, 0xa4, 0x0f, 0x02, 0xf3, 0x11, 0x31, 0x5a, 0xfb, 0xa9,
+	0x02, 0xab, 0x89, 0xb8, 0x83, 0x9e, 0xc2, 0xd9, 0x84, 0x09, 0x86, 0xa2, 0x71, 0x76, 0xb3, 0xb8,
+	0x75, 0x7d, 0x16, 0x33, 0xc4, 0x82, 0xf2, 0x19, 0x6f, 0xfc, 0x40, 0x0d, 0x41, 0x39, 0xe9, 0x87,
+	0xb5, 0x5f, 0x03, 0x9c, 0x99, 0x00, 0x84, 0x56, 0x20, 0x13, 0x2e, 0x90, 0x8c, 0x65, 0x22, 0x07,
+	0xc0, 0x27, 0x86, 0x43, 0xbb, 0x2e, 0xe9, 0xd3, 0x6a, 0x86, 0x2b, 0x7b, 0x77, 0x3e, 0x65, 0xeb,
+	0x07, 0x21, 0x60, 0xd3, 0xf1, 0xc9, 0xb1, 0x16, 0x63, 0x40, 0x3e, 0x94, 0xbc, 0x8e, 0x6b, 0xdb,
+	0x98, 0x2f, 0x4b, 0x5a, 0xcd, 0x72, 0xc6, 0xf6, 0x9c, 0x8c, 0xed, 0x18, 0xa4, 0xe0, 0x1c, 0x62,
+	0x41, 0x3f, 0x56, 0x60, 0xfd, 0xb1, 0xe5, 0x98, 0xee, 0x63, 0xcb, 0xe9, 0xe9, 0xd4, 0x27, 0x86,
+	0x8f, 0x7b, 0x16, 0xa6, 0xd5, 0x1c, 0xa7, 0x7f, 0x30, 0x27, 0xfd, 0x83, 0x00, 0x7a, 0x3f, 0x44,
+	0x16, 0x5a, 0xac, 0x3d, 0x1e, 0x1d, 0x41, 0x87, 0xb0, 0xc8, 0xb7, 0x4e, 0x5a, 0x5d, 0xe0, 0xec,
+	0x6f, 0xcd, 0xc9, 0xde, 0xe0, 0x60, 0x82, 0x50, 0x22, 0x33, 0x33, 0x63, 0xe7, 0xc8, 0x22, 0xae,
+	0xd3, 0xc7, 0x8e, 0x4f, 0xab, 0x8b, 0x27, 0x62, 0xe6, 0x66, 0x0c, 0x52, 0x9a, 0x39, 0xce, 0x82,
+	0x9e, 0xc0, 0x39, 0xea, 0x1b, 0x3e, 0xd6, 0x27, 0x64, 0x26, 0x4b, 0xf3, 0x65, 0x26, 0x67, 0x39,
+	0xf8, 0xb8, 0x21, 0xd5, 0x86, 0xd5, 0x84, 0xd7, 0xa1, 0x32, 0x64, 0x1f, 0xe1, 0x63, 0xe9, 0xea,
+	0xec, 0x11, 0x35, 0x60, 0xe1, 0xc8, 0xb0, 0x07, 0x98, 0xef, 0x00, 0xc5, 0xad, 0x97, 0x52, 0xe8,
+	0xd1, 0x0e, 0x51, 0x35, 0x21, 0xfb, 0x6a, 0xe6, 0xaa, 0xa2, 0xba, 0x50, 0x19, 0xf1, 0xb8, 0x31,
+	0x7c, 0xbb, 0xc3, 0x7c, 0xf5, 0x34, 0x7c, 0x8d, 0x10, 0x36, 0x4e, 0xf8, 0x01, 0x54, 0x27, 0xf9,
+	0xd8, 0x18, 0xde, 0xb7, 0x86, 0x79, 0xaf, 0xa4, 0xe0, 0x4d, 0xa2, 0x1f, 0xc7, 0xd9, 0x3b, 0x50,
+	0x8c, 0xf9, 0xd8, 0x18, 0xc2, 0xeb, 0xc3, 0x84, 0x9b, 0x29, 0x08, 0x39, 0x60, 0xc2, 0xa6, 0x23,
+	0xee, 0x75, 0x32, 0x36, 0x8d, 0xc1, 0xc6, 0x08, 0x6b, 0xff, 0xca, 0x42, 0x45, 0x78, 0xf8, 0xb6,
+	0xe7, 0xd9, 0x56, 0x87, 0xa7, 0xe7, 0xe8, 0x79, 0x28, 0x85, 0xd1, 0x2a, 0x4a, 0x25, 0x8a, 0x61,
+	0x5f, 0xcb, 0x64, 0xa9, 0xb0, 0xe5, 0x78, 0x03, 0x3f, 0x96, 0x0a, 0xf3, 0x76, 0xcb, 0x44, 0x55,
+	0x58, 0xc2, 0x36, 0x66, 0x4c, 0xd5, 0xec, 0x79, 0x65, 0xb3, 0xa4, 0x05, 0x4d, 0xf4, 0x2d, 0xa8,
+	0xb8, 0x03, 0x9f, 0x49, 0x3d, 0x36, 0x7c, 0x4c, 0xfa, 0x06, 0x79, 0x14, 0x44, 0x9f, 0xb4, 0xe1,
+	0x76, 0x44, 0xd9, 0xfa, 0x3d, 0x8e, 0xf8, 0x20, 0x04, 0x14, 0x6b, 0xb2, 0xec, 0x26, 0xba, 0x51,
+	0x1b, 0xc0, 0xa2, 0xfa, 0xa1, 0x3b, 0x70, 0x4c, 0x6c, 0x56, 0x17, 0xce, 0x2b, 0x9b, 0x2b, 0x5b,
+	0x97, 0x52, 0x58, 0xae, 0x45, 0x77, 0x84, 0x4c, 0xbd, 0xe9, 0x0c, 0xfa, 0x5a, 0xc1, 0x0a, 0xda,
+	0xe8, 0x9b, 0x50, 0xee, 0xbb, 0x8e, 0xe5, 0xbb, 0x84, 0x05, 0x54, 0xcb, 0xe9, 0xba, 0x41, 0x8c,
+	0x49, 0x83, 0x7b, 0x27, 0x14, 0x6d, 0x39, 0x5d, 0x57, 0x5b, 0xed, 0x0f, 0xb5, 0xa9, 0xaa, 0xc3,
+	0xc6, 0xd8, 0xa9, 0x8d, 0xf1, 0x87, 0x8b, 0xc3, 0xfe, 0xa0, 0xd6, 0x45, 0x61, 0x55, 0x0f, 0x0a,
+	0xab, 0xfa, 0x41, 0x50, 0x99, 0xc5, 0xbf, 0xfd, 0x27, 0x19, 0xa8, 0xee, 0x62, 0xdb, 0x38, 0xc6,
+	0xe6, 0xa8, 0x0b, 0x1c, 0x40, 0x55, 0xa6, 0x9c, 0xd8, 0x8c, 0xbe, 0x80, 0xce, 0x4a, 0x3c, 0x59,
+	0x5b, 0x3d, 0x8b, 0xe5, 0x74, 0x28, 0xdb, 0x0c, 0x44, 0xd9, 0x20, 0x7a, 0x1b, 0x8a, 0x46, 0x44,
+	0x22, 0xd5, 0xbd, 0x3a, 0xeb, 0xa7, 0xd7, 0xe2, 0x60, 0xe8, 0x36, 0xac, 0x47, 0x1a, 0x33, 0x3d,
+	0x75, 0x93, 0x4d, 0x8e, 0xfb, 0x60, 0x71, 0xeb, 0xec, 0x88, 0xb6, 0xbb, 0xb2, 0x18, 0xd5, 0x50,
+	0x28, 0xc6, 0x74, 0xe4, 0x16, 0xa9, 0xfd, 0x2c, 0x07, 0xeb, 0xe3, 0x8a, 0x1f, 0xf4, 0x06, 0x9c,
+	0x9b, 0x98, 0xe6, 0x44, 0x4b, 0xe5, 0xec, 0x84, 0x4c, 0xa5, 0x65, 0x22, 0x0b, 0x4a, 0x1d, 0x36,
+	0x53, 0xdd, 0x77, 0x1f, 0x61, 0x27, 0xc8, 0x36, 0x6e, 0xcc, 0x51, 0x90, 0xd5, 0x1b, 0x4c, 0xea,
+	0x80, 0xc1, 0x69, 0xc5, 0x4e, 0xf8, 0x4c, 0xd5, 0xdf, 0x67, 0x00, 0xa2, 0x31, 0xf4, 0x3e, 0xc0,
+	0x80, 0x62, 0xa2, 0xf3, 0x0d, 0x44, 0x7e, 0xc4, 0xf6, 0xc9, 0xf0, 0xd6, 0xef, 0x53, 0x4c, 0xf6,
+	0x19, 0xee, 0xad, 0x53, 0x5a, 0x61, 0x10, 0x34, 0x18, 0x25, 0xb5, 0x4c, 0xac, 0xf3, 0xd0, 0x20,
+	0x3f, 0xf7, 0x49, 0x51, 0xee, 0x5b, 0x26, 0x6e, 0x31, 0x5c, 0x46, 0x49, 0x83, 0x06, 0xab, 0x70,
+	0xb8, 0x65, 0xab, 0xc0, 0x63, 0x8f, 0x68, 0xa8, 0x45, 0x28, 0x84, 0x2a, 0xaa, 0x2f, 0x42, 0x21,
+	0x14, 0x46, 0xcf, 0x0d, 0xa9, 0x28, 0x3e, 0x5f, 0x04, 0xb7, 0xb3, 0x08, 0x39, 0xff, 0xd8, 0xc3,
+	0xb5, 0xcf, 0x32, 0xb0, 0x31, 0xb6, 0x1a, 0x41, 0xb7, 0x60, 0x49, 0x9e, 0x53, 0x48, 0x9b, 0xd6,
+	0x53, 0x4e, 0xf0, 0x8e, 0x90, 0xd2, 0x02, 0x71, 0x56, 0x2e, 0x11, 0x4c, 0x2d, 0x73, 0x60, 0xd8,
+	0x3a, 0x71, 0x5d, 0x3f, 0x70, 0x8e, 0x37, 0x52, 0x02, 0x4e, 0x5a, 0xcc, 0xda, 0x72, 0x00, 0xab,
+	0x31, 0xd4, 0xb1, 0x71, 0x2b, 0x7b, 0x52, 0x71, 0x0b, 0x5d, 0x86, 0x0d, 0xb6, 0xa0, 0x2c, 0x82,
+	0xa9, 0x2e, 0x6b, 0x08, 0xb1, 0xda, 0x73, 0xe7, 0x95, 0xcd, 0xbc, 0xb6, 0x1e, 0x0c, 0xde, 0x88,
+	0x8d, 0xd5, 0x9a, 0x70, 0xee, 0x59, 0xb5, 0x7f, 0xca, 0xf2, 0xb6, 0xf6, 0xf1, 0x1a, 0x2c, 0x49,
+	0xb3, 0x22, 0x03, 0x8a, 0x5e, 0x2c, 0xab, 0x57, 0xa6, 0x32, 0xa5, 0x04, 0xa9, 0xb7, 0xfd, 0x44,
+	0x1a, 0x1f, 0xc7, 0x54, 0x3f, 0x2b, 0x02, 0x44, 0xc9, 0x11, 0x7a, 0x0a, 0x41, 0x8d, 0xc6, 0x62,
+	0xa6, 0xd8, 0xf3, 0x02, 0xa7, 0xb8, 0x3d, 0x2d, 0x71, 0x08, 0x1b, 0x2c, 0x04, 0x6c, 0x36, 0x25,
+	0xa4, 0x56, 0xf1, 0x92, 0x5d, 0xe8, 0x7d, 0x58, 0x35, 0x3a, 0xbe, 0x75, 0x84, 0x23, 0x62, 0xb1,
+	0xdc, 0x6e, 0xcd, 0x4e, 0xbc, 0xcd, 0x01, 0x43, 0xd6, 0x15, 0x63, 0xa8, 0x8d, 0x2c, 0x80, 0xd8,
+	0x36, 0x2e, 0x1c, 0xa8, 0x35, 0x3b, 0x5b, 0x72, 0x07, 0x8f, 0x81, 0xa3, 0x9b, 0x90, 0x63, 0x41,
+	0x45, 0xe6, 0x0a, 0x97, 0xa7, 0x24, 0x61, 0x2b, 0x5f, 0xe3, 0x00, 0xea, 0x9f, 0xb3, 0x90, 0xbf,
+	0x83, 0x0d, 0x3a, 0x20, 0xd8, 0x44, 0x3f, 0x51, 0x60, 0x5d, 0x24, 0x31, 0xd2, 0x66, 0x7a, 0xc7,
+	0x1d, 0x88, 0x4f, 0xc6, 0x68, 0xde, 0x9e, 0x7d, 0x2e, 0x01, 0x45, 0x9d, 0x07, 0x11, 0x69, 0xb1,
+	0x06, 0x07, 0x17, 0x93, 0x43, 0xd6, 0xc8, 0x00, 0xfa, 0x48, 0x81, 0x0d, 0x99, 0x1e, 0x25, 0xf4,
+	0x11, 0x61, 0xe0, 0x9d, 0x13, 0xd0, 0x47, 0x64, 0x14, 0x63, 0x14, 0x5a, 0x73, 0x47, 0x47, 0xd0,
+	0x26, 0x94, 0x7d, 0xd7, 0x37, 0x6c, 0xb1, 0x9d, 0x52, 0x2f, 0x48, 0xe9, 0x14, 0x6d, 0x85, 0xf7,
+	0xb3, 0xfd, 0x72, 0x9f, 0xf5, 0xaa, 0x4d, 0x38, 0x33, 0x61, 0xaa, 0x63, 0xd2, 0x95, 0xf5, 0x78,
+	0xba, 0x92, 0x8d, 0xe7, 0xbf, 0x37, 0xa0, 0x3a, 0x49, 0xc3, 0xa9, 0x70, 0x28, 0x54, 0x46, 0x56,
+	0x0d, 0x7a, 0x0f, 0xf2, 0x7d, 0x69, 0x07, 0xb9, 0x28, 0x77, 0xe6, 0xb7, 0xa8, 0x16, 0x62, 0xaa,
+	0x1f, 0x65, 0x61, 0x65, 0x78, 0xc9, 0x7c, 0xd9, 0x94, 0xe8, 0x25, 0x40, 0x5d, 0x62, 0x88, 0x98,
+	0x48, 0x70, 0xdf, 0xb0, 0x1c, 0xcb, 0xe9, 0x71, 0x73, 0x28, 0x5a, 0x25, 0x18, 0xd1, 0x82, 0x01,
+	0xf4, 0x4b, 0x05, 0xce, 0x0e, 0x7b, 0x18, 0x8d, 0x89, 0x89, 0x15, 0x8c, 0x4f, 0x2a, 0x5e, 0x0c,
+	0xfb, 0x1a, 0x0d, 0xb5, 0x10, 0xfe, 0x76, 0xc6, 0x1d, 0x3f, 0xaa, 0xbe, 0x05, 0xe7, 0x9e, 0x25,
+	0x38, 0x95, 0x1b, 0xbc, 0x0e, 0xab, 0x5f, 0x9c, 0x3c, 0x4f, 0x16, 0xff, 0xe3, 0x02, 0xe4, 0x58,
+	0xec, 0x40, 0x3a, 0x14, 0xc5, 0x1e, 0xad, 0x3b, 0x46, 0x98, 0xff, 0x5e, 0x9f, 0x21, 0x0a, 0xc9,
+	0xc6, 0x5d, 0xa3, 0x8f, 0x35, 0xe8, 0x87, 0xcf, 0x08, 0x43, 0x89, 0x2f, 0x75, 0x4c, 0x74, 0xd3,
+	0xf0, 0x8d, 0xe0, 0x98, 0xf4, 0x8d, 0x59, 0x28, 0x1a, 0x02, 0x68, 0xd7, 0xf0, 0x8d, 0x5b, 0xa7,
+	0xb4, 0x62, 0x27, 0x6a, 0x22, 0x1f, 0x2a, 0xa6, 0x45, 0x7d, 0x62, 0x1d, 0x8a, 0x6c, 0x9e, 0x73,
+	0x4d, 0x79, 0x42, 0x3a, 0xc4, 0xb5, 0x1b, 0x43, 0x93, 0x84, 0x65, 0x33, 0xd1, 0x87, 0x74, 0x80,
+	0x9e, 0x31, 0xe8, 0x61, 0x41, 0xf7, 0xf9, 0x74, 0xe7, 0x93, 0x43, 0x74, 0x37, 0x19, 0x8c, 0xe4,
+	0x29, 0xf4, 0x82, 0x86, 0x7a, 0x1d, 0x20, 0xb2, 0x2b, 0x3a, 0x07, 0x05, 0xf6, 0x95, 0xa8, 0x67,
+	0x74, 0xb0, 0x2c, 0x4d, 0xa3, 0x0e, 0x84, 0x20, 0xc7, 0xbf, 0x61, 0x96, 0x0f, 0xf0, 0x67, 0xf5,
+	0x7f, 0x59, 0x69, 0x1f, 0x59, 0x29, 0x74, 0x08, 0x25, 0xe6, 0x10, 0xea, 0x7b, 0x50, 0x4e, 0xce,
+	0x96, 0xbd, 0xc9, 0xcd, 0x1b, 0xbc, 0xc9, 0x1b, 0xcc, 0xc5, 0xe8, 0xa0, 0x2f, 0xdd, 0x89, 0x3d,
+	0xb2, 0x9e, 0xbe, 0xe5, 0x70, 0xce, 0xac, 0xc6, 0x1e, 0x79, 0x8f, 0xf1, 0x84, 0xa7, 0x44, 0xac,
+	0xc7, 0x78, 0xa2, 0xbe, 0x03, 0x85, 0x70, 0x7a, 0xe3, 0x55, 0x40, 0x57, 0xa1, 0x10, 0x5e, 0xb1,
+	0xa5, 0x28, 0xf5, 0xa2, 0x97, 0x59, 0x16, 0xcb, 0x8c, 0xaf, 0x1e, 0x43, 0x39, 0x99, 0xd1, 0x8c,
+	0x59, 0x11, 0xf7, 0x86, 0xcb, 0xc9, 0x6b, 0x33, 0x47, 0x84, 0x78, 0xb5, 0xf9, 0x9b, 0x0c, 0x3c,
+	0xf7, 0xcc, 0xd3, 0xf5, 0x13, 0x4c, 0xa4, 0xbf, 0xdc, 0x04, 0xf7, 0x5d, 0x58, 0xf6, 0x88, 0xd5,
+	0x37, 0xc8, 0xb1, 0xcc, 0xd2, 0x45, 0x56, 0x32, 0x7b, 0x19, 0x5b, 0x92, 0x70, 0x3c, 0x3b, 0xaf,
+	0x7d, 0x27, 0x07, 0x67, 0x27, 0x5e, 0x45, 0xa5, 0xbd, 0xe7, 0x79, 0x0a, 0x2b, 0x26, 0xa6, 0x16,
+	0xc1, 0xa6, 0xb8, 0x89, 0x08, 0xe6, 0xbf, 0x3f, 0xef, 0x5d, 0x58, 0x7d, 0x57, 0xc0, 0xf2, 0x3e,
+	0x99, 0x3b, 0x2c, 0x9b, 0xf1, 0x3e, 0xf5, 0x77, 0x0a, 0x94, 0xe2, 0x6f, 0xa1, 0x2d, 0xd8, 0x08,
+	0x77, 0x29, 0xb7, 0x2b, 0x77, 0x1c, 0x13, 0x8b, 0x4b, 0xda, 0x8c, 0xb6, 0x16, 0x0c, 0xde, 0xeb,
+	0x6a, 0xc1, 0x10, 0xba, 0x08, 0xeb, 0x86, 0x6d, 0xbb, 0x8f, 0x83, 0x09, 0xe8, 0xe2, 0x72, 0x9a,
+	0x4f, 0x23, 0xab, 0x21, 0x39, 0xc6, 0xf1, 0xdb, 0x7c, 0x04, 0x5d, 0x85, 0x2a, 0xa6, 0xbe, 0xd5,
+	0x37, 0x58, 0xfd, 0x3f, 0x94, 0xd6, 0x51, 0xb9, 0x16, 0x4f, 0x87, 0xe3, 0xf1, 0x5c, 0x85, 0xaa,
+	0x1f, 0x29, 0x80, 0x46, 0xa7, 0x35, 0x66, 0x61, 0x74, 0x86, 0x17, 0xc6, 0x9d, 0x13, 0x35, 0x66,
+	0x7c, 0xb1, 0xfc, 0x33, 0x0b, 0xea, 0xe4, 0xcb, 0xa0, 0x51, 0x0f, 0x54, 0x4e, 0xd2, 0x03, 0xff,
+	0x63, 0x75, 0xe8, 0x00, 0x56, 0x3a, 0x0f, 0x0d, 0xc7, 0xc1, 0xf6, 0xb0, 0x93, 0xde, 0x9d, 0xfb,
+	0xba, 0xac, 0xde, 0x10, 0xb8, 0xa2, 0x73, 0xb9, 0x13, 0x6b, 0x51, 0xf5, 0x13, 0x05, 0x4a, 0xf1,
+	0xf1, 0x34, 0xc7, 0x9d, 0x17, 0x61, 0xdd, 0x36, 0xa8, 0xaf, 0x07, 0x66, 0x0f, 0x0e, 0x38, 0x99,
+	0x23, 0x2c, 0x68, 0x88, 0x8d, 0xb5, 0xc5, 0x90, 0xf4, 0x2a, 0x74, 0x05, 0x4e, 0x77, 0x2d, 0x42,
+	0x7d, 0x3d, 0x34, 0x65, 0xfc, 0x50, 0x74, 0x41, 0x5b, 0xe7, 0xa3, 0x9a, 0x1c, 0x94, 0x52, 0xb5,
+	0xeb, 0xb0, 0x31, 0xf6, 0x52, 0x38, 0x6d, 0x01, 0x5c, 0x85, 0xd3, 0xe3, 0x6f, 0xf4, 0x6a, 0x9f,
+	0x2a, 0x90, 0x0f, 0xf3, 0xd2, 0x5b, 0x62, 0x3f, 0x90, 0x7e, 0x73, 0x25, 0xa5, 0xbd, 0xc3, 0xcc,
+	0x8e, 0xed, 0x51, 0x9a, 0xd8, 0x51, 0x4c, 0xc8, 0xf1, 0x1d, 0x2b, 0x65, 0x5c, 0x4a, 0x9a, 0x3a,
+	0x33, 0x6a, 0x6a, 0x24, 0x75, 0x13, 0x67, 0xc7, 0xfc, 0xb9, 0xf6, 0xf3, 0x2c, 0x94, 0xf8, 0xd9,
+	0x4d, 0x60, 0x8e, 0xe4, 0x0d, 0xde, 0x28, 0x7d, 0x66, 0x1c, 0xfd, 0x1e, 0x14, 0xc4, 0xdd, 0x0c,
+	0x5b, 0xd8, 0xe2, 0x60, 0xf0, 0x42, 0xca, 0xc9, 0x73, 0xfa, 0xdb, 0xf8, 0x58, 0xcb, 0x53, 0xf9,
+	0x84, 0x6e, 0x43, 0xb6, 0x87, 0xfd, 0x69, 0x7f, 0xd8, 0xe0, 0x40, 0x37, 0x71, 0xec, 0xe7, 0x02,
+	0x86, 0x82, 0x0e, 0x60, 0xd1, 0xf0, 0x3c, 0xec, 0x98, 0x41, 0xf2, 0x77, 0x6d, 0x1a, 0xbc, 0x6d,
+	0x2e, 0x1a, 0x41, 0x4a, 0x2c, 0xf4, 0x55, 0x58, 0xe8, 0xd8, 0xd8, 0x20, 0x41, 0x96, 0x77, 0x75,
+	0x1a, 0xd0, 0x06, 0x93, 0x8c, 0x30, 0x05, 0x52, 0xfc, 0x67, 0x84, 0x4f, 0x33, 0xb0, 0x2c, 0x3f,
+	0x8b, 0x8c, 0x4c, 0xc9, 0xef, 0x32, 0xfe, 0x7f, 0x83, 0xbd, 0x21, 0xc3, 0xbd, 0x32, 0xb5, 0xe1,
+	0xc2, 0x4b, 0x6a, 0x6e, 0xb9, 0xfb, 0x49, 0xcb, 0xbd, 0x3a, 0x8b, 0xe5, 0x42, 0xcc, 0xc0, 0x74,
+	0x5a, 0xc2, 0x74, 0xd7, 0x66, 0x30, 0x5d, 0x08, 0x2a, 0x6d, 0x17, 0xbf, 0x44, 0xff, 0xc3, 0x22,
+	0xe4, 0x03, 0xa7, 0x42, 0x6d, 0x58, 0x14, 0xbf, 0x64, 0xc9, 0xd4, 0xe7, 0xe5, 0x29, 0xbd, 0xb2,
+	0xae, 0x71, 0x69, 0xa6, 0xbe, 0xc0, 0x41, 0x14, 0xd6, 0xfa, 0x03, 0x9b, 0xed, 0x77, 0x9e, 0x3e,
+	0x72, 0x06, 0xbb, 0x3d, 0x2d, 0xfc, 0x1d, 0x09, 0x15, 0x3f, 0x74, 0xad, 0xf4, 0x93, 0x9d, 0xc8,
+	0x84, 0x95, 0x43, 0xa3, 0xa7, 0xc7, 0x8e, 0x99, 0xb3, 0x53, 0xfd, 0xef, 0x11, 0xf2, 0xed, 0x18,
+	0xbd, 0xf8, 0x91, 0x72, 0xe9, 0x30, 0xd6, 0x66, 0x53, 0xb3, 0x7c, 0x4c, 0x8c, 0x43, 0x1b, 0xc7,
+	0xa7, 0x96, 0x9b, 0x6d, 0x6a, 0x2d, 0x09, 0x35, 0x34, 0x35, 0x2b, 0xd9, 0xa9, 0xaa, 0xb0, 0x28,
+	0x6c, 0x1c, 0xcf, 0x0b, 0x4a, 0x3c, 0x2f, 0x50, 0xbf, 0xaf, 0x40, 0x65, 0xc4, 0x42, 0x69, 0xb6,
+	0x95, 0x1a, 0x2c, 0x47, 0x13, 0x88, 0xc5, 0xc3, 0xf0, 0xfc, 0xb9, 0x65, 0xa2, 0xd3, 0xb0, 0x28,
+	0x2e, 0xd0, 0x65, 0x44, 0x94, 0xad, 0x40, 0x8d, 0x5c, 0xa4, 0xc6, 0x87, 0x50, 0x8a, 0xdb, 0x2d,
+	0xa5, 0x02, 0xd1, 0xc7, 0x8a, 0x29, 0x10, 0x1e, 0xe1, 0x4f, 0xa5, 0x00, 0x81, 0xca, 0x88, 0x35,
+	0xbf, 0x64, 0x33, 0x84, 0x07, 0xf4, 0x6f, 0xc2, 0x6a, 0x22, 0xb2, 0xa2, 0x97, 0x00, 0x75, 0x5c,
+	0xc7, 0xb7, 0x9c, 0x81, 0x21, 0x2e, 0xaf, 0xf8, 0xbd, 0x80, 0xf8, 0x6e, 0x95, 0xf8, 0x08, 0xbf,
+	0x50, 0xa8, 0xdd, 0x87, 0x72, 0x32, 0xc4, 0x4c, 0x09, 0x11, 0xee, 0x5d, 0x99, 0xd8, 0xde, 0xb5,
+	0x09, 0x68, 0x34, 0x44, 0x87, 0x6f, 0x2a, 0xb1, 0x37, 0x37, 0x60, 0x6d, 0x4c, 0x48, 0xaa, 0xad,
+	0x41, 0x65, 0x24, 0x1c, 0xd7, 0xd6, 0x25, 0xea, 0x50, 0xa0, 0xa9, 0xfd, 0x2a, 0x07, 0xf9, 0x3d,
+	0x57, 0x9e, 0x94, 0x7c, 0x03, 0xf2, 0x14, 0x1f, 0x61, 0x62, 0xf9, 0xc2, 0x59, 0x57, 0x52, 0x17,
+	0xdd, 0x01, 0x44, 0x7d, 0x5f, 0xca, 0x8b, 0xab, 0xcf, 0x10, 0x6e, 0xf6, 0x4a, 0x14, 0x55, 0x59,
+	0x91, 0x47, 0xa9, 0xd1, 0x0b, 0x4a, 0xf0, 0xa0, 0xc9, 0x2f, 0x6e, 0x08, 0xab, 0xd9, 0x73, 0x62,
+	0xab, 0xe0, 0x8d, 0x31, 0x1b, 0xfb, 0x42, 0x9a, 0xbc, 0x62, 0x71, 0xd4, 0xc9, 0x9e, 0x87, 0x92,
+	0xed, 0xf6, 0x74, 0xdb, 0x95, 0x97, 0x8f, 0x4b, 0xe2, 0x15, 0xdb, 0xed, 0xed, 0xc9, 0x2e, 0xe6,
+	0x63, 0xfe, 0x43, 0x82, 0x0d, 0xb3, 0x9a, 0xe7, 0x83, 0xb2, 0xa5, 0x7e, 0x1d, 0x72, 0x7b, 0x16,
+	0xf5, 0x51, 0x1b, 0xd8, 0xeb, 0x3a, 0x76, 0x7c, 0x62, 0xe1, 0x20, 0xeb, 0xbe, 0x30, 0xa5, 0x51,
+	0x35, 0xb0, 0xc5, 0x93, 0x85, 0xa9, 0x4a, 0x20, 0x1f, 0xd8, 0xb8, 0xd6, 0x85, 0x1c, 0x33, 0x33,
+	0x5a, 0x85, 0xe2, 0xfd, 0xbb, 0xfb, 0xed, 0x66, 0xa3, 0x75, 0xa3, 0xd5, 0xdc, 0x2d, 0x9f, 0x42,
+	0x05, 0x58, 0x38, 0xd0, 0xb6, 0x1b, 0xcd, 0xb2, 0xc2, 0x1e, 0x77, 0x9b, 0x3b, 0xf7, 0x6f, 0x96,
+	0x33, 0x28, 0x0f, 0xb9, 0xd6, 0xdd, 0x1b, 0xf7, 0xca, 0x59, 0x04, 0xb0, 0x78, 0xf7, 0xde, 0x41,
+	0xab, 0xd1, 0x2c, 0xe7, 0x58, 0xef, 0x83, 0x6d, 0xed, 0x6e, 0x79, 0x81, 0xbd, 0xda, 0xd4, 0xb4,
+	0x7b, 0x5a, 0x79, 0x11, 0x95, 0x20, 0xdf, 0xd0, 0x5a, 0x07, 0xad, 0xc6, 0xf6, 0x5e, 0x79, 0xa9,
+	0x56, 0x02, 0xd8, 0x73, 0x7b, 0x0d, 0xd7, 0xf1, 0x89, 0x6b, 0xd7, 0xfe, 0x94, 0xe3, 0x9e, 0x44,
+	0xfc, 0x07, 0x2e, 0x79, 0x14, 0xfd, 0xce, 0xf5, 0x5f, 0x50, 0x78, 0xcc, 0x3b, 0xa2, 0x25, 0x9b,
+	0x17, 0x1d, 0x2d, 0x13, 0x1d, 0x42, 0xb9, 0x23, 0xc4, 0xf5, 0xe0, 0xb7, 0x61, 0xe9, 0x05, 0x33,
+	0xff, 0xd6, 0xb2, 0x2a, 0x01, 0x9b, 0x12, 0x8f, 0x71, 0xd8, 0x6e, 0xaf, 0xc7, 0x0a, 0xf8, 0x90,
+	0x23, 0x3b, 0x27, 0x87, 0x04, 0x0c, 0x39, 0x4c, 0xa8, 0x18, 0xc4, 0xb7, 0xba, 0x46, 0xc7, 0x8f,
+	0x48, 0x72, 0xf3, 0x91, 0x94, 0x03, 0xc4, 0x90, 0xa5, 0xcb, 0xaf, 0x85, 0x8e, 0x2c, 0xca, 0x1c,
+	0x38, 0xa4, 0x59, 0x98, 0x8f, 0xa6, 0x12, 0x42, 0x86, 0x3c, 0xef, 0xc2, 0xa2, 0x67, 0x10, 0xa3,
+	0x4f, 0xab, 0xc0, 0x1d, 0xb3, 0x99, 0x7e, 0x27, 0x4c, 0x7c, 0xfd, 0x7a, 0x9b, 0xe3, 0xc8, 0xbf,
+	0xa9, 0x04, 0xa8, 0x7a, 0x0d, 0x8a, 0xb1, 0xee, 0x2f, 0x3a, 0x48, 0x2d, 0xc4, 0xcb, 0xd9, 0xff,
+	0xe7, 0x81, 0x2d, 0x22, 0x91, 0xc1, 0x35, 0x4c, 0x0e, 0x95, 0x58, 0x72, 0x58, 0xbb, 0xc8, 0xc2,
+	0x9d, 0xeb, 0xa5, 0x77, 0xc7, 0xda, 0x8b, 0xcc, 0x83, 0x23, 0x89, 0x67, 0xa1, 0x6f, 0x7d, 0xac,
+	0xc0, 0xf2, 0x0e, 0x36, 0xfa, 0x37, 0x1c, 0xb9, 0x00, 0xd0, 0x0f, 0x14, 0x58, 0x0a, 0x9e, 0xd3,
+	0x66, 0x8e, 0x63, 0xfe, 0xc0, 0x55, 0xaf, 0xcd, 0x22, 0x2b, 0x82, 0xf9, 0xa9, 0x4d, 0xe5, 0xa2,
+	0xb2, 0xf5, 0x01, 0x80, 0xd0, 0x8c, 0x17, 0x54, 0x8e, 0x2c, 0xac, 0x2e, 0x4c, 0x59, 0x9c, 0xa9,
+	0xd3, 0x0a, 0x48, 0xf6, 0x1f, 0x2a, 0x50, 0x14, 0xf4, 0x22, 0x79, 0x78, 0x02, 0x0b, 0xe2, 0xe1,
+	0xf2, 0x34, 0x09, 0x95, 0x9c, 0x91, 0x7a, 0x65, 0x3a, 0x21, 0xb9, 0x7d, 0x09, 0x4d, 0x7e, 0x14,
+	0x7e, 0xa2, 0x3d, 0xb1, 0x5e, 0xd1, 0x13, 0x58, 0x0a, 0x1e, 0xaf, 0x4c, 0xbb, 0x85, 0xb1, 0xc0,
+	0xad, 0x5e, 0x4a, 0x2f, 0x15, 0xc4, 0x45, 0xa1, 0xcb, 0x6f, 0x33, 0x50, 0x15, 0xba, 0x34, 0x9f,
+	0xf8, 0x98, 0x38, 0x86, 0x2d, 0xbc, 0xac, 0xed, 0x0a, 0xcf, 0x29, 0xc6, 0xfc, 0x1a, 0x5d, 0x9b,
+	0x79, 0xc1, 0xa9, 0xaf, 0xce, 0x22, 0x1a, 0x58, 0x0d, 0x7d, 0x4f, 0x01, 0x88, 0x56, 0x00, 0x4a,
+	0x5f, 0xe4, 0x25, 0x96, 0x99, 0x7a, 0x6d, 0x06, 0xc9, 0x40, 0x8b, 0x9d, 0x6d, 0xf8, 0xbf, 0x49,
+	0xd2, 0x71, 0xe1, 0x9d, 0x82, 0x30, 0xe8, 0xb6, 0x67, 0xbd, 0xbd, 0x12, 0x1b, 0xd2, 0x8f, 0x2e,
+	0x1d, 0x2e, 0xf2, 0xe4, 0xe1, 0xf2, 0xbf, 0x03, 0x00, 0x00, 0xff, 0xff, 0xd0, 0x49, 0x45, 0xe8,
+	0xdf, 0x32, 0x00, 0x00,
 }
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 55f8b16..661c75f 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
@@ -57,23 +57,36 @@
 	return proto.EnumName(JobMessage_MessageImportance_name, int32(x))
 }
 func (JobMessage_MessageImportance) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_beam_job_api_0a1706a5eaabebe4, []int{14, 0}
+	return fileDescriptor_beam_job_api_cf64c696c499a6a1, []int{14, 0}
 }
 
 type JobState_Enum int32
 
 const (
+	// The job state reported by a runner cannot be interpreted by the SDK.
 	JobState_UNSPECIFIED JobState_Enum = 0
-	JobState_STOPPED     JobState_Enum = 1
-	JobState_RUNNING     JobState_Enum = 2
-	JobState_DONE        JobState_Enum = 3
-	JobState_FAILED      JobState_Enum = 4
-	JobState_CANCELLED   JobState_Enum = 5
-	JobState_UPDATED     JobState_Enum = 6
-	JobState_DRAINING    JobState_Enum = 7
-	JobState_DRAINED     JobState_Enum = 8
-	JobState_STARTING    JobState_Enum = 9
-	JobState_CANCELLING  JobState_Enum = 10
+	// The job has not yet started.
+	JobState_STOPPED JobState_Enum = 1
+	// The job is currently running.
+	JobState_RUNNING JobState_Enum = 2
+	// The job has successfully completed. (terminal)
+	JobState_DONE JobState_Enum = 3
+	// The job has failed. (terminal)
+	JobState_FAILED JobState_Enum = 4
+	// The job has been explicitly cancelled. (terminal)
+	JobState_CANCELLED JobState_Enum = 5
+	// The job has been updated. (terminal)
+	JobState_UPDATED JobState_Enum = 6
+	// The job is draining its data. (optional)
+	JobState_DRAINING JobState_Enum = 7
+	// The job has completed draining its data. (terminal)
+	JobState_DRAINED JobState_Enum = 8
+	// The job is starting up.
+	JobState_STARTING JobState_Enum = 9
+	// The job is cancelling. (optional)
+	JobState_CANCELLING JobState_Enum = 10
+	// The job is in the process of being updated. (optional)
+	JobState_UPDATING JobState_Enum = 11
 )
 
 var JobState_Enum_name = map[int32]string{
@@ -88,6 +101,7 @@
 	8:  "DRAINED",
 	9:  "STARTING",
 	10: "CANCELLING",
+	11: "UPDATING",
 }
 var JobState_Enum_value = map[string]int32{
 	"UNSPECIFIED": 0,
@@ -101,13 +115,14 @@
 	"DRAINED":     8,
 	"STARTING":    9,
 	"CANCELLING":  10,
+	"UPDATING":    11,
 }
 
 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_0a1706a5eaabebe4, []int{16, 0}
+	return fileDescriptor_beam_job_api_cf64c696c499a6a1, []int{16, 0}
 }
 
 type PipelineOptionType_Enum int32
@@ -143,7 +158,7 @@
 	return proto.EnumName(PipelineOptionType_Enum_name, int32(x))
 }
 func (PipelineOptionType_Enum) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_beam_job_api_0a1706a5eaabebe4, []int{21, 0}
+	return fileDescriptor_beam_job_api_cf64c696c499a6a1, []int{21, 0}
 }
 
 // Prepare is a synchronous request that returns a preparationId back
@@ -163,7 +178,7 @@
 func (m *PrepareJobRequest) String() string { return proto.CompactTextString(m) }
 func (*PrepareJobRequest) ProtoMessage()    {}
 func (*PrepareJobRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_job_api_0a1706a5eaabebe4, []int{0}
+	return fileDescriptor_beam_job_api_cf64c696c499a6a1, []int{0}
 }
 func (m *PrepareJobRequest) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_PrepareJobRequest.Unmarshal(m, b)
@@ -223,7 +238,7 @@
 func (m *PrepareJobResponse) String() string { return proto.CompactTextString(m) }
 func (*PrepareJobResponse) ProtoMessage()    {}
 func (*PrepareJobResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_job_api_0a1706a5eaabebe4, []int{1}
+	return fileDescriptor_beam_job_api_cf64c696c499a6a1, []int{1}
 }
 func (m *PrepareJobResponse) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_PrepareJobResponse.Unmarshal(m, b)
@@ -284,7 +299,7 @@
 func (m *RunJobRequest) String() string { return proto.CompactTextString(m) }
 func (*RunJobRequest) ProtoMessage()    {}
 func (*RunJobRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_job_api_0a1706a5eaabebe4, []int{2}
+	return fileDescriptor_beam_job_api_cf64c696c499a6a1, []int{2}
 }
 func (m *RunJobRequest) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_RunJobRequest.Unmarshal(m, b)
@@ -329,7 +344,7 @@
 func (m *RunJobResponse) String() string { return proto.CompactTextString(m) }
 func (*RunJobResponse) ProtoMessage()    {}
 func (*RunJobResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_job_api_0a1706a5eaabebe4, []int{3}
+	return fileDescriptor_beam_job_api_cf64c696c499a6a1, []int{3}
 }
 func (m *RunJobResponse) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_RunJobResponse.Unmarshal(m, b)
@@ -370,7 +385,7 @@
 func (m *CancelJobRequest) String() string { return proto.CompactTextString(m) }
 func (*CancelJobRequest) ProtoMessage()    {}
 func (*CancelJobRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_job_api_0a1706a5eaabebe4, []int{4}
+	return fileDescriptor_beam_job_api_cf64c696c499a6a1, []int{4}
 }
 func (m *CancelJobRequest) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_CancelJobRequest.Unmarshal(m, b)
@@ -409,7 +424,7 @@
 func (m *CancelJobResponse) String() string { return proto.CompactTextString(m) }
 func (*CancelJobResponse) ProtoMessage()    {}
 func (*CancelJobResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_job_api_0a1706a5eaabebe4, []int{5}
+	return fileDescriptor_beam_job_api_cf64c696c499a6a1, []int{5}
 }
 func (m *CancelJobResponse) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_CancelJobResponse.Unmarshal(m, b)
@@ -451,7 +466,7 @@
 func (m *JobInfo) String() string { return proto.CompactTextString(m) }
 func (*JobInfo) ProtoMessage()    {}
 func (*JobInfo) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_job_api_0a1706a5eaabebe4, []int{6}
+	return fileDescriptor_beam_job_api_cf64c696c499a6a1, []int{6}
 }
 func (m *JobInfo) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_JobInfo.Unmarshal(m, b)
@@ -511,7 +526,7 @@
 func (m *GetJobsRequest) String() string { return proto.CompactTextString(m) }
 func (*GetJobsRequest) ProtoMessage()    {}
 func (*GetJobsRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_job_api_0a1706a5eaabebe4, []int{7}
+	return fileDescriptor_beam_job_api_cf64c696c499a6a1, []int{7}
 }
 func (m *GetJobsRequest) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_GetJobsRequest.Unmarshal(m, b)
@@ -542,7 +557,7 @@
 func (m *GetJobsResponse) String() string { return proto.CompactTextString(m) }
 func (*GetJobsResponse) ProtoMessage()    {}
 func (*GetJobsResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_job_api_0a1706a5eaabebe4, []int{8}
+	return fileDescriptor_beam_job_api_cf64c696c499a6a1, []int{8}
 }
 func (m *GetJobsResponse) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_GetJobsResponse.Unmarshal(m, b)
@@ -583,7 +598,7 @@
 func (m *GetJobStateRequest) String() string { return proto.CompactTextString(m) }
 func (*GetJobStateRequest) ProtoMessage()    {}
 func (*GetJobStateRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_job_api_0a1706a5eaabebe4, []int{9}
+	return fileDescriptor_beam_job_api_cf64c696c499a6a1, []int{9}
 }
 func (m *GetJobStateRequest) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_GetJobStateRequest.Unmarshal(m, b)
@@ -621,7 +636,7 @@
 func (m *GetJobStateResponse) String() string { return proto.CompactTextString(m) }
 func (*GetJobStateResponse) ProtoMessage()    {}
 func (*GetJobStateResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_job_api_0a1706a5eaabebe4, []int{10}
+	return fileDescriptor_beam_job_api_cf64c696c499a6a1, []int{10}
 }
 func (m *GetJobStateResponse) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_GetJobStateResponse.Unmarshal(m, b)
@@ -662,7 +677,7 @@
 func (m *GetJobPipelineRequest) String() string { return proto.CompactTextString(m) }
 func (*GetJobPipelineRequest) ProtoMessage()    {}
 func (*GetJobPipelineRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_job_api_0a1706a5eaabebe4, []int{11}
+	return fileDescriptor_beam_job_api_cf64c696c499a6a1, []int{11}
 }
 func (m *GetJobPipelineRequest) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_GetJobPipelineRequest.Unmarshal(m, b)
@@ -700,7 +715,7 @@
 func (m *GetJobPipelineResponse) String() string { return proto.CompactTextString(m) }
 func (*GetJobPipelineResponse) ProtoMessage()    {}
 func (*GetJobPipelineResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_job_api_0a1706a5eaabebe4, []int{12}
+	return fileDescriptor_beam_job_api_cf64c696c499a6a1, []int{12}
 }
 func (m *GetJobPipelineResponse) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_GetJobPipelineResponse.Unmarshal(m, b)
@@ -742,7 +757,7 @@
 func (m *JobMessagesRequest) String() string { return proto.CompactTextString(m) }
 func (*JobMessagesRequest) ProtoMessage()    {}
 func (*JobMessagesRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_job_api_0a1706a5eaabebe4, []int{13}
+	return fileDescriptor_beam_job_api_cf64c696c499a6a1, []int{13}
 }
 func (m *JobMessagesRequest) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_JobMessagesRequest.Unmarshal(m, b)
@@ -783,7 +798,7 @@
 func (m *JobMessage) String() string { return proto.CompactTextString(m) }
 func (*JobMessage) ProtoMessage()    {}
 func (*JobMessage) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_job_api_0a1706a5eaabebe4, []int{14}
+	return fileDescriptor_beam_job_api_cf64c696c499a6a1, []int{14}
 }
 func (m *JobMessage) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_JobMessage.Unmarshal(m, b)
@@ -845,7 +860,7 @@
 func (m *JobMessagesResponse) String() string { return proto.CompactTextString(m) }
 func (*JobMessagesResponse) ProtoMessage()    {}
 func (*JobMessagesResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_job_api_0a1706a5eaabebe4, []int{15}
+	return fileDescriptor_beam_job_api_cf64c696c499a6a1, []int{15}
 }
 func (m *JobMessagesResponse) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_JobMessagesResponse.Unmarshal(m, b)
@@ -975,6 +990,16 @@
 }
 
 // Enumeration of all JobStates
+//
+// The state transition diagram is:
+//   STOPPED -> STARTING -> RUNNING -> DONE
+//                                  \> FAILED
+//                                  \> CANCELLING -> CANCELLED
+//                                  \> UPDATING -> UPDATED
+//                                  \> DRAINING -> DRAINED
+//
+// Transitions are optional such that a job may go from STOPPED to RUNNING
+// without needing to pass through STARTING.
 type JobState struct {
 	XXX_NoUnkeyedLiteral struct{} `json:"-"`
 	XXX_unrecognized     []byte   `json:"-"`
@@ -985,7 +1010,7 @@
 func (m *JobState) String() string { return proto.CompactTextString(m) }
 func (*JobState) ProtoMessage()    {}
 func (*JobState) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_job_api_0a1706a5eaabebe4, []int{16}
+	return fileDescriptor_beam_job_api_cf64c696c499a6a1, []int{16}
 }
 func (m *JobState) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_JobState.Unmarshal(m, b)
@@ -1016,7 +1041,7 @@
 func (m *GetJobMetricsRequest) String() string { return proto.CompactTextString(m) }
 func (*GetJobMetricsRequest) ProtoMessage()    {}
 func (*GetJobMetricsRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_job_api_0a1706a5eaabebe4, []int{17}
+	return fileDescriptor_beam_job_api_cf64c696c499a6a1, []int{17}
 }
 func (m *GetJobMetricsRequest) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_GetJobMetricsRequest.Unmarshal(m, b)
@@ -1054,7 +1079,7 @@
 func (m *GetJobMetricsResponse) String() string { return proto.CompactTextString(m) }
 func (*GetJobMetricsResponse) ProtoMessage()    {}
 func (*GetJobMetricsResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_job_api_0a1706a5eaabebe4, []int{18}
+	return fileDescriptor_beam_job_api_cf64c696c499a6a1, []int{18}
 }
 func (m *GetJobMetricsResponse) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_GetJobMetricsResponse.Unmarshal(m, b)
@@ -1094,7 +1119,7 @@
 func (m *MetricResults) String() string { return proto.CompactTextString(m) }
 func (*MetricResults) ProtoMessage()    {}
 func (*MetricResults) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_job_api_0a1706a5eaabebe4, []int{19}
+	return fileDescriptor_beam_job_api_cf64c696c499a6a1, []int{19}
 }
 func (m *MetricResults) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_MetricResults.Unmarshal(m, b)
@@ -1142,7 +1167,7 @@
 func (m *DescribePipelineOptionsRequest) String() string { return proto.CompactTextString(m) }
 func (*DescribePipelineOptionsRequest) ProtoMessage()    {}
 func (*DescribePipelineOptionsRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_job_api_0a1706a5eaabebe4, []int{20}
+	return fileDescriptor_beam_job_api_cf64c696c499a6a1, []int{20}
 }
 func (m *DescribePipelineOptionsRequest) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_DescribePipelineOptionsRequest.Unmarshal(m, b)
@@ -1174,7 +1199,7 @@
 func (m *PipelineOptionType) String() string { return proto.CompactTextString(m) }
 func (*PipelineOptionType) ProtoMessage()    {}
 func (*PipelineOptionType) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_job_api_0a1706a5eaabebe4, []int{21}
+	return fileDescriptor_beam_job_api_cf64c696c499a6a1, []int{21}
 }
 func (m *PipelineOptionType) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_PipelineOptionType.Unmarshal(m, b)
@@ -1215,7 +1240,7 @@
 func (m *PipelineOptionDescriptor) String() string { return proto.CompactTextString(m) }
 func (*PipelineOptionDescriptor) ProtoMessage()    {}
 func (*PipelineOptionDescriptor) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_job_api_0a1706a5eaabebe4, []int{22}
+	return fileDescriptor_beam_job_api_cf64c696c499a6a1, []int{22}
 }
 func (m *PipelineOptionDescriptor) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_PipelineOptionDescriptor.Unmarshal(m, b)
@@ -1282,7 +1307,7 @@
 func (m *DescribePipelineOptionsResponse) String() string { return proto.CompactTextString(m) }
 func (*DescribePipelineOptionsResponse) ProtoMessage()    {}
 func (*DescribePipelineOptionsResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_job_api_0a1706a5eaabebe4, []int{23}
+	return fileDescriptor_beam_job_api_cf64c696c499a6a1, []int{23}
 }
 func (m *DescribePipelineOptionsResponse) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_DescribePipelineOptionsResponse.Unmarshal(m, b)
@@ -1785,96 +1810,97 @@
 	Metadata: "beam_job_api.proto",
 }
 
-func init() { proto.RegisterFile("beam_job_api.proto", fileDescriptor_beam_job_api_0a1706a5eaabebe4) }
+func init() { proto.RegisterFile("beam_job_api.proto", fileDescriptor_beam_job_api_cf64c696c499a6a1) }
 
-var fileDescriptor_beam_job_api_0a1706a5eaabebe4 = []byte{
-	// 1401 bytes of a gzipped FileDescriptorProto
-	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x57, 0xdd, 0x8e, 0xdb, 0x44,
-	0x14, 0xae, 0xf3, 0x9f, 0x93, 0x26, 0xeb, 0x9d, 0xed, 0xb2, 0x69, 0x04, 0x65, 0x31, 0x82, 0x16,
-	0x55, 0xb8, 0xdd, 0x54, 0xa2, 0xd0, 0x42, 0xc1, 0xd9, 0xb8, 0x69, 0xc2, 0x6e, 0x12, 0x4d, 0xb2,
-	0x20, 0x40, 0x22, 0x38, 0xc9, 0x6c, 0x70, 0x89, 0x3d, 0xc6, 0x9e, 0x44, 0xad, 0x84, 0x40, 0x02,
-	0x71, 0x09, 0xbc, 0x00, 0x2f, 0x80, 0x84, 0x84, 0xb8, 0xe6, 0x19, 0x78, 0x08, 0x24, 0xae, 0x78,
-	0x05, 0x6e, 0xd0, 0xd8, 0xe3, 0x6c, 0xbc, 0x3f, 0x6c, 0x92, 0x16, 0x71, 0x15, 0xcf, 0xf9, 0xf9,
-	0xce, 0xef, 0x9c, 0x39, 0x01, 0xd4, 0x27, 0x86, 0xd5, 0x7b, 0x48, 0xfb, 0x3d, 0xc3, 0x31, 0x55,
-	0xc7, 0xa5, 0x8c, 0xa2, 0xab, 0xd4, 0x1d, 0xa9, 0x86, 0x63, 0x0c, 0x3e, 0x25, 0x2a, 0x67, 0xab,
-	0x16, 0x1d, 0x92, 0xb1, 0xca, 0x85, 0x2c, 0xc3, 0x36, 0x46, 0xc4, 0x22, 0x36, 0x53, 0xa7, 0x3b,
-	0xa5, 0x4d, 0x5f, 0xd9, 0x9d, 0xd8, 0x36, 0x71, 0x8f, 0xf4, 0x4b, 0x6b, 0xc4, 0x1e, 0x3a, 0xd4,
-	0xb4, 0x99, 0x27, 0x08, 0xcf, 0x8e, 0x28, 0x1d, 0x8d, 0xc9, 0x0d, 0xff, 0xd4, 0x9f, 0x1c, 0xde,
-	0xf0, 0x98, 0x3b, 0x19, 0x30, 0xc1, 0xcd, 0x5b, 0x84, 0xb9, 0xe6, 0x40, 0x08, 0x2b, 0xbf, 0x49,
-	0xb0, 0xde, 0x76, 0x89, 0x63, 0xb8, 0xa4, 0x41, 0xfb, 0x98, 0x7c, 0x3e, 0x21, 0x1e, 0x43, 0x35,
-	0xc8, 0x38, 0xa6, 0x43, 0xc6, 0xa6, 0x4d, 0x8a, 0xd2, 0xb6, 0x74, 0x2d, 0x57, 0xbe, 0xae, 0x9e,
-	0xee, 0x66, 0x28, 0xa6, 0x4e, 0x77, 0xd4, 0xb6, 0xf8, 0xc6, 0x33, 0x65, 0x54, 0x01, 0x39, 0xfc,
-	0xee, 0x51, 0x87, 0x99, 0xd4, 0xf6, 0x8a, 0x31, 0x1f, 0x70, 0x4b, 0x0d, 0xdc, 0x54, 0x43, 0x37,
-	0xd5, 0x8e, 0xef, 0x26, 0x5e, 0x0b, 0x15, 0x5a, 0x81, 0x3c, 0xba, 0x0c, 0x19, 0x9e, 0x0c, 0xdb,
-	0xb0, 0x48, 0x31, 0xbe, 0x2d, 0x5d, 0xcb, 0xe2, 0xf4, 0x43, 0xda, 0x6f, 0x1a, 0x16, 0x51, 0xfe,
-	0x94, 0x00, 0xcd, 0x7b, 0xef, 0x39, 0xd4, 0xf6, 0x08, 0x7a, 0x09, 0x0a, 0x8e, 0x4f, 0x35, 0x38,
-	0x42, 0xcf, 0x1c, 0xfa, 0x41, 0x64, 0x71, 0x7e, 0x8e, 0x5a, 0x1f, 0x22, 0x0f, 0x2e, 0x1b, 0x2e,
-	0x33, 0x0f, 0x8d, 0x01, 0xeb, 0x79, 0xcc, 0x18, 0x99, 0xf6, 0xa8, 0x17, 0x26, 0x53, 0x78, 0x79,
-	0x7b, 0x81, 0xb0, 0x35, 0xc7, 0xec, 0x10, 0x77, 0x6a, 0x0e, 0x48, 0x95, 0x78, 0x03, 0xd7, 0x74,
-	0x18, 0x75, 0xf1, 0x56, 0x88, 0xdc, 0x09, 0x80, 0x75, 0x81, 0x8b, 0xca, 0xb0, 0x19, 0xda, 0xf2,
-	0x88, 0xe7, 0x71, 0xff, 0x18, 0xfd, 0x8c, 0xd8, 0x22, 0xb4, 0x0d, 0xc1, 0xec, 0x04, 0xbc, 0x2e,
-	0x67, 0x29, 0x3d, 0xc8, 0xe3, 0x89, 0x3d, 0x57, 0x9f, 0x05, 0x03, 0xbc, 0x0a, 0x6b, 0x2e, 0xaf,
-	0x36, 0x99, 0x1a, 0x63, 0x61, 0x25, 0xe6, 0xcb, 0x15, 0x66, 0xe4, 0xc0, 0xc0, 0x55, 0x28, 0x84,
-	0x06, 0x44, 0x0a, 0x37, 0x21, 0xc5, 0x93, 0x3e, 0x43, 0x4e, 0x3e, 0xa4, 0xfd, 0xfa, 0x50, 0x79,
-	0x05, 0xe4, 0x5d, 0xc3, 0x1e, 0x90, 0xf1, 0x9c, 0x33, 0x67, 0x88, 0x1a, 0xb0, 0x3e, 0x27, 0x2a,
-	0x60, 0xf7, 0x20, 0xe9, 0x31, 0x83, 0x05, 0x5d, 0x55, 0x28, 0xbf, 0xa6, 0x2e, 0xd8, 0xfc, 0x6a,
-	0x83, 0xf6, 0x3b, 0x5c, 0x51, 0xd5, 0xed, 0x89, 0x85, 0x03, 0x10, 0xe5, 0x77, 0x09, 0xd2, 0x0d,
-	0xda, 0xaf, 0xdb, 0x87, 0xf4, 0x0c, 0x2f, 0x22, 0xcd, 0x13, 0x8b, 0x34, 0xcf, 0xa9, 0xbd, 0x19,
-	0x5f, 0xb2, 0x37, 0x67, 0xf1, 0x24, 0x9e, 0x46, 0x3c, 0x32, 0x14, 0x6a, 0x84, 0x35, 0x68, 0xdf,
-	0x13, 0xb9, 0x55, 0x3e, 0x86, 0xb5, 0x19, 0x45, 0xa4, 0xf0, 0xdd, 0x20, 0x22, 0xd3, 0x3e, 0xa4,
-	0x45, 0x69, 0x3b, 0x7e, 0x2d, 0x57, 0xbe, 0xb9, 0x8c, 0x55, 0x9e, 0x2c, 0x3f, 0x07, 0xfc, 0x43,
-	0xb9, 0x0e, 0x28, 0xc0, 0xf7, 0x9d, 0x39, 0xa7, 0xa2, 0x03, 0xd8, 0x88, 0x08, 0xff, 0x27, 0x35,
-	0x55, 0x61, 0x33, 0x30, 0x32, 0x9b, 0x26, 0xe7, 0xb5, 0xd9, 0x33, 0xc7, 0xe5, 0x85, 0x5f, 0x4f,
-	0x6b, 0x88, 0xf1, 0x24, 0x35, 0x68, 0x7f, 0x9f, 0x78, 0x9e, 0x31, 0x22, 0xde, 0x39, 0xfe, 0xfc,
-	0x1d, 0x03, 0x38, 0x92, 0x46, 0xcf, 0x01, 0x58, 0xc1, 0xe7, 0x91, 0x64, 0x56, 0x50, 0xea, 0x43,
-	0x84, 0x20, 0xc1, 0xcc, 0x59, 0x6b, 0xfa, 0xdf, 0x88, 0x00, 0x98, 0x96, 0x43, 0x5d, 0xc6, 0x6f,
-	0x8f, 0xdf, 0x91, 0x85, 0xb2, 0xbe, 0x4c, 0x52, 0x85, 0x6d, 0x55, 0xfc, 0xd6, 0x67, 0x60, 0x78,
-	0x0e, 0x18, 0xbd, 0x00, 0x17, 0x43, 0xcf, 0x18, 0x79, 0xc4, 0xfc, 0x0e, 0xce, 0xe2, 0x9c, 0xa0,
-	0x75, 0xc9, 0x23, 0xa6, 0xfc, 0x22, 0xc1, 0xfa, 0x09, 0x10, 0xa4, 0xc0, 0x95, 0x7d, 0xbd, 0xd3,
-	0xd1, 0x6a, 0x7a, 0xaf, 0xbe, 0xdf, 0x6e, 0xe1, 0xae, 0xd6, 0xdc, 0xd5, 0x7b, 0x07, 0xcd, 0x4e,
-	0x5b, 0xdf, 0xad, 0xdf, 0xaf, 0xeb, 0x55, 0xf9, 0x02, 0xda, 0x84, 0xf5, 0x46, 0xab, 0xd2, 0x0b,
-	0xe5, 0xaa, 0x7a, 0xe5, 0xa0, 0x26, 0x4b, 0xa8, 0x08, 0x97, 0xa2, 0xe4, 0xae, 0x56, 0xdf, 0xd3,
-	0xab, 0x72, 0xec, 0xb8, 0x42, 0x45, 0xeb, 0xd4, 0x77, 0xe5, 0x38, 0xda, 0x82, 0x8d, 0x79, 0xf2,
-	0xfb, 0x1a, 0x6e, 0xd6, 0x9b, 0x35, 0x39, 0x71, 0x5c, 0x5e, 0xc7, 0xb8, 0x85, 0xe5, 0xa4, 0xf2,
-	0x97, 0x04, 0x1b, 0x91, 0x5a, 0x89, 0x5e, 0xf8, 0x04, 0xe4, 0x30, 0x58, 0x57, 0xd0, 0x44, 0x4f,
-	0xdc, 0x5a, 0x21, 0xb3, 0x0f, 0x2e, 0xe0, 0x35, 0x01, 0x37, 0xb3, 0x40, 0xa0, 0xe0, 0x37, 0xf0,
-	0x11, 0x7e, 0xf0, 0x82, 0xbc, 0xb9, 0x30, 0xfe, 0x29, 0x77, 0xeb, 0xc1, 0x05, 0x9c, 0xf7, 0xe6,
-	0x09, 0x15, 0x80, 0x4c, 0x68, 0x40, 0xf9, 0x49, 0x82, 0x4c, 0xa8, 0xa1, 0xfc, 0x28, 0x41, 0x82,
-	0xdf, 0x23, 0xb4, 0x06, 0xb9, 0x68, 0x2d, 0x72, 0x90, 0xee, 0x74, 0x5b, 0xed, 0xb6, 0x5e, 0x95,
-	0x25, 0x7e, 0xc0, 0x07, 0x4d, 0x3f, 0x89, 0x31, 0x94, 0x81, 0x44, 0xb5, 0xd5, 0xd4, 0xe5, 0x38,
-	0x02, 0x48, 0xdd, 0x0f, 0x4a, 0x91, 0x40, 0x79, 0xc8, 0xee, 0xf2, 0x92, 0xee, 0xf1, 0x63, 0x92,
-	0x6b, 0x1c, 0xb4, 0xab, 0x5a, 0x57, 0xaf, 0xca, 0x29, 0x74, 0x11, 0x32, 0x55, 0xac, 0xd5, 0x7d,
-	0xfd, 0x34, 0x67, 0xf9, 0x27, 0xbd, 0x2a, 0x67, 0x38, 0xab, 0xd3, 0xd5, 0x70, 0x97, 0xb3, 0xb2,
-	0xa8, 0x00, 0x20, 0x40, 0xf8, 0x19, 0x94, 0x57, 0xe1, 0x52, 0x10, 0xdf, 0x7e, 0xb0, 0x7e, 0x9c,
-	0x73, 0x8b, 0xcc, 0x70, 0x0a, 0xcc, 0xc4, 0x45, 0x9a, 0xdb, 0x90, 0x16, 0x0b, 0x8c, 0xa8, 0xdf,
-	0xe2, 0xe3, 0x26, 0x80, 0xc2, 0xc4, 0x9b, 0x8c, 0x99, 0x87, 0x43, 0x18, 0xe5, 0x57, 0x09, 0xf2,
-	0x11, 0x16, 0x6a, 0x41, 0xd6, 0x60, 0x8c, 0x58, 0x0e, 0x23, 0x43, 0x31, 0x62, 0x77, 0x16, 0x98,
-	0x1c, 0xfb, 0xd4, 0x36, 0x19, 0x75, 0x4d, 0x7b, 0xe4, 0xcf, 0xd8, 0x23, 0x0c, 0x0e, 0x38, 0xa0,
-	0x96, 0x65, 0x32, 0x0e, 0x18, 0x5b, 0x19, 0x70, 0x86, 0xa1, 0x6c, 0xc3, 0x95, 0x60, 0xd7, 0xe8,
-	0x93, 0x76, 0xf4, 0x45, 0x0a, 0x1f, 0x0e, 0x02, 0x28, 0xca, 0xe9, 0x3e, 0x76, 0x88, 0xd2, 0x12,
-	0x3d, 0x02, 0x90, 0xea, 0x74, 0x31, 0xaf, 0x8c, 0xdf, 0x1e, 0x95, 0x56, 0x6b, 0x4f, 0xd7, 0x9a,
-	0x41, 0x7b, 0xd4, 0x9b, 0x5d, 0xbd, 0xa6, 0x63, 0x39, 0xc6, 0xa5, 0x9a, 0x07, 0xfb, 0x15, 0x1d,
-	0xcb, 0x71, 0x94, 0x85, 0xa4, 0x86, 0xb1, 0xf6, 0x81, 0x9c, 0xe0, 0xe4, 0x56, 0xa5, 0xa1, 0xef,
-	0x76, 0xe5, 0xa4, 0xf2, 0x87, 0x04, 0xc5, 0xa8, 0x9d, 0xa3, 0x1d, 0x88, 0x0f, 0x37, 0xff, 0xdd,
-	0x0d, 0x2a, 0xeb, 0x7f, 0xa3, 0x2e, 0x24, 0xd8, 0x63, 0x27, 0xb8, 0x1c, 0x85, 0xf2, 0x3b, 0x0b,
-	0x17, 0xef, 0x64, 0x30, 0xc1, 0xab, 0xe1, 0xa3, 0xa1, 0x6d, 0xc8, 0x0d, 0x85, 0x5d, 0x93, 0x86,
-	0xab, 0xd4, 0x3c, 0x09, 0xbd, 0x08, 0xf9, 0x21, 0x39, 0x34, 0x26, 0x63, 0xd6, 0x9b, 0x1a, 0xe3,
-	0x09, 0x11, 0xe3, 0xee, 0xa2, 0x20, 0xbe, 0xc7, 0x69, 0xe8, 0x12, 0x24, 0x47, 0x2e, 0x9d, 0x38,
-	0xc5, 0x64, 0xd0, 0x8b, 0xfe, 0x41, 0xf9, 0x12, 0x9e, 0x3f, 0x33, 0xd9, 0xa2, 0x2b, 0x3f, 0x82,
-	0x74, 0xb8, 0x41, 0x04, 0xfd, 0xa2, 0xad, 0x18, 0xd8, 0xdc, 0x06, 0x19, 0x22, 0x96, 0xbf, 0xc9,
-	0xf9, 0x2f, 0x8a, 0xd8, 0x31, 0xd1, 0xd7, 0x12, 0xa4, 0xc5, 0xce, 0x8b, 0xee, 0x2c, 0x6e, 0xe6,
-	0xf8, 0x8e, 0x5f, 0xba, 0xbb, 0x92, 0xae, 0x08, 0x78, 0x0a, 0x71, 0x3c, 0xb1, 0xd1, 0xe2, 0x97,
-	0x2f, 0xb2, 0xbf, 0x96, 0x6e, 0x2f, 0xad, 0x27, 0xec, 0x7e, 0x01, 0x69, 0xb1, 0x0f, 0xa1, 0xdb,
-	0x4b, 0x0e, 0xd6, 0xf0, 0x6a, 0x94, 0x5e, 0x5f, 0x5e, 0x51, 0x58, 0xff, 0x56, 0x82, 0x4c, 0x8d,
-	0x30, 0x7f, 0xe0, 0xa2, 0xbb, 0xab, 0x0d, 0xf6, 0xc0, 0x87, 0x27, 0x7a, 0x15, 0xd0, 0xf7, 0x12,
-	0xe4, 0x6a, 0x84, 0x85, 0xad, 0x83, 0xee, 0x2d, 0x89, 0x76, 0x6c, 0xb5, 0x2a, 0xbd, 0xbd, 0xb2,
-	0xbe, 0x70, 0xe8, 0x2b, 0x48, 0x05, 0xbb, 0x3e, 0x7a, 0x63, 0x61, 0xa8, 0xe3, 0xff, 0x23, 0x4a,
-	0x77, 0x56, 0x51, 0x15, 0x0e, 0x7c, 0x27, 0xf9, 0xab, 0xb3, 0x9f, 0xa6, 0x0e, 0x73, 0x89, 0x61,
-	0xfd, 0x8f, 0xf5, 0xb9, 0x29, 0xa1, 0x1f, 0x24, 0x90, 0x6b, 0x84, 0x89, 0x7d, 0x61, 0x69, 0x8f,
-	0x4e, 0xae, 0x9b, 0x4b, 0x78, 0x74, 0xca, 0xfe, 0x73, 0x53, 0xe2, 0x3d, 0x93, 0x8f, 0x3c, 0xa9,
-	0xe8, 0xad, 0x25, 0x63, 0x8c, 0xbe, 0xdc, 0xa5, 0x7b, 0xab, 0xaa, 0x8b, 0x92, 0xfd, 0x2c, 0xc1,
-	0xd6, 0x19, 0x73, 0x15, 0xd5, 0x16, 0xc6, 0xfe, 0xf7, 0x67, 0xb0, 0xf4, 0xe0, 0xc9, 0x81, 0xc4,
-	0xe2, 0x55, 0x81, 0x97, 0xcf, 0x84, 0x8a, 0x20, 0x55, 0x52, 0x0d, 0xda, 0xd7, 0x1c, 0xf3, 0x43,
-	0x39, 0xc2, 0xe9, 0x4d, 0x77, 0xfa, 0x29, 0xff, 0xff, 0xe4, 0xad, 0x7f, 0x02, 0x00, 0x00, 0xff,
-	0xff, 0xa3, 0xce, 0x16, 0x90, 0x07, 0x12, 0x00, 0x00,
+var fileDescriptor_beam_job_api_cf64c696c499a6a1 = []byte{
+	// 1410 bytes of a gzipped FileDescriptorProto
+	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x57, 0xdf, 0x6e, 0x1b, 0x45,
+	0x17, 0xef, 0x3a, 0x76, 0x6c, 0x1f, 0xd7, 0xce, 0x66, 0xd2, 0x7c, 0x49, 0xad, 0xef, 0xeb, 0x17,
+	0x16, 0x41, 0x8b, 0x2a, 0xb6, 0x8d, 0x2b, 0x51, 0x68, 0xa1, 0xb0, 0x8e, 0xb7, 0xae, 0x4d, 0x62,
+	0x5b, 0x63, 0x07, 0x04, 0x48, 0x98, 0xb5, 0x3d, 0x31, 0x5b, 0xbc, 0x3b, 0xcb, 0xee, 0xd8, 0x6a,
+	0x25, 0x04, 0x12, 0x88, 0x4b, 0xe0, 0x3d, 0x10, 0x12, 0xe2, 0x8a, 0x0b, 0x9e, 0x81, 0x87, 0x40,
+	0xe2, 0x8a, 0x57, 0xe0, 0x06, 0xcd, 0xec, 0xac, 0xe3, 0x4d, 0x13, 0x62, 0xbb, 0x45, 0x5c, 0x79,
+	0xe7, 0xfc, 0xf9, 0x9d, 0xbf, 0x73, 0xe6, 0x18, 0x50, 0x8f, 0x58, 0x4e, 0xf7, 0x21, 0xed, 0x75,
+	0x2d, 0xcf, 0xd6, 0x3d, 0x9f, 0x32, 0x8a, 0xae, 0x52, 0x7f, 0xa8, 0x5b, 0x9e, 0xd5, 0xff, 0x98,
+	0xe8, 0x9c, 0xad, 0x3b, 0x74, 0x40, 0x46, 0x3a, 0x17, 0x72, 0x2c, 0xd7, 0x1a, 0x12, 0x87, 0xb8,
+	0x4c, 0x9f, 0xec, 0x16, 0x37, 0x85, 0xb2, 0x3f, 0x76, 0x5d, 0xe2, 0x1f, 0xeb, 0x17, 0xd7, 0x88,
+	0x3b, 0xf0, 0xa8, 0xed, 0xb2, 0x40, 0x12, 0xfe, 0x3b, 0xa4, 0x74, 0x38, 0x22, 0x37, 0xc4, 0xa9,
+	0x37, 0x3e, 0xba, 0x11, 0x30, 0x7f, 0xdc, 0x67, 0x92, 0x9b, 0x77, 0x08, 0xf3, 0xed, 0xbe, 0x14,
+	0xd6, 0x7e, 0x51, 0x60, 0xbd, 0xe5, 0x13, 0xcf, 0xf2, 0x49, 0x9d, 0xf6, 0x30, 0xf9, 0x74, 0x4c,
+	0x02, 0x86, 0xaa, 0x90, 0xf1, 0x6c, 0x8f, 0x8c, 0x6c, 0x97, 0x6c, 0x2b, 0x3b, 0xca, 0xb5, 0x5c,
+	0xe9, 0xba, 0x7e, 0xba, 0x9b, 0x91, 0x98, 0x3e, 0xd9, 0xd5, 0x5b, 0xf2, 0x1b, 0x4f, 0x95, 0x51,
+	0x19, 0xd4, 0xe8, 0xbb, 0x4b, 0x3d, 0x66, 0x53, 0x37, 0xd8, 0x4e, 0x08, 0xc0, 0x2d, 0x3d, 0x74,
+	0x53, 0x8f, 0xdc, 0xd4, 0xdb, 0xc2, 0x4d, 0xbc, 0x16, 0x29, 0x34, 0x43, 0x79, 0x74, 0x19, 0x32,
+	0x3c, 0x19, 0xae, 0xe5, 0x90, 0xed, 0x95, 0x1d, 0xe5, 0x5a, 0x16, 0xa7, 0x1f, 0xd2, 0x5e, 0xc3,
+	0x72, 0x88, 0xf6, 0xbb, 0x02, 0x68, 0xd6, 0xfb, 0xc0, 0xa3, 0x6e, 0x40, 0xd0, 0x0b, 0x50, 0xf0,
+	0x04, 0xd5, 0xe2, 0x08, 0x5d, 0x7b, 0x20, 0x82, 0xc8, 0xe2, 0xfc, 0x0c, 0xb5, 0x36, 0x40, 0x01,
+	0x5c, 0xb6, 0x7c, 0x66, 0x1f, 0x59, 0x7d, 0xd6, 0x0d, 0x98, 0x35, 0xb4, 0xdd, 0x61, 0x37, 0x4a,
+	0xa6, 0xf4, 0xf2, 0xf6, 0x1c, 0x61, 0x1b, 0x9e, 0xdd, 0x26, 0xfe, 0xc4, 0xee, 0x93, 0x0a, 0x09,
+	0xfa, 0xbe, 0xed, 0x31, 0xea, 0xe3, 0xad, 0x08, 0xb9, 0x1d, 0x02, 0x9b, 0x12, 0x17, 0x95, 0x60,
+	0x33, 0xb2, 0x15, 0x90, 0x20, 0xe0, 0xfe, 0x31, 0xfa, 0x09, 0x71, 0x65, 0x68, 0x1b, 0x92, 0xd9,
+	0x0e, 0x79, 0x1d, 0xce, 0xd2, 0xba, 0x90, 0xc7, 0x63, 0x77, 0xa6, 0x3e, 0x73, 0x06, 0x78, 0x15,
+	0xd6, 0x7c, 0x5e, 0x6d, 0x32, 0xb1, 0x46, 0xd2, 0x4a, 0x42, 0xc8, 0x15, 0xa6, 0xe4, 0xd0, 0xc0,
+	0x55, 0x28, 0x44, 0x06, 0x64, 0x0a, 0x37, 0x61, 0x95, 0x27, 0x7d, 0x8a, 0x9c, 0x7a, 0x48, 0x7b,
+	0xb5, 0x81, 0xf6, 0x12, 0xa8, 0x7b, 0x96, 0xdb, 0x27, 0xa3, 0x19, 0x67, 0xce, 0x10, 0xb5, 0x60,
+	0x7d, 0x46, 0x54, 0xc2, 0xee, 0x43, 0x2a, 0x60, 0x16, 0x0b, 0xbb, 0xaa, 0x50, 0x7a, 0x45, 0x9f,
+	0xb3, 0xf9, 0xf5, 0x3a, 0xed, 0xb5, 0xb9, 0xa2, 0x6e, 0xba, 0x63, 0x07, 0x87, 0x20, 0xda, 0xaf,
+	0x0a, 0xa4, 0xeb, 0xb4, 0x57, 0x73, 0x8f, 0xe8, 0x19, 0x5e, 0xc4, 0x9a, 0x27, 0x11, 0x6b, 0x9e,
+	0x53, 0x7b, 0x73, 0x65, 0xc1, 0xde, 0x9c, 0xc6, 0x93, 0x7c, 0x16, 0xf1, 0xa8, 0x50, 0xa8, 0x12,
+	0x56, 0xa7, 0xbd, 0x40, 0xe6, 0x56, 0xfb, 0x10, 0xd6, 0xa6, 0x14, 0x99, 0xc2, 0xb7, 0xc3, 0x88,
+	0x6c, 0xf7, 0x88, 0x6e, 0x2b, 0x3b, 0x2b, 0xd7, 0x72, 0xa5, 0x9b, 0x8b, 0x58, 0xe5, 0xc9, 0x12,
+	0x39, 0xe0, 0x1f, 0xda, 0x75, 0x40, 0x21, 0xbe, 0x70, 0xe6, 0x9c, 0x8a, 0xf6, 0x61, 0x23, 0x26,
+	0xfc, 0x8f, 0xd4, 0x54, 0x87, 0xcd, 0xd0, 0xc8, 0x74, 0x9a, 0x9c, 0xd7, 0x66, 0xff, 0x39, 0x29,
+	0x2f, 0xfd, 0x7a, 0x56, 0x43, 0x8c, 0x27, 0xa9, 0x4e, 0x7b, 0x07, 0x24, 0x08, 0xac, 0x21, 0x09,
+	0xce, 0xf1, 0xe7, 0xcf, 0x04, 0xc0, 0xb1, 0x34, 0xfa, 0x1f, 0x80, 0x13, 0x7e, 0x1e, 0x4b, 0x66,
+	0x25, 0xa5, 0x36, 0x40, 0x08, 0x92, 0xcc, 0x9e, 0xb6, 0xa6, 0xf8, 0x46, 0x04, 0xc0, 0x76, 0x3c,
+	0xea, 0x33, 0x7e, 0x7b, 0x44, 0x47, 0x16, 0x4a, 0xe6, 0x22, 0x49, 0x95, 0xb6, 0x75, 0xf9, 0x5b,
+	0x9b, 0x82, 0xe1, 0x19, 0x60, 0xf4, 0x1c, 0x5c, 0x8c, 0x3c, 0x63, 0xe4, 0x11, 0x13, 0x1d, 0x9c,
+	0xc5, 0x39, 0x49, 0xeb, 0x90, 0x47, 0x4c, 0xfb, 0x51, 0x81, 0xf5, 0x27, 0x40, 0x90, 0x06, 0x57,
+	0x0e, 0xcc, 0x76, 0xdb, 0xa8, 0x9a, 0xdd, 0xda, 0x41, 0xab, 0x89, 0x3b, 0x46, 0x63, 0xcf, 0xec,
+	0x1e, 0x36, 0xda, 0x2d, 0x73, 0xaf, 0x76, 0xbf, 0x66, 0x56, 0xd4, 0x0b, 0x68, 0x13, 0xd6, 0xeb,
+	0xcd, 0x72, 0x37, 0x92, 0xab, 0x98, 0xe5, 0xc3, 0xaa, 0xaa, 0xa0, 0x6d, 0xb8, 0x14, 0x27, 0x77,
+	0x8c, 0xda, 0xbe, 0x59, 0x51, 0x13, 0x27, 0x15, 0xca, 0x46, 0xbb, 0xb6, 0xa7, 0xae, 0xa0, 0x2d,
+	0xd8, 0x98, 0x25, 0xbf, 0x6b, 0xe0, 0x46, 0xad, 0x51, 0x55, 0x93, 0x27, 0xe5, 0x4d, 0x8c, 0x9b,
+	0x58, 0x4d, 0x69, 0x7f, 0x28, 0xb0, 0x11, 0xab, 0x95, 0xec, 0x85, 0x8f, 0x40, 0x8d, 0x82, 0xf5,
+	0x25, 0x4d, 0xf6, 0xc4, 0xad, 0x25, 0x32, 0xfb, 0xe0, 0x02, 0x5e, 0x93, 0x70, 0x53, 0x0b, 0x04,
+	0x0a, 0xa2, 0x81, 0x8f, 0xf1, 0xc3, 0x17, 0xe4, 0xf5, 0xb9, 0xf1, 0x4f, 0xb9, 0x5b, 0x0f, 0x2e,
+	0xe0, 0x7c, 0x30, 0x4b, 0x28, 0x03, 0x64, 0x22, 0x03, 0xda, 0xcf, 0x0a, 0x64, 0x22, 0x0d, 0xed,
+	0x7b, 0x05, 0x92, 0xfc, 0x1e, 0xa1, 0x35, 0xc8, 0xc5, 0x6b, 0x91, 0x83, 0x74, 0xbb, 0xd3, 0x6c,
+	0xb5, 0xcc, 0x8a, 0xaa, 0xf0, 0x03, 0x3e, 0x6c, 0x88, 0x24, 0x26, 0x50, 0x06, 0x92, 0x95, 0x66,
+	0xc3, 0x54, 0x57, 0x10, 0xc0, 0xea, 0xfd, 0xb0, 0x14, 0x49, 0x94, 0x87, 0xec, 0x1e, 0x2f, 0xe9,
+	0x3e, 0x3f, 0xa6, 0xb8, 0xc6, 0x61, 0xab, 0x62, 0x74, 0xcc, 0x8a, 0xba, 0x8a, 0x2e, 0x42, 0xa6,
+	0x82, 0x8d, 0x9a, 0xd0, 0x4f, 0x73, 0x96, 0x38, 0x99, 0x15, 0x35, 0xc3, 0x59, 0xed, 0x8e, 0x81,
+	0x3b, 0x9c, 0x95, 0x45, 0x05, 0x00, 0x09, 0xc2, 0xcf, 0xc0, 0xb9, 0x02, 0x85, 0x9f, 0x72, 0xda,
+	0xcb, 0x70, 0x29, 0x8c, 0xf6, 0x20, 0x5c, 0x46, 0xce, 0xb9, 0x53, 0x76, 0x34, 0x13, 0xa6, 0xe2,
+	0x32, 0xe9, 0x2d, 0x48, 0xcb, 0x75, 0x46, 0x56, 0x73, 0xfe, 0xe1, 0x13, 0x42, 0x61, 0x12, 0x8c,
+	0x47, 0x2c, 0xc0, 0x11, 0x8c, 0xf6, 0x93, 0x02, 0xf9, 0x18, 0x0b, 0x35, 0x21, 0x6b, 0x31, 0x46,
+	0x1c, 0x8f, 0x91, 0x81, 0x1c, 0xb8, 0xbb, 0x73, 0xcc, 0x91, 0x03, 0xea, 0xda, 0x8c, 0xfa, 0xb6,
+	0x3b, 0x14, 0x13, 0xf7, 0x18, 0x83, 0x03, 0xf6, 0xa9, 0xe3, 0xd8, 0x8c, 0x03, 0x26, 0x96, 0x06,
+	0x9c, 0x62, 0x68, 0x3b, 0x70, 0x25, 0xdc, 0x3c, 0x7a, 0xa4, 0x15, 0x7f, 0x9f, 0xa2, 0x67, 0x84,
+	0x00, 0x8a, 0x73, 0x3a, 0x8f, 0x3d, 0xa2, 0x35, 0x65, 0xc7, 0x00, 0xac, 0xb6, 0x3b, 0x98, 0x57,
+	0x46, 0x34, 0x4b, 0xb9, 0xd9, 0xdc, 0x37, 0x8d, 0x46, 0xd8, 0x2c, 0xb5, 0x46, 0xc7, 0xac, 0x9a,
+	0x58, 0x4d, 0x70, 0xa9, 0xc6, 0xe1, 0x41, 0xd9, 0xc4, 0xea, 0x0a, 0xca, 0x42, 0xca, 0xc0, 0xd8,
+	0x78, 0x4f, 0x4d, 0x72, 0x72, 0xb3, 0x5c, 0x37, 0xf7, 0x3a, 0x6a, 0x4a, 0xfb, 0x4d, 0x81, 0xed,
+	0xb8, 0x9d, 0xe3, 0x8d, 0x88, 0x8f, 0x3a, 0xf1, 0x0a, 0x87, 0x95, 0x15, 0xdf, 0xa8, 0x03, 0x49,
+	0xf6, 0xd8, 0x0b, 0xaf, 0x4a, 0xa1, 0xf4, 0xd6, 0xdc, 0xc5, 0x7b, 0x32, 0x98, 0xf0, 0x0d, 0x11,
+	0x68, 0x68, 0x07, 0x72, 0x03, 0x69, 0xd7, 0xa6, 0xd1, 0x62, 0x35, 0x4b, 0x42, 0xcf, 0x43, 0x7e,
+	0x40, 0x8e, 0xac, 0xf1, 0x88, 0x75, 0x27, 0xd6, 0x68, 0x4c, 0xe4, 0xf0, 0xbb, 0x28, 0x89, 0xef,
+	0x70, 0x1a, 0xba, 0x04, 0xa9, 0xa1, 0x4f, 0xc7, 0xde, 0x76, 0x2a, 0xec, 0x45, 0x71, 0xd0, 0x3e,
+	0x87, 0xff, 0x9f, 0x99, 0x6c, 0xd9, 0x95, 0x1f, 0x40, 0x3a, 0xda, 0x27, 0xc2, 0x7e, 0x31, 0x96,
+	0x0c, 0x6c, 0x66, 0x9f, 0x8c, 0x10, 0x4b, 0x5f, 0xe5, 0xc4, 0xfb, 0x22, 0x37, 0x4e, 0xf4, 0xa5,
+	0x02, 0x69, 0xb9, 0x01, 0xa3, 0x3b, 0xf3, 0x9b, 0x39, 0xb9, 0xf1, 0x17, 0xef, 0x2e, 0xa5, 0x2b,
+	0x03, 0x9e, 0xc0, 0x0a, 0x1e, 0xbb, 0x68, 0xfe, 0xcb, 0x17, 0xdb, 0x66, 0x8b, 0xb7, 0x17, 0xd6,
+	0x93, 0x76, 0x3f, 0x83, 0xb4, 0xdc, 0x8e, 0xd0, 0xed, 0x05, 0xc7, 0x6c, 0x74, 0x35, 0x8a, 0xaf,
+	0x2e, 0xae, 0x28, 0xad, 0x7f, 0xad, 0x40, 0xa6, 0x4a, 0x98, 0x18, 0xbf, 0xe8, 0xee, 0x72, 0x63,
+	0x3e, 0xf4, 0xe1, 0xa9, 0xde, 0x08, 0xf4, 0xad, 0x02, 0xb9, 0x2a, 0x61, 0x51, 0xeb, 0xa0, 0x7b,
+	0x0b, 0xa2, 0x9d, 0x58, 0xb4, 0x8a, 0x6f, 0x2e, 0xad, 0x2f, 0x1d, 0xfa, 0x02, 0x56, 0xc3, 0xcd,
+	0x1f, 0xbd, 0x36, 0x37, 0xd4, 0xc9, 0x7f, 0x15, 0xc5, 0x3b, 0xcb, 0xa8, 0x4a, 0x07, 0xbe, 0x51,
+	0xc4, 0x22, 0x2d, 0xd2, 0xd4, 0x66, 0x3e, 0xb1, 0x9c, 0x7f, 0xb1, 0x3e, 0x37, 0x15, 0xf4, 0x9d,
+	0x02, 0x6a, 0x95, 0x30, 0xb9, 0x3d, 0x2c, 0xec, 0xd1, 0x93, 0xcb, 0xe7, 0x02, 0x1e, 0x9d, 0xb2,
+	0x0d, 0xdd, 0x54, 0x78, 0xcf, 0xe4, 0x63, 0x4f, 0x2a, 0x7a, 0x63, 0xc1, 0x18, 0xe3, 0x2f, 0x77,
+	0xf1, 0xde, 0xb2, 0xea, 0xb2, 0x64, 0x3f, 0x28, 0xb0, 0x75, 0xc6, 0x5c, 0x45, 0xd5, 0xb9, 0xb1,
+	0xff, 0xfe, 0x19, 0x2c, 0x3e, 0x78, 0x7a, 0x20, 0xb9, 0x86, 0x95, 0xe1, 0xc5, 0x33, 0xa1, 0x62,
+	0x48, 0xe5, 0xd5, 0x3a, 0xed, 0x19, 0x9e, 0xfd, 0xbe, 0x1a, 0xe3, 0x74, 0x27, 0xbb, 0xbd, 0x55,
+	0xf1, 0xef, 0xf2, 0xd6, 0x5f, 0x01, 0x00, 0x00, 0xff, 0xff, 0xf8, 0xd6, 0x25, 0x25, 0x15, 0x12,
+	0x00, 0x00,
 }
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 8d8face..dcb1ba1 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
@@ -51,7 +51,7 @@
 	return proto.EnumName(BeamConstants_Constants_name, int32(x))
 }
 func (BeamConstants_Constants) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d5fa30116074ddde, []int{0, 0}
+	return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{0, 0}
 }
 
 type StandardPTransforms_Primitives int32
@@ -130,7 +130,7 @@
 	return proto.EnumName(StandardPTransforms_Primitives_name, int32(x))
 }
 func (StandardPTransforms_Primitives) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d5fa30116074ddde, []int{4, 0}
+	return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{4, 0}
 }
 
 type StandardPTransforms_DeprecatedPrimitives int32
@@ -157,7 +157,7 @@
 	return proto.EnumName(StandardPTransforms_DeprecatedPrimitives_name, int32(x))
 }
 func (StandardPTransforms_DeprecatedPrimitives) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d5fa30116074ddde, []int{4, 1}
+	return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{4, 1}
 }
 
 type StandardPTransforms_Composites int32
@@ -196,7 +196,7 @@
 	return proto.EnumName(StandardPTransforms_Composites_name, int32(x))
 }
 func (StandardPTransforms_Composites) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d5fa30116074ddde, []int{4, 2}
+	return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{4, 2}
 }
 
 // Payload for all of these: CombinePayload
@@ -242,7 +242,7 @@
 	return proto.EnumName(StandardPTransforms_CombineComponents_name, int32(x))
 }
 func (StandardPTransforms_CombineComponents) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d5fa30116074ddde, []int{4, 3}
+	return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{4, 3}
 }
 
 // Payload for all of these: ParDoPayload containing the user's SDF
@@ -302,13 +302,20 @@
 	return proto.EnumName(StandardPTransforms_SplittableParDoComponents_name, int32(x))
 }
 func (StandardPTransforms_SplittableParDoComponents) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d5fa30116074ddde, []int{4, 4}
+	return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{4, 4}
 }
 
 type StandardSideInputTypes_Enum int32
 
 const (
+	// Represents a view over a PCollection<V>.
+	//
+	// The SDK is limited to perform state get requests using the StateKey.IterableSideInput.
 	StandardSideInputTypes_ITERABLE StandardSideInputTypes_Enum = 0
+	// Represents a view over a PCollection<KV<K, V>>.
+	//
+	// The SDK is able to perform state get requests with the StateKey.IterableSideInput and
+	// StateKey.MultimapSideInput
 	StandardSideInputTypes_MULTIMAP StandardSideInputTypes_Enum = 1
 )
 
@@ -325,7 +332,7 @@
 	return proto.EnumName(StandardSideInputTypes_Enum_name, int32(x))
 }
 func (StandardSideInputTypes_Enum) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d5fa30116074ddde, []int{5, 0}
+	return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{5, 0}
 }
 
 type Parameter_Type_Enum int32
@@ -354,7 +361,7 @@
 	return proto.EnumName(Parameter_Type_Enum_name, int32(x))
 }
 func (Parameter_Type_Enum) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d5fa30116074ddde, []int{8, 0, 0}
+	return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{8, 0, 0}
 }
 
 type IsBounded_Enum int32
@@ -380,7 +387,7 @@
 	return proto.EnumName(IsBounded_Enum_name, int32(x))
 }
 func (IsBounded_Enum) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d5fa30116074ddde, []int{16, 0}
+	return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{16, 0}
 }
 
 type StandardCoders_Enum int32
@@ -392,6 +399,8 @@
 	StandardCoders_STRING_UTF8 StandardCoders_Enum = 10
 	// Components: The key and value coder, in that order.
 	StandardCoders_KV StandardCoders_Enum = 1
+	// Components: None
+	StandardCoders_BOOL StandardCoders_Enum = 12
 	// Variable length Encodes a 64-bit integer.
 	// Components: None
 	StandardCoders_VARINT StandardCoders_Enum = 2
@@ -462,12 +471,50 @@
 	// Components: Coder for a single element.
 	// Experimental.
 	StandardCoders_STATE_BACKED_ITERABLE StandardCoders_Enum = 9
+	// Encodes a "row", an element with a known schema, defined by an
+	// instance of Schema from schema.proto.
+	//
+	// A row is encoded as the concatenation of:
+	//   - The number of attributes in the schema, encoded with
+	//     beam:coder:varint:v1. This makes it possible to detect certain
+	//     allowed schema changes (appending or removing columns) in
+	//     long-running streaming pipelines.
+	//   - A byte array representing a packed bitset indicating null fields (a
+	//     1 indicating a null) encoded with beam:coder:bytes:v1. The unused
+	//     bits in the last byte must be set to 0. If there are no nulls an
+	//     empty byte array is encoded.
+	//     The two-byte bitset (not including the lenghth-prefix) for the row
+	//     [NULL, 0, 0, 0, NULL, 0, 0, NULL, 0, NULL] would be
+	//     [0b10010001, 0b00000010]
+	//   - An encoding for each non-null field, concatenated together.
+	//
+	// Schema types are mapped to coders as follows:
+	//   AtomicType:
+	//     BYTE:      not yet a standard coder (BEAM-7996)
+	//     INT16:     not yet a standard coder (BEAM-7996)
+	//     INT32:     beam:coder:varint:v1
+	//     INT64:     beam:coder:varint:v1
+	//     FLOAT:     not yet a standard coder (BEAM-7996)
+	//     DOUBLE:    beam:coder:double:v1
+	//     STRING:    beam:coder:string_utf8:v1
+	//     BOOLEAN:   beam:coder:bool:v1
+	//     BYTES:     beam:coder:bytes:v1
+	//   ArrayType:   beam:coder:iterable:v1 (always has a known length)
+	//   MapType:     not yet a standard coder (BEAM-7996)
+	//   RowType:     beam:coder:row:v1
+	//   LogicalType: Uses the coder for its representation.
+	//
+	// The payload for RowCoder is an instance of Schema.
+	// Components: None
+	// Experimental.
+	StandardCoders_ROW StandardCoders_Enum = 13
 )
 
 var StandardCoders_Enum_name = map[int32]string{
 	0:  "BYTES",
 	10: "STRING_UTF8",
 	1:  "KV",
+	12: "BOOL",
 	2:  "VARINT",
 	11: "DOUBLE",
 	3:  "ITERABLE",
@@ -477,11 +524,13 @@
 	7:  "GLOBAL_WINDOW",
 	8:  "WINDOWED_VALUE",
 	9:  "STATE_BACKED_ITERABLE",
+	13: "ROW",
 }
 var StandardCoders_Enum_value = map[string]int32{
 	"BYTES":                 0,
 	"STRING_UTF8":           10,
 	"KV":                    1,
+	"BOOL":                  12,
 	"VARINT":                2,
 	"DOUBLE":                11,
 	"ITERABLE":              3,
@@ -491,13 +540,14 @@
 	"GLOBAL_WINDOW":         7,
 	"WINDOWED_VALUE":        8,
 	"STATE_BACKED_ITERABLE": 9,
+	"ROW": 13,
 }
 
 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_d5fa30116074ddde, []int{23, 0}
+	return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{23, 0}
 }
 
 type MergeStatus_Enum int32
@@ -534,7 +584,7 @@
 	return proto.EnumName(MergeStatus_Enum_name, int32(x))
 }
 func (MergeStatus_Enum) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d5fa30116074ddde, []int{25, 0}
+	return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{25, 0}
 }
 
 type AccumulationMode_Enum int32
@@ -566,7 +616,7 @@
 	return proto.EnumName(AccumulationMode_Enum_name, int32(x))
 }
 func (AccumulationMode_Enum) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d5fa30116074ddde, []int{26, 0}
+	return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{26, 0}
 }
 
 type ClosingBehavior_Enum int32
@@ -595,7 +645,7 @@
 	return proto.EnumName(ClosingBehavior_Enum_name, int32(x))
 }
 func (ClosingBehavior_Enum) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d5fa30116074ddde, []int{27, 0}
+	return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{27, 0}
 }
 
 type OnTimeBehavior_Enum int32
@@ -624,7 +674,7 @@
 	return proto.EnumName(OnTimeBehavior_Enum_name, int32(x))
 }
 func (OnTimeBehavior_Enum) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d5fa30116074ddde, []int{28, 0}
+	return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{28, 0}
 }
 
 type OutputTime_Enum int32
@@ -658,7 +708,7 @@
 	return proto.EnumName(OutputTime_Enum_name, int32(x))
 }
 func (OutputTime_Enum) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d5fa30116074ddde, []int{29, 0}
+	return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{29, 0}
 }
 
 type TimeDomain_Enum int32
@@ -695,7 +745,7 @@
 	return proto.EnumName(TimeDomain_Enum_name, int32(x))
 }
 func (TimeDomain_Enum) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d5fa30116074ddde, []int{30, 0}
+	return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{30, 0}
 }
 
 type StandardEnvironments_Environments int32
@@ -721,7 +771,7 @@
 	return proto.EnumName(StandardEnvironments_Environments_name, int32(x))
 }
 func (StandardEnvironments_Environments) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d5fa30116074ddde, []int{35, 0}
+	return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{35, 0}
 }
 
 type DisplayData_Type_Enum int32
@@ -762,7 +812,7 @@
 	return proto.EnumName(DisplayData_Type_Enum_name, int32(x))
 }
 func (DisplayData_Type_Enum) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d5fa30116074ddde, []int{41, 2, 0}
+	return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{41, 2, 0}
 }
 
 type BeamConstants struct {
@@ -775,7 +825,7 @@
 func (m *BeamConstants) String() string { return proto.CompactTextString(m) }
 func (*BeamConstants) ProtoMessage()    {}
 func (*BeamConstants) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d5fa30116074ddde, []int{0}
+	return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{0}
 }
 func (m *BeamConstants) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_BeamConstants.Unmarshal(m, b)
@@ -817,7 +867,7 @@
 func (m *Components) String() string { return proto.CompactTextString(m) }
 func (*Components) ProtoMessage()    {}
 func (*Components) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d5fa30116074ddde, []int{1}
+	return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{1}
 }
 func (m *Components) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Components.Unmarshal(m, b)
@@ -901,7 +951,7 @@
 func (m *Pipeline) String() string { return proto.CompactTextString(m) }
 func (*Pipeline) ProtoMessage()    {}
 func (*Pipeline) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d5fa30116074ddde, []int{2}
+	return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{2}
 }
 func (m *Pipeline) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Pipeline.Unmarshal(m, b)
@@ -1015,7 +1065,7 @@
 func (m *PTransform) String() string { return proto.CompactTextString(m) }
 func (*PTransform) ProtoMessage()    {}
 func (*PTransform) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d5fa30116074ddde, []int{3}
+	return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{3}
 }
 func (m *PTransform) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_PTransform.Unmarshal(m, b)
@@ -1087,7 +1137,7 @@
 func (m *StandardPTransforms) String() string { return proto.CompactTextString(m) }
 func (*StandardPTransforms) ProtoMessage()    {}
 func (*StandardPTransforms) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d5fa30116074ddde, []int{4}
+	return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{4}
 }
 func (m *StandardPTransforms) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_StandardPTransforms.Unmarshal(m, b)
@@ -1117,7 +1167,7 @@
 func (m *StandardSideInputTypes) String() string { return proto.CompactTextString(m) }
 func (*StandardSideInputTypes) ProtoMessage()    {}
 func (*StandardSideInputTypes) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d5fa30116074ddde, []int{5}
+	return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{5}
 }
 func (m *StandardSideInputTypes) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_StandardSideInputTypes.Unmarshal(m, b)
@@ -1167,7 +1217,7 @@
 func (m *PCollection) String() string { return proto.CompactTextString(m) }
 func (*PCollection) ProtoMessage()    {}
 func (*PCollection) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d5fa30116074ddde, []int{6}
+	return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{6}
 }
 func (m *PCollection) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_PCollection.Unmarshal(m, b)
@@ -1252,7 +1302,7 @@
 func (m *ParDoPayload) String() string { return proto.CompactTextString(m) }
 func (*ParDoPayload) ProtoMessage()    {}
 func (*ParDoPayload) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d5fa30116074ddde, []int{7}
+	return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{7}
 }
 func (m *ParDoPayload) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_ParDoPayload.Unmarshal(m, b)
@@ -1353,7 +1403,7 @@
 func (m *Parameter) String() string { return proto.CompactTextString(m) }
 func (*Parameter) ProtoMessage()    {}
 func (*Parameter) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d5fa30116074ddde, []int{8}
+	return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{8}
 }
 func (m *Parameter) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Parameter.Unmarshal(m, b)
@@ -1390,7 +1440,7 @@
 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_d5fa30116074ddde, []int{8, 0}
+	return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{8, 0}
 }
 func (m *Parameter_Type) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Parameter_Type.Unmarshal(m, b)
@@ -1412,7 +1462,7 @@
 
 type StateSpec struct {
 	// Types that are valid to be assigned to Spec:
-	//	*StateSpec_ValueSpec
+	//	*StateSpec_ReadModifyWriteSpec
 	//	*StateSpec_BagSpec
 	//	*StateSpec_CombiningSpec
 	//	*StateSpec_MapSpec
@@ -1427,7 +1477,7 @@
 func (m *StateSpec) String() string { return proto.CompactTextString(m) }
 func (*StateSpec) ProtoMessage()    {}
 func (*StateSpec) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d5fa30116074ddde, []int{9}
+	return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{9}
 }
 func (m *StateSpec) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_StateSpec.Unmarshal(m, b)
@@ -1451,8 +1501,8 @@
 	isStateSpec_Spec()
 }
 
-type StateSpec_ValueSpec struct {
-	ValueSpec *ValueStateSpec `protobuf:"bytes,1,opt,name=value_spec,json=valueSpec,proto3,oneof"`
+type StateSpec_ReadModifyWriteSpec struct {
+	ReadModifyWriteSpec *ReadModifyWriteStateSpec `protobuf:"bytes,1,opt,name=read_modify_write_spec,json=readModifyWriteSpec,proto3,oneof"`
 }
 type StateSpec_BagSpec struct {
 	BagSpec *BagStateSpec `protobuf:"bytes,2,opt,name=bag_spec,json=bagSpec,proto3,oneof"`
@@ -1467,11 +1517,11 @@
 	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_CombiningSpec) isStateSpec_Spec() {}
-func (*StateSpec_MapSpec) isStateSpec_Spec()       {}
-func (*StateSpec_SetSpec) isStateSpec_Spec()       {}
+func (*StateSpec_ReadModifyWriteSpec) isStateSpec_Spec() {}
+func (*StateSpec_BagSpec) isStateSpec_Spec()             {}
+func (*StateSpec_CombiningSpec) isStateSpec_Spec()       {}
+func (*StateSpec_MapSpec) isStateSpec_Spec()             {}
+func (*StateSpec_SetSpec) isStateSpec_Spec()             {}
 
 func (m *StateSpec) GetSpec() isStateSpec_Spec {
 	if m != nil {
@@ -1480,9 +1530,9 @@
 	return nil
 }
 
-func (m *StateSpec) GetValueSpec() *ValueStateSpec {
-	if x, ok := m.GetSpec().(*StateSpec_ValueSpec); ok {
-		return x.ValueSpec
+func (m *StateSpec) GetReadModifyWriteSpec() *ReadModifyWriteStateSpec {
+	if x, ok := m.GetSpec().(*StateSpec_ReadModifyWriteSpec); ok {
+		return x.ReadModifyWriteSpec
 	}
 	return nil
 }
@@ -1518,7 +1568,7 @@
 // XXX_OneofFuncs is for the internal use of the proto package.
 func (*StateSpec) XXX_OneofFuncs() (func(msg proto.Message, b *proto.Buffer) error, func(msg proto.Message, tag, wire int, b *proto.Buffer) (bool, error), func(msg proto.Message) (n int), []interface{}) {
 	return _StateSpec_OneofMarshaler, _StateSpec_OneofUnmarshaler, _StateSpec_OneofSizer, []interface{}{
-		(*StateSpec_ValueSpec)(nil),
+		(*StateSpec_ReadModifyWriteSpec)(nil),
 		(*StateSpec_BagSpec)(nil),
 		(*StateSpec_CombiningSpec)(nil),
 		(*StateSpec_MapSpec)(nil),
@@ -1530,9 +1580,9 @@
 	m := msg.(*StateSpec)
 	// spec
 	switch x := m.Spec.(type) {
-	case *StateSpec_ValueSpec:
+	case *StateSpec_ReadModifyWriteSpec:
 		b.EncodeVarint(1<<3 | proto.WireBytes)
-		if err := b.EncodeMessage(x.ValueSpec); err != nil {
+		if err := b.EncodeMessage(x.ReadModifyWriteSpec); err != nil {
 			return err
 		}
 	case *StateSpec_BagSpec:
@@ -1565,13 +1615,13 @@
 func _StateSpec_OneofUnmarshaler(msg proto.Message, tag, wire int, b *proto.Buffer) (bool, error) {
 	m := msg.(*StateSpec)
 	switch tag {
-	case 1: // spec.value_spec
+	case 1: // spec.read_modify_write_spec
 		if wire != proto.WireBytes {
 			return true, proto.ErrInternalBadWireType
 		}
-		msg := new(ValueStateSpec)
+		msg := new(ReadModifyWriteStateSpec)
 		err := b.DecodeMessage(msg)
-		m.Spec = &StateSpec_ValueSpec{msg}
+		m.Spec = &StateSpec_ReadModifyWriteSpec{msg}
 		return true, err
 	case 2: // spec.bag_spec
 		if wire != proto.WireBytes {
@@ -1614,8 +1664,8 @@
 	m := msg.(*StateSpec)
 	// spec
 	switch x := m.Spec.(type) {
-	case *StateSpec_ValueSpec:
-		s := proto.Size(x.ValueSpec)
+	case *StateSpec_ReadModifyWriteSpec:
+		s := proto.Size(x.ReadModifyWriteSpec)
 		n += 1 // tag and wire
 		n += proto.SizeVarint(uint64(s))
 		n += s
@@ -1646,38 +1696,38 @@
 	return n
 }
 
-type ValueStateSpec struct {
+type ReadModifyWriteStateSpec struct {
 	CoderId              string   `protobuf:"bytes,1,opt,name=coder_id,json=coderId,proto3" json:"coder_id,omitempty"`
 	XXX_NoUnkeyedLiteral struct{} `json:"-"`
 	XXX_unrecognized     []byte   `json:"-"`
 	XXX_sizecache        int32    `json:"-"`
 }
 
-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_d5fa30116074ddde, []int{10}
+func (m *ReadModifyWriteStateSpec) Reset()         { *m = ReadModifyWriteStateSpec{} }
+func (m *ReadModifyWriteStateSpec) String() string { return proto.CompactTextString(m) }
+func (*ReadModifyWriteStateSpec) ProtoMessage()    {}
+func (*ReadModifyWriteStateSpec) Descriptor() ([]byte, []int) {
+	return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{10}
 }
-func (m *ValueStateSpec) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_ValueStateSpec.Unmarshal(m, b)
+func (m *ReadModifyWriteStateSpec) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_ReadModifyWriteStateSpec.Unmarshal(m, b)
 }
-func (m *ValueStateSpec) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_ValueStateSpec.Marshal(b, m, deterministic)
+func (m *ReadModifyWriteStateSpec) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_ReadModifyWriteStateSpec.Marshal(b, m, deterministic)
 }
-func (dst *ValueStateSpec) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ValueStateSpec.Merge(dst, src)
+func (dst *ReadModifyWriteStateSpec) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_ReadModifyWriteStateSpec.Merge(dst, src)
 }
-func (m *ValueStateSpec) XXX_Size() int {
-	return xxx_messageInfo_ValueStateSpec.Size(m)
+func (m *ReadModifyWriteStateSpec) XXX_Size() int {
+	return xxx_messageInfo_ReadModifyWriteStateSpec.Size(m)
 }
-func (m *ValueStateSpec) XXX_DiscardUnknown() {
-	xxx_messageInfo_ValueStateSpec.DiscardUnknown(m)
+func (m *ReadModifyWriteStateSpec) XXX_DiscardUnknown() {
+	xxx_messageInfo_ReadModifyWriteStateSpec.DiscardUnknown(m)
 }
 
-var xxx_messageInfo_ValueStateSpec proto.InternalMessageInfo
+var xxx_messageInfo_ReadModifyWriteStateSpec proto.InternalMessageInfo
 
-func (m *ValueStateSpec) GetCoderId() string {
+func (m *ReadModifyWriteStateSpec) GetCoderId() string {
 	if m != nil {
 		return m.CoderId
 	}
@@ -1695,7 +1745,7 @@
 func (m *BagStateSpec) String() string { return proto.CompactTextString(m) }
 func (*BagStateSpec) ProtoMessage()    {}
 func (*BagStateSpec) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d5fa30116074ddde, []int{11}
+	return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{11}
 }
 func (m *BagStateSpec) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_BagStateSpec.Unmarshal(m, b)
@@ -1734,7 +1784,7 @@
 func (m *CombiningStateSpec) String() string { return proto.CompactTextString(m) }
 func (*CombiningStateSpec) ProtoMessage()    {}
 func (*CombiningStateSpec) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d5fa30116074ddde, []int{12}
+	return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{12}
 }
 func (m *CombiningStateSpec) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_CombiningStateSpec.Unmarshal(m, b)
@@ -1780,7 +1830,7 @@
 func (m *MapStateSpec) String() string { return proto.CompactTextString(m) }
 func (*MapStateSpec) ProtoMessage()    {}
 func (*MapStateSpec) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d5fa30116074ddde, []int{13}
+	return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{13}
 }
 func (m *MapStateSpec) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_MapStateSpec.Unmarshal(m, b)
@@ -1825,7 +1875,7 @@
 func (m *SetStateSpec) String() string { return proto.CompactTextString(m) }
 func (*SetStateSpec) ProtoMessage()    {}
 func (*SetStateSpec) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d5fa30116074ddde, []int{14}
+	return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{14}
 }
 func (m *SetStateSpec) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_SetStateSpec.Unmarshal(m, b)
@@ -1864,7 +1914,7 @@
 func (m *TimerSpec) String() string { return proto.CompactTextString(m) }
 func (*TimerSpec) ProtoMessage()    {}
 func (*TimerSpec) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d5fa30116074ddde, []int{15}
+	return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{15}
 }
 func (m *TimerSpec) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_TimerSpec.Unmarshal(m, b)
@@ -1908,7 +1958,7 @@
 func (m *IsBounded) String() string { return proto.CompactTextString(m) }
 func (*IsBounded) ProtoMessage()    {}
 func (*IsBounded) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d5fa30116074ddde, []int{16}
+	return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{16}
 }
 func (m *IsBounded) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_IsBounded.Unmarshal(m, b)
@@ -1943,7 +1993,7 @@
 func (m *ReadPayload) String() string { return proto.CompactTextString(m) }
 func (*ReadPayload) ProtoMessage()    {}
 func (*ReadPayload) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d5fa30116074ddde, []int{17}
+	return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{17}
 }
 func (m *ReadPayload) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_ReadPayload.Unmarshal(m, b)
@@ -1990,7 +2040,7 @@
 func (m *WindowIntoPayload) String() string { return proto.CompactTextString(m) }
 func (*WindowIntoPayload) ProtoMessage()    {}
 func (*WindowIntoPayload) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d5fa30116074ddde, []int{18}
+	return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{18}
 }
 func (m *WindowIntoPayload) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_WindowIntoPayload.Unmarshal(m, b)
@@ -2032,7 +2082,7 @@
 func (m *CombinePayload) String() string { return proto.CompactTextString(m) }
 func (*CombinePayload) ProtoMessage()    {}
 func (*CombinePayload) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d5fa30116074ddde, []int{19}
+	return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{19}
 }
 func (m *CombinePayload) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_CombinePayload.Unmarshal(m, b)
@@ -2080,7 +2130,7 @@
 func (m *TestStreamPayload) String() string { return proto.CompactTextString(m) }
 func (*TestStreamPayload) ProtoMessage()    {}
 func (*TestStreamPayload) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d5fa30116074ddde, []int{20}
+	return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{20}
 }
 func (m *TestStreamPayload) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_TestStreamPayload.Unmarshal(m, b)
@@ -2129,7 +2179,7 @@
 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_d5fa30116074ddde, []int{20, 0}
+	return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{20, 0}
 }
 func (m *TestStreamPayload_Event) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_TestStreamPayload_Event.Unmarshal(m, b)
@@ -2301,7 +2351,7 @@
 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_d5fa30116074ddde, []int{20, 0, 0}
+	return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{20, 0, 0}
 }
 func (m *TestStreamPayload_Event_AdvanceWatermark) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_TestStreamPayload_Event_AdvanceWatermark.Unmarshal(m, b)
@@ -2343,7 +2393,7 @@
 }
 func (*TestStreamPayload_Event_AdvanceProcessingTime) ProtoMessage() {}
 func (*TestStreamPayload_Event_AdvanceProcessingTime) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d5fa30116074ddde, []int{20, 0, 1}
+	return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{20, 0, 1}
 }
 func (m *TestStreamPayload_Event_AdvanceProcessingTime) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_TestStreamPayload_Event_AdvanceProcessingTime.Unmarshal(m, b)
@@ -2381,7 +2431,7 @@
 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_d5fa30116074ddde, []int{20, 0, 2}
+	return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{20, 0, 2}
 }
 func (m *TestStreamPayload_Event_AddElements) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_TestStreamPayload_Event_AddElements.Unmarshal(m, b)
@@ -2420,7 +2470,7 @@
 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_d5fa30116074ddde, []int{20, 1}
+	return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{20, 1}
 }
 func (m *TestStreamPayload_TimestampedElement) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_TestStreamPayload_TimestampedElement.Unmarshal(m, b)
@@ -2472,7 +2522,7 @@
 func (m *WriteFilesPayload) String() string { return proto.CompactTextString(m) }
 func (*WriteFilesPayload) ProtoMessage()    {}
 func (*WriteFilesPayload) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d5fa30116074ddde, []int{21}
+	return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{21}
 }
 func (m *WriteFilesPayload) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_WriteFilesPayload.Unmarshal(m, b)
@@ -2549,7 +2599,7 @@
 func (m *Coder) String() string { return proto.CompactTextString(m) }
 func (*Coder) ProtoMessage()    {}
 func (*Coder) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d5fa30116074ddde, []int{22}
+	return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{22}
 }
 func (m *Coder) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Coder.Unmarshal(m, b)
@@ -2593,7 +2643,7 @@
 func (m *StandardCoders) String() string { return proto.CompactTextString(m) }
 func (*StandardCoders) ProtoMessage()    {}
 func (*StandardCoders) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d5fa30116074ddde, []int{23}
+	return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{23}
 }
 func (m *StandardCoders) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_StandardCoders.Unmarshal(m, b)
@@ -2664,7 +2714,7 @@
 func (m *WindowingStrategy) String() string { return proto.CompactTextString(m) }
 func (*WindowingStrategy) ProtoMessage()    {}
 func (*WindowingStrategy) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d5fa30116074ddde, []int{24}
+	return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{24}
 }
 func (m *WindowingStrategy) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_WindowingStrategy.Unmarshal(m, b)
@@ -2767,7 +2817,7 @@
 func (m *MergeStatus) String() string { return proto.CompactTextString(m) }
 func (*MergeStatus) ProtoMessage()    {}
 func (*MergeStatus) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d5fa30116074ddde, []int{25}
+	return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{25}
 }
 func (m *MergeStatus) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_MergeStatus.Unmarshal(m, b)
@@ -2800,7 +2850,7 @@
 func (m *AccumulationMode) String() string { return proto.CompactTextString(m) }
 func (*AccumulationMode) ProtoMessage()    {}
 func (*AccumulationMode) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d5fa30116074ddde, []int{26}
+	return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{26}
 }
 func (m *AccumulationMode) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_AccumulationMode.Unmarshal(m, b)
@@ -2832,7 +2882,7 @@
 func (m *ClosingBehavior) String() string { return proto.CompactTextString(m) }
 func (*ClosingBehavior) ProtoMessage()    {}
 func (*ClosingBehavior) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d5fa30116074ddde, []int{27}
+	return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{27}
 }
 func (m *ClosingBehavior) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_ClosingBehavior.Unmarshal(m, b)
@@ -2864,7 +2914,7 @@
 func (m *OnTimeBehavior) String() string { return proto.CompactTextString(m) }
 func (*OnTimeBehavior) ProtoMessage()    {}
 func (*OnTimeBehavior) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d5fa30116074ddde, []int{28}
+	return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{28}
 }
 func (m *OnTimeBehavior) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_OnTimeBehavior.Unmarshal(m, b)
@@ -2896,7 +2946,7 @@
 func (m *OutputTime) String() string { return proto.CompactTextString(m) }
 func (*OutputTime) ProtoMessage()    {}
 func (*OutputTime) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d5fa30116074ddde, []int{29}
+	return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{29}
 }
 func (m *OutputTime) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_OutputTime.Unmarshal(m, b)
@@ -2927,7 +2977,7 @@
 func (m *TimeDomain) String() string { return proto.CompactTextString(m) }
 func (*TimeDomain) ProtoMessage()    {}
 func (*TimeDomain) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d5fa30116074ddde, []int{30}
+	return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{30}
 }
 func (m *TimeDomain) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_TimeDomain.Unmarshal(m, b)
@@ -2977,7 +3027,7 @@
 func (m *Trigger) String() string { return proto.CompactTextString(m) }
 func (*Trigger) ProtoMessage()    {}
 func (*Trigger) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d5fa30116074ddde, []int{31}
+	return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{31}
 }
 func (m *Trigger) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Trigger.Unmarshal(m, b)
@@ -3418,7 +3468,7 @@
 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_d5fa30116074ddde, []int{31, 0}
+	return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{31, 0}
 }
 func (m *Trigger_AfterAll) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Trigger_AfterAll.Unmarshal(m, b)
@@ -3457,7 +3507,7 @@
 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_d5fa30116074ddde, []int{31, 1}
+	return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{31, 1}
 }
 func (m *Trigger_AfterAny) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Trigger_AfterAny.Unmarshal(m, b)
@@ -3497,7 +3547,7 @@
 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_d5fa30116074ddde, []int{31, 2}
+	return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{31, 2}
 }
 func (m *Trigger_AfterEach) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Trigger_AfterEach.Unmarshal(m, b)
@@ -3543,7 +3593,7 @@
 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_d5fa30116074ddde, []int{31, 3}
+	return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{31, 3}
 }
 func (m *Trigger_AfterEndOfWindow) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Trigger_AfterEndOfWindow.Unmarshal(m, b)
@@ -3591,7 +3641,7 @@
 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_d5fa30116074ddde, []int{31, 4}
+	return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{31, 4}
 }
 func (m *Trigger_AfterProcessingTime) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Trigger_AfterProcessingTime.Unmarshal(m, b)
@@ -3632,7 +3682,7 @@
 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_d5fa30116074ddde, []int{31, 5}
+	return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{31, 5}
 }
 func (m *Trigger_AfterSynchronizedProcessingTime) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Trigger_AfterSynchronizedProcessingTime.Unmarshal(m, b)
@@ -3664,7 +3714,7 @@
 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_d5fa30116074ddde, []int{31, 6}
+	return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{31, 6}
 }
 func (m *Trigger_Default) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Trigger_Default.Unmarshal(m, b)
@@ -3696,7 +3746,7 @@
 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_d5fa30116074ddde, []int{31, 7}
+	return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{31, 7}
 }
 func (m *Trigger_ElementCount) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Trigger_ElementCount.Unmarshal(m, b)
@@ -3735,7 +3785,7 @@
 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_d5fa30116074ddde, []int{31, 8}
+	return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{31, 8}
 }
 func (m *Trigger_Never) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Trigger_Never.Unmarshal(m, b)
@@ -3767,7 +3817,7 @@
 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_d5fa30116074ddde, []int{31, 9}
+	return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{31, 9}
 }
 func (m *Trigger_Always) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Trigger_Always.Unmarshal(m, b)
@@ -3803,7 +3853,7 @@
 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_d5fa30116074ddde, []int{31, 10}
+	return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{31, 10}
 }
 func (m *Trigger_OrFinally) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Trigger_OrFinally.Unmarshal(m, b)
@@ -3851,7 +3901,7 @@
 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_d5fa30116074ddde, []int{31, 11}
+	return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{31, 11}
 }
 func (m *Trigger_Repeat) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Trigger_Repeat.Unmarshal(m, b)
@@ -3896,7 +3946,7 @@
 func (m *TimestampTransform) String() string { return proto.CompactTextString(m) }
 func (*TimestampTransform) ProtoMessage()    {}
 func (*TimestampTransform) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d5fa30116074ddde, []int{32}
+	return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{32}
 }
 func (m *TimestampTransform) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_TimestampTransform.Unmarshal(m, b)
@@ -4037,7 +4087,7 @@
 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_d5fa30116074ddde, []int{32, 0}
+	return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{32, 0}
 }
 func (m *TimestampTransform_Delay) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_TimestampTransform_Delay.Unmarshal(m, b)
@@ -4080,7 +4130,7 @@
 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_d5fa30116074ddde, []int{32, 1}
+	return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{32, 1}
 }
 func (m *TimestampTransform_AlignTo) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_TimestampTransform_AlignTo.Unmarshal(m, b)
@@ -4148,7 +4198,7 @@
 func (m *SideInput) String() string { return proto.CompactTextString(m) }
 func (*SideInput) ProtoMessage()    {}
 func (*SideInput) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d5fa30116074ddde, []int{33}
+	return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{33}
 }
 func (m *SideInput) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_SideInput.Unmarshal(m, b)
@@ -4206,7 +4256,7 @@
 func (m *Environment) String() string { return proto.CompactTextString(m) }
 func (*Environment) ProtoMessage()    {}
 func (*Environment) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d5fa30116074ddde, []int{34}
+	return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{34}
 }
 func (m *Environment) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Environment.Unmarshal(m, b)
@@ -4250,7 +4300,7 @@
 func (m *StandardEnvironments) String() string { return proto.CompactTextString(m) }
 func (*StandardEnvironments) ProtoMessage()    {}
 func (*StandardEnvironments) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d5fa30116074ddde, []int{35}
+	return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{35}
 }
 func (m *StandardEnvironments) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_StandardEnvironments.Unmarshal(m, b)
@@ -4282,7 +4332,7 @@
 func (m *DockerPayload) String() string { return proto.CompactTextString(m) }
 func (*DockerPayload) ProtoMessage()    {}
 func (*DockerPayload) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d5fa30116074ddde, []int{36}
+	return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{36}
 }
 func (m *DockerPayload) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_DockerPayload.Unmarshal(m, b)
@@ -4323,7 +4373,7 @@
 func (m *ProcessPayload) String() string { return proto.CompactTextString(m) }
 func (*ProcessPayload) ProtoMessage()    {}
 func (*ProcessPayload) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d5fa30116074ddde, []int{37}
+	return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{37}
 }
 func (m *ProcessPayload) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_ProcessPayload.Unmarshal(m, b)
@@ -4383,7 +4433,7 @@
 func (m *ExternalPayload) String() string { return proto.CompactTextString(m) }
 func (*ExternalPayload) ProtoMessage()    {}
 func (*ExternalPayload) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d5fa30116074ddde, []int{38}
+	return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{38}
 }
 func (m *ExternalPayload) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_ExternalPayload.Unmarshal(m, b)
@@ -4434,7 +4484,7 @@
 func (m *SdkFunctionSpec) String() string { return proto.CompactTextString(m) }
 func (*SdkFunctionSpec) ProtoMessage()    {}
 func (*SdkFunctionSpec) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d5fa30116074ddde, []int{39}
+	return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{39}
 }
 func (m *SdkFunctionSpec) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_SdkFunctionSpec.Unmarshal(m, b)
@@ -4513,7 +4563,7 @@
 func (m *FunctionSpec) String() string { return proto.CompactTextString(m) }
 func (*FunctionSpec) ProtoMessage()    {}
 func (*FunctionSpec) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d5fa30116074ddde, []int{40}
+	return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{40}
 }
 func (m *FunctionSpec) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_FunctionSpec.Unmarshal(m, b)
@@ -4560,7 +4610,7 @@
 func (m *DisplayData) String() string { return proto.CompactTextString(m) }
 func (*DisplayData) ProtoMessage()    {}
 func (*DisplayData) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d5fa30116074ddde, []int{41}
+	return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{41}
 }
 func (m *DisplayData) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_DisplayData.Unmarshal(m, b)
@@ -4604,7 +4654,7 @@
 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_d5fa30116074ddde, []int{41, 0}
+	return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{41, 0}
 }
 func (m *DisplayData_Identifier) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_DisplayData_Identifier.Unmarshal(m, b)
@@ -4668,7 +4718,7 @@
 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_d5fa30116074ddde, []int{41, 1}
+	return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{41, 1}
 }
 func (m *DisplayData_Item) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_DisplayData_Item.Unmarshal(m, b)
@@ -4740,7 +4790,7 @@
 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_d5fa30116074ddde, []int{41, 2}
+	return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{41, 2}
 }
 func (m *DisplayData_Type) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_DisplayData_Type.Unmarshal(m, b)
@@ -4794,7 +4844,7 @@
 func (m *MessageWithComponents) String() string { return proto.CompactTextString(m) }
 func (*MessageWithComponents) ProtoMessage()    {}
 func (*MessageWithComponents) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d5fa30116074ddde, []int{42}
+	return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{42}
 }
 func (m *MessageWithComponents) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_MessageWithComponents.Unmarshal(m, b)
@@ -5238,7 +5288,7 @@
 func (m *ExecutableStagePayload) String() string { return proto.CompactTextString(m) }
 func (*ExecutableStagePayload) ProtoMessage()    {}
 func (*ExecutableStagePayload) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d5fa30116074ddde, []int{43}
+	return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{43}
 }
 func (m *ExecutableStagePayload) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_ExecutableStagePayload.Unmarshal(m, b)
@@ -5330,7 +5380,7 @@
 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_d5fa30116074ddde, []int{43, 0}
+	return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{43, 0}
 }
 func (m *ExecutableStagePayload_SideInputId) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_ExecutableStagePayload_SideInputId.Unmarshal(m, b)
@@ -5380,7 +5430,7 @@
 func (m *ExecutableStagePayload_UserStateId) String() string { return proto.CompactTextString(m) }
 func (*ExecutableStagePayload_UserStateId) ProtoMessage()    {}
 func (*ExecutableStagePayload_UserStateId) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d5fa30116074ddde, []int{43, 1}
+	return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{43, 1}
 }
 func (m *ExecutableStagePayload_UserStateId) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_ExecutableStagePayload_UserStateId.Unmarshal(m, b)
@@ -5430,7 +5480,7 @@
 func (m *ExecutableStagePayload_TimerId) String() string { return proto.CompactTextString(m) }
 func (*ExecutableStagePayload_TimerId) ProtoMessage()    {}
 func (*ExecutableStagePayload_TimerId) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d5fa30116074ddde, []int{43, 2}
+	return fileDescriptor_beam_runner_api_b87c0d18be5b2d09, []int{43, 2}
 }
 func (m *ExecutableStagePayload_TimerId) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_ExecutableStagePayload_TimerId.Unmarshal(m, b)
@@ -5504,7 +5554,7 @@
 	proto.RegisterType((*Parameter)(nil), "org.apache.beam.model.pipeline.v1.Parameter")
 	proto.RegisterType((*Parameter_Type)(nil), "org.apache.beam.model.pipeline.v1.Parameter.Type")
 	proto.RegisterType((*StateSpec)(nil), "org.apache.beam.model.pipeline.v1.StateSpec")
-	proto.RegisterType((*ValueStateSpec)(nil), "org.apache.beam.model.pipeline.v1.ValueStateSpec")
+	proto.RegisterType((*ReadModifyWriteStateSpec)(nil), "org.apache.beam.model.pipeline.v1.ReadModifyWriteStateSpec")
 	proto.RegisterType((*BagStateSpec)(nil), "org.apache.beam.model.pipeline.v1.BagStateSpec")
 	proto.RegisterType((*CombiningStateSpec)(nil), "org.apache.beam.model.pipeline.v1.CombiningStateSpec")
 	proto.RegisterType((*MapStateSpec)(nil), "org.apache.beam.model.pipeline.v1.MapStateSpec")
@@ -5589,327 +5639,330 @@
 }
 
 func init() {
-	proto.RegisterFile("beam_runner_api.proto", fileDescriptor_beam_runner_api_d5fa30116074ddde)
+	proto.RegisterFile("beam_runner_api.proto", fileDescriptor_beam_runner_api_b87c0d18be5b2d09)
 }
 
-var fileDescriptor_beam_runner_api_d5fa30116074ddde = []byte{
-	// 5086 bytes of a gzipped FileDescriptorProto
-	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xbc, 0x5c, 0xdd, 0x6f, 0xdb, 0x58,
-	0x76, 0xd7, 0xb7, 0xa5, 0x23, 0x59, 0xa6, 0xaf, 0x9d, 0xac, 0xc3, 0x9d, 0x9d, 0x24, 0x9c, 0xec,
-	0x4c, 0x76, 0x76, 0x46, 0x93, 0x38, 0xc9, 0x24, 0xf1, 0xcc, 0x66, 0x56, 0xb2, 0xa8, 0x98, 0x89,
-	0xbe, 0x86, 0x92, 0x9d, 0x64, 0x76, 0x76, 0xb8, 0xb4, 0x78, 0x65, 0x13, 0xa6, 0x48, 0x2d, 0x49,
-	0x39, 0xa3, 0xc5, 0x2e, 0x0a, 0xf4, 0x61, 0x50, 0xa0, 0x40, 0xd1, 0x3e, 0xf4, 0x61, 0x9e, 0x0a,
-	0xec, 0x02, 0x05, 0xda, 0x3e, 0xf4, 0x63, 0xdb, 0x02, 0x7d, 0xdd, 0x6e, 0xff, 0x82, 0x16, 0x28,
-	0xd0, 0xff, 0xa2, 0x2d, 0xf6, 0xa1, 0x7d, 0x2a, 0xee, 0x07, 0x29, 0x4a, 0xb6, 0x33, 0x92, 0x1d,
-	0xf4, 0x4d, 0x3c, 0xbc, 0xe7, 0x77, 0xee, 0x3d, 0xf7, 0xde, 0x73, 0xcf, 0x39, 0xf7, 0x50, 0x70,
-	0x69, 0x1f, 0xeb, 0x03, 0xcd, 0x1d, 0xd9, 0x36, 0x76, 0x35, 0x7d, 0x68, 0x96, 0x86, 0xae, 0xe3,
-	0x3b, 0xe8, 0xba, 0xe3, 0x1e, 0x94, 0xf4, 0xa1, 0xde, 0x3b, 0xc4, 0x25, 0xd2, 0xa2, 0x34, 0x70,
-	0x0c, 0x6c, 0x95, 0x86, 0xe6, 0x10, 0x5b, 0xa6, 0x8d, 0x4b, 0xc7, 0xb7, 0xc5, 0x15, 0x6c, 0x1b,
-	0x43, 0xc7, 0xb4, 0x7d, 0x8f, 0xf1, 0x88, 0x57, 0x0e, 0x1c, 0xe7, 0xc0, 0xc2, 0x1f, 0xd0, 0xa7,
-	0xfd, 0x51, 0xff, 0x03, 0xdd, 0x1e, 0xf3, 0x57, 0xd7, 0x66, 0x5f, 0x19, 0xd8, 0xeb, 0xb9, 0xe6,
-	0xd0, 0x77, 0x5c, 0xd6, 0x42, 0xfa, 0x4d, 0x1c, 0x96, 0x2b, 0x58, 0x1f, 0x6c, 0x3b, 0xb6, 0xe7,
-	0xeb, 0xb6, 0xef, 0x49, 0x7f, 0x13, 0x87, 0x5c, 0xf8, 0x84, 0x6e, 0xc3, 0x7a, 0x43, 0x69, 0x6a,
-	0x5d, 0xa5, 0x21, 0x77, 0xba, 0xe5, 0x46, 0x5b, 0x6b, 0x28, 0xf5, 0xba, 0xd2, 0x11, 0x62, 0xe2,
-	0xb7, 0xfe, 0xf2, 0xef, 0xff, 0xf7, 0x37, 0xe9, 0xd5, 0xf7, 0x1f, 0x6e, 0x6e, 0xde, 0xb9, 0x73,
-	0x7f, 0xf3, 0xd6, 0x9d, 0x0f, 0x1f, 0xdc, 0xbb, 0x7b, 0xff, 0xfe, 0x3d, 0x74, 0x0b, 0xd6, 0x1b,
-	0xe5, 0xe7, 0x27, 0x59, 0xe2, 0xe2, 0x65, 0xca, 0x22, 0x9c, 0xe0, 0x78, 0x04, 0xd2, 0xe3, 0x7a,
-	0xab, 0x52, 0xae, 0x6b, 0xcf, 0x94, 0x66, 0xb5, 0xf5, 0x4c, 0x3b, 0x95, 0x3f, 0x31, 0xcd, 0x7f,
-	0xfb, 0xe1, 0xbd, 0x5b, 0x77, 0x29, 0xbf, 0xf4, 0x8f, 0x59, 0x80, 0x6d, 0x67, 0x30, 0x74, 0x6c,
-	0x4c, 0xfa, 0xfc, 0x63, 0x00, 0xdf, 0xd5, 0x6d, 0xaf, 0xef, 0xb8, 0x03, 0x6f, 0x23, 0x7e, 0x2d,
-	0x79, 0x33, 0xbf, 0xf9, 0x83, 0xd2, 0x37, 0x6a, 0xb6, 0x34, 0x81, 0x28, 0x75, 0x43, 0x7e, 0xd9,
-	0xf6, 0xdd, 0xb1, 0x1a, 0x01, 0x44, 0x3d, 0x28, 0x0c, 0x7b, 0x8e, 0x65, 0xe1, 0x9e, 0x6f, 0x3a,
-	0xb6, 0xb7, 0x91, 0xa0, 0x02, 0x3e, 0x59, 0x4c, 0x40, 0x3b, 0x82, 0xc0, 0x44, 0x4c, 0x81, 0xa2,
-	0x31, 0xac, 0xbf, 0x34, 0x6d, 0xc3, 0x79, 0x69, 0xda, 0x07, 0x9a, 0xe7, 0xbb, 0xba, 0x8f, 0x0f,
-	0x4c, 0xec, 0x6d, 0x24, 0xa9, 0xb0, 0xda, 0x62, 0xc2, 0x9e, 0x05, 0x48, 0x9d, 0x10, 0x88, 0xc9,
-	0x5c, 0x7b, 0x79, 0xf2, 0x0d, 0xfa, 0x14, 0x32, 0x3d, 0xc7, 0xc0, 0xae, 0xb7, 0x91, 0xa2, 0xc2,
-	0x1e, 0x2e, 0x26, 0x6c, 0x9b, 0xf2, 0x32, 0x7c, 0x0e, 0x44, 0x54, 0x86, 0xed, 0x63, 0xd3, 0x75,
-	0xec, 0x01, 0x69, 0xb3, 0x91, 0x3e, 0x8f, 0xca, 0xe4, 0x08, 0x02, 0x57, 0x59, 0x14, 0x54, 0xb4,
-	0x60, 0x65, 0x66, 0xda, 0x90, 0x00, 0xc9, 0x23, 0x3c, 0xde, 0x88, 0x5f, 0x8b, 0xdf, 0xcc, 0xa9,
-	0xe4, 0x27, 0xda, 0x86, 0xf4, 0xb1, 0x6e, 0x8d, 0xf0, 0x46, 0xe2, 0x5a, 0xfc, 0x66, 0x7e, 0xf3,
-	0xfd, 0x39, 0xba, 0xd0, 0x0e, 0x51, 0x55, 0xc6, 0xbb, 0x95, 0x78, 0x10, 0x17, 0x1d, 0x58, 0x3d,
-	0x31, 0x87, 0xa7, 0xc8, 0xab, 0x4e, 0xcb, 0x2b, 0xcd, 0x23, 0x6f, 0x3b, 0x84, 0x8d, 0x0a, 0xfc,
-	0x39, 0x6c, 0x9c, 0x35, 0x8f, 0xa7, 0xc8, 0x7d, 0x32, 0x2d, 0xf7, 0xee, 0x1c, 0x72, 0x67, 0xd1,
-	0xc7, 0x51, 0xe9, 0x3d, 0xc8, 0x47, 0x26, 0xf6, 0x14, 0x81, 0x8f, 0xa6, 0x05, 0xde, 0x9c, 0x6b,
-	0x6e, 0x0d, 0xec, 0xce, 0xe8, 0xf4, 0xc4, 0x24, 0xbf, 0x1e, 0x9d, 0x46, 0x60, 0x23, 0x02, 0xa5,
-	0xff, 0x88, 0x43, 0xb6, 0xcd, 0x9b, 0xa1, 0x06, 0x40, 0x2f, 0x5c, 0x6d, 0x54, 0xde, 0x7c, 0xeb,
-	0x63, 0xb2, 0x44, 0xd5, 0x08, 0x00, 0x7a, 0x0f, 0x90, 0xeb, 0x38, 0xbe, 0x16, 0x5a, 0x0e, 0xcd,
-	0x34, 0x98, 0xb1, 0xc8, 0xa9, 0x02, 0x79, 0x13, 0x2e, 0x2b, 0xc5, 0x20, 0x9b, 0xae, 0x60, 0x98,
-	0xde, 0xd0, 0xd2, 0xc7, 0x9a, 0xa1, 0xfb, 0xfa, 0x46, 0x72, 0xee, 0xa1, 0x55, 0x19, 0x5b, 0x55,
-	0xf7, 0x75, 0x35, 0x6f, 0x4c, 0x1e, 0xa4, 0x3f, 0x4c, 0x01, 0x4c, 0xd6, 0x2e, 0xba, 0x0a, 0xf9,
-	0x91, 0x6d, 0xfe, 0x74, 0x84, 0x35, 0x5b, 0x1f, 0xe0, 0x8d, 0x34, 0xd5, 0x27, 0x30, 0x52, 0x53,
-	0x1f, 0x60, 0xb4, 0x0d, 0x29, 0x6f, 0x88, 0x7b, 0x7c, 0xe4, 0x1f, 0xcc, 0x21, 0xba, 0x36, 0xb2,
-	0xe9, 0x32, 0xed, 0x0c, 0x71, 0x4f, 0xa5, 0xcc, 0xe8, 0x06, 0x2c, 0x7b, 0xa3, 0xfd, 0x88, 0xf9,
-	0x65, 0x03, 0x9e, 0x26, 0x12, 0x13, 0x63, 0xda, 0xc3, 0x91, 0x1f, 0xd8, 0xb3, 0x87, 0x0b, 0x6d,
-	0xc3, 0x92, 0x42, 0x79, 0xb9, 0x89, 0x61, 0x40, 0xa8, 0x0b, 0x4b, 0xce, 0xc8, 0xa7, 0x98, 0xcc,
-	0x6c, 0x6d, 0x2d, 0x86, 0xd9, 0x62, 0xcc, 0x0c, 0x34, 0x80, 0x3a, 0x31, 0x2d, 0x99, 0x0b, 0x4f,
-	0x8b, 0xf8, 0x10, 0xf2, 0x91, 0xfe, 0x9f, 0xb2, 0xbc, 0xd7, 0xa3, 0xcb, 0x3b, 0x17, 0xdd, 0x1f,
-	0x5b, 0x50, 0x88, 0x76, 0x73, 0x11, 0x5e, 0xe9, 0x1f, 0x96, 0x61, 0xad, 0xe3, 0xeb, 0xb6, 0xa1,
-	0xbb, 0xc6, 0x64, 0xd8, 0x9e, 0xf4, 0x17, 0x49, 0x80, 0xb6, 0x6b, 0x0e, 0x4c, 0xdf, 0x3c, 0xc6,
-	0x1e, 0xfa, 0x1e, 0x64, 0xda, 0x65, 0x55, 0xab, 0xb6, 0x84, 0x98, 0xf8, 0x9d, 0x5f, 0x92, 0xe3,
-	0xf6, 0x5b, 0x64, 0x80, 0x5b, 0xe1, 0xe4, 0x6d, 0x0d, 0x75, 0xd7, 0x70, 0xb6, 0x8e, 0x6f, 0xa3,
-	0xf7, 0x60, 0xa9, 0x56, 0x2f, 0x77, 0xbb, 0x72, 0x53, 0x88, 0x8b, 0x57, 0x69, 0xdb, 0x2b, 0x33,
-	0x6d, 0xfb, 0x96, 0xee, 0xfb, 0xd8, 0x26, 0xad, 0x3f, 0x84, 0xc2, 0x63, 0xb5, 0xb5, 0xdb, 0xd6,
-	0x2a, 0x2f, 0xb4, 0xa7, 0xf2, 0x0b, 0x21, 0x21, 0xde, 0xa0, 0x2c, 0x6f, 0xce, 0xb0, 0x1c, 0xb8,
-	0xce, 0x68, 0xa8, 0xed, 0x8f, 0xb5, 0x23, 0x3c, 0xe6, 0x52, 0x94, 0x46, 0x7b, 0xb7, 0xde, 0x91,
-	0x85, 0xe4, 0x19, 0x52, 0xcc, 0xc1, 0x70, 0x64, 0x79, 0x98, 0xb4, 0xbe, 0x0f, 0xc5, 0x72, 0xa7,
-	0xa3, 0x3c, 0x6e, 0x72, 0x4f, 0xa2, 0x23, 0xa4, 0xc4, 0xb7, 0x28, 0xd3, 0x77, 0x66, 0x98, 0xd8,
-	0xc9, 0xa7, 0x99, 0xb6, 0x4f, 0x07, 0x73, 0x07, 0xf2, 0x5d, 0xb9, 0xd3, 0xd5, 0x3a, 0x5d, 0x55,
-	0x2e, 0x37, 0x84, 0xb4, 0x28, 0x51, 0xae, 0x37, 0x66, 0xb8, 0x7c, 0xec, 0xf9, 0x9e, 0xef, 0x12,
-	0xe2, 0xf1, 0x6d, 0x74, 0x17, 0xf2, 0x8d, 0x72, 0x3b, 0x14, 0x95, 0x39, 0x43, 0xd4, 0x40, 0x1f,
-	0x6a, 0x4c, 0x9c, 0x47, 0xb8, 0x1e, 0xc0, 0x72, 0x43, 0x56, 0x1f, 0xcb, 0x21, 0xdf, 0x92, 0xf8,
-	0x5d, 0xca, 0x77, 0x75, 0x96, 0x0f, 0xbb, 0x07, 0x38, 0xc2, 0x29, 0xf9, 0xb0, 0x5e, 0xc5, 0x43,
-	0x17, 0xf7, 0x74, 0x1f, 0x1b, 0x91, 0x49, 0x7b, 0x1b, 0x52, 0xaa, 0x5c, 0xae, 0x0a, 0x31, 0xf1,
-	0x0d, 0x0a, 0x74, 0x79, 0x06, 0xc8, 0xc5, 0xba, 0xc1, 0xfb, 0xbb, 0xad, 0xca, 0xe5, 0xae, 0xac,
-	0xed, 0x29, 0xf2, 0x33, 0x21, 0x7e, 0x46, 0x7f, 0x7b, 0x2e, 0xd6, 0x7d, 0xac, 0x1d, 0x9b, 0xf8,
-	0x25, 0x91, 0xfa, 0x5f, 0x71, 0xee, 0x5d, 0x79, 0xa6, 0x8f, 0x3d, 0xf4, 0x31, 0xac, 0x6c, 0xb7,
-	0x1a, 0x15, 0xa5, 0x29, 0x6b, 0x6d, 0x59, 0xa5, 0x73, 0x19, 0x13, 0xdf, 0xa1, 0x40, 0xd7, 0x67,
-	0x81, 0x9c, 0xc1, 0xbe, 0x69, 0x63, 0x6d, 0x88, 0xdd, 0x60, 0x3a, 0x1f, 0x81, 0x10, 0x70, 0x33,
-	0x97, 0xaf, 0xfe, 0x42, 0x88, 0x8b, 0x37, 0x29, 0xbb, 0x74, 0x06, 0xfb, 0x81, 0xe5, 0xec, 0xeb,
-	0x96, 0x45, 0xf9, 0x6f, 0x41, 0x4e, 0x95, 0x3b, 0x3b, 0xbb, 0xb5, 0x5a, 0x5d, 0x16, 0x12, 0xe2,
-	0x75, 0xca, 0xf8, 0xed, 0x13, 0xe3, 0xf5, 0x0e, 0x47, 0xfd, 0xbe, 0x85, 0xf9, 0xa0, 0x9f, 0xa9,
-	0x4a, 0x57, 0xd6, 0x6a, 0x4a, 0x5d, 0xee, 0x08, 0xc9, 0xb3, 0xd6, 0x83, 0x6b, 0xfa, 0x58, 0xeb,
-	0x9b, 0x16, 0xa6, 0xaa, 0xfe, 0x5d, 0x02, 0x56, 0xb7, 0x99, 0xfc, 0x88, 0x67, 0xa9, 0x82, 0x38,
-	0x33, 0x76, 0xad, 0xad, 0xca, 0x9c, 0x24, 0xc4, 0xc4, 0x4d, 0x0a, 0xfd, 0xde, 0xab, 0xd5, 0xa0,
-	0x91, 0x19, 0x64, 0x24, 0xd2, 0xbf, 0x7d, 0x90, 0x66, 0x31, 0xd9, 0xf2, 0x28, 0x6f, 0x6f, 0xef,
-	0x36, 0x76, 0xeb, 0xe5, 0x6e, 0x4b, 0x25, 0xce, 0xf3, 0x16, 0xc5, 0xbe, 0xfb, 0x0d, 0xd8, 0x6c,
-	0xcd, 0xe8, 0xbd, 0xde, 0x68, 0x30, 0xb2, 0x74, 0xdf, 0x71, 0xe9, 0x92, 0xfb, 0x1c, 0xae, 0xce,
-	0xca, 0x90, 0x9f, 0x77, 0xd5, 0xf2, 0x76, 0x57, 0x6b, 0xed, 0x76, 0xdb, 0xbb, 0x5d, 0xe2, 0x5d,
-	0xdf, 0xa7, 0x02, 0x6e, 0x7f, 0x83, 0x00, 0xfc, 0xa5, 0xef, 0xea, 0x3d, 0x5f, 0xe3, 0x16, 0x92,
-	0xa0, 0x3f, 0x81, 0xcb, 0xe1, 0x9c, 0x92, 0x2d, 0x2e, 0x57, 0xb5, 0xbd, 0x72, 0x7d, 0x97, 0x2a,
-	0xbb, 0x44, 0x41, 0x6f, 0x9e, 0x35, 0xb3, 0x64, 0xb3, 0x63, 0x43, 0xa3, 0x66, 0x8a, 0xea, 0xfd,
-	0x8f, 0x52, 0x70, 0xa5, 0x33, 0xb4, 0x4c, 0xdf, 0xd7, 0xf7, 0x2d, 0xdc, 0xd6, 0xdd, 0xaa, 0x13,
-	0xd1, 0x7f, 0x1d, 0x2e, 0xb5, 0xcb, 0x8a, 0xaa, 0x3d, 0x53, 0xba, 0x3b, 0x9a, 0x2a, 0x77, 0xba,
-	0xaa, 0xb2, 0xdd, 0x55, 0x5a, 0x4d, 0x21, 0x26, 0xde, 0xa6, 0x82, 0xbe, 0x3f, 0x23, 0xc8, 0x33,
-	0xfa, 0xda, 0x50, 0x37, 0x5d, 0xed, 0xa5, 0xe9, 0x1f, 0x6a, 0x2e, 0xf6, 0x7c, 0xd7, 0xa4, 0x47,
-	0x16, 0xe9, 0x77, 0x15, 0x56, 0x3b, 0xed, 0xba, 0xd2, 0x9d, 0x42, 0x8a, 0x8b, 0xef, 0x53, 0xa4,
-	0x77, 0x4e, 0x41, 0xf2, 0x48, 0xc7, 0x66, 0x51, 0x9a, 0x70, 0xb9, 0xad, 0xb6, 0xb6, 0xe5, 0x4e,
-	0x87, 0xe8, 0x55, 0xae, 0x6a, 0x72, 0x5d, 0x6e, 0xc8, 0x4d, 0xaa, 0xd2, 0xd3, 0xd7, 0x03, 0xed,
-	0x94, 0xeb, 0xf4, 0xb0, 0xe7, 0x11, 0x95, 0x62, 0x43, 0xc3, 0x16, 0xa6, 0x1e, 0x0f, 0xc1, 0xab,
-	0x80, 0x10, 0xe0, 0x85, 0x48, 0x49, 0xf1, 0x3d, 0x8a, 0xf4, 0xf6, 0x2b, 0x90, 0xa2, 0x18, 0xcf,
-	0xe1, 0xdb, 0x6c, 0x64, 0xe5, 0x66, 0x55, 0xeb, 0x28, 0x9f, 0xc9, 0xd1, 0x21, 0x12, 0x9b, 0x78,
-	0xfa, 0x5c, 0x4f, 0xc6, 0xa8, 0xdb, 0x86, 0xe6, 0x99, 0x3f, 0xc3, 0xd1, 0xc1, 0x52, 0x64, 0x07,
-	0xde, 0x09, 0x7a, 0x47, 0x70, 0x27, 0xa3, 0xa5, 0xa2, 0xa6, 0xa4, 0xa4, 0xc5, 0x0a, 0x95, 0xf2,
-	0xf1, 0x2b, 0x3a, 0x4d, 0x64, 0x84, 0xc3, 0xa7, 0x52, 0x67, 0x04, 0x4a, 0xbf, 0x1f, 0x87, 0xcb,
-	0xc1, 0xb9, 0xd5, 0x31, 0x0d, 0x4c, 0xcf, 0xce, 0xee, 0x78, 0x88, 0x3d, 0xe9, 0x10, 0x52, 0xb2,
-	0x3d, 0x1a, 0xa0, 0x0f, 0x20, 0xab, 0x74, 0x65, 0xb5, 0x5c, 0xa9, 0x93, 0x3d, 0x18, 0x35, 0x09,
-	0x9e, 0x69, 0x60, 0x8d, 0x3a, 0x08, 0x5b, 0xa6, 0x8f, 0x5d, 0xb2, 0xa4, 0xc8, 0x20, 0x3e, 0x80,
-	0x6c, 0x63, 0xb7, 0xde, 0x55, 0x1a, 0xe5, 0xb6, 0x10, 0x3f, 0x8b, 0x61, 0x30, 0xb2, 0x7c, 0x73,
-	0xa0, 0x0f, 0x49, 0x27, 0x7e, 0x99, 0x80, 0x7c, 0xc4, 0x2d, 0x9f, 0xf5, 0xa5, 0xe2, 0x27, 0x7c,
-	0xa9, 0x2b, 0x90, 0xa5, 0xa1, 0x8f, 0x66, 0x1a, 0xfc, 0x28, 0x5e, 0xa2, 0xcf, 0x8a, 0x81, 0xda,
-	0x00, 0xa6, 0xa7, 0xed, 0x3b, 0x23, 0xdb, 0xc0, 0x06, 0xf5, 0xf3, 0x8a, 0x9b, 0xb7, 0xe7, 0x70,
-	0x28, 0x14, 0xaf, 0xc2, 0x78, 0x4a, 0x64, 0xd0, 0x6a, 0xce, 0x0c, 0x9e, 0xd1, 0x26, 0x5c, 0x3a,
-	0x11, 0x2b, 0x8e, 0x89, 0xe4, 0x14, 0x95, 0x7c, 0x22, 0xc8, 0x1b, 0x2b, 0xc6, 0x09, 0xc7, 0x26,
-	0x7d, 0x71, 0x7f, 0xf3, 0xeb, 0x25, 0x28, 0xd0, 0x0d, 0xdb, 0xd6, 0xc7, 0x96, 0xa3, 0x1b, 0xe8,
-	0x31, 0xa4, 0x0d, 0x47, 0xeb, 0xdb, 0xdc, 0xa3, 0xdc, 0x9c, 0x03, 0xbc, 0x63, 0x1c, 0x4d, 0x3b,
-	0x95, 0x86, 0x53, 0xb3, 0x51, 0x1d, 0x60, 0xa8, 0xbb, 0xfa, 0x00, 0xfb, 0x24, 0x2a, 0x65, 0xf1,
-	0xf6, 0x7b, 0xf3, 0xb8, 0x77, 0x01, 0x93, 0x1a, 0xe1, 0x47, 0x3f, 0x81, 0xfc, 0x64, 0x9a, 0x03,
-	0x0f, 0xf4, 0x93, 0xf9, 0xe0, 0xc2, 0xc1, 0x95, 0xc2, 0xb5, 0x18, 0x64, 0x08, 0xbc, 0x90, 0x40,
-	0x25, 0xf8, 0xe4, 0x08, 0x25, 0x2e, 0x71, 0xe0, 0x8f, 0x2e, 0x2e, 0x81, 0x40, 0x10, 0x2d, 0x84,
-	0x12, 0x42, 0x02, 0x91, 0xe0, 0x9b, 0x03, 0xec, 0x72, 0x09, 0xe9, 0xf3, 0x49, 0xe8, 0x12, 0x88,
-	0xa8, 0x04, 0x3f, 0x24, 0xa0, 0x37, 0x01, 0xbc, 0xd0, 0x0e, 0x53, 0xbf, 0x37, 0xab, 0x46, 0x28,
-	0xe8, 0x16, 0xac, 0x47, 0xb6, 0xaa, 0x16, 0xae, 0xf6, 0x25, 0xba, 0xe6, 0x50, 0xe4, 0xdd, 0x36,
-	0x5f, 0xf8, 0x77, 0xe0, 0x92, 0x8b, 0x7f, 0x3a, 0x22, 0x1e, 0x94, 0xd6, 0x37, 0x6d, 0xdd, 0x32,
-	0x7f, 0xa6, 0x93, 0xf7, 0x1b, 0x59, 0x0a, 0xbe, 0x1e, 0xbc, 0xac, 0x45, 0xde, 0x89, 0x47, 0xb0,
-	0x32, 0xa3, 0xe9, 0x53, 0xbc, 0xde, 0xca, 0x74, 0x40, 0x38, 0xcf, 0xd2, 0x08, 0x41, 0xa3, 0xfe,
-	0x35, 0x11, 0x36, 0xad, 0xf4, 0xd7, 0x24, 0x2c, 0x00, 0x9d, 0x11, 0x36, 0xa3, 0xff, 0xd7, 0x23,
-	0x2c, 0x04, 0x8d, 0x7a, 0xff, 0xbf, 0x8e, 0x43, 0x2e, 0xdc, 0x0d, 0xe8, 0x09, 0xa4, 0xfc, 0xf1,
-	0x90, 0xd9, 0xad, 0xe2, 0xe6, 0x87, 0x8b, 0xec, 0xa4, 0x12, 0x31, 0xbd, 0xcc, 0x02, 0x51, 0x0c,
-	0xf1, 0x33, 0x48, 0x11, 0x92, 0xa4, 0x72, 0x63, 0xbc, 0x02, 0xf9, 0xdd, 0x66, 0xa7, 0x2d, 0x6f,
-	0x2b, 0x35, 0x45, 0xae, 0x0a, 0x31, 0x04, 0x90, 0x61, 0x8e, 0xae, 0x10, 0x47, 0xeb, 0x20, 0xb4,
-	0x95, 0xb6, 0x5c, 0x27, 0xae, 0x42, 0xab, 0xcd, 0x8e, 0x89, 0x04, 0xfa, 0x16, 0xac, 0x45, 0x0e,
-	0x0e, 0x8d, 0xf8, 0x25, 0x4f, 0x65, 0x55, 0x48, 0x4a, 0x7f, 0x9b, 0x84, 0x5c, 0xa8, 0x3b, 0xa4,
-	0x02, 0xd0, 0x01, 0x69, 0x91, 0x28, 0x75, 0x1e, 0xc3, 0xb9, 0x47, 0x98, 0x42, 0x98, 0x9d, 0x98,
-	0x9a, 0xa3, 0x30, 0x14, 0xb3, 0x0e, 0xd9, 0x7d, 0xfd, 0x80, 0x21, 0x26, 0xe6, 0x8e, 0x7b, 0x2b,
-	0xfa, 0x41, 0x14, 0x6f, 0x69, 0x5f, 0x3f, 0xa0, 0x68, 0x5f, 0x40, 0x91, 0x79, 0x36, 0xd4, 0x10,
-	0x13, 0x4c, 0x16, 0xc6, 0xdf, 0x9b, 0x2f, 0x8b, 0xc0, 0x18, 0xa3, 0xc8, 0xcb, 0x21, 0x5c, 0xd0,
-	0x5b, 0x12, 0x4b, 0x50, 0xe4, 0xd4, 0xdc, 0xbd, 0x6d, 0xe8, 0xc3, 0xa9, 0xde, 0x0e, 0xf4, 0x61,
-	0x80, 0xe6, 0x61, 0x9f, 0xa1, 0xa5, 0xe7, 0x46, 0xeb, 0x60, 0x7f, 0x0a, 0xcd, 0xc3, 0x3e, 0xf9,
-	0x59, 0xc9, 0xb0, 0xec, 0x81, 0xf4, 0x7d, 0x28, 0x4e, 0x2b, 0x7c, 0xea, 0x2c, 0x8c, 0x4f, 0x9d,
-	0x85, 0xd2, 0x03, 0x28, 0x44, 0x75, 0x89, 0x6e, 0x82, 0x10, 0xf8, 0x02, 0x33, 0x2c, 0x45, 0x4e,
-	0xe7, 0xc6, 0x44, 0xfa, 0x3a, 0x0e, 0xe8, 0xa4, 0xca, 0x88, 0x55, 0x8a, 0xf8, 0xbe, 0xb3, 0x20,
-	0x28, 0xf2, 0x2e, 0xb0, 0x4a, 0x9f, 0xd2, 0xac, 0x0f, 0xf5, 0x46, 0xfb, 0x36, 0x5f, 0x03, 0xe7,
-	0x39, 0xa9, 0x72, 0x1c, 0xa5, 0x66, 0x4b, 0x7b, 0x50, 0x88, 0xea, 0x1c, 0x5d, 0x83, 0x02, 0xf1,
-	0x9c, 0x67, 0x3a, 0x03, 0x47, 0x78, 0x1c, 0x74, 0xe2, 0x06, 0x14, 0xd9, 0xd2, 0x9e, 0x71, 0x1a,
-	0x0a, 0x94, 0xba, 0x3d, 0xd1, 0x56, 0x54, 0xfb, 0x0b, 0x68, 0xeb, 0xab, 0x38, 0xe4, 0x42, 0xbb,
-	0x80, 0x3a, 0xec, 0xf0, 0xd0, 0x0c, 0x67, 0xa0, 0x9b, 0x36, 0xb7, 0x02, 0x9b, 0x73, 0x9a, 0x96,
-	0x2a, 0x65, 0x62, 0x16, 0x80, 0x9e, 0x17, 0x8c, 0x40, 0x86, 0xc0, 0x4e, 0xa4, 0xd9, 0x21, 0x50,
-	0x6a, 0xd0, 0x91, 0x1f, 0x42, 0x2e, 0xf4, 0x63, 0xa4, 0x3b, 0x67, 0x99, 0x8c, 0x65, 0xc8, 0xed,
-	0x36, 0x2b, 0xad, 0xdd, 0x66, 0x55, 0xae, 0x0a, 0x71, 0x94, 0x87, 0xa5, 0xe0, 0x21, 0x21, 0xfd,
-	0x55, 0x1c, 0xf2, 0x2a, 0xd6, 0x8d, 0xc0, 0xc9, 0x78, 0x02, 0x19, 0xcf, 0x19, 0xb9, 0x3d, 0x7c,
-	0x01, 0x2f, 0x83, 0x23, 0xcc, 0xb8, 0x66, 0x89, 0x8b, 0xbb, 0x66, 0x92, 0x01, 0xab, 0x2c, 0xad,
-	0xaa, 0xd8, 0x7e, 0xe8, 0x17, 0xb5, 0x20, 0xc7, 0xb3, 0x0f, 0x17, 0xf2, 0x8d, 0xb2, 0x0c, 0xa4,
-	0x66, 0x4b, 0x7f, 0x1a, 0x87, 0x22, 0x0f, 0x56, 0x03, 0x19, 0xd3, 0xcb, 0x3a, 0xfe, 0x1a, 0x96,
-	0xf5, 0x99, 0x7b, 0x2b, 0x71, 0xd6, 0xde, 0x92, 0xfe, 0x35, 0x03, 0xab, 0x5d, 0xec, 0xf9, 0x1d,
-	0x9a, 0x31, 0x09, 0xba, 0x76, 0xb6, 0x3d, 0x40, 0x2a, 0x64, 0xf0, 0x31, 0x4d, 0xbf, 0x26, 0xe6,
-	0xce, 0xe1, 0x9d, 0x10, 0x50, 0x92, 0x09, 0x84, 0xca, 0x91, 0xc4, 0xff, 0x4c, 0x41, 0x9a, 0x52,
-	0xd0, 0x31, 0xac, 0xbc, 0xd4, 0x7d, 0xec, 0x0e, 0x74, 0xf7, 0x48, 0xa3, 0x6f, 0xb9, 0x62, 0x9e,
-	0x9e, 0x5f, 0x4c, 0xa9, 0x6c, 0x1c, 0xeb, 0x76, 0x0f, 0x3f, 0x0b, 0x80, 0x77, 0x62, 0x6a, 0x31,
-	0x94, 0xc2, 0xe4, 0x7e, 0x15, 0x87, 0x4b, 0x3c, 0xe0, 0x21, 0x07, 0x03, 0xdd, 0x7b, 0x4c, 0x3c,
-	0x33, 0x37, 0xed, 0x8b, 0x8b, 0x6f, 0x87, 0xf0, 0x64, 0x8f, 0xee, 0xc4, 0xd4, 0xb5, 0xe1, 0x14,
-	0x85, 0x75, 0x64, 0x00, 0xcb, 0x81, 0xc1, 0x60, 0xf2, 0xd9, 0xf1, 0x54, 0xbb, 0x90, 0x7c, 0x43,
-	0xe6, 0x81, 0xe7, 0x4e, 0x4c, 0x2d, 0x70, 0x78, 0xfa, 0x4e, 0xbc, 0x0f, 0xc2, 0xac, 0x76, 0xd0,
-	0x5b, 0xb0, 0x6c, 0xe3, 0x97, 0x5a, 0xa8, 0x21, 0x3a, 0x03, 0x49, 0xb5, 0x60, 0xe3, 0x97, 0x61,
-	0x23, 0xb1, 0x02, 0x97, 0x4e, 0x1d, 0x17, 0xfa, 0x1e, 0x08, 0x3a, 0x7b, 0xa1, 0x19, 0x23, 0x97,
-	0x79, 0x8f, 0x0c, 0x60, 0x85, 0xd3, 0xab, 0x9c, 0x2c, 0xba, 0x90, 0x8f, 0xf4, 0x0d, 0xf5, 0x20,
-	0x1b, 0x04, 0xc8, 0xfc, 0x46, 0xf0, 0xf1, 0xb9, 0x46, 0x4d, 0xba, 0xe1, 0xf9, 0xfa, 0x60, 0x88,
-	0x03, 0x6c, 0x35, 0x04, 0xae, 0x2c, 0x41, 0x9a, 0xea, 0x55, 0xfc, 0x11, 0xa0, 0x93, 0x0d, 0xd1,
-	0x3b, 0xb0, 0x82, 0x6d, 0xb2, 0xd4, 0xc3, 0x88, 0x97, 0x76, 0xbe, 0xa0, 0x16, 0x39, 0x39, 0x68,
-	0xf8, 0x06, 0xe4, 0xfc, 0x80, 0x9d, 0xae, 0x91, 0xa4, 0x3a, 0x21, 0x48, 0xff, 0x9d, 0x84, 0xd5,
-	0x67, 0xae, 0xe9, 0xe3, 0x9a, 0x69, 0x61, 0x2f, 0xd8, 0x55, 0x35, 0x48, 0x79, 0xa6, 0x7d, 0x74,
-	0x91, 0x58, 0x8b, 0xf0, 0xa3, 0x1f, 0xc1, 0x0a, 0x89, 0xd2, 0x75, 0x5f, 0xeb, 0xf3, 0x97, 0x17,
-	0x38, 0x14, 0x8b, 0x0c, 0x2a, 0xa0, 0x11, 0x0d, 0x30, 0xa3, 0x85, 0x0d, 0x8d, 0x26, 0xdc, 0x3c,
-	0xba, 0x04, 0xb3, 0x6a, 0x31, 0x20, 0xd3, 0x81, 0x79, 0xe8, 0x63, 0x10, 0xf9, 0xdd, 0xb8, 0x41,
-	0xbc, 0xce, 0x81, 0x69, 0x63, 0x43, 0xf3, 0x0e, 0x75, 0xd7, 0x30, 0xed, 0x03, 0xea, 0xfb, 0x64,
-	0xd5, 0x0d, 0xd6, 0xa2, 0x1a, 0x36, 0xe8, 0xf0, 0xf7, 0x08, 0x4f, 0x47, 0x78, 0x2c, 0x3a, 0xaa,
-	0xce, 0x73, 0x05, 0x36, 0xab, 0xd6, 0x57, 0x85, 0x79, 0xff, 0xaf, 0xb1, 0x89, 0xf4, 0x73, 0x48,
-	0x53, 0xb3, 0xfa, 0x7a, 0xae, 0x69, 0x4a, 0xb0, 0x16, 0x5e, 0x55, 0x85, 0x96, 0x3c, 0xb8, 0xac,
-	0x59, 0x0d, 0x5f, 0x71, 0x43, 0xee, 0x49, 0xff, 0x9e, 0x82, 0x62, 0x90, 0x85, 0x61, 0xf7, 0x80,
-	0xd2, 0x6f, 0x53, 0xfc, 0xf8, 0xbe, 0x01, 0xe9, 0xca, 0x8b, 0xae, 0xdc, 0x11, 0x62, 0xe2, 0x15,
-	0x9a, 0x4a, 0x59, 0xa3, 0xa9, 0x14, 0x8a, 0xba, 0xb5, 0x3f, 0xf6, 0x69, 0x62, 0x0f, 0xdd, 0x82,
-	0x3c, 0x71, 0xf1, 0x9b, 0x8f, 0xb5, 0xdd, 0x6e, 0xed, 0x81, 0x00, 0x53, 0xb9, 0x7c, 0xd6, 0x96,
-	0x44, 0x8c, 0xf6, 0x81, 0x36, 0xf2, 0xfb, 0x0f, 0x08, 0xc7, 0x9b, 0x90, 0x78, 0xba, 0x27, 0xc4,
-	0xc5, 0xcb, 0xb4, 0xa1, 0x10, 0x69, 0x78, 0x74, 0x4c, 0xde, 0xbf, 0x0d, 0x99, 0xbd, 0xb2, 0xaa,
-	0x34, 0xbb, 0x42, 0x42, 0x14, 0x69, 0x9b, 0xf5, 0x48, 0x9b, 0x63, 0xdd, 0x35, 0x6d, 0x9f, 0xb7,
-	0xab, 0xb6, 0x76, 0x2b, 0x75, 0x59, 0xc8, 0x9f, 0xd2, 0xce, 0x70, 0x46, 0x3c, 0x2b, 0xf4, 0x6e,
-	0x24, 0x8d, 0x94, 0x9c, 0xca, 0xa4, 0xb3, 0x96, 0xd1, 0x0c, 0xd2, 0x0d, 0x48, 0x77, 0x95, 0x86,
-	0xac, 0x0a, 0xa9, 0x53, 0xc6, 0x4c, 0x3d, 0x1e, 0x96, 0xe9, 0x5f, 0x51, 0x9a, 0x5d, 0x59, 0xdd,
-	0x0b, 0x2b, 0x1b, 0x84, 0xf4, 0x54, 0xfa, 0x99, 0x03, 0xdb, 0x3e, 0x76, 0x8f, 0x75, 0x8b, 0xa7,
-	0xfa, 0x59, 0xd2, 0x7a, 0xb9, 0x2e, 0x37, 0x1f, 0x77, 0x77, 0xb4, 0xb6, 0x2a, 0xd7, 0x94, 0xe7,
-	0x42, 0x66, 0x2a, 0x4d, 0xc5, 0xf8, 0x2c, 0x6c, 0x1f, 0xf8, 0x87, 0xda, 0xd0, 0xc5, 0x7d, 0xf3,
-	0x4b, 0xce, 0x35, 0x55, 0x47, 0x21, 0x2c, 0x9d, 0xc2, 0xc5, 0xb2, 0xe9, 0x11, 0x59, 0x1f, 0x42,
-	0x91, 0x35, 0x0f, 0xf2, 0xb6, 0x42, 0x76, 0xea, 0xf6, 0x83, 0xb1, 0x85, 0xfb, 0x96, 0x2d, 0x49,
-	0x9a, 0x3e, 0xbd, 0xd4, 0xe9, 0x96, 0xbb, 0xb2, 0x56, 0x21, 0xf1, 0x5a, 0x55, 0x0b, 0x95, 0x97,
-	0x13, 0xbf, 0x47, 0xd9, 0xdf, 0x9a, 0x9a, 0x5b, 0xdd, 0xc7, 0xda, 0xbe, 0xde, 0x3b, 0xc2, 0x86,
-	0x16, 0xd1, 0xa4, 0xf4, 0x07, 0x99, 0xc0, 0x45, 0x8a, 0x24, 0xa8, 0x5e, 0xbb, 0x8b, 0x84, 0xf6,
-	0xa0, 0xc0, 0x52, 0xe3, 0xa4, 0x23, 0x23, 0x8f, 0x3b, 0x77, 0x77, 0xe6, 0x09, 0x9f, 0x08, 0x5b,
-	0x87, 0x72, 0x31, 0xf7, 0x2e, 0x3f, 0x98, 0x50, 0xd0, 0xdb, 0x81, 0x45, 0x9b, 0xf8, 0x43, 0x49,
-	0xba, 0xf9, 0x97, 0x19, 0x39, 0xf0, 0xf0, 0xab, 0xb0, 0xe4, 0xbb, 0xe6, 0xc1, 0x01, 0x76, 0x79,
-	0xe4, 0xf6, 0xee, 0x3c, 0xc7, 0x0f, 0xe3, 0x50, 0x03, 0x56, 0x84, 0x61, 0x35, 0x74, 0xb3, 0x4c,
-	0xc7, 0xd6, 0x08, 0x0b, 0x8d, 0xdd, 0x8a, 0x9b, 0x0f, 0xe6, 0xc0, 0x2b, 0x47, 0x78, 0x1b, 0x8e,
-	0xc1, 0xe3, 0x78, 0x41, 0x9f, 0x21, 0x93, 0x00, 0x81, 0xa5, 0xf7, 0xa9, 0xaf, 0x42, 0x93, 0x3f,
-	0xf3, 0x05, 0x08, 0xec, 0x76, 0x92, 0x1c, 0x7d, 0x3c, 0x40, 0x70, 0x42, 0x02, 0xda, 0x07, 0xa1,
-	0x67, 0x39, 0xd4, 0x03, 0xda, 0xc7, 0x87, 0xfa, 0xb1, 0xe9, 0xb8, 0x34, 0x59, 0x54, 0xdc, 0xbc,
-	0x3f, 0x4f, 0x78, 0xcc, 0x58, 0x2b, 0x9c, 0x93, 0xc1, 0xaf, 0xf4, 0xa6, 0xa9, 0xd4, 0x3f, 0xb0,
-	0x2c, 0xba, 0x4c, 0x2d, 0xdd, 0xc7, 0x36, 0xf6, 0x3c, 0x9a, 0x5d, 0x22, 0xfe, 0x01, 0xa3, 0xd7,
-	0x39, 0x99, 0xc4, 0xea, 0x2d, 0x9b, 0x74, 0x2c, 0x60, 0xde, 0xc8, 0xcd, 0x9d, 0x0d, 0x99, 0x66,
-	0x64, 0x7d, 0x99, 0x41, 0x43, 0xb7, 0xe1, 0x92, 0xee, 0x79, 0xe6, 0x81, 0xed, 0x69, 0xbe, 0xa3,
-	0x39, 0x76, 0x70, 0x91, 0xb7, 0x01, 0xf4, 0xf0, 0x42, 0xfc, 0x65, 0xd7, 0x69, 0xd9, 0x98, 0xad,
-	0x7f, 0xe9, 0x73, 0xc8, 0x47, 0x16, 0x9b, 0xd4, 0x38, 0x2b, 0x3c, 0x5a, 0x81, 0x7c, 0xb3, 0xd5,
-	0xa4, 0xb7, 0x44, 0x4a, 0xf3, 0xb1, 0x10, 0xa7, 0x04, 0x59, 0xae, 0x76, 0xd8, 0xc5, 0x91, 0x90,
-	0x40, 0x08, 0x8a, 0xe5, 0xba, 0x2a, 0x97, 0xab, 0xfc, 0x2e, 0xa9, 0x2a, 0x24, 0xa5, 0x1f, 0x83,
-	0x30, 0x3b, 0xff, 0x92, 0x72, 0x96, 0x88, 0x22, 0x40, 0x55, 0xe9, 0x6c, 0x97, 0xd5, 0x2a, 0x93,
-	0x20, 0x40, 0x21, 0xbc, 0x8e, 0x22, 0x94, 0x04, 0x69, 0xa1, 0xca, 0xf4, 0x0a, 0x89, 0x3c, 0x27,
-	0xa5, 0x4f, 0x61, 0x65, 0x66, 0x8e, 0xa4, 0x47, 0xaf, 0x18, 0x80, 0xdc, 0x50, 0xba, 0x5a, 0xb9,
-	0xfe, 0xac, 0xfc, 0xa2, 0xc3, 0xf2, 0x42, 0x94, 0xa0, 0xd4, 0xb4, 0x66, 0xab, 0x29, 0x37, 0xda,
-	0xdd, 0x17, 0x42, 0x42, 0x6a, 0xcf, 0x4e, 0xd1, 0x2b, 0x11, 0x6b, 0x8a, 0x2a, 0x4f, 0x21, 0x52,
-	0xc2, 0x34, 0xe2, 0x3e, 0xc0, 0x64, 0x89, 0x4a, 0xdd, 0xb3, 0xd0, 0x56, 0x61, 0x59, 0x6e, 0x56,
-	0xb5, 0x56, 0x4d, 0x0b, 0x33, 0x57, 0x08, 0x8a, 0xf5, 0x32, 0xbd, 0x21, 0x56, 0x9a, 0x5a, 0xbb,
-	0xdc, 0x24, 0x5a, 0x26, 0xbd, 0x2e, 0xab, 0x75, 0x25, 0x4a, 0x4d, 0x4a, 0x16, 0xc0, 0x24, 0x4e,
-	0x96, 0xbe, 0x78, 0x85, 0x86, 0xe5, 0x3d, 0xb9, 0xd9, 0xa5, 0x75, 0x6e, 0x42, 0x1c, 0xad, 0xc1,
-	0x0a, 0xbf, 0x58, 0x21, 0x67, 0x24, 0x25, 0x26, 0xd0, 0x35, 0x78, 0xa3, 0xf3, 0xa2, 0xb9, 0xbd,
-	0xa3, 0xb6, 0x9a, 0xf4, 0xb2, 0x65, 0xb6, 0x45, 0x52, 0xfa, 0x95, 0x00, 0x4b, 0xdc, 0x4c, 0x20,
-	0x15, 0x72, 0x7a, 0xdf, 0xc7, 0xae, 0xa6, 0x5b, 0x16, 0x37, 0x9a, 0x77, 0xe6, 0xb7, 0x32, 0xa5,
-	0x32, 0xe1, 0x2d, 0x5b, 0xd6, 0x4e, 0x4c, 0xcd, 0xea, 0xfc, 0x77, 0x04, 0xd3, 0x1e, 0x73, 0x17,
-	0x66, 0x71, 0x4c, 0x7b, 0x3c, 0xc1, 0xb4, 0xc7, 0x68, 0x17, 0x80, 0x61, 0x62, 0xbd, 0x77, 0xc8,
-	0x63, 0x90, 0xbb, 0x8b, 0x82, 0xca, 0x7a, 0xef, 0x70, 0x27, 0xa6, 0xb2, 0xde, 0x91, 0x07, 0x64,
-	0xc1, 0x1a, 0x87, 0xb5, 0x0d, 0xcd, 0xe9, 0x07, 0xfb, 0x8d, 0x99, 0xdb, 0x8f, 0x16, 0xc6, 0xb7,
-	0x8d, 0x56, 0x9f, 0x6d, 0xcc, 0x9d, 0x98, 0x2a, 0xe8, 0x33, 0x34, 0xe4, 0xc3, 0x25, 0x26, 0x6d,
-	0x26, 0xb2, 0xe3, 0xa9, 0xb4, 0x47, 0x8b, 0xca, 0x3b, 0x19, 0xc1, 0xe9, 0x27, 0xc9, 0xe8, 0xeb,
-	0x38, 0x48, 0x4c, 0xac, 0x37, 0xb6, 0x7b, 0x87, 0xae, 0x63, 0xd3, 0x0b, 0xb4, 0xd9, 0x3e, 0xb0,
-	0x32, 0x95, 0x27, 0x8b, 0xf6, 0xa1, 0x13, 0xc1, 0x3c, 0xd1, 0x9f, 0xab, 0xfa, 0xab, 0x9b, 0xa0,
-	0xa7, 0x90, 0xd1, 0xad, 0x97, 0xfa, 0xd8, 0xdb, 0x28, 0xcc, 0x9d, 0x9b, 0x0d, 0xc5, 0x53, 0xc6,
-	0x9d, 0x98, 0xca, 0x21, 0x50, 0x13, 0x96, 0x0c, 0xdc, 0xd7, 0x47, 0x96, 0x4f, 0x0f, 0x89, 0xf9,
-	0x8e, 0xff, 0x00, 0xad, 0xca, 0x38, 0x77, 0x62, 0x6a, 0x00, 0x82, 0xbe, 0x98, 0x84, 0xbe, 0x3d,
-	0x67, 0x64, 0xfb, 0xf4, 0x58, 0xc8, 0xcf, 0x75, 0xf4, 0x04, 0xa8, 0x72, 0x90, 0x53, 0x1b, 0xd9,
-	0x7e, 0x24, 0xd6, 0xa5, 0xcf, 0x68, 0x07, 0xd2, 0x36, 0x3e, 0xc6, 0xec, 0x14, 0xc9, 0x6f, 0xde,
-	0x5a, 0x00, 0xb7, 0x49, 0xf8, 0x76, 0x62, 0x2a, 0x03, 0x20, 0xbb, 0xc3, 0x71, 0xd9, 0x05, 0x89,
-	0x35, 0xa6, 0xa7, 0xc5, 0x62, 0xbb, 0xa3, 0xe5, 0xd6, 0x18, 0x2f, 0xd9, 0x1d, 0x4e, 0xf0, 0x40,
-	0x66, 0xc7, 0xc5, 0x43, 0xac, 0xfb, 0x1b, 0xf9, 0x85, 0x67, 0x47, 0xa5, 0x8c, 0x64, 0x76, 0x18,
-	0x84, 0xf8, 0x1c, 0xb2, 0x81, 0xb5, 0x40, 0x75, 0xc8, 0xd3, 0xe2, 0x2e, 0xda, 0x34, 0x08, 0xae,
-	0x17, 0xf1, 0x6e, 0xa2, 0xec, 0x13, 0x64, 0x7b, 0xfc, 0x9a, 0x91, 0x5f, 0x40, 0x2e, 0x34, 0x1c,
-	0xaf, 0x19, 0xfa, 0xef, 0xe2, 0x20, 0xcc, 0x1a, 0x0d, 0xd4, 0x82, 0x65, 0xac, 0xbb, 0xd6, 0x58,
-	0xeb, 0x9b, 0x24, 0xac, 0x09, 0x2a, 0x0a, 0x17, 0x11, 0x52, 0xa0, 0x00, 0x35, 0xc6, 0x8f, 0x1a,
-	0x50, 0x20, 0x4e, 0x4d, 0x88, 0x97, 0x58, 0x18, 0x2f, 0x4f, 0xf8, 0x39, 0x9c, 0xf8, 0x7b, 0xb0,
-	0x76, 0x8a, 0xe1, 0x41, 0x87, 0xb0, 0x1e, 0xa6, 0x1a, 0xb4, 0x13, 0x65, 0xd4, 0xf7, 0xe6, 0xcc,
-	0x12, 0x53, 0xf6, 0x49, 0xdd, 0xec, 0x9a, 0x7f, 0x82, 0xe6, 0x89, 0xd7, 0xe1, 0xea, 0x37, 0x58,
-	0x1d, 0x31, 0x07, 0x4b, 0x7c, 0x2f, 0x8b, 0x77, 0xa0, 0x10, 0xdd, 0x80, 0xe8, 0xad, 0xd9, 0x0d,
-	0x4d, 0xd4, 0x9b, 0x9e, 0xde, 0x95, 0xe2, 0x12, 0xa4, 0xe9, 0xee, 0x12, 0xb3, 0x90, 0x61, 0x26,
-	0x46, 0xfc, 0x93, 0x38, 0xe4, 0xc2, 0x2d, 0x82, 0x1e, 0x41, 0x2a, 0xcc, 0x81, 0x2f, 0xa6, 0x4b,
-	0xca, 0x47, 0xdc, 0xfa, 0x60, 0xa7, 0x2e, 0x3e, 0x1d, 0x01, 0xab, 0xd8, 0x85, 0x0c, 0xdb, 0x62,
-	0xe8, 0x09, 0xc0, 0x64, 0x61, 0x9d, 0xa3, 0x57, 0x11, 0xee, 0x4a, 0x2e, 0x0c, 0x39, 0xa4, 0x7f,
-	0x4e, 0x44, 0x12, 0x52, 0x93, 0x92, 0xd0, 0x0e, 0xa4, 0x0d, 0x6c, 0xe9, 0x63, 0x2e, 0xe8, 0xa3,
-	0x73, 0x4d, 0x6e, 0xa9, 0x4a, 0x20, 0x88, 0xfd, 0xa2, 0x58, 0xe8, 0x33, 0xc8, 0xea, 0x96, 0x79,
-	0x60, 0x6b, 0xbe, 0xc3, 0x75, 0xf2, 0x83, 0xf3, 0xe1, 0x96, 0x09, 0x4a, 0xd7, 0x21, 0x56, 0x5c,
-	0x67, 0x3f, 0xc5, 0x77, 0x21, 0x4d, 0xa5, 0xa1, 0xeb, 0x50, 0xa0, 0xd2, 0xb4, 0x81, 0x69, 0x59,
-	0xa6, 0xc7, 0x93, 0x80, 0x79, 0x4a, 0x6b, 0x50, 0x92, 0xf8, 0x10, 0x96, 0x38, 0x02, 0xba, 0x0c,
-	0x99, 0x21, 0x76, 0x4d, 0x87, 0xc5, 0x66, 0x49, 0x95, 0x3f, 0x11, 0xba, 0xd3, 0xef, 0x7b, 0xd8,
-	0xa7, 0x4e, 0x42, 0x52, 0xe5, 0x4f, 0x95, 0x4b, 0xb0, 0x76, 0xca, 0x1e, 0x90, 0xfe, 0x38, 0x01,
-	0xb9, 0x30, 0x37, 0x83, 0xf6, 0xa0, 0xa8, 0xf7, 0x68, 0x11, 0xcb, 0x50, 0xf7, 0x7d, 0xec, 0xda,
-	0xe7, 0xcd, 0xc8, 0x2c, 0x33, 0x98, 0x36, 0x43, 0x41, 0x4f, 0x61, 0xe9, 0xd8, 0xc4, 0x2f, 0x2f,
-	0x76, 0x1b, 0x95, 0x21, 0x10, 0x35, 0x1b, 0x7d, 0x01, 0xab, 0x3c, 0x3c, 0x1d, 0xe8, 0xc3, 0x21,
-	0xf1, 0x0f, 0xfa, 0x36, 0xf7, 0xb8, 0xce, 0x03, 0xcb, 0x63, 0xdd, 0x06, 0xc3, 0xaa, 0xd9, 0xd2,
-	0x27, 0x90, 0x8f, 0x94, 0x56, 0x23, 0x01, 0x92, 0x23, 0xd7, 0xe6, 0x37, 0x02, 0xe4, 0x27, 0xda,
-	0x80, 0xa5, 0x21, 0x4b, 0xa5, 0x51, 0xb1, 0x05, 0x35, 0x78, 0x7c, 0x92, 0xca, 0xc6, 0x85, 0x84,
-	0xf4, 0x67, 0x71, 0x58, 0x0f, 0x12, 0x4b, 0xd1, 0xda, 0x6f, 0xe9, 0xab, 0x38, 0x14, 0xa2, 0x04,
-	0x74, 0x03, 0x32, 0xd5, 0x16, 0xbd, 0x18, 0x8e, 0x89, 0x1b, 0x34, 0xbf, 0x80, 0x68, 0x7e, 0x01,
-	0xdb, 0xc7, 0x5b, 0x86, 0xd3, 0x3b, 0x62, 0x29, 0x97, 0xb7, 0x61, 0x89, 0x3b, 0xc9, 0x42, 0x7c,
-	0x2a, 0x35, 0x43, 0x9a, 0x71, 0x37, 0x89, 0xb4, 0xbb, 0x09, 0x59, 0xf9, 0x79, 0x57, 0x56, 0x9b,
-	0xe5, 0xfa, 0x4c, 0xfa, 0x88, 0x34, 0xc4, 0x5f, 0x92, 0xa9, 0xd0, 0xad, 0xad, 0xe3, 0xdb, 0xd2,
-	0x03, 0x58, 0xae, 0x52, 0xf8, 0x20, 0xd3, 0xfa, 0x0e, 0xac, 0xf4, 0x1c, 0xdb, 0xd7, 0x4d, 0x9b,
-	0xc4, 0xfb, 0x03, 0xfd, 0x20, 0x28, 0x00, 0x2a, 0x86, 0x64, 0x85, 0x50, 0xa5, 0x7f, 0x8b, 0x43,
-	0x91, 0x1b, 0xb4, 0x80, 0xb7, 0x08, 0x09, 0xc7, 0xe3, 0xcd, 0x13, 0x8e, 0x87, 0x10, 0xa4, 0x74,
-	0xb7, 0x77, 0xc8, 0x35, 0x46, 0x7f, 0x13, 0x95, 0xf5, 0x9c, 0xc1, 0x40, 0xb7, 0x83, 0x54, 0x42,
-	0xf0, 0x88, 0xea, 0x90, 0xc4, 0xf6, 0xf1, 0x22, 0xf5, 0xcd, 0x53, 0xd2, 0x4b, 0xb2, 0x7d, 0xcc,
-	0xb2, 0x98, 0x04, 0x46, 0xfc, 0x10, 0xb2, 0x01, 0x61, 0xa1, 0x4a, 0xe2, 0xff, 0x89, 0xc3, 0x8a,
-	0xcc, 0x15, 0x14, 0x8c, 0xab, 0x03, 0xd9, 0xe0, 0xb3, 0x24, 0xbe, 0x0d, 0xe6, 0xf1, 0xac, 0xca,
-	0x43, 0xb3, 0x83, 0xdd, 0x63, 0xb3, 0x87, 0xab, 0xe1, 0x77, 0x49, 0x6a, 0x08, 0x84, 0xf6, 0x20,
-	0x43, 0xcb, 0x76, 0x82, 0xdb, 0xa0, 0x79, 0x7c, 0xea, 0x99, 0x8e, 0xb1, 0xc2, 0x85, 0xa0, 0x54,
-	0x9c, 0xa1, 0x89, 0x0f, 0x21, 0x1f, 0x21, 0x2f, 0x34, 0xf6, 0x5f, 0xc0, 0xca, 0xcc, 0x9e, 0x78,
-	0x3d, 0xf9, 0xd8, 0xef, 0x42, 0x31, 0xf2, 0x2d, 0xcb, 0xe4, 0x56, 0x6d, 0x39, 0x42, 0x55, 0x0c,
-	0x69, 0x0b, 0x0a, 0x53, 0xb2, 0xf9, 0x7e, 0x8b, 0xcf, 0xb1, 0xdf, 0xa4, 0xdf, 0xa5, 0x20, 0x1f,
-	0xa9, 0xdd, 0x42, 0x0a, 0xa4, 0x4d, 0x1f, 0x87, 0x27, 0xfb, 0x9d, 0xc5, 0x4a, 0xbf, 0x4a, 0x8a,
-	0x8f, 0x07, 0x2a, 0x43, 0x10, 0xfb, 0x00, 0x8a, 0x81, 0x6d, 0xdf, 0xec, 0x9b, 0xd8, 0x25, 0xb6,
-	0x39, 0xfa, 0xcd, 0x03, 0xef, 0x5d, 0xde, 0x9f, 0x7c, 0xee, 0x40, 0x0e, 0xef, 0x49, 0x93, 0x89,
-	0xc5, 0x98, 0xf0, 0xed, 0xba, 0x76, 0x30, 0x2f, 0xc9, 0x70, 0x5e, 0xc4, 0x5f, 0x27, 0x20, 0x45,
-	0xe4, 0x22, 0x05, 0x12, 0x1c, 0x78, 0xbe, 0x6f, 0x07, 0xa6, 0x3a, 0x1e, 0xf6, 0x54, 0x4d, 0x98,
-	0x64, 0x4f, 0xb1, 0x5a, 0x98, 0xc4, 0xdc, 0x59, 0xb4, 0x28, 0xd8, 0x4c, 0x35, 0x0c, 0x7a, 0x37,
-	0x58, 0x39, 0xcc, 0xc6, 0xae, 0x97, 0xd8, 0x07, 0x78, 0xa5, 0xe0, 0x03, 0xbc, 0x52, 0xd9, 0x0e,
-	0x3e, 0xab, 0x41, 0xf7, 0x20, 0xef, 0x1d, 0x3a, 0xae, 0xcf, 0x32, 0xaa, 0x3c, 0x4e, 0x3d, 0x9d,
-	0x03, 0x68, 0x43, 0x5a, 0x57, 0x41, 0x16, 0xa7, 0xa5, 0xef, 0x63, 0x8b, 0x7f, 0xc1, 0xc1, 0x1e,
-	0xd0, 0x15, 0xc8, 0x5a, 0xa6, 0x7d, 0xa4, 0x8d, 0x5c, 0x8b, 0x46, 0x7f, 0x39, 0x75, 0x89, 0x3c,
-	0xef, 0xba, 0x96, 0xf8, 0x0b, 0x5e, 0xa1, 0x33, 0x7a, 0x45, 0x85, 0x0e, 0x4b, 0xcd, 0xb3, 0xbb,
-	0x76, 0xa5, 0xd9, 0x95, 0x1f, 0xcb, 0xaa, 0x90, 0x40, 0x39, 0x48, 0xd7, 0xea, 0xad, 0x72, 0x57,
-	0x48, 0xb2, 0x3b, 0xf8, 0x56, 0x5d, 0x2e, 0x37, 0x85, 0x14, 0x5a, 0x86, 0x5c, 0xf8, 0x75, 0x9e,
-	0x90, 0x46, 0x05, 0xc8, 0x56, 0x77, 0xd5, 0x32, 0x2d, 0x9f, 0xcd, 0xa0, 0x22, 0xc0, 0x93, 0xf2,
-	0x5e, 0x59, 0xdb, 0xae, 0x97, 0x3b, 0x1d, 0x61, 0x49, 0xfa, 0xa7, 0x2c, 0x5c, 0x6a, 0x60, 0xcf,
-	0xd3, 0x0f, 0xf0, 0x33, 0xd3, 0x3f, 0x8c, 0x54, 0xf3, 0xbe, 0xe6, 0x0f, 0x6e, 0x7e, 0x08, 0x69,
-	0x9a, 0x83, 0x5d, 0xf4, 0x0b, 0x24, 0xe2, 0xba, 0x50, 0x46, 0xf4, 0x39, 0xb1, 0xec, 0xbc, 0xdc,
-	0x39, 0xb2, 0x89, 0xe6, 0x0b, 0x96, 0xa6, 0x2f, 0xe0, 0x77, 0x62, 0x2a, 0xaf, 0x05, 0x0a, 0xaf,
-	0xe4, 0x7f, 0x02, 0xab, 0x9e, 0x71, 0x14, 0x5e, 0xab, 0x45, 0xcb, 0x78, 0xce, 0x71, 0x16, 0xef,
-	0xc4, 0xd4, 0x15, 0x6f, 0xc6, 0x14, 0x3d, 0x83, 0xe2, 0x50, 0x77, 0x35, 0xc3, 0x09, 0xbb, 0x9f,
-	0x99, 0xdb, 0x28, 0x45, 0x0b, 0x03, 0x49, 0x74, 0x3b, 0x8c, 0x56, 0x72, 0xb6, 0x00, 0x86, 0xe1,
-	0xde, 0xe4, 0x01, 0xf9, 0x62, 0x9f, 0xce, 0xed, 0xc4, 0xd4, 0x08, 0x04, 0x52, 0x21, 0x1f, 0xf9,
-	0xdc, 0x91, 0x07, 0xe3, 0x0b, 0x7e, 0x1c, 0xb7, 0x13, 0x53, 0xa3, 0x20, 0xa8, 0x03, 0x05, 0x17,
-	0xeb, 0x46, 0x38, 0xf6, 0xdc, 0xdc, 0xa0, 0x91, 0x7a, 0x12, 0x02, 0xea, 0x46, 0xca, 0x4b, 0x1a,
-	0x00, 0x93, 0xab, 0x44, 0x1e, 0x3a, 0x2f, 0x74, 0x87, 0x47, 0xa2, 0xf0, 0xf0, 0xce, 0x10, 0xf5,
-	0x61, 0x2d, 0xf2, 0xe1, 0x49, 0xd8, 0xd5, 0xc2, 0x82, 0x1f, 0xe9, 0x45, 0xaa, 0x49, 0x76, 0x62,
-	0x2a, 0x77, 0xf1, 0xa2, 0x25, 0x26, 0x18, 0xd0, 0xc9, 0x92, 0xe0, 0x8d, 0xe5, 0xf3, 0x7f, 0x0b,
-	0x38, 0x11, 0x13, 0xbd, 0xa6, 0xd9, 0x83, 0xe5, 0xe9, 0xe5, 0x5c, 0x3c, 0xd7, 0x21, 0x48, 0xd6,
-	0x5b, 0x3f, 0xf2, 0x5c, 0xc9, 0x40, 0xca, 0x75, 0x1c, 0x5f, 0xfa, 0x55, 0x06, 0x2e, 0xcb, 0x5f,
-	0xe2, 0xde, 0x88, 0xd6, 0x9c, 0x76, 0x7c, 0xfd, 0x20, 0xdc, 0x4d, 0x6d, 0xc8, 0x47, 0xce, 0x46,
-	0x6e, 0x3d, 0x16, 0xfd, 0x14, 0x30, 0x0a, 0x41, 0x0c, 0x2b, 0x9b, 0x65, 0x7e, 0xea, 0x9b, 0x7c,
-	0xc6, 0x4e, 0xa9, 0x16, 0x96, 0xe7, 0xf2, 0x44, 0x4e, 0xeb, 0xf7, 0x64, 0x61, 0x28, 0xc6, 0x54,
-	0xcd, 0xf0, 0x9b, 0x53, 0x1f, 0x2d, 0xa7, 0xe8, 0x45, 0x6c, 0xf4, 0xab, 0xe3, 0x8d, 0xc9, 0xf7,
-	0x6d, 0x69, 0xfa, 0x32, 0xfc, 0x46, 0x6d, 0xda, 0x8c, 0x66, 0x2e, 0x6a, 0x46, 0xfb, 0x90, 0x1f,
-	0x79, 0xd8, 0xa5, 0x17, 0x65, 0xd8, 0xdb, 0x58, 0xba, 0xe8, 0x80, 0x77, 0x3d, 0xec, 0xd2, 0x9a,
-	0x35, 0x32, 0xe0, 0x51, 0xf0, 0xe0, 0xa1, 0x17, 0x90, 0xa1, 0x17, 0xa5, 0xde, 0x46, 0x96, 0x8a,
-	0x28, 0x9f, 0x5f, 0x04, 0x2d, 0x6d, 0x53, 0x0c, 0x95, 0x03, 0x8a, 0x2d, 0xc8, 0x47, 0xd4, 0x3c,
-	0x8f, 0x43, 0xf2, 0x1d, 0x00, 0xcb, 0xe9, 0xe9, 0x16, 0xab, 0xe7, 0x67, 0x0b, 0x20, 0x47, 0x29,
-	0x4d, 0x7d, 0x80, 0x09, 0x60, 0x64, 0x18, 0xaf, 0x01, 0xf0, 0x29, 0x2c, 0xf1, 0x4e, 0x5f, 0x1c,
-	0x6c, 0xeb, 0x13, 0xc8, 0xd2, 0x7f, 0x13, 0x20, 0xfe, 0xdf, 0xf5, 0x13, 0xfe, 0x03, 0x39, 0xf3,
-	0xa9, 0xe7, 0xd0, 0x1a, 0xb2, 0xef, 0xd5, 0x7f, 0xfb, 0xe7, 0x7f, 0xfd, 0x9c, 0x79, 0x08, 0x84,
-	0x6b, 0xd7, 0xb5, 0xb7, 0x14, 0x58, 0xa6, 0x00, 0x3d, 0xfe, 0xd9, 0xff, 0x3c, 0x28, 0xff, 0x12,
-	0xa0, 0x14, 0xf6, 0x23, 0x7f, 0x1f, 0x50, 0xf9, 0x08, 0xbe, 0xf9, 0x2f, 0x0c, 0x2a, 0x39, 0x95,
-	0x56, 0x6e, 0x94, 0x87, 0xe6, 0x67, 0xf9, 0x80, 0xae, 0x1d, 0xdf, 0xde, 0xcf, 0x50, 0x71, 0x77,
-	0xfe, 0x2f, 0x00, 0x00, 0xff, 0xff, 0xa0, 0xb1, 0x4c, 0x75, 0x1d, 0x41, 0x00, 0x00,
+var fileDescriptor_beam_runner_api_b87c0d18be5b2d09 = []byte{
+	// 5131 bytes of a gzipped FileDescriptorProto
+	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xbc, 0x7c, 0xdf, 0x6f, 0x1b, 0x57,
+	0x76, 0xbf, 0xf8, 0x53, 0xe4, 0x21, 0x45, 0x8d, 0xae, 0x64, 0x47, 0x9e, 0xcd, 0xc6, 0xf6, 0xc4,
+	0x9b, 0x38, 0xf9, 0x26, 0x8c, 0x25, 0xdb, 0xb1, 0xad, 0xcd, 0x3a, 0x4b, 0x8a, 0x43, 0x6b, 0x6c,
+	0xfe, 0xca, 0x90, 0x92, 0xec, 0x6c, 0x36, 0xb3, 0x23, 0xce, 0xa5, 0x34, 0xd0, 0x70, 0x86, 0x3b,
+	0x33, 0x94, 0xc3, 0xc5, 0x2e, 0xbe, 0x40, 0x1f, 0x82, 0xa2, 0x05, 0x8a, 0xf6, 0xa1, 0x0f, 0x79,
+	0x28, 0x0a, 0xec, 0x02, 0x05, 0xda, 0x3e, 0xb4, 0xe8, 0xb6, 0x05, 0xfa, 0xba, 0x6d, 0xff, 0x80,
+	0xa2, 0x7d, 0xea, 0x7f, 0xd1, 0x16, 0xfb, 0xd0, 0x3e, 0x15, 0xf7, 0xc7, 0x0c, 0x87, 0x94, 0xe4,
+	0x90, 0x92, 0xd1, 0x37, 0xce, 0xb9, 0xf7, 0x7c, 0xce, 0xfd, 0x79, 0xee, 0x39, 0xe7, 0x9e, 0x4b,
+	0xb8, 0x72, 0x80, 0xf5, 0xbe, 0xe6, 0x0e, 0x6d, 0x1b, 0xbb, 0x9a, 0x3e, 0x30, 0x8b, 0x03, 0xd7,
+	0xf1, 0x1d, 0x74, 0xd3, 0x71, 0x0f, 0x8b, 0xfa, 0x40, 0xef, 0x1e, 0xe1, 0x22, 0xa9, 0x51, 0xec,
+	0x3b, 0x06, 0xb6, 0x8a, 0x03, 0x73, 0x80, 0x2d, 0xd3, 0xc6, 0xc5, 0x93, 0x0d, 0x71, 0x19, 0xdb,
+	0xc6, 0xc0, 0x31, 0x6d, 0xdf, 0x63, 0x3c, 0xe2, 0xb5, 0x43, 0xc7, 0x39, 0xb4, 0xf0, 0x47, 0xf4,
+	0xeb, 0x60, 0xd8, 0xfb, 0x48, 0xb7, 0x47, 0xbc, 0xe8, 0xc6, 0x74, 0x91, 0x81, 0xbd, 0xae, 0x6b,
+	0x0e, 0x7c, 0xc7, 0x65, 0x35, 0xa4, 0xdf, 0xc4, 0x60, 0xa9, 0x8c, 0xf5, 0xfe, 0xb6, 0x63, 0x7b,
+	0xbe, 0x6e, 0xfb, 0x9e, 0xf4, 0xd7, 0x31, 0xc8, 0x86, 0x5f, 0x68, 0x03, 0xd6, 0xea, 0x4a, 0x43,
+	0xeb, 0x28, 0x75, 0xb9, 0xdd, 0x29, 0xd5, 0x5b, 0x5a, 0x5d, 0xa9, 0xd5, 0x94, 0xb6, 0xb0, 0x20,
+	0xbe, 0xf1, 0x17, 0x7f, 0xfb, 0x3f, 0xbf, 0x49, 0xad, 0x7c, 0xf8, 0x68, 0x73, 0xf3, 0xee, 0xdd,
+	0x07, 0x9b, 0x77, 0xee, 0x7e, 0xfc, 0xf0, 0xfe, 0xbd, 0x07, 0x0f, 0xee, 0xa3, 0x3b, 0xb0, 0x56,
+	0x2f, 0x3d, 0x3f, 0xcd, 0x12, 0x13, 0xaf, 0x52, 0x16, 0xe1, 0x14, 0xc7, 0x63, 0x90, 0x9e, 0xd4,
+	0x9a, 0xe5, 0x52, 0x4d, 0xdb, 0x57, 0x1a, 0x95, 0xe6, 0xbe, 0x76, 0x26, 0x7f, 0x7c, 0x92, 0x7f,
+	0xe3, 0xd1, 0xfd, 0x3b, 0xf7, 0x28, 0xbf, 0xf4, 0xf7, 0x19, 0x80, 0x6d, 0xa7, 0x3f, 0x70, 0x6c,
+	0x4c, 0xda, 0xfc, 0x63, 0x00, 0xdf, 0xd5, 0x6d, 0xaf, 0xe7, 0xb8, 0x7d, 0x6f, 0x3d, 0x76, 0x23,
+	0x71, 0x3b, 0xb7, 0xf9, 0x83, 0xe2, 0xb7, 0x8e, 0x6c, 0x71, 0x0c, 0x51, 0xec, 0x84, 0xfc, 0xb2,
+	0xed, 0xbb, 0x23, 0x35, 0x02, 0x88, 0xba, 0x90, 0x1f, 0x74, 0x1d, 0xcb, 0xc2, 0x5d, 0xdf, 0x74,
+	0x6c, 0x6f, 0x3d, 0x4e, 0x05, 0x7c, 0x3a, 0x9f, 0x80, 0x56, 0x04, 0x81, 0x89, 0x98, 0x00, 0x45,
+	0x23, 0x58, 0x7b, 0x69, 0xda, 0x86, 0xf3, 0xd2, 0xb4, 0x0f, 0x35, 0xcf, 0x77, 0x75, 0x1f, 0x1f,
+	0x9a, 0xd8, 0x5b, 0x4f, 0x50, 0x61, 0xd5, 0xf9, 0x84, 0xed, 0x07, 0x48, 0xed, 0x10, 0x88, 0xc9,
+	0x5c, 0x7d, 0x79, 0xba, 0x04, 0x7d, 0x06, 0xe9, 0xae, 0x63, 0x60, 0xd7, 0x5b, 0x4f, 0x52, 0x61,
+	0x8f, 0xe6, 0x13, 0xb6, 0x4d, 0x79, 0x19, 0x3e, 0x07, 0x22, 0x43, 0x86, 0xed, 0x13, 0xd3, 0x75,
+	0xec, 0x3e, 0xa9, 0xb3, 0x9e, 0xba, 0xc8, 0x90, 0xc9, 0x11, 0x04, 0x3e, 0x64, 0x51, 0x50, 0xd1,
+	0x82, 0xe5, 0xa9, 0x69, 0x43, 0x02, 0x24, 0x8e, 0xf1, 0x68, 0x3d, 0x76, 0x23, 0x76, 0x3b, 0xab,
+	0x92, 0x9f, 0x68, 0x1b, 0x52, 0x27, 0xba, 0x35, 0xc4, 0xeb, 0xf1, 0x1b, 0xb1, 0xdb, 0xb9, 0xcd,
+	0x0f, 0x67, 0x68, 0x42, 0x2b, 0x44, 0x55, 0x19, 0xef, 0x56, 0xfc, 0x61, 0x4c, 0x74, 0x60, 0xe5,
+	0xd4, 0x1c, 0x9e, 0x21, 0xaf, 0x32, 0x29, 0xaf, 0x38, 0x8b, 0xbc, 0xed, 0x10, 0x36, 0x2a, 0xf0,
+	0xe7, 0xb0, 0x7e, 0xde, 0x3c, 0x9e, 0x21, 0xf7, 0xe9, 0xa4, 0xdc, 0x7b, 0x33, 0xc8, 0x9d, 0x46,
+	0x1f, 0x45, 0xa5, 0x77, 0x21, 0x17, 0x99, 0xd8, 0x33, 0x04, 0x3e, 0x9e, 0x14, 0x78, 0x7b, 0xa6,
+	0xb9, 0x35, 0xb0, 0x3b, 0x35, 0xa6, 0xa7, 0x26, 0xf9, 0xf5, 0x8c, 0x69, 0x04, 0x36, 0x22, 0x50,
+	0xfa, 0xf7, 0x18, 0x64, 0x5a, 0xbc, 0x1a, 0xaa, 0x03, 0x74, 0xc3, 0xd5, 0x46, 0xe5, 0xcd, 0xb6,
+	0x3e, 0xc6, 0x4b, 0x54, 0x8d, 0x00, 0xa0, 0x0f, 0x00, 0xb9, 0x8e, 0xe3, 0x6b, 0xa1, 0xe6, 0xd0,
+	0x4c, 0x83, 0x29, 0x8b, 0xac, 0x2a, 0x90, 0x92, 0x70, 0x59, 0x29, 0x06, 0xd9, 0x74, 0x79, 0xc3,
+	0xf4, 0x06, 0x96, 0x3e, 0xd2, 0x0c, 0xdd, 0xd7, 0xd7, 0x13, 0x33, 0x77, 0xad, 0xc2, 0xd8, 0x2a,
+	0xba, 0xaf, 0xab, 0x39, 0x63, 0xfc, 0x21, 0xfd, 0x7e, 0x12, 0x60, 0xbc, 0x76, 0xd1, 0x75, 0xc8,
+	0x0d, 0x6d, 0xf3, 0xa7, 0x43, 0xac, 0xd9, 0x7a, 0x1f, 0xaf, 0xa7, 0xe8, 0x78, 0x02, 0x23, 0x35,
+	0xf4, 0x3e, 0x46, 0xdb, 0x90, 0xf4, 0x06, 0xb8, 0xcb, 0x7b, 0xfe, 0xd1, 0x0c, 0xa2, 0xab, 0x43,
+	0x9b, 0x2e, 0xd3, 0xf6, 0x00, 0x77, 0x55, 0xca, 0x8c, 0x6e, 0xc1, 0x92, 0x37, 0x3c, 0x88, 0xa8,
+	0x5f, 0xd6, 0xe1, 0x49, 0x22, 0x51, 0x31, 0xa6, 0x3d, 0x18, 0xfa, 0x81, 0x3e, 0x7b, 0x34, 0xd7,
+	0x36, 0x2c, 0x2a, 0x94, 0x97, 0xab, 0x18, 0x06, 0x84, 0x3a, 0xb0, 0xe8, 0x0c, 0x7d, 0x8a, 0xc9,
+	0xd4, 0xd6, 0xd6, 0x7c, 0x98, 0x4d, 0xc6, 0xcc, 0x40, 0x03, 0xa8, 0x53, 0xd3, 0x92, 0xbe, 0xf4,
+	0xb4, 0x88, 0x8f, 0x20, 0x17, 0x69, 0xff, 0x19, 0xcb, 0x7b, 0x2d, 0xba, 0xbc, 0xb3, 0xd1, 0xfd,
+	0xb1, 0x05, 0xf9, 0x68, 0x33, 0xe7, 0xe1, 0x95, 0xfe, 0x6e, 0x09, 0x56, 0xdb, 0xbe, 0x6e, 0x1b,
+	0xba, 0x6b, 0x8c, 0xbb, 0xed, 0x49, 0x7f, 0x9e, 0x00, 0x68, 0xb9, 0x66, 0xdf, 0xf4, 0xcd, 0x13,
+	0xec, 0xa1, 0xf7, 0x20, 0xdd, 0x2a, 0xa9, 0x5a, 0xa5, 0x29, 0x2c, 0x88, 0xdf, 0xfd, 0x25, 0x39,
+	0x6e, 0xdf, 0x20, 0x1d, 0xdc, 0x0a, 0x27, 0x6f, 0x6b, 0xa0, 0xbb, 0x86, 0xb3, 0x75, 0xb2, 0x81,
+	0x3e, 0x80, 0xc5, 0x6a, 0xad, 0xd4, 0xe9, 0xc8, 0x0d, 0x21, 0x26, 0x5e, 0xa7, 0x75, 0xaf, 0x4d,
+	0xd5, 0xed, 0x59, 0xba, 0xef, 0x63, 0x9b, 0xd4, 0xfe, 0x18, 0xf2, 0x4f, 0xd4, 0xe6, 0x6e, 0x4b,
+	0x2b, 0xbf, 0xd0, 0x9e, 0xc9, 0x2f, 0x84, 0xb8, 0x78, 0x8b, 0xb2, 0xbc, 0x35, 0xc5, 0x72, 0xe8,
+	0x3a, 0xc3, 0x81, 0x76, 0x30, 0xd2, 0x8e, 0xf1, 0x88, 0x4b, 0x51, 0xea, 0xad, 0xdd, 0x5a, 0x5b,
+	0x16, 0x12, 0xe7, 0x48, 0x31, 0xfb, 0x83, 0xa1, 0xe5, 0x61, 0x52, 0xfb, 0x01, 0x14, 0x4a, 0xed,
+	0xb6, 0xf2, 0xa4, 0xc1, 0x2d, 0x89, 0xb6, 0x90, 0x14, 0xdf, 0xa6, 0x4c, 0xdf, 0x9d, 0x62, 0x62,
+	0x27, 0x9f, 0x66, 0xda, 0x3e, 0xed, 0xcc, 0x5d, 0xc8, 0x75, 0xe4, 0x76, 0x47, 0x6b, 0x77, 0x54,
+	0xb9, 0x54, 0x17, 0x52, 0xa2, 0x44, 0xb9, 0xde, 0x9c, 0xe2, 0xf2, 0xb1, 0xe7, 0x7b, 0xbe, 0x4b,
+	0x88, 0x27, 0x1b, 0xe8, 0x1e, 0xe4, 0xea, 0xa5, 0x56, 0x28, 0x2a, 0x7d, 0x8e, 0xa8, 0xbe, 0x3e,
+	0xd0, 0x98, 0x38, 0x8f, 0x70, 0x3d, 0x84, 0xa5, 0xba, 0xac, 0x3e, 0x91, 0x43, 0xbe, 0x45, 0xf1,
+	0x7b, 0x94, 0xef, 0xfa, 0x34, 0x1f, 0x76, 0x0f, 0x71, 0x84, 0x53, 0xf2, 0x61, 0xad, 0x82, 0x07,
+	0x2e, 0xee, 0xea, 0x3e, 0x36, 0x22, 0x93, 0xf6, 0x0e, 0x24, 0x55, 0xb9, 0x54, 0x11, 0x16, 0xc4,
+	0x37, 0x29, 0xd0, 0xd5, 0x29, 0x20, 0x17, 0xeb, 0x06, 0x6f, 0xef, 0xb6, 0x2a, 0x97, 0x3a, 0xb2,
+	0xb6, 0xa7, 0xc8, 0xfb, 0x42, 0xec, 0x9c, 0xf6, 0x76, 0x5d, 0xac, 0xfb, 0x58, 0x3b, 0x31, 0xf1,
+	0x4b, 0x22, 0xf5, 0x3f, 0x63, 0xdc, 0xba, 0xf2, 0x4c, 0x1f, 0x7b, 0xe8, 0x13, 0x58, 0xde, 0x6e,
+	0xd6, 0xcb, 0x4a, 0x43, 0xd6, 0x5a, 0xb2, 0x4a, 0xe7, 0x72, 0x41, 0x7c, 0x97, 0x02, 0xdd, 0x9c,
+	0x06, 0x72, 0xfa, 0x07, 0xa6, 0x8d, 0xb5, 0x01, 0x76, 0x83, 0xe9, 0x7c, 0x0c, 0x42, 0xc0, 0xcd,
+	0x4c, 0xbe, 0xda, 0x0b, 0x21, 0x26, 0xde, 0xa6, 0xec, 0xd2, 0x39, 0xec, 0x87, 0x96, 0x73, 0xa0,
+	0x5b, 0x16, 0xe5, 0xbf, 0x03, 0x59, 0x55, 0x6e, 0xef, 0xec, 0x56, 0xab, 0x35, 0x59, 0x88, 0x8b,
+	0x37, 0x29, 0xe3, 0x77, 0x4e, 0xf5, 0xd7, 0x3b, 0x1a, 0xf6, 0x7a, 0x16, 0xe6, 0x9d, 0xde, 0x57,
+	0x95, 0x8e, 0xac, 0x55, 0x95, 0x9a, 0xdc, 0x16, 0x12, 0xe7, 0xad, 0x07, 0xd7, 0xf4, 0xb1, 0xd6,
+	0x33, 0x2d, 0x4c, 0x87, 0xfa, 0xb7, 0x71, 0x58, 0xd9, 0x66, 0xf2, 0x23, 0x96, 0xa5, 0x0a, 0xe2,
+	0x54, 0xdf, 0xb5, 0x96, 0x2a, 0x73, 0x92, 0xb0, 0x20, 0x6e, 0x52, 0xe8, 0x0f, 0x5e, 0x3d, 0x0c,
+	0x1a, 0x99, 0x41, 0x46, 0x22, 0xed, 0x3b, 0x00, 0x69, 0x1a, 0x93, 0x2d, 0x8f, 0xd2, 0xf6, 0xf6,
+	0x6e, 0x7d, 0xb7, 0x56, 0xea, 0x34, 0x55, 0x62, 0x3c, 0x6f, 0x51, 0xec, 0x7b, 0xdf, 0x82, 0xcd,
+	0xd6, 0x8c, 0xde, 0xed, 0x0e, 0xfb, 0x43, 0x4b, 0xf7, 0x1d, 0x97, 0x2e, 0xb9, 0x2f, 0xe0, 0xfa,
+	0xb4, 0x0c, 0xf9, 0x79, 0x47, 0x2d, 0x6d, 0x77, 0xb4, 0xe6, 0x6e, 0xa7, 0xb5, 0xdb, 0x21, 0xd6,
+	0xf5, 0x03, 0x2a, 0x60, 0xe3, 0x5b, 0x04, 0xe0, 0xaf, 0x7c, 0x57, 0xef, 0xfa, 0x1a, 0xd7, 0x90,
+	0x04, 0xfd, 0x29, 0x5c, 0x0d, 0xe7, 0x94, 0x6c, 0x71, 0xb9, 0xa2, 0xed, 0x95, 0x6a, 0xbb, 0x74,
+	0xb0, 0x8b, 0x14, 0xf4, 0xf6, 0x79, 0x33, 0x4b, 0x36, 0x3b, 0x36, 0x34, 0xaa, 0xa6, 0xe8, 0xb8,
+	0xff, 0x41, 0x12, 0xae, 0xb5, 0x07, 0x96, 0xe9, 0xfb, 0xfa, 0x81, 0x85, 0x5b, 0xba, 0x5b, 0x71,
+	0x22, 0xe3, 0x5f, 0x83, 0x2b, 0xad, 0x92, 0xa2, 0x6a, 0xfb, 0x4a, 0x67, 0x47, 0x53, 0xe5, 0x76,
+	0x47, 0x55, 0xb6, 0x3b, 0x4a, 0xb3, 0x21, 0x2c, 0x88, 0x1b, 0x54, 0xd0, 0xff, 0x9b, 0x12, 0xe4,
+	0x19, 0x3d, 0x6d, 0xa0, 0x9b, 0xae, 0xf6, 0xd2, 0xf4, 0x8f, 0x34, 0x17, 0x7b, 0xbe, 0x6b, 0xd2,
+	0x23, 0x8b, 0xb4, 0xbb, 0x02, 0x2b, 0xed, 0x56, 0x4d, 0xe9, 0x4c, 0x20, 0xc5, 0xc4, 0x0f, 0x29,
+	0xd2, 0xbb, 0x67, 0x20, 0x79, 0xa4, 0x61, 0xd3, 0x28, 0x0d, 0xb8, 0xda, 0x52, 0x9b, 0xdb, 0x72,
+	0xbb, 0x4d, 0xc6, 0x55, 0xae, 0x68, 0x72, 0x4d, 0xae, 0xcb, 0x0d, 0x3a, 0xa4, 0x67, 0xaf, 0x07,
+	0xda, 0x28, 0xd7, 0xe9, 0x62, 0xcf, 0x23, 0x43, 0x8a, 0x0d, 0x0d, 0x5b, 0x98, 0x5a, 0x3c, 0x04,
+	0xaf, 0x0c, 0x42, 0x80, 0x17, 0x22, 0x25, 0xc4, 0x0f, 0x28, 0xd2, 0x3b, 0xaf, 0x40, 0x8a, 0x62,
+	0x3c, 0x87, 0xef, 0xb0, 0x9e, 0x95, 0x1a, 0x15, 0xad, 0xad, 0x7c, 0x2e, 0x47, 0xbb, 0x48, 0x74,
+	0xe2, 0xd9, 0x73, 0x3d, 0xee, 0xa3, 0x6e, 0x1b, 0x9a, 0x67, 0xfe, 0x0c, 0x47, 0x3b, 0x4b, 0x91,
+	0x1d, 0x78, 0x37, 0x68, 0x1d, 0xc1, 0x1d, 0xf7, 0x96, 0x8a, 0x9a, 0x90, 0x92, 0x12, 0xcb, 0x54,
+	0xca, 0x27, 0xaf, 0x68, 0x34, 0x91, 0x11, 0x76, 0x9f, 0x4a, 0x9d, 0x12, 0x28, 0xfd, 0x4e, 0x0c,
+	0xae, 0x06, 0xe7, 0x56, 0xdb, 0x34, 0x30, 0x3d, 0x3b, 0x3b, 0xa3, 0x01, 0xf6, 0xa4, 0x23, 0x48,
+	0xca, 0xf6, 0xb0, 0x8f, 0x3e, 0x82, 0x8c, 0xd2, 0x91, 0xd5, 0x52, 0xb9, 0x46, 0xf6, 0x60, 0x54,
+	0x25, 0x78, 0xa6, 0x81, 0x35, 0x6a, 0x20, 0x6c, 0x99, 0x3e, 0x76, 0xc9, 0x92, 0x22, 0x9d, 0xf8,
+	0x08, 0x32, 0xf5, 0xdd, 0x5a, 0x47, 0xa9, 0x97, 0x5a, 0x42, 0xec, 0x3c, 0x86, 0xfe, 0xd0, 0xf2,
+	0xcd, 0xbe, 0x3e, 0x20, 0x8d, 0xf8, 0x65, 0x1c, 0x72, 0x11, 0xb3, 0x7c, 0xda, 0x96, 0x8a, 0x9d,
+	0xb2, 0xa5, 0xae, 0x41, 0x86, 0xba, 0x3e, 0x9a, 0x69, 0xf0, 0xa3, 0x78, 0x91, 0x7e, 0x2b, 0x06,
+	0x6a, 0x01, 0x98, 0x9e, 0x76, 0xe0, 0x0c, 0x6d, 0x03, 0x1b, 0xd4, 0xce, 0x2b, 0x6c, 0x6e, 0xcc,
+	0x60, 0x50, 0x28, 0x5e, 0x99, 0xf1, 0x14, 0x49, 0xa7, 0xd5, 0xac, 0x19, 0x7c, 0xa3, 0x4d, 0xb8,
+	0x72, 0xca, 0x57, 0x1c, 0x11, 0xc9, 0x49, 0x2a, 0xf9, 0x94, 0x93, 0x37, 0x52, 0x8c, 0x53, 0x86,
+	0x4d, 0xea, 0xf2, 0xf6, 0xe6, 0x37, 0x8b, 0x90, 0xa7, 0x1b, 0xb6, 0xa5, 0x8f, 0x2c, 0x47, 0x37,
+	0xd0, 0x13, 0x48, 0x19, 0x8e, 0xd6, 0xb3, 0xb9, 0x45, 0xb9, 0x39, 0x03, 0x78, 0xdb, 0x38, 0x9e,
+	0x34, 0x2a, 0x0d, 0xa7, 0x6a, 0xa3, 0x1a, 0xc0, 0x40, 0x77, 0xf5, 0x3e, 0xf6, 0x89, 0x57, 0xca,
+	0xfc, 0xed, 0x0f, 0x66, 0x31, 0xef, 0x02, 0x26, 0x35, 0xc2, 0x8f, 0x7e, 0x02, 0xb9, 0xf1, 0x34,
+	0x07, 0x16, 0xe8, 0xa7, 0xb3, 0xc1, 0x85, 0x9d, 0x2b, 0x86, 0x6b, 0x31, 0x88, 0x10, 0x78, 0x21,
+	0x81, 0x4a, 0xf0, 0xc9, 0x11, 0x4a, 0x4c, 0xe2, 0xc0, 0x1e, 0x9d, 0x5f, 0x02, 0x81, 0x20, 0xa3,
+	0x10, 0x4a, 0x08, 0x09, 0x44, 0x82, 0x6f, 0xf6, 0xb1, 0xcb, 0x25, 0xa4, 0x2e, 0x26, 0xa1, 0x43,
+	0x20, 0xa2, 0x12, 0xfc, 0x90, 0x80, 0xde, 0x02, 0xf0, 0x42, 0x3d, 0x4c, 0xed, 0xde, 0x8c, 0x1a,
+	0xa1, 0xa0, 0x3b, 0xb0, 0x16, 0xd9, 0xaa, 0x5a, 0xb8, 0xda, 0x17, 0xe9, 0x9a, 0x43, 0x91, 0xb2,
+	0x6d, 0xbe, 0xf0, 0xef, 0xc2, 0x15, 0x17, 0xff, 0x74, 0x48, 0x2c, 0x28, 0xad, 0x67, 0xda, 0xba,
+	0x65, 0xfe, 0x4c, 0x27, 0xe5, 0xeb, 0x19, 0x0a, 0xbe, 0x16, 0x14, 0x56, 0x23, 0x65, 0xe2, 0x31,
+	0x2c, 0x4f, 0x8d, 0xf4, 0x19, 0x56, 0x6f, 0x79, 0xd2, 0x21, 0x9c, 0x65, 0x69, 0x84, 0xa0, 0x51,
+	0xfb, 0x9a, 0x08, 0x9b, 0x1c, 0xf4, 0xd7, 0x24, 0x2c, 0x00, 0x9d, 0x12, 0x36, 0x35, 0xfe, 0xaf,
+	0x47, 0x58, 0x08, 0x1a, 0xb5, 0xfe, 0x7f, 0x1d, 0x83, 0x6c, 0xb8, 0x1b, 0xd0, 0x53, 0x48, 0xfa,
+	0xa3, 0x01, 0xd3, 0x5b, 0x85, 0xcd, 0x8f, 0xe7, 0xd9, 0x49, 0x45, 0xa2, 0x7a, 0x99, 0x06, 0xa2,
+	0x18, 0xe2, 0xe7, 0x90, 0x24, 0x24, 0x49, 0xe5, 0xca, 0x78, 0x19, 0x72, 0xbb, 0x8d, 0x76, 0x4b,
+	0xde, 0x56, 0xaa, 0x8a, 0x5c, 0x11, 0x16, 0x10, 0x40, 0x9a, 0x19, 0xba, 0x42, 0x0c, 0xad, 0x81,
+	0xd0, 0x52, 0x5a, 0x72, 0x8d, 0x98, 0x0a, 0xcd, 0x16, 0x3b, 0x26, 0xe2, 0xe8, 0x0d, 0x58, 0x8d,
+	0x1c, 0x1c, 0x1a, 0xb1, 0x4b, 0x9e, 0xc9, 0xaa, 0x90, 0x90, 0xfe, 0x25, 0x01, 0xd9, 0x70, 0xec,
+	0x90, 0x0b, 0x57, 0x89, 0x21, 0xab, 0xf5, 0x1d, 0xc3, 0xec, 0x8d, 0x34, 0x66, 0xb0, 0x45, 0x3c,
+	0xd6, 0xef, 0xcf, 0xd0, 0x0f, 0x15, 0xeb, 0x46, 0x9d, 0xf2, 0xef, 0x13, 0xf6, 0x10, 0x7c, 0x67,
+	0x41, 0x5d, 0x75, 0xa7, 0xca, 0x88, 0xcc, 0x1a, 0x64, 0x0e, 0xf4, 0x43, 0x26, 0x25, 0x3e, 0xb3,
+	0x5f, 0x5c, 0xd6, 0x0f, 0xa3, 0xc8, 0x8b, 0x07, 0xfa, 0x21, 0x45, 0xfb, 0x12, 0x0a, 0xcc, 0xf2,
+	0xa1, 0x8a, 0x9a, 0x60, 0x32, 0x37, 0xff, 0xfe, 0x6c, 0x51, 0x06, 0xc6, 0x18, 0x45, 0x5e, 0x0a,
+	0xe1, 0x82, 0xd6, 0x12, 0x5f, 0x83, 0x22, 0x27, 0x67, 0x6e, 0x6d, 0x5d, 0x1f, 0x4c, 0xb4, 0xb6,
+	0xaf, 0x0f, 0x02, 0x34, 0x0f, 0xfb, 0x0c, 0x2d, 0x35, 0x33, 0x5a, 0x1b, 0xfb, 0x13, 0x68, 0x1e,
+	0xf6, 0xc9, 0xcf, 0x72, 0x9a, 0x45, 0x17, 0xa4, 0xfb, 0xb0, 0x7e, 0xde, 0x24, 0x4c, 0x9c, 0x9a,
+	0xb1, 0x89, 0x53, 0x53, 0x7a, 0x08, 0xf9, 0xe8, 0xa8, 0xa2, 0xdb, 0x20, 0x04, 0x56, 0xc3, 0x14,
+	0x4b, 0x81, 0xd3, 0xb9, 0xda, 0x91, 0xbe, 0x89, 0x01, 0x3a, 0x3d, 0x78, 0x44, 0x7f, 0x45, 0xac,
+	0xe4, 0x69, 0x10, 0x14, 0x29, 0x0b, 0xf4, 0xd7, 0x67, 0x34, 0x3e, 0x44, 0xed, 0xd6, 0x9e, 0xcd,
+	0x57, 0xc3, 0x45, 0xce, 0xb4, 0x2c, 0x47, 0xa9, 0xda, 0xd2, 0x1e, 0xe4, 0xa3, 0xa3, 0x8f, 0x6e,
+	0x40, 0x9e, 0xd8, 0xd8, 0x53, 0x8d, 0x81, 0x63, 0x3c, 0x0a, 0x1a, 0x71, 0x0b, 0x0a, 0x74, 0x57,
+	0x6b, 0x53, 0xe6, 0x45, 0x9e, 0x52, 0xb7, 0xc7, 0xa3, 0x15, 0x9d, 0x87, 0x39, 0x46, 0xeb, 0xeb,
+	0x18, 0x64, 0x43, 0x0d, 0x82, 0xda, 0xec, 0x98, 0xd1, 0x0c, 0xa7, 0xaf, 0x9b, 0x36, 0xd7, 0x17,
+	0x9b, 0x33, 0x2a, 0xa1, 0x0a, 0x65, 0x62, 0xba, 0x82, 0x9e, 0x2c, 0x8c, 0x40, 0xba, 0xc0, 0xce,
+	0xae, 0xe9, 0x2e, 0x50, 0x6a, 0xd0, 0x90, 0x1f, 0x42, 0x36, 0xb4, 0x78, 0xa4, 0xbb, 0xe7, 0x29,
+	0x97, 0x25, 0xc8, 0xee, 0x36, 0xca, 0xcd, 0xdd, 0x46, 0x45, 0xae, 0x08, 0x31, 0x94, 0x83, 0xc5,
+	0xe0, 0x23, 0x2e, 0xfd, 0x65, 0x0c, 0x72, 0x64, 0xa9, 0x05, 0xe6, 0xc8, 0x53, 0x48, 0x7b, 0xce,
+	0xd0, 0xed, 0xe2, 0x4b, 0xd8, 0x23, 0x1c, 0x61, 0xca, 0x88, 0x8b, 0x5f, 0xde, 0x88, 0x93, 0x0c,
+	0x58, 0x61, 0x01, 0x58, 0xc5, 0xf6, 0x43, 0x0b, 0xaa, 0x09, 0x59, 0x1e, 0xa7, 0xb8, 0x94, 0x15,
+	0x95, 0x61, 0x20, 0x55, 0x5b, 0xfa, 0xe3, 0x18, 0x14, 0xb8, 0x5b, 0x1b, 0xc8, 0x98, 0x5c, 0xd6,
+	0xb1, 0xd7, 0xb0, 0xac, 0xcf, 0xdd, 0x5b, 0xf1, 0xf3, 0xf6, 0x96, 0xf4, 0xaf, 0x69, 0x58, 0xe9,
+	0x60, 0xcf, 0x6f, 0xd3, 0xd8, 0x4a, 0xd0, 0xb4, 0xf3, 0xf5, 0x01, 0x52, 0x21, 0x8d, 0x4f, 0x68,
+	0xa0, 0x36, 0x3e, 0x73, 0xb4, 0xef, 0x94, 0x80, 0xa2, 0x4c, 0x20, 0x54, 0x8e, 0x24, 0xfe, 0x47,
+	0x12, 0x52, 0x94, 0x82, 0x4e, 0x60, 0xf9, 0xa5, 0xee, 0x63, 0xb7, 0xaf, 0xbb, 0xc7, 0x1a, 0x2d,
+	0xe5, 0x03, 0xf3, 0xec, 0xe2, 0x62, 0x8a, 0x25, 0xe3, 0x44, 0xb7, 0xbb, 0x78, 0x3f, 0x00, 0xde,
+	0x59, 0x50, 0x0b, 0xa1, 0x14, 0x26, 0xf7, 0xeb, 0x18, 0x5c, 0xe1, 0xae, 0x11, 0x39, 0x22, 0xe8,
+	0xde, 0x63, 0xe2, 0x99, 0xba, 0x69, 0x5d, 0x5e, 0x7c, 0x2b, 0x84, 0x27, 0x7b, 0x94, 0x9c, 0x7b,
+	0x83, 0x09, 0x0a, 0x6b, 0x48, 0x1f, 0x96, 0x02, 0x85, 0xc1, 0xe4, 0xb3, 0x83, 0xaa, 0x7a, 0x29,
+	0xf9, 0x86, 0xcc, 0x5d, 0xd4, 0x9d, 0x05, 0x35, 0xcf, 0xe1, 0x69, 0x99, 0xf8, 0x00, 0x84, 0xe9,
+	0xd1, 0x41, 0x6f, 0xc3, 0x92, 0x8d, 0x5f, 0x6a, 0xe1, 0x08, 0xd1, 0x19, 0x48, 0xa8, 0x79, 0x1b,
+	0xbf, 0x0c, 0x2b, 0x89, 0x65, 0xb8, 0x72, 0x66, 0xbf, 0xd0, 0x7b, 0x20, 0xe8, 0xac, 0x40, 0x33,
+	0x86, 0x2e, 0xb3, 0x33, 0x19, 0xc0, 0x32, 0xa7, 0x57, 0x38, 0x59, 0x74, 0x21, 0x17, 0x69, 0x1b,
+	0xea, 0x42, 0x26, 0x70, 0xa5, 0xf9, 0xdd, 0xe1, 0x93, 0x0b, 0xf5, 0x9a, 0x34, 0xc3, 0xf3, 0xf5,
+	0xfe, 0x00, 0x07, 0xd8, 0x6a, 0x08, 0x5c, 0x5e, 0x84, 0x14, 0x1d, 0x57, 0xf1, 0x47, 0x80, 0x4e,
+	0x57, 0x44, 0xef, 0xc2, 0x32, 0xb6, 0xc9, 0x52, 0x0f, 0x7d, 0x63, 0xda, 0xf8, 0xbc, 0x5a, 0xe0,
+	0xe4, 0xa0, 0xe2, 0x9b, 0x90, 0xf5, 0x03, 0x76, 0xba, 0x46, 0x12, 0xea, 0x98, 0x20, 0xfd, 0x57,
+	0x02, 0x56, 0xe8, 0x11, 0x5b, 0x35, 0x2d, 0xec, 0x05, 0xbb, 0xaa, 0x0a, 0x49, 0xcf, 0xb4, 0x8f,
+	0x2f, 0xe3, 0x95, 0x11, 0x7e, 0xf4, 0x23, 0x58, 0x26, 0xfe, 0xbc, 0xee, 0x6b, 0x3d, 0x5e, 0x78,
+	0x89, 0x43, 0xb1, 0xc0, 0xa0, 0x02, 0x1a, 0x19, 0x01, 0xa6, 0xb4, 0xb0, 0xc1, 0x2c, 0x3d, 0x8f,
+	0x2e, 0xc1, 0x8c, 0x5a, 0x08, 0xc8, 0xb4, 0x63, 0x1e, 0xfa, 0x04, 0x44, 0x7e, 0x8b, 0x6e, 0x10,
+	0xfb, 0xb4, 0x6f, 0xda, 0xd8, 0xd0, 0xbc, 0x23, 0xdd, 0x35, 0x4c, 0xfb, 0x90, 0x5a, 0x41, 0x19,
+	0x75, 0x9d, 0xd5, 0xa8, 0x84, 0x15, 0xda, 0xbc, 0x1c, 0xe1, 0x49, 0x5f, 0x90, 0xf9, 0x51, 0x95,
+	0x59, 0x2e, 0xcb, 0xa6, 0x87, 0xf5, 0x55, 0x0e, 0xe1, 0xff, 0xa9, 0x17, 0x23, 0xfd, 0x1c, 0x52,
+	0x54, 0xad, 0xbe, 0x9e, 0x0b, 0x9d, 0x22, 0xac, 0x86, 0x97, 0x5a, 0xa1, 0x26, 0x0f, 0xae, 0x75,
+	0x56, 0xc2, 0x22, 0xae, 0xc8, 0x3d, 0xe9, 0x4f, 0x52, 0x50, 0x08, 0xe2, 0x35, 0xec, 0xc6, 0x50,
+	0xfa, 0xbd, 0x14, 0x3f, 0xbe, 0x6f, 0x41, 0xaa, 0xfc, 0xa2, 0x23, 0xb7, 0x85, 0x05, 0xf1, 0x1a,
+	0x0d, 0xba, 0xac, 0xd2, 0xa0, 0x0b, 0x45, 0xdd, 0x3a, 0x18, 0xf9, 0x34, 0x04, 0x88, 0xee, 0x40,
+	0x8e, 0x38, 0x03, 0x8d, 0x27, 0xda, 0x6e, 0xa7, 0xfa, 0x50, 0x80, 0x89, 0xa8, 0x3f, 0xab, 0x4b,
+	0x7c, 0x4b, 0xfb, 0x50, 0x1b, 0xfa, 0xbd, 0x87, 0x84, 0xe3, 0x2d, 0x88, 0x3f, 0xdb, 0x13, 0x62,
+	0xe2, 0x55, 0x5a, 0x51, 0x88, 0x54, 0x3c, 0x3e, 0x21, 0xe5, 0x12, 0x24, 0xcb, 0xcd, 0x66, 0x4d,
+	0xc8, 0x8b, 0xeb, 0xb4, 0x06, 0x8a, 0x8a, 0x75, 0x1c, 0x8b, 0xd4, 0x79, 0x07, 0xd2, 0x7b, 0x25,
+	0x55, 0x69, 0x74, 0x84, 0xb8, 0x28, 0xd2, 0x5a, 0x6b, 0x91, 0x5a, 0x27, 0xba, 0x6b, 0xda, 0x3e,
+	0xaf, 0x57, 0x69, 0xee, 0x96, 0x6b, 0xb2, 0x90, 0x3b, 0xa3, 0x9e, 0xe1, 0x0c, 0x79, 0x8c, 0xe9,
+	0xfd, 0x48, 0x50, 0x2a, 0x31, 0x11, 0x97, 0x67, 0x35, 0xa3, 0xf1, 0xa8, 0x5b, 0x90, 0xea, 0x28,
+	0x75, 0x59, 0x15, 0x92, 0x67, 0x8c, 0x0b, 0xb5, 0x8a, 0xd8, 0xbd, 0xc1, 0xb2, 0xd2, 0xe8, 0xc8,
+	0xea, 0x5e, 0x98, 0x27, 0x21, 0xa4, 0x26, 0x82, 0xd9, 0x1c, 0xd8, 0xf6, 0xb1, 0x7b, 0xa2, 0x5b,
+	0xfc, 0xe2, 0x80, 0x85, 0xc0, 0x97, 0x6a, 0x72, 0xe3, 0x49, 0x67, 0x47, 0x6b, 0xa9, 0x72, 0x55,
+	0x79, 0x2e, 0xa4, 0x27, 0x82, 0x5e, 0x8c, 0xcf, 0xc2, 0xf6, 0xa1, 0x7f, 0xa4, 0x0d, 0x5c, 0xdc,
+	0x33, 0xbf, 0xe2, 0x5c, 0x13, 0x59, 0x19, 0xc2, 0xe2, 0x19, 0x5c, 0x2c, 0x36, 0x1f, 0x91, 0xf5,
+	0x31, 0x14, 0x58, 0xf5, 0x20, 0x0a, 0x2c, 0x64, 0x26, 0xee, 0x52, 0x18, 0x5b, 0xb8, 0xb7, 0xd9,
+	0xb2, 0xa5, 0xc1, 0xd8, 0x2b, 0xed, 0x4e, 0xa9, 0x23, 0x6b, 0x65, 0xe2, 0xfd, 0x55, 0xb4, 0x70,
+	0xf0, 0xb2, 0xe2, 0x7b, 0x94, 0xfd, 0xed, 0x89, 0xf9, 0xd7, 0x7d, 0xac, 0x1d, 0xe8, 0xdd, 0x63,
+	0x6c, 0x68, 0xd1, 0x91, 0xbc, 0x01, 0x09, 0xb5, 0xb9, 0x2f, 0x2c, 0x89, 0x6f, 0x50, 0x9e, 0x95,
+	0x08, 0x8f, 0x4b, 0xdb, 0x27, 0xfd, 0x6e, 0x3a, 0x30, 0xb4, 0x22, 0x01, 0xb1, 0xd7, 0x6e, 0x68,
+	0xa1, 0x3d, 0xc8, 0xb3, 0x50, 0x3c, 0x69, 0xea, 0xd0, 0xe3, 0x26, 0xe2, 0xdd, 0x59, 0xdc, 0x31,
+	0xc2, 0xd6, 0xa6, 0x5c, 0xcc, 0x48, 0xcc, 0xf5, 0xc7, 0x14, 0xf4, 0x4e, 0xa0, 0x17, 0xc7, 0x56,
+	0x55, 0x82, 0xaa, 0x90, 0x25, 0x46, 0x0e, 0xfc, 0x84, 0x0a, 0x2c, 0xfa, 0xae, 0x79, 0x78, 0x88,
+	0x5d, 0xee, 0x09, 0xbe, 0x3f, 0xcb, 0x21, 0xc6, 0x38, 0xd4, 0x80, 0x15, 0x61, 0x58, 0x09, 0x8d,
+	0x35, 0xd3, 0xb1, 0x89, 0xeb, 0xcd, 0x6e, 0x8e, 0x0b, 0x9b, 0x0f, 0x67, 0xc0, 0x2b, 0x45, 0x78,
+	0xeb, 0x8e, 0xc1, 0xe3, 0x06, 0x82, 0x3e, 0x45, 0x26, 0x6e, 0x06, 0xbb, 0x4e, 0xa0, 0x16, 0x0f,
+	0x0d, 0x36, 0xcd, 0xe6, 0x66, 0xb0, 0xdb, 0x50, 0x72, 0x80, 0x72, 0x37, 0xc3, 0x09, 0x09, 0xe8,
+	0x00, 0x84, 0xae, 0xe5, 0x50, 0x3b, 0xea, 0x00, 0x1f, 0xe9, 0x27, 0xa6, 0xe3, 0xd2, 0xe0, 0x54,
+	0x61, 0xf3, 0xc1, 0x2c, 0xee, 0x36, 0x63, 0x2d, 0x73, 0x4e, 0x06, 0xbf, 0xdc, 0x9d, 0xa4, 0x52,
+	0x2b, 0xc3, 0xb2, 0xe8, 0x42, 0xb6, 0x74, 0x1f, 0xdb, 0xd8, 0xf3, 0x68, 0x34, 0x8b, 0x58, 0x19,
+	0x8c, 0x5e, 0xe3, 0x64, 0xe2, 0xfb, 0x37, 0x6d, 0xd2, 0xb0, 0x80, 0x79, 0x3d, 0x3b, 0x73, 0xf4,
+	0x65, 0x92, 0x91, 0xb5, 0x65, 0x0a, 0x0d, 0x6d, 0xc0, 0x15, 0xdd, 0xf3, 0xcc, 0x43, 0xdb, 0xd3,
+	0x7c, 0x47, 0x73, 0xec, 0xe0, 0xe2, 0x70, 0x1d, 0xe8, 0x11, 0x88, 0x78, 0x61, 0xc7, 0x69, 0xda,
+	0x98, 0xad, 0x7f, 0xe9, 0x0b, 0xc8, 0x45, 0x16, 0x9b, 0x54, 0x3f, 0xcf, 0xc9, 0x5a, 0x86, 0x5c,
+	0xa3, 0xd9, 0xa0, 0xb7, 0x52, 0x4a, 0xe3, 0x89, 0x10, 0xa3, 0x04, 0x59, 0xae, 0xb4, 0xd9, 0x45,
+	0x95, 0x10, 0x47, 0x08, 0x0a, 0xa5, 0x9a, 0x2a, 0x97, 0x2a, 0xfc, 0xee, 0xaa, 0x22, 0x24, 0xa4,
+	0x1f, 0x83, 0x30, 0x3d, 0xff, 0x92, 0x72, 0x9e, 0x88, 0x02, 0x40, 0x45, 0x69, 0x6f, 0x97, 0xd4,
+	0x0a, 0x93, 0x20, 0x40, 0x3e, 0xbc, 0xfe, 0x22, 0x94, 0x38, 0xa9, 0xa1, 0xca, 0xf4, 0xca, 0x8a,
+	0x7c, 0x27, 0xa4, 0xcf, 0x60, 0x79, 0x6a, 0x8e, 0xa4, 0xc7, 0xaf, 0xe8, 0x80, 0x5c, 0x57, 0x3a,
+	0x5a, 0xa9, 0xb6, 0x5f, 0x7a, 0xd1, 0x66, 0x71, 0x28, 0x4a, 0x50, 0xaa, 0x5a, 0xa3, 0xd9, 0x90,
+	0xeb, 0xad, 0xce, 0x0b, 0x21, 0x2e, 0xb5, 0xa6, 0xa7, 0xe8, 0x95, 0x88, 0x55, 0x45, 0x95, 0x27,
+	0x10, 0x29, 0x61, 0x12, 0xf1, 0x00, 0x60, 0xbc, 0x44, 0xa5, 0xce, 0x79, 0x68, 0x2b, 0xb0, 0x24,
+	0x37, 0x2a, 0x5a, 0xb3, 0xaa, 0x85, 0x91, 0x32, 0x04, 0x85, 0x5a, 0x89, 0xde, 0x48, 0x2b, 0x0d,
+	0xad, 0x55, 0x6a, 0x90, 0x51, 0x26, 0xad, 0x2e, 0xa9, 0x35, 0x25, 0x4a, 0x4d, 0x48, 0x16, 0xc0,
+	0xd8, 0xdb, 0x96, 0xbe, 0x7c, 0xc5, 0x08, 0xcb, 0x7b, 0x72, 0xa3, 0x43, 0xf3, 0xea, 0x84, 0x18,
+	0x5a, 0x85, 0x65, 0x7e, 0x91, 0x43, 0x4e, 0x5a, 0x4a, 0x8c, 0xa3, 0x1b, 0xf0, 0x66, 0xfb, 0x45,
+	0x63, 0x7b, 0x47, 0x6d, 0x36, 0xe8, 0xe5, 0xce, 0x74, 0x8d, 0x84, 0xf4, 0x2b, 0x01, 0x16, 0xb9,
+	0x9a, 0x40, 0x2a, 0x64, 0xf5, 0x9e, 0x8f, 0x5d, 0x4d, 0xb7, 0x2c, 0xae, 0x34, 0xef, 0xce, 0xae,
+	0x65, 0x8a, 0x25, 0xc2, 0x5b, 0xb2, 0xac, 0x9d, 0x05, 0x35, 0xa3, 0xf3, 0xdf, 0x11, 0x4c, 0x7b,
+	0xc4, 0x0d, 0xa1, 0xf9, 0x31, 0xed, 0xd1, 0x18, 0xd3, 0x1e, 0xa1, 0x5d, 0x00, 0x86, 0x89, 0xf5,
+	0xee, 0x11, 0xf7, 0x64, 0xee, 0xcd, 0x0b, 0x2a, 0xeb, 0xdd, 0xa3, 0x9d, 0x05, 0x95, 0xb5, 0x8e,
+	0x7c, 0x20, 0x0b, 0x56, 0x39, 0xac, 0x6d, 0x68, 0x4e, 0x2f, 0xd8, 0x6f, 0xc9, 0x99, 0x83, 0x91,
+	0x93, 0xf8, 0xb6, 0xd1, 0xec, 0xb1, 0x8d, 0xb9, 0xb3, 0xa0, 0x0a, 0xfa, 0x14, 0x0d, 0xf9, 0x70,
+	0x85, 0x49, 0x9b, 0xf2, 0x0f, 0x79, 0x68, 0xee, 0xf1, 0xbc, 0xf2, 0x4e, 0xfb, 0x81, 0xfa, 0x69,
+	0x32, 0xfa, 0x26, 0x06, 0x12, 0x13, 0xeb, 0x8d, 0xec, 0xee, 0x91, 0xeb, 0xd8, 0xf4, 0xc2, 0x6e,
+	0xba, 0x0d, 0x2c, 0x2d, 0xe6, 0xe9, 0xbc, 0x6d, 0x68, 0x47, 0x30, 0x4f, 0xb5, 0xe7, 0xba, 0xfe,
+	0xea, 0x2a, 0xe8, 0x19, 0xa4, 0x75, 0xeb, 0xa5, 0x3e, 0xf2, 0xd6, 0xf3, 0x54, 0xfc, 0xc6, 0x3c,
+	0xe2, 0x29, 0xe3, 0xce, 0x82, 0xca, 0x21, 0x50, 0x03, 0x16, 0x0d, 0xdc, 0xd3, 0x87, 0x96, 0x4f,
+	0x0f, 0x89, 0xd9, 0x8e, 0xff, 0x00, 0xad, 0xc2, 0x38, 0x77, 0x16, 0xd4, 0x00, 0x04, 0x7d, 0x39,
+	0x76, 0xa0, 0xbb, 0xce, 0xd0, 0xf6, 0xe9, 0xb1, 0x90, 0x9b, 0xe9, 0xe8, 0x09, 0x50, 0xe5, 0x20,
+	0x32, 0x37, 0xb4, 0xfd, 0x88, 0xc7, 0x4c, 0xbf, 0xd1, 0x0e, 0xa4, 0x6c, 0x7c, 0x82, 0xd9, 0x29,
+	0x92, 0xdb, 0xbc, 0x33, 0x07, 0x6e, 0x83, 0xf0, 0xed, 0x2c, 0xa8, 0x0c, 0x80, 0xec, 0x0e, 0xc7,
+	0x65, 0x17, 0x32, 0xd6, 0x88, 0x9e, 0x16, 0xf3, 0xed, 0x8e, 0xa6, 0x5b, 0x65, 0xbc, 0x64, 0x77,
+	0x38, 0xc1, 0x07, 0x99, 0x1d, 0x17, 0x0f, 0xb0, 0xee, 0xaf, 0xe7, 0xe6, 0x9e, 0x1d, 0x95, 0x32,
+	0x92, 0xd9, 0x61, 0x10, 0xe2, 0x73, 0xc8, 0x04, 0xda, 0x02, 0xd5, 0x20, 0x47, 0x93, 0xc9, 0x68,
+	0xd5, 0xc0, 0x45, 0x9f, 0xc7, 0xba, 0x89, 0xb2, 0x8f, 0x91, 0xed, 0xd1, 0x6b, 0x46, 0x7e, 0x01,
+	0xd9, 0x50, 0x71, 0xbc, 0x66, 0xe8, 0xbf, 0x89, 0x81, 0x30, 0xad, 0x34, 0x50, 0x13, 0x96, 0xb0,
+	0xee, 0x5a, 0x23, 0xad, 0x67, 0x12, 0xe7, 0x28, 0xc8, 0x60, 0x9c, 0x47, 0x48, 0x9e, 0x02, 0x54,
+	0x19, 0x3f, 0xaa, 0x43, 0x9e, 0x18, 0x35, 0x21, 0x5e, 0x7c, 0x6e, 0xbc, 0x1c, 0xe1, 0xe7, 0x70,
+	0xe2, 0xff, 0x87, 0xd5, 0x33, 0x14, 0x0f, 0x3a, 0x82, 0xb5, 0x30, 0x60, 0xa1, 0x9d, 0x4a, 0xdb,
+	0xbe, 0x3f, 0x63, 0xac, 0x99, 0xb2, 0x8f, 0xf3, 0x74, 0x57, 0xfd, 0x53, 0x34, 0x4f, 0xbc, 0x09,
+	0xd7, 0xbf, 0x45, 0xeb, 0x88, 0x59, 0x58, 0xe4, 0x7b, 0x59, 0xbc, 0x0b, 0xf9, 0xe8, 0x06, 0x44,
+	0x6f, 0x4f, 0x6f, 0x68, 0x32, 0xbc, 0xa9, 0xc9, 0x5d, 0x29, 0x2e, 0x42, 0x8a, 0xee, 0x2e, 0x31,
+	0x03, 0x69, 0xa6, 0x62, 0xc4, 0x3f, 0x8a, 0x41, 0x36, 0xdc, 0x22, 0xe8, 0x31, 0x24, 0xc3, 0x48,
+	0xfa, 0x7c, 0x63, 0x49, 0xf9, 0x88, 0x59, 0x1f, 0xec, 0xd4, 0xf9, 0xa7, 0x23, 0x60, 0x15, 0x3b,
+	0x90, 0x66, 0x5b, 0x0c, 0x3d, 0x05, 0x18, 0x2f, 0xac, 0x0b, 0xb4, 0x2a, 0xc2, 0x5d, 0xce, 0x86,
+	0x2e, 0x87, 0xf4, 0x8f, 0xf1, 0x48, 0x58, 0x6b, 0x9c, 0x82, 0xda, 0x86, 0x94, 0x81, 0x2d, 0x7d,
+	0x34, 0xc7, 0x85, 0xdd, 0x69, 0x94, 0x62, 0x85, 0x40, 0x10, 0xfd, 0x45, 0xb1, 0xd0, 0xe7, 0x90,
+	0xd1, 0x2d, 0xf3, 0xd0, 0xd6, 0x7c, 0x87, 0x8f, 0xc9, 0x0f, 0x2e, 0x86, 0x5b, 0x22, 0x28, 0x1d,
+	0x87, 0x68, 0x71, 0x9d, 0xfd, 0x14, 0xdf, 0x87, 0x14, 0x95, 0x86, 0x6e, 0x42, 0x9e, 0x4a, 0xd3,
+	0xfa, 0xa6, 0x65, 0x99, 0x1e, 0x0f, 0x25, 0xe6, 0x28, 0xad, 0x4e, 0x49, 0xe2, 0x23, 0x58, 0xe4,
+	0x08, 0xe8, 0x2a, 0xa4, 0x07, 0xd8, 0x35, 0x1d, 0xe6, 0x9b, 0x25, 0x54, 0xfe, 0x45, 0xe8, 0x4e,
+	0xaf, 0xe7, 0x61, 0x9f, 0x1a, 0x09, 0x09, 0x95, 0x7f, 0x95, 0xaf, 0xc0, 0xea, 0x19, 0x7b, 0x40,
+	0xfa, 0xc3, 0x38, 0x64, 0xc3, 0x08, 0x0f, 0xda, 0x83, 0x82, 0xde, 0xa5, 0x49, 0x33, 0x03, 0xdd,
+	0xf7, 0xb1, 0x6b, 0x5f, 0x34, 0xae, 0xb3, 0xc4, 0x60, 0x5a, 0x0c, 0x05, 0x3d, 0x83, 0xc5, 0x13,
+	0x13, 0xbf, 0xbc, 0xdc, 0x9d, 0x56, 0x9a, 0x40, 0x54, 0x6d, 0xf4, 0x25, 0xac, 0x70, 0xf7, 0xb4,
+	0xaf, 0x0f, 0x06, 0xc4, 0x3e, 0xe8, 0xd9, 0xdc, 0xe2, 0xba, 0x08, 0x2c, 0xf7, 0x75, 0xeb, 0x0c,
+	0xab, 0x6a, 0x4b, 0x9f, 0x42, 0x2e, 0x92, 0xca, 0x8d, 0x04, 0x48, 0x0c, 0x5d, 0x9b, 0xdf, 0x2b,
+	0x90, 0x9f, 0x68, 0x1d, 0x16, 0x07, 0x2c, 0x20, 0x47, 0xc5, 0xe6, 0xd5, 0xe0, 0xf3, 0x69, 0x32,
+	0x13, 0x13, 0xe2, 0xd2, 0x9f, 0xc6, 0x60, 0x2d, 0x08, 0x4f, 0x45, 0x73, 0xcd, 0xa5, 0xaf, 0x63,
+	0x90, 0x8f, 0x12, 0xd0, 0x2d, 0x48, 0x57, 0x9a, 0xf4, 0x22, 0x7a, 0x61, 0x22, 0x6c, 0x84, 0xed,
+	0x93, 0x2d, 0xc3, 0xe9, 0x1e, 0xb3, 0xa0, 0xcc, 0x3b, 0xb0, 0xc8, 0x8d, 0x64, 0x21, 0x36, 0x11,
+	0xbc, 0x21, 0xd5, 0xb8, 0x99, 0x44, 0xea, 0xdd, 0x86, 0x8c, 0xfc, 0xbc, 0x23, 0xab, 0x8d, 0x52,
+	0x6d, 0x2a, 0xc0, 0x44, 0x2a, 0xe2, 0xaf, 0xc8, 0x54, 0xe8, 0xd6, 0xd6, 0xc9, 0x86, 0xf4, 0x10,
+	0x96, 0x2a, 0x14, 0x3e, 0x88, 0xd7, 0xbe, 0x0b, 0xcb, 0x5d, 0xc7, 0xf6, 0x75, 0xd3, 0x26, 0xfe,
+	0x7e, 0x5f, 0x3f, 0x0c, 0x12, 0x8e, 0x0a, 0x21, 0x59, 0x21, 0x54, 0xe9, 0xdf, 0x62, 0x50, 0xe0,
+	0x0a, 0x2d, 0xe0, 0x2d, 0x40, 0xdc, 0xf1, 0x78, 0xf5, 0xb8, 0xe3, 0x21, 0x04, 0x49, 0xdd, 0xed,
+	0x1e, 0xf1, 0x11, 0xa3, 0xbf, 0xc9, 0x90, 0x75, 0x9d, 0x7e, 0x5f, 0xb7, 0x83, 0x50, 0x42, 0xf0,
+	0x89, 0x6a, 0x90, 0xc0, 0xf6, 0xc9, 0x3c, 0xf9, 0xd4, 0x13, 0xd2, 0x8b, 0xb2, 0x7d, 0xc2, 0x62,
+	0xa1, 0x04, 0x46, 0xfc, 0x18, 0x32, 0x01, 0x61, 0xae, 0xcc, 0xe5, 0xff, 0x8e, 0xc1, 0xb2, 0xcc,
+	0x07, 0x28, 0xe8, 0x57, 0x1b, 0x32, 0xc1, 0x33, 0x28, 0xbe, 0x0d, 0x66, 0xb1, 0xac, 0x4a, 0x03,
+	0xb3, 0x8d, 0xdd, 0x13, 0xb3, 0x8b, 0x2b, 0xe1, 0x3b, 0x28, 0x35, 0x04, 0x42, 0x7b, 0x90, 0xa6,
+	0x69, 0x42, 0xc1, 0x9d, 0xd2, 0x2c, 0x36, 0xf5, 0x54, 0xc3, 0x58, 0xa2, 0x44, 0x90, 0x9a, 0xce,
+	0xd0, 0xc4, 0x47, 0x90, 0x8b, 0x90, 0xe7, 0xea, 0xfb, 0x2f, 0x60, 0x79, 0x6a, 0x4f, 0xbc, 0x9e,
+	0xa8, 0xee, 0xf7, 0xa0, 0x10, 0x79, 0x3b, 0x33, 0xbe, 0x9b, 0x5b, 0x8a, 0x50, 0x15, 0x43, 0xda,
+	0x82, 0xfc, 0x84, 0x6c, 0xbe, 0xdf, 0x62, 0x33, 0xec, 0x37, 0xe9, 0xb7, 0x49, 0xc8, 0x45, 0x72,
+	0xc5, 0x90, 0x02, 0x29, 0xd3, 0xc7, 0xe1, 0xc9, 0x7e, 0x77, 0xbe, 0x54, 0xb3, 0xa2, 0xe2, 0xe3,
+	0xbe, 0xca, 0x10, 0xc4, 0x1e, 0x80, 0x62, 0x60, 0xdb, 0x37, 0x7b, 0x26, 0x76, 0x89, 0x6e, 0x8e,
+	0xbe, 0xb1, 0xe0, 0xad, 0xcb, 0xf9, 0xe3, 0xe7, 0x15, 0xe4, 0xf0, 0x1e, 0x57, 0x19, 0x6b, 0x8c,
+	0x31, 0xdf, 0xae, 0x6b, 0x07, 0xf3, 0x92, 0x08, 0xe7, 0x45, 0xfc, 0x75, 0x1c, 0x92, 0x44, 0x2e,
+	0x52, 0x20, 0xce, 0x81, 0x67, 0x7b, 0xab, 0x30, 0xd1, 0xf0, 0xb0, 0xa5, 0x6a, 0xdc, 0x24, 0x7b,
+	0x8a, 0xe5, 0xde, 0xc4, 0x67, 0x8e, 0xa2, 0x45, 0xc1, 0xa6, 0xb2, 0x6f, 0xd0, 0xfb, 0xc1, 0xca,
+	0x61, 0x3a, 0x76, 0xad, 0xc8, 0x1e, 0xfc, 0x15, 0x83, 0x07, 0x7f, 0xc5, 0x92, 0x1d, 0x3c, 0xe3,
+	0x41, 0xf7, 0x21, 0xe7, 0x1d, 0x39, 0xae, 0xcf, 0x62, 0xae, 0xdc, 0x4f, 0x3d, 0x9b, 0x03, 0x68,
+	0xc5, 0x3d, 0xca, 0xb6, 0x06, 0x29, 0x4b, 0x3f, 0xc0, 0x16, 0x7f, 0x31, 0xc2, 0x3e, 0xd0, 0x35,
+	0xc8, 0x58, 0xa6, 0x7d, 0xac, 0x0d, 0x5d, 0x8b, 0x7a, 0x7f, 0x59, 0x75, 0x91, 0x7c, 0xef, 0xba,
+	0x96, 0xf8, 0x0b, 0x9e, 0x11, 0x34, 0x7c, 0x45, 0x46, 0x10, 0x0b, 0xf0, 0xb3, 0x1b, 0x7b, 0xa5,
+	0xd1, 0x91, 0x9f, 0xc8, 0xaa, 0x10, 0x47, 0x59, 0x48, 0x55, 0x6b, 0xcd, 0x52, 0x47, 0x48, 0xb0,
+	0x9b, 0xfc, 0x66, 0x4d, 0x2e, 0x35, 0x84, 0x24, 0x5a, 0x82, 0x6c, 0xf8, 0x1a, 0x50, 0x48, 0xa1,
+	0x3c, 0x64, 0x2a, 0xbb, 0x6a, 0x89, 0xa6, 0xeb, 0xa6, 0x51, 0x01, 0xe0, 0x69, 0x69, 0xaf, 0xa4,
+	0x6d, 0xd7, 0x4a, 0xed, 0xb6, 0xb0, 0x28, 0xfd, 0x43, 0x06, 0xae, 0xd4, 0xb1, 0xe7, 0xe9, 0x87,
+	0x78, 0xdf, 0xf4, 0x8f, 0x22, 0xd9, 0xc3, 0xaf, 0xf9, 0x81, 0xcf, 0x0f, 0x21, 0x45, 0x63, 0xb0,
+	0xf3, 0xbe, 0x78, 0x22, 0xa6, 0x0b, 0x65, 0x44, 0x5f, 0x10, 0xcd, 0xce, 0xd3, 0xab, 0x23, 0x9b,
+	0x68, 0x36, 0x67, 0x69, 0xf2, 0x1a, 0x7f, 0x67, 0x41, 0xe5, 0xb9, 0x45, 0xe1, 0xc5, 0xfe, 0x4f,
+	0x60, 0xc5, 0x33, 0x8e, 0xc3, 0xcb, 0xb9, 0x68, 0x5a, 0xd0, 0x05, 0xce, 0xe2, 0x9d, 0x05, 0x75,
+	0xd9, 0x9b, 0x52, 0x45, 0xfb, 0x50, 0x18, 0xe8, 0xae, 0x66, 0x38, 0x61, 0xf3, 0xd3, 0x33, 0x2b,
+	0xa5, 0x68, 0x22, 0x22, 0xf1, 0x6e, 0x07, 0xd1, 0xcc, 0xd1, 0x26, 0xc0, 0x20, 0xdc, 0x9b, 0xdc,
+	0x21, 0x9f, 0xef, 0xa9, 0xde, 0xce, 0x82, 0x1a, 0x81, 0x40, 0x2a, 0xe4, 0x22, 0xcf, 0x2b, 0xb9,
+	0x33, 0x3e, 0xe7, 0x63, 0xbc, 0x9d, 0x05, 0x35, 0x0a, 0x82, 0xda, 0x90, 0xa7, 0xf9, 0x68, 0x41,
+	0xdf, 0xb3, 0x33, 0x83, 0x46, 0xb2, 0x52, 0x08, 0xa8, 0x1b, 0x49, 0x52, 0xa9, 0x03, 0x8c, 0x2f,
+	0x24, 0xb9, 0xeb, 0x3c, 0xd7, 0x4d, 0x20, 0xf1, 0xc2, 0xc3, 0x9b, 0x47, 0xd4, 0x83, 0xd5, 0xc8,
+	0x43, 0x97, 0xb0, 0xa9, 0xf9, 0x39, 0x1f, 0x05, 0x46, 0x72, 0x52, 0x76, 0x16, 0x54, 0x6e, 0xe2,
+	0x45, 0x13, 0x55, 0x30, 0xa0, 0xd3, 0x29, 0xc8, 0xeb, 0x4b, 0x17, 0x7f, 0x7b, 0x38, 0x16, 0x13,
+	0xbd, 0xa6, 0xd9, 0x83, 0xa5, 0xc9, 0xe5, 0x5c, 0xb8, 0xd0, 0x21, 0x48, 0xd6, 0x5b, 0x2f, 0xf2,
+	0x5d, 0x4e, 0x43, 0xd2, 0x75, 0x1c, 0x5f, 0xfa, 0x55, 0x1a, 0xae, 0xca, 0x5f, 0xe1, 0xee, 0x90,
+	0xe6, 0xb8, 0xb6, 0x7d, 0xfd, 0x30, 0xdc, 0x4d, 0x2d, 0xc8, 0x45, 0xce, 0x46, 0xae, 0x3d, 0xe6,
+	0x7d, 0x7a, 0x18, 0x85, 0x20, 0x8a, 0x95, 0xcd, 0x32, 0x3f, 0xf5, 0x4d, 0x3e, 0x63, 0x67, 0x64,
+	0x27, 0xcb, 0x33, 0x59, 0x22, 0x67, 0xb5, 0x7b, 0xbc, 0x30, 0x14, 0x63, 0x22, 0x47, 0xf9, 0xad,
+	0x89, 0x47, 0xd2, 0x49, 0x7a, 0x9d, 0x1b, 0x7d, 0xe5, 0xbc, 0x3e, 0x7e, 0x4f, 0x97, 0xa2, 0x85,
+	0xe1, 0x9b, 0xb8, 0x49, 0x35, 0x9a, 0xbe, 0xac, 0x1a, 0xed, 0x41, 0x6e, 0xe8, 0x61, 0x97, 0x5e,
+	0x94, 0x61, 0x6f, 0x7d, 0xf1, 0xb2, 0x1d, 0xde, 0xf5, 0xb0, 0x4b, 0x33, 0xdf, 0x48, 0x87, 0x87,
+	0xc1, 0x87, 0x87, 0x5e, 0x40, 0x9a, 0x5e, 0xa5, 0x7a, 0xeb, 0x19, 0x2a, 0xa2, 0x74, 0x71, 0x11,
+	0x34, 0x41, 0x4e, 0x31, 0x54, 0x0e, 0x28, 0x36, 0x21, 0x17, 0x19, 0xe6, 0x59, 0x0c, 0x92, 0xef,
+	0x02, 0x58, 0x4e, 0x57, 0xb7, 0xd8, 0xfb, 0x01, 0xb6, 0x00, 0xb2, 0x94, 0xd2, 0xd0, 0xfb, 0x98,
+	0x00, 0x46, 0xba, 0xf1, 0x1a, 0x00, 0x9f, 0xc1, 0x22, 0x6f, 0xf4, 0xe5, 0xc1, 0xb6, 0x3e, 0x85,
+	0x0c, 0xfd, 0xf7, 0x02, 0x62, 0xff, 0xdd, 0x3c, 0x65, 0x3f, 0x90, 0x33, 0x9f, 0x5a, 0x0e, 0xcd,
+	0x01, 0x7b, 0x1f, 0xff, 0x4f, 0x7f, 0xf6, 0x57, 0xcf, 0x99, 0x85, 0x40, 0xb8, 0x76, 0x5d, 0x7b,
+	0x4b, 0x81, 0x25, 0x0a, 0xd0, 0xe5, 0x7f, 0x33, 0x30, 0x0b, 0xca, 0x3f, 0x07, 0x28, 0xf9, 0x83,
+	0xc8, 0xdf, 0x15, 0x94, 0xbf, 0x0f, 0xdf, 0xfe, 0x97, 0x09, 0xe5, 0xac, 0x4a, 0xf3, 0x3f, 0x4a,
+	0x03, 0xf3, 0xf3, 0x5c, 0x40, 0xd7, 0x4e, 0x36, 0x0e, 0xd2, 0x54, 0xdc, 0xdd, 0xff, 0x0d, 0x00,
+	0x00, 0xff, 0xff, 0xbe, 0xe9, 0x00, 0xc5, 0x8d, 0x41, 0x00, 0x00,
 }
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/RowJson.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/RowJson.java
index 49ded07..96ad120 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/RowJson.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/RowJson.java
@@ -137,6 +137,14 @@
         return jsonArrayToList(fieldValue);
       }
 
+      if (fieldValue.typeName().isLogicalType()) {
+        return extractJsonNodeValue(
+            FieldValue.of(
+                fieldValue.name(),
+                fieldValue.type().getLogicalType().getBaseType(),
+                fieldValue.jsonValue()));
+      }
+
       return extractJsonPrimitiveValue(fieldValue);
     }
 
@@ -351,6 +359,9 @@
         case ROW:
           writeRow((Row) value, type.getRowSchema(), gen);
           break;
+        case LOGICAL_TYPE:
+          writeValue(gen, type.getLogicalType().getBaseType(), value);
+          break;
         default:
           throw new IllegalArgumentException("Unsupported field type: " + type);
       }
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/RowJsonValidation.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/RowJsonValidation.java
index 69cd1d3..af929ff 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/RowJsonValidation.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/RowJsonValidation.java
@@ -59,6 +59,11 @@
       return;
     }
 
+    if (fieldTypeName.isLogicalType()) {
+      verifyFieldTypeSupported(fieldType.getLogicalType().getBaseType());
+      return;
+    }
+
     if (!SUPPORTED_TYPES.contains(fieldTypeName)) {
       throw new RowJson.RowJsonDeserializer.UnsupportedRowJsonException(
           fieldTypeName.name()
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/RowJsonTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/RowJsonTest.java
index 3277bb0..d9ab410 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/RowJsonTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/RowJsonTest.java
@@ -28,6 +28,7 @@
 import java.math.BigDecimal;
 import java.util.Arrays;
 import java.util.Collection;
+import org.apache.beam.sdk.schemas.LogicalTypes;
 import org.apache.beam.sdk.schemas.Schema;
 import org.apache.beam.sdk.schemas.Schema.FieldType;
 import org.apache.beam.sdk.util.RowJson.RowJsonDeserializer.UnsupportedRowJsonException;
@@ -66,6 +67,7 @@
     public static Collection<Object[]> data() {
       return ImmutableList.of(
           makeFlatRowTestCase(),
+          makeLogicalTypeTestCase(),
           makeArrayFieldTestCase(),
           makeArrayOfArraysTestCase(),
           makeNestedRowTestCase(),
@@ -117,6 +119,22 @@
       return new Object[] {"Flat row", schema, rowString, expectedRow};
     }
 
+    private static Object[] makeLogicalTypeTestCase() {
+      Schema schema =
+          Schema.builder()
+              .addLogicalTypeField(
+                  "f_passThroughString",
+                  new LogicalTypes.PassThroughLogicalType<String>(
+                      "SqlCharType", "", FieldType.STRING) {})
+              .build();
+
+      String rowString = "{\n" + "\"f_passThroughString\" : \"hello\"\n" + "}";
+
+      Row expectedRow = Row.withSchema(schema).addValues("hello").build();
+
+      return new Object[] {"Logical Types", schema, rowString, expectedRow};
+    }
+
     private static Object[] makeArrayFieldTestCase() {
 
       Schema schema =
diff --git a/sdks/java/extensions/google-cloud-platform-core/src/test/java/org/apache/beam/sdk/extensions/gcp/options/GcpOptionsTest.java b/sdks/java/extensions/google-cloud-platform-core/src/test/java/org/apache/beam/sdk/extensions/gcp/options/GcpOptionsTest.java
index ab03de0..0f0b5b3 100644
--- a/sdks/java/extensions/google-cloud-platform-core/src/test/java/org/apache/beam/sdk/extensions/gcp/options/GcpOptionsTest.java
+++ b/sdks/java/extensions/google-cloud-platform-core/src/test/java/org/apache/beam/sdk/extensions/gcp/options/GcpOptionsTest.java
@@ -159,21 +159,6 @@
       options.getGcpTempLocation();
     }
 
-    @Test
-    public void testDefaultGcpTempLocationDoesNotExist() {
-      GcpOptions options = PipelineOptionsFactory.as(GcpOptions.class);
-      String tempLocation = "gs://does/not/exist";
-      options.setTempLocation(tempLocation);
-      thrown.expect(IllegalArgumentException.class);
-      thrown.expectMessage(
-          "Error constructing default value for gcpTempLocation: tempLocation is not"
-              + " a valid GCS path");
-      thrown.expectCause(
-          hasMessage(containsString("Output path does not exist or is not writeable")));
-
-      options.getGcpTempLocation();
-    }
-
     private static void makePropertiesFileWithProject(File path, String projectId)
         throws IOException {
       String properties =
@@ -221,6 +206,21 @@
     }
 
     @Test
+    public void testDefaultGcpTempLocationDoesNotExist() throws IOException {
+      String tempLocation = "gs://does/not/exist";
+      options.setTempLocation(tempLocation);
+      when(mockGcsUtil.bucketAccessible(any(GcsPath.class))).thenReturn(false);
+      thrown.expect(IllegalArgumentException.class);
+      thrown.expectMessage(
+          "Error constructing default value for gcpTempLocation: tempLocation is not"
+              + " a valid GCS path");
+      thrown.expectCause(
+          hasMessage(containsString("Output path does not exist or is not writeable")));
+
+      options.as(GcpOptions.class).getGcpTempLocation();
+    }
+
+    @Test
     public void testCreateBucket() throws Exception {
       doReturn(fakeProject).when(mockGet).execute();
       when(mockGcsUtil.bucketOwner(any(GcsPath.class))).thenReturn(1L);
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/mongodb/MongoDbTable.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/mongodb/MongoDbTable.java
index eaa9661..7b8ce03 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/mongodb/MongoDbTable.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/mongodb/MongoDbTable.java
@@ -31,8 +31,11 @@
 import org.apache.beam.sdk.schemas.Schema;
 import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.transforms.JsonToRow;
+import org.apache.beam.sdk.transforms.MapElements;
 import org.apache.beam.sdk.transforms.PTransform;
 import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.SimpleFunction;
+import org.apache.beam.sdk.transforms.ToJson;
 import org.apache.beam.sdk.values.PBegin;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PCollection.IsBounded;
@@ -83,7 +86,9 @@
 
   @Override
   public POutput buildIOWriter(PCollection<Row> input) {
-    throw new UnsupportedOperationException("Writing to a MongoDB is not supported");
+    return input
+        .apply(new RowToDocument())
+        .apply(MongoDbIO.write().withUri(dbUri).withDatabase(dbName).withCollection(dbCollection));
   }
 
   @Override
@@ -140,4 +145,29 @@
       }
     }
   }
+
+  public static class RowToDocument extends PTransform<PCollection<Row>, PCollection<Document>> {
+
+    private RowToDocument() {}
+
+    public static RowToDocument convert() {
+      return new RowToDocument();
+    }
+
+    @Override
+    public PCollection<Document> expand(PCollection<Row> input) {
+      return input
+          // TODO(BEAM-8498): figure out a way convert Row directly to Document.
+          .apply("Transform Rows to JSON", ToJson.of())
+          .apply("Produce documents from JSON", MapElements.via(new ObjectToDocumentFn()));
+    }
+
+    @VisibleForTesting
+    static class ObjectToDocumentFn extends SimpleFunction<String, Document> {
+      @Override
+      public Document apply(String input) {
+        return Document.parse(input);
+      }
+    }
+  }
 }
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/mongodb/MongoDbReadWriteIT.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/mongodb/MongoDbReadWriteIT.java
index 9c4d4cd..82cafb9 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/mongodb/MongoDbReadWriteIT.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/mongodb/MongoDbReadWriteIT.java
@@ -29,23 +29,17 @@
 
 import com.mongodb.MongoClient;
 import java.util.Arrays;
-import org.apache.beam.sdk.PipelineResult;
 import org.apache.beam.sdk.extensions.sql.impl.BeamSqlEnv;
+import org.apache.beam.sdk.extensions.sql.impl.rel.BeamRelNode;
 import org.apache.beam.sdk.extensions.sql.impl.rel.BeamSqlRelUtils;
 import org.apache.beam.sdk.io.mongodb.MongoDBIOIT.MongoDBPipelineOptions;
-import org.apache.beam.sdk.io.mongodb.MongoDbIO;
 import org.apache.beam.sdk.options.PipelineOptionsFactory;
 import org.apache.beam.sdk.schemas.Schema;
 import org.apache.beam.sdk.schemas.Schema.FieldType;
 import org.apache.beam.sdk.testing.PAssert;
 import org.apache.beam.sdk.testing.TestPipeline;
-import org.apache.beam.sdk.transforms.Create;
-import org.apache.beam.sdk.transforms.MapElements;
-import org.apache.beam.sdk.transforms.SimpleFunction;
-import org.apache.beam.sdk.transforms.ToJson;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.Row;
-import org.bson.Document;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Ignore;
@@ -115,8 +109,6 @@
 
   @Test
   public void testWriteAndRead() {
-    final String mongoUrl =
-        String.format("mongodb://%s:%d", options.getMongoDBHostName(), options.getMongoDBPort());
     final String mongoSqlUrl =
         String.format(
             "mongodb://%s:%d/%s/%s",
@@ -139,20 +131,6 @@
             "varchar",
             Arrays.asList("123", "456"));
 
-    writePipeline
-        .apply(Create.of(testRow))
-        .setRowSchema(SOURCE_SCHEMA)
-        .apply("Transform Rows to JSON", ToJson.of())
-        .apply("Produce documents from JSON", MapElements.via(new ObjectToDocumentFn()))
-        .apply(
-            "Write documents to MongoDB",
-            MongoDbIO.write()
-                .withUri(mongoUrl)
-                .withDatabase(options.getMongoDBDatabaseName())
-                .withCollection(collection));
-    PipelineResult writeResult = writePipeline.run();
-    writeResult.waitUntilFinish();
-
     String createTableStatement =
         "CREATE EXTERNAL TABLE TEST( \n"
             + "   _id VARCHAR, \n "
@@ -170,10 +148,27 @@
             + "LOCATION '"
             + mongoSqlUrl
             + "'";
-
     BeamSqlEnv sqlEnv = BeamSqlEnv.inMemory(new MongoDbTableProvider());
     sqlEnv.executeDdl(createTableStatement);
 
+    String insertStatement =
+        "INSERT INTO TEST VALUES ("
+            + "'object_id', "
+            + "9223372036854775807, "
+            + "127, "
+            + "32767, "
+            + "2147483647, "
+            + "1.0, "
+            + "1.0, "
+            + "TRUE, "
+            + "'varchar', "
+            + "ARRAY['123', '456']"
+            + ")";
+
+    BeamRelNode insertRelNode = sqlEnv.parseQuery(insertStatement);
+    BeamSqlRelUtils.toPCollection(writePipeline, insertRelNode);
+    writePipeline.run().waitUntilFinish();
+
     PCollection<Row> output =
         BeamSqlRelUtils.toPCollection(readPipeline, sqlEnv.parseQuery("select * from TEST"));
 
@@ -184,13 +179,6 @@
     readPipeline.run().waitUntilFinish();
   }
 
-  private static class ObjectToDocumentFn extends SimpleFunction<String, Document> {
-    @Override
-    public Document apply(String input) {
-      return Document.parse(input);
-    }
-  }
-
   private Row row(Schema schema, Object... values) {
     return Row.withSchema(schema).addValues(values).build();
   }
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/mongodb/MongoDbTableTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/mongodb/MongoDbTableTest.java
index cccac9c..cd5bdff 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/mongodb/MongoDbTableTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/mongodb/MongoDbTableTest.java
@@ -27,7 +27,9 @@
 import static org.apache.beam.sdk.schemas.Schema.FieldType.STRING;
 
 import java.util.Arrays;
+import org.apache.beam.sdk.extensions.sql.impl.utils.CalciteUtils;
 import org.apache.beam.sdk.extensions.sql.meta.provider.mongodb.MongoDbTable.DocumentToRow;
+import org.apache.beam.sdk.extensions.sql.meta.provider.mongodb.MongoDbTable.RowToDocument;
 import org.apache.beam.sdk.schemas.Schema;
 import org.apache.beam.sdk.schemas.Schema.FieldType;
 import org.apache.beam.sdk.testing.PAssert;
@@ -53,7 +55,7 @@
           .addNullableField("bool", BOOLEAN)
           .addNullableField("double", DOUBLE)
           .addNullableField("float", FLOAT)
-          .addNullableField("string", STRING)
+          .addNullableField("string", CalciteUtils.CHAR)
           .addRowField("nested", Schema.builder().addNullableField("int32", INT32).build())
           .addNullableField("arr", FieldType.array(STRING))
           .build();
@@ -78,7 +80,7 @@
     PCollection<Row> output =
         pipeline
             .apply("Create document from JSON", Create.<Document>of(Document.parse(JSON_ROW)))
-            .apply("CConvert document to Row", DocumentToRow.withSchema(SCHEMA));
+            .apply("Convert document to Row", DocumentToRow.withSchema(SCHEMA));
 
     // Make sure proper rows are constructed from JSON.
     PAssert.that(output)
@@ -99,6 +101,35 @@
     pipeline.run().waitUntilFinish();
   }
 
+  @Test
+  public void testRowToDocumentConverter() {
+    PCollection<Document> output =
+        pipeline
+            .apply(
+                "Create a row",
+                Create.of(
+                        row(
+                            SCHEMA,
+                            9223372036854775807L,
+                            2147483647,
+                            (short) 32767,
+                            (byte) 127,
+                            true,
+                            1.0,
+                            (float) 1.0,
+                            "string",
+                            row(
+                                Schema.builder().addNullableField("int32", INT32).build(),
+                                2147483645),
+                            Arrays.asList("str1", "str2", "str3")))
+                    .withRowSchema(SCHEMA))
+            .apply("Convert row to document", RowToDocument.convert());
+
+    PAssert.that(output).containsInAnyOrder(Document.parse(JSON_ROW));
+
+    pipeline.run().waitUntilFinish();
+  }
+
   private Row row(Schema schema, Object... values) {
     return Row.withSchema(schema).addValues(values).build();
   }
diff --git a/sdks/java/io/kinesis/build.gradle b/sdks/java/io/kinesis/build.gradle
index ccc0dce..6cdaf3b 100644
--- a/sdks/java/io/kinesis/build.gradle
+++ b/sdks/java/io/kinesis/build.gradle
@@ -37,7 +37,7 @@
   compile library.java.aws_java_sdk_cloudwatch
   compile library.java.aws_java_sdk_core
   compile library.java.aws_java_sdk_kinesis
-  compile "com.amazonaws:amazon-kinesis-client:1.10.0"
+  compile "com.amazonaws:amazon-kinesis-client:1.13.0"
   compile "com.amazonaws:amazon-kinesis-producer:0.13.1"
   compile "commons-lang:commons-lang:2.6"
   testCompile project(path: ":sdks:java:io:common", configuration: "testRuntime")
diff --git a/sdks/python/apache_beam/io/iobase.py b/sdks/python/apache_beam/io/iobase.py
index 5b66730..e21052f 100644
--- a/sdks/python/apache_beam/io/iobase.py
+++ b/sdks/python/apache_beam/io/iobase.py
@@ -35,6 +35,7 @@
 import logging
 import math
 import random
+import threading
 import uuid
 from builtins import object
 from builtins import range
@@ -1104,13 +1105,17 @@
 class RestrictionTracker(object):
   """Manages concurrent access to a restriction.
 
-  Experimental; no backwards-compatibility guarantees.
-
   Keeps track of the restrictions claimed part for a Splittable DoFn.
 
+  The restriction may be modified by different threads, however the system will
+  ensure sufficient locking such that no methods on the restriction tracker
+  will be called concurrently.
+
   See following documents for more details.
   * https://s.apache.org/splittable-do-fn
   * https://s.apache.org/splittable-do-fn-python-sdk
+
+  Experimental; no backwards-compatibility guarantees.
   """
 
   def current_restriction(self):
@@ -1121,54 +1126,22 @@
 
     The current restriction returned by method may be updated dynamically due
     to due to concurrent invocation of other methods of the
-    ``RestrictionTracker``, For example, ``checkpoint()``.
+    ``RestrictionTracker``, For example, ``split()``.
 
-    ** Thread safety **
+    This API is required to be implemented.
 
-    Methods of the class ``RestrictionTracker`` including this method may get
-    invoked by different threads, hence must be made thread-safe, e.g. by using
-    a single lock object.
-
-    TODO(BEAM-7473): Remove thread safety requirements from API implementation.
+    Returns: a restriction object.
     """
     raise NotImplementedError
 
   def current_progress(self):
     """Returns a RestrictionProgress object representing the current progress.
+
+    This API is recommended to be implemented. The runner can do a better job
+    at parallel processing with better progress signals.
     """
     raise NotImplementedError
 
-  def current_watermark(self):
-    """Returns current watermark. By default, not report watermark.
-
-    TODO(BEAM-7473): Provide synchronization guarantee by using a wrapper.
-    """
-    return None
-
-  def checkpoint(self):
-    """Performs a checkpoint of the current restriction.
-
-    Signals that the current ``DoFn.process()`` call should terminate as soon as
-    possible. After this method returns, the tracker MUST refuse all future
-    claim calls, and ``RestrictionTracker.check_done()`` MUST succeed.
-
-    This invocation modifies the value returned by ``current_restriction()``
-    invocation and returns a restriction representing the rest of the work. The
-    old value of ``current_restriction()`` is equivalent to the new value of
-    ``current_restriction()`` and the return value of this method invocation
-    combined.
-
-    ** Thread safety **
-
-    Methods of the class ``RestrictionTracker`` including this method may get
-    invoked by different threads, hence must be made thread-safe, e.g. by using
-    a single lock object.
-
-    TODO(BEAM-7473): Remove thread safety requirements from API implementation.
-    """
-
-    raise NotImplementedError
-
   def check_done(self):
     """Checks whether the restriction has been fully processed.
 
@@ -1179,13 +1152,8 @@
     remaining in the restriction when this method is invoked. Exception raised
     must have an informative error message.
 
-    ** Thread safety **
-
-    Methods of the class ``RestrictionTracker`` including this method may get
-    invoked by different threads, hence must be made thread-safe, e.g. by using
-    a single lock object.
-
-    TODO(BEAM-7473): Remove thread safety requirements from API implementation.
+    This API is required to be implemented in order to make sure no data loss
+    during SDK processing.
 
     Returns: ``True`` if current restriction has been fully processed.
     Raises:
@@ -1215,8 +1183,12 @@
     restrictions returned would be [100, 179), [179, 200) (note: current_offset
     + fraction_of_remainder * remaining_work = 130 + 0.7 * 70 = 179).
 
-    It is very important for pipeline scaling and end to end pipeline execution
-    that try_split is implemented well.
+    ``fraction_of_remainder`` = 0 means a checkpoint is required.
+
+    The API is recommended to be implemented for batch pipeline given that it is
+    very important for pipeline scaling and end to end pipeline execution.
+
+    The API is required to be implemented for a streaming pipeline.
 
     Args:
       fraction_of_remainder: A hint as to the fraction of work the primary
@@ -1226,19 +1198,11 @@
     Returns:
       (primary_restriction, residual_restriction) if a split was possible,
       otherwise returns ``None``.
-
-    ** Thread safety **
-
-    Methods of the class ``RestrictionTracker`` including this method may get
-    invoked by different threads, hence must be made thread-safe, e.g. by using
-    a single lock object.
-
-    TODO(BEAM-7473): Remove thread safety requirements from API implementation.
     """
     raise NotImplementedError
 
   def try_claim(self, position):
-    """ Attempts to claim the block of work in the current restriction
+    """Attempts to claim the block of work in the current restriction
     identified by the given position.
 
     If this succeeds, the DoFn MUST execute the entire block of work. If it
@@ -1247,40 +1211,137 @@
     work from ``DoFn.process()`` is also not allowed before the first call of
     this method).
 
+    The API is required to be implemented.
+
     Args:
       position: current position that wants to be claimed.
 
     Returns: ``True`` if the position can be claimed as current_position.
     Otherwise, returns ``False``.
-
-    ** Thread safety **
-
-    Methods of the class ``RestrictionTracker`` including this method may get
-    invoked by different threads, hence must be made thread-safe, e.g. by using
-    a single lock object.
-
-    TODO(BEAM-7473): Remove thread safety requirements from API implementation.
     """
     raise NotImplementedError
 
-  def defer_remainder(self, watermark=None):
-    """ Invokes checkpoint() in an SDF.process().
 
-    TODO(BEAM-7472): Remove defer_remainder() once SDF.process() uses
-    ``ProcessContinuation``.
+class ThreadsafeRestrictionTracker(object):
+  """A thread-safe wrapper which wraps a `RestritionTracker`.
+
+  This wrapper guarantees synchronization of modifying restrictions across
+  multi-thread.
+  """
+
+  def __init__(self, restriction_tracker):
+    if not isinstance(restriction_tracker, RestrictionTracker):
+      raise ValueError(
+          'Initialize ThreadsafeRestrictionTracker requires'
+          'RestrictionTracker.')
+    self._restriction_tracker = restriction_tracker
+    # Records an absolute timestamp when defer_remainder is called.
+    self._deferred_timestamp = None
+    self._lock = threading.RLock()
+    self._deferred_residual = None
+    self._deferred_watermark = None
+
+  def current_restriction(self):
+    with self._lock:
+      return self._restriction_tracker.current_restriction()
+
+  def try_claim(self, position):
+    with self._lock:
+      return self._restriction_tracker.try_claim(position)
+
+  def defer_remainder(self, deferred_time=None):
+    """Performs self-checkpoint on current processing restriction with an
+    expected resuming time.
+
+    Self-checkpoint could happen during processing elements. When executing an
+    DoFn.process(), you may want to stop processing an element and resuming
+    later if current element has been processed quit a long time or you also
+    want to have some outputs from other elements. ``defer_remainder()`` can be
+    called on per element if needed.
 
     Args:
-      watermark
+      deferred_time: A relative ``timestamp.Duration`` that indicates the ideal
+      time gap between now and resuming, or an absolute ``timestamp.Timestamp``
+      for resuming execution time. If the time_delay is None, the deferred work
+      will be executed as soon as possible.
     """
-    raise NotImplementedError
+
+    # Record current time for calculating deferred_time later.
+    self._deferred_timestamp = timestamp.Timestamp.now()
+    if (deferred_time and
+        not isinstance(deferred_time, timestamp.Duration) and
+        not isinstance(deferred_time, timestamp.Timestamp)):
+      raise ValueError('The timestamp of deter_remainder() should be a '
+                       'Duration or a Timestamp, or None.')
+    self._deferred_watermark = deferred_time
+    checkpoint = self.try_split(0)
+    if checkpoint:
+      _, self._deferred_residual = checkpoint
+
+  def check_done(self):
+    with self._lock:
+      return self._restriction_tracker.check_done()
+
+  def current_progress(self):
+    with self._lock:
+      return self._restriction_tracker.current_progress()
+
+  def try_split(self, fraction_of_remainder):
+    with self._lock:
+      return self._restriction_tracker.try_split(fraction_of_remainder)
 
   def deferred_status(self):
-    """ Returns deferred_residual with deferred_watermark.
+    """Returns deferred work which is produced by ``defer_remainder()``.
 
-    TODO(BEAM-7472): Remove defer_status() once SDF.process() uses
-    ``ProcessContinuation``.
+    When there is a self-checkpoint performed, the system needs to fulfill the
+    DelayedBundleApplication with deferred_work for a  ProcessBundleResponse.
+    The system calls this API to get deferred_residual with watermark together
+    to help the runner to schedule a future work.
+
+    Returns: (deferred_residual, time_delay) if having any residual, else None.
     """
-    raise NotImplementedError
+    if self._deferred_residual:
+      # If _deferred_watermark is None, create Duration(0).
+      if not self._deferred_watermark:
+        self._deferred_watermark = timestamp.Duration()
+      # If an absolute timestamp is provided, calculate the delta between
+      # the absoluted time and the time deferred_status() is called.
+      elif isinstance(self._deferred_watermark, timestamp.Timestamp):
+        self._deferred_watermark = (self._deferred_watermark -
+                                    timestamp.Timestamp.now())
+      # If a Duration is provided, the deferred time should be:
+      # provided duration - the spent time since the defer_remainder() is
+      # called.
+      elif isinstance(self._deferred_watermark, timestamp.Duration):
+        self._deferred_watermark -= (timestamp.Timestamp.now() -
+                                     self._deferred_timestamp)
+      return self._deferred_residual, self._deferred_watermark
+
+
+class RestrictionTrackerView(object):
+  """A DoFn view of thread-safe RestrictionTracker.
+
+  The RestrictionTrackerView wraps a ThreadsafeRestrictionTracker and only
+  exposes APIs that will be called by a ``DoFn.process()``. During execution
+  time, the RestrictionTrackerView will be fed into the ``DoFn.process`` as a
+  restriction_tracker.
+  """
+
+  def __init__(self, threadsafe_restriction_tracker):
+    if not isinstance(threadsafe_restriction_tracker,
+                      ThreadsafeRestrictionTracker):
+      raise ValueError('Initialize RestrictionTrackerView requires '
+                       'ThreadsafeRestrictionTracker.')
+    self._threadsafe_restriction_tracker = threadsafe_restriction_tracker
+
+  def current_restriction(self):
+    return self._threadsafe_restriction_tracker.current_restriction()
+
+  def try_claim(self, position):
+    return self._threadsafe_restriction_tracker.try_claim(position)
+
+  def defer_remainder(self, deferred_time=None):
+    self._threadsafe_restriction_tracker.defer_remainder(deferred_time)
 
 
 class RestrictionProgress(object):
@@ -1400,17 +1461,8 @@
                 SourceBundle(residual_weight, self._source, split_pos,
                              stop_pos))
 
-    def deferred_status(self):
-      return None
-
-    def current_watermark(self):
-      return None
-
-    def get_delegate_range_tracker(self):
-      return self._delegate_range_tracker
-
-    def get_tracking_source(self):
-      return self._source
+    def check_done(self):
+      return self._delegate_range_tracker.fraction_consumed() >= 1.0
 
   class _SDFBoundedSourceRestrictionProvider(core.RestrictionProvider):
     """A `RestrictionProvider` that is used by SDF for `BoundedSource`."""
@@ -1463,8 +1515,13 @@
           restriction_tracker=core.DoFn.RestrictionParam(
               _SDFBoundedSourceWrapper._SDFBoundedSourceRestrictionProvider(
                   source, chunk_size))):
-        return restriction_tracker.get_tracking_source().read(
-            restriction_tracker.get_delegate_range_tracker())
+        current_restriction = restriction_tracker.current_restriction()
+        assert isinstance(current_restriction, SourceBundle)
+        tracking_source = current_restriction.source
+        start = current_restriction.start_position
+        stop = current_restriction.stop_position
+        return tracking_source.read(tracking_source.get_range_tracker(start,
+                                                                      stop))
 
     return SDFBoundedSourceDoFn(self.source)
 
diff --git a/sdks/python/apache_beam/io/iobase_test.py b/sdks/python/apache_beam/io/iobase_test.py
index 7adb764..0a6afae 100644
--- a/sdks/python/apache_beam/io/iobase_test.py
+++ b/sdks/python/apache_beam/io/iobase_test.py
@@ -19,6 +19,7 @@
 
 from __future__ import absolute_import
 
+import time
 import unittest
 
 import mock
@@ -28,6 +29,9 @@
 from apache_beam.io.concat_source_test import RangeSource
 from apache_beam.io import iobase
 from apache_beam.io.iobase import SourceBundle
+from apache_beam.io.restriction_trackers import OffsetRange
+from apache_beam.io.restriction_trackers import OffsetRestrictionTracker
+from apache_beam.utils import timestamp
 from apache_beam.options.pipeline_options import DebugOptions
 from apache_beam.testing.util import assert_that
 from apache_beam.testing.util import equal_to
@@ -191,5 +195,87 @@
     self._run_sdf_wrapper_pipeline(RangeSource(0, 4), [0, 1, 2, 3])
 
 
+class ThreadsafeRestrictionTrackerTest(unittest.TestCase):
+
+  def test_initialization(self):
+    with self.assertRaises(ValueError):
+      iobase.ThreadsafeRestrictionTracker(RangeSource(0, 1))
+
+  def test_defer_remainder_with_wrong_time_type(self):
+    threadsafe_tracker = iobase.ThreadsafeRestrictionTracker(
+        OffsetRestrictionTracker(OffsetRange(0, 10)))
+    with self.assertRaises(ValueError):
+      threadsafe_tracker.defer_remainder(10)
+
+  def test_self_checkpoint_immediately(self):
+    restriction_tracker = OffsetRestrictionTracker(OffsetRange(0, 10))
+    threadsafe_tracker = iobase.ThreadsafeRestrictionTracker(
+        restriction_tracker)
+    threadsafe_tracker.defer_remainder()
+    deferred_residual, deferred_time = threadsafe_tracker.deferred_status()
+    expected_residual = OffsetRange(0, 10)
+    self.assertEqual(deferred_residual, expected_residual)
+    self.assertTrue(isinstance(deferred_time, timestamp.Duration))
+    self.assertEqual(deferred_time, 0)
+
+  def test_self_checkpoint_with_relative_time(self):
+    threadsafe_tracker = iobase.ThreadsafeRestrictionTracker(
+        OffsetRestrictionTracker(OffsetRange(0, 10)))
+    threadsafe_tracker.defer_remainder(timestamp.Duration(100))
+    time.sleep(2)
+    _, deferred_time = threadsafe_tracker.deferred_status()
+    self.assertTrue(isinstance(deferred_time, timestamp.Duration))
+    # The expectation = 100 - 2 - some_delta
+    self.assertTrue(deferred_time <= 98)
+
+  def test_self_checkpoint_with_absolute_time(self):
+    threadsafe_tracker = iobase.ThreadsafeRestrictionTracker(
+        OffsetRestrictionTracker(OffsetRange(0, 10)))
+    now = timestamp.Timestamp.now()
+    schedule_time = now + timestamp.Duration(100)
+    self.assertTrue(isinstance(schedule_time, timestamp.Timestamp))
+    threadsafe_tracker.defer_remainder(schedule_time)
+    time.sleep(2)
+    _, deferred_time = threadsafe_tracker.deferred_status()
+    self.assertTrue(isinstance(deferred_time, timestamp.Duration))
+    # The expectation =
+    # schedule_time - the time when deferred_status is called - some_delta
+    self.assertTrue(deferred_time <= 98)
+
+
+class RestrictionTrackerViewTest(unittest.TestCase):
+
+  def test_initialization(self):
+    with self.assertRaises(ValueError):
+      iobase.RestrictionTrackerView(
+          OffsetRestrictionTracker(OffsetRange(0, 10)))
+
+  def test_api_expose(self):
+    threadsafe_tracker = iobase.ThreadsafeRestrictionTracker(
+        OffsetRestrictionTracker(OffsetRange(0, 10)))
+    tracker_view = iobase.RestrictionTrackerView(threadsafe_tracker)
+    current_restriction = tracker_view.current_restriction()
+    self.assertEqual(current_restriction, OffsetRange(0, 10))
+    self.assertTrue(tracker_view.try_claim(0))
+    tracker_view.defer_remainder()
+    deferred_remainder, deferred_watermark = (
+        threadsafe_tracker.deferred_status())
+    self.assertEqual(deferred_remainder, OffsetRange(1, 10))
+    self.assertEqual(deferred_watermark, timestamp.Duration())
+
+  def test_non_expose_apis(self):
+    threadsafe_tracker = iobase.ThreadsafeRestrictionTracker(
+        OffsetRestrictionTracker(OffsetRange(0, 10)))
+    tracker_view = iobase.RestrictionTrackerView(threadsafe_tracker)
+    with self.assertRaises(AttributeError):
+      tracker_view.check_done()
+    with self.assertRaises(AttributeError):
+      tracker_view.current_progress()
+    with self.assertRaises(AttributeError):
+      tracker_view.try_split()
+    with self.assertRaises(AttributeError):
+      tracker_view.deferred_status()
+
+
 if __name__ == '__main__':
   unittest.main()
diff --git a/sdks/python/apache_beam/io/restriction_trackers.py b/sdks/python/apache_beam/io/restriction_trackers.py
index 0ba5b23..20bb5c1 100644
--- a/sdks/python/apache_beam/io/restriction_trackers.py
+++ b/sdks/python/apache_beam/io/restriction_trackers.py
@@ -19,7 +19,6 @@
 from __future__ import absolute_import
 from __future__ import division
 
-import threading
 from builtins import object
 
 from apache_beam.io.iobase import RestrictionProgress
@@ -86,104 +85,69 @@
     assert isinstance(offset_range, OffsetRange)
     self._range = offset_range
     self._current_position = None
-    self._current_watermark = None
     self._last_claim_attempt = None
-    self._deferred_residual = None
     self._checkpointed = False
-    self._lock = threading.RLock()
 
   def check_done(self):
-    with self._lock:
-      if self._last_claim_attempt < self._range.stop - 1:
-        raise ValueError(
-            'OffsetRestrictionTracker is not done since work in range [%s, %s) '
-            'has not been claimed.'
-            % (self._last_claim_attempt if self._last_claim_attempt is not None
-               else self._range.start,
-               self._range.stop))
+    if self._last_claim_attempt < self._range.stop - 1:
+      raise ValueError(
+          'OffsetRestrictionTracker is not done since work in range [%s, %s) '
+          'has not been claimed.'
+          % (self._last_claim_attempt if self._last_claim_attempt is not None
+             else self._range.start,
+             self._range.stop))
 
   def current_restriction(self):
-    with self._lock:
-      return self._range
-
-  def current_watermark(self):
-    return self._current_watermark
+    return self._range
 
   def current_progress(self):
-    with self._lock:
-      if self._current_position is None:
-        fraction = 0.0
-      elif self._range.stop == self._range.start:
-        # If self._current_position is not None, we must be done.
-        fraction = 1.0
-      else:
-        fraction = (
-            float(self._current_position - self._range.start)
-            / (self._range.stop - self._range.start))
+    if self._current_position is None:
+      fraction = 0.0
+    elif self._range.stop == self._range.start:
+      # If self._current_position is not None, we must be done.
+      fraction = 1.0
+    else:
+      fraction = (
+          float(self._current_position - self._range.start)
+          / (self._range.stop - self._range.start))
     return RestrictionProgress(fraction=fraction)
 
   def start_position(self):
-    with self._lock:
-      return self._range.start
+    return self._range.start
 
   def stop_position(self):
-    with self._lock:
-      return self._range.stop
-
-  def default_size(self):
-    return self._range.size()
+    return self._range.stop
 
   def try_claim(self, position):
-    with self._lock:
-      if self._last_claim_attempt and position <= self._last_claim_attempt:
-        raise ValueError(
-            'Positions claimed should strictly increase. Trying to claim '
-            'position %d while last claim attempt was %d.'
-            % (position, self._last_claim_attempt))
+    if self._last_claim_attempt and position <= self._last_claim_attempt:
+      raise ValueError(
+          'Positions claimed should strictly increase. Trying to claim '
+          'position %d while last claim attempt was %d.'
+          % (position, self._last_claim_attempt))
 
-      self._last_claim_attempt = position
-      if position < self._range.start:
-        raise ValueError(
-            'Position to be claimed cannot be smaller than the start position '
-            'of the range. Tried to claim position %r for the range [%r, %r)'
-            % (position, self._range.start, self._range.stop))
+    self._last_claim_attempt = position
+    if position < self._range.start:
+      raise ValueError(
+          'Position to be claimed cannot be smaller than the start position '
+          'of the range. Tried to claim position %r for the range [%r, %r)'
+          % (position, self._range.start, self._range.stop))
 
-      if position >= self._range.start and position < self._range.stop:
-        self._current_position = position
-        return True
+    if position >= self._range.start and position < self._range.stop:
+      self._current_position = position
+      return True
 
-      return False
+    return False
 
   def try_split(self, fraction_of_remainder):
-    with self._lock:
-      if not self._checkpointed:
-        if self._current_position is None:
-          cur = self._range.start - 1
-        else:
-          cur = self._current_position
-        split_point = (
-            cur + int(max(1, (self._range.stop - cur) * fraction_of_remainder)))
-        if split_point < self._range.stop:
-          self._range, residual_range = self._range.split_at(split_point)
-          return self._range, residual_range
-
-  # TODO(SDF): Replace all calls with try_claim(0).
-  def checkpoint(self):
-    with self._lock:
-      # If self._current_position is 'None' no records have been claimed so
-      # residual should start from self._range.start.
+    if not self._checkpointed:
       if self._current_position is None:
-        end_position = self._range.start
+        cur = self._range.start - 1
       else:
-        end_position = self._current_position + 1
-      self._range, residual_range = self._range.split_at(end_position)
-      return residual_range
-
-  def defer_remainder(self, watermark=None):
-    with self._lock:
-      self._deferred_watermark = watermark or self._current_watermark
-      self._deferred_residual = self.checkpoint()
-
-  def deferred_status(self):
-    if self._deferred_residual:
-      return (self._deferred_residual, self._deferred_watermark)
+        cur = self._current_position
+      split_point = (
+          cur + int(max(1, (self._range.stop - cur) * fraction_of_remainder)))
+      if split_point < self._range.stop:
+        if fraction_of_remainder == 0:
+          self._checkpointed = True
+        self._range, residual_range = self._range.split_at(split_point)
+        return self._range, residual_range
diff --git a/sdks/python/apache_beam/io/restriction_trackers_test.py b/sdks/python/apache_beam/io/restriction_trackers_test.py
index 459b039..4a57d98 100644
--- a/sdks/python/apache_beam/io/restriction_trackers_test.py
+++ b/sdks/python/apache_beam/io/restriction_trackers_test.py
@@ -81,14 +81,14 @@
 
   def test_checkpoint_unstarted(self):
     tracker = OffsetRestrictionTracker(OffsetRange(100, 200))
-    checkpoint = tracker.checkpoint()
+    _, checkpoint = tracker.try_split(0)
     self.assertEqual(OffsetRange(100, 100), tracker.current_restriction())
     self.assertEqual(OffsetRange(100, 200), checkpoint)
 
   def test_checkpoint_just_started(self):
     tracker = OffsetRestrictionTracker(OffsetRange(100, 200))
     self.assertTrue(tracker.try_claim(100))
-    checkpoint = tracker.checkpoint()
+    _, checkpoint = tracker.try_split(0)
     self.assertEqual(OffsetRange(100, 101), tracker.current_restriction())
     self.assertEqual(OffsetRange(101, 200), checkpoint)
 
@@ -96,7 +96,7 @@
     tracker = OffsetRestrictionTracker(OffsetRange(100, 200))
     self.assertTrue(tracker.try_claim(105))
     self.assertTrue(tracker.try_claim(110))
-    checkpoint = tracker.checkpoint()
+    _, checkpoint = tracker.try_split(0)
     self.assertEqual(OffsetRange(100, 111), tracker.current_restriction())
     self.assertEqual(OffsetRange(111, 200), checkpoint)
 
@@ -105,9 +105,9 @@
     self.assertTrue(tracker.try_claim(105))
     self.assertTrue(tracker.try_claim(110))
     self.assertTrue(tracker.try_claim(199))
-    checkpoint = tracker.checkpoint()
+    checkpoint = tracker.try_split(0)
     self.assertEqual(OffsetRange(100, 200), tracker.current_restriction())
-    self.assertEqual(OffsetRange(200, 200), checkpoint)
+    self.assertEqual(None, checkpoint)
 
   def test_checkpoint_after_failed_claim(self):
     tracker = OffsetRestrictionTracker(OffsetRange(100, 200))
@@ -116,7 +116,7 @@
     self.assertTrue(tracker.try_claim(160))
     self.assertFalse(tracker.try_claim(240))
 
-    checkpoint = tracker.checkpoint()
+    _, checkpoint = tracker.try_split(0)
     self.assertTrue(OffsetRange(100, 161), tracker.current_restriction())
     self.assertTrue(OffsetRange(161, 200), checkpoint)
 
diff --git a/sdks/python/apache_beam/metrics/cells.py b/sdks/python/apache_beam/metrics/cells.py
index d30dd2a..1df52d9 100644
--- a/sdks/python/apache_beam/metrics/cells.py
+++ b/sdks/python/apache_beam/metrics/cells.py
@@ -404,7 +404,8 @@
     else:
       self.sum = self.count = 0
       self.min = 2**63 - 1
-      self.max = -2**63
+      # Avoid Wimplicitly-unsigned-literal caused by -2**63.
+      self.max = -self.min - 1
 
   def __eq__(self, other):
     return (self.sum == other.sum and
diff --git a/sdks/python/apache_beam/options/pipeline_options.py b/sdks/python/apache_beam/options/pipeline_options.py
index 462c5ab..b92ee44 100644
--- a/sdks/python/apache_beam/options/pipeline_options.py
+++ b/sdks/python/apache_beam/options/pipeline_options.py
@@ -23,6 +23,7 @@
 import json
 import logging
 import os
+import subprocess
 from builtins import list
 from builtins import object
 
@@ -514,11 +515,17 @@
       return environment_region
     try:
       cmd = ['gcloud', 'config', 'get-value', 'compute/region']
-      output = processes.check_output(cmd).decode('utf-8').strip()
-      if output:
+      # Use subprocess.DEVNULL in Python 3.3+.
+      if hasattr(subprocess, 'DEVNULL'):
+        DEVNULL = subprocess.DEVNULL
+      else:
+        DEVNULL = open(os.devnull, 'ab')
+      raw_output = processes.check_output(cmd, stderr=DEVNULL)
+      formatted_output = raw_output.decode('utf-8').strip()
+      if formatted_output:
         logging.info('Using default GCP region %s from `%s`',
-                     output, ' '.join(cmd))
-        return output
+                     formatted_output, ' '.join(cmd))
+        return formatted_output
     except RuntimeError:
       pass
     logging.warning(
diff --git a/sdks/python/apache_beam/runners/common.pxd b/sdks/python/apache_beam/runners/common.pxd
index 2ffe432..37e05bf 100644
--- a/sdks/python/apache_beam/runners/common.pxd
+++ b/sdks/python/apache_beam/runners/common.pxd
@@ -42,6 +42,8 @@
   cdef object key_arg_name
   cdef object restriction_provider
   cdef object restriction_provider_arg_name
+  cdef object watermark_estimator
+  cdef object watermark_estimator_arg_name
 
 
 cdef class DoFnSignature(object):
@@ -91,7 +93,9 @@
   cdef bint cache_globally_windowed_args
   cdef object process_method
   cdef bint is_splittable
-  cdef object restriction_tracker
+  cdef object threadsafe_restriction_tracker
+  cdef object watermark_estimator
+  cdef object watermark_estimator_param
   cdef WindowedValue current_windowed_value
   cdef bint is_key_param_required
 
diff --git a/sdks/python/apache_beam/runners/common.py b/sdks/python/apache_beam/runners/common.py
index 3e14f3b..8632cfd 100644
--- a/sdks/python/apache_beam/runners/common.py
+++ b/sdks/python/apache_beam/runners/common.py
@@ -14,7 +14,6 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 #
-
 # cython: profile=True
 
 """Worker operations executor.
@@ -167,6 +166,8 @@
     self.key_arg_name = None
     self.restriction_provider = None
     self.restriction_provider_arg_name = None
+    self.watermark_estimator = None
+    self.watermark_estimator_arg_name = None
 
     for kw, v in zip(self.args[-len(self.defaults):], self.defaults):
       if isinstance(v, core.DoFn.StateParam):
@@ -184,6 +185,9 @@
       elif isinstance(v, core.DoFn.RestrictionParam):
         self.restriction_provider = v.restriction_provider
         self.restriction_provider_arg_name = kw
+      elif isinstance(v, core.DoFn.WatermarkEstimatorParam):
+        self.watermark_estimator = v.watermark_estimator
+        self.watermark_estimator_arg_name = kw
 
   def invoke_timer_callback(self,
                             user_state_context,
@@ -264,6 +268,9 @@
   def get_restriction_provider(self):
     return self.process_method.restriction_provider
 
+  def get_watermark_estimator(self):
+    return self.process_method.watermark_estimator
+
   def _validate(self):
     self._validate_process()
     self._validate_bundle_method(self.start_bundle_method)
@@ -458,7 +465,11 @@
         signature.is_stateful_dofn())
     self.user_state_context = user_state_context
     self.is_splittable = signature.is_splittable_dofn()
-    self.restriction_tracker = None
+    self.watermark_estimator = self.signature.get_watermark_estimator()
+    self.watermark_estimator_param = (
+        self.signature.process_method.watermark_estimator_arg_name
+        if self.watermark_estimator else None)
+    self.threadsafe_restriction_tracker = None
     self.current_windowed_value = None
     self.bundle_finalizer_param = bundle_finalizer_param
     self.is_key_param_required = False
@@ -569,15 +580,24 @@
         raise ValueError(
             'A RestrictionTracker %r was provided but DoFn does not have a '
             'RestrictionTrackerParam defined' % restriction_tracker)
-      additional_kwargs[restriction_tracker_param] = restriction_tracker
+      from apache_beam.io import iobase
+      self.threadsafe_restriction_tracker = iobase.ThreadsafeRestrictionTracker(
+          restriction_tracker)
+      additional_kwargs[restriction_tracker_param] = (
+          iobase.RestrictionTrackerView(self.threadsafe_restriction_tracker))
+
+      if self.watermark_estimator:
+        # The watermark estimator needs to be reset for every element.
+        self.watermark_estimator.reset()
+        additional_kwargs[self.watermark_estimator_param] = (
+            self.watermark_estimator)
       try:
         self.current_windowed_value = windowed_value
-        self.restriction_tracker = restriction_tracker
         return self._invoke_process_per_window(
             windowed_value, additional_args, additional_kwargs,
             output_processor)
       finally:
-        self.restriction_tracker = None
+        self.threadsafe_restriction_tracker = None
         self.current_windowed_value = windowed_value
 
     elif self.has_windowed_inputs and len(windowed_value.windows) != 1:
@@ -664,24 +684,34 @@
           windowed_value, self.process_method(*args_for_process))
 
     if self.is_splittable:
-      deferred_status = self.restriction_tracker.deferred_status()
+      # TODO: Consider calling check_done right after SDF.Process() finishing.
+      # In order to do this, we need to know that current invoking dofn is
+      # ProcessSizedElementAndRestriction.
+      self.threadsafe_restriction_tracker.check_done()
+      deferred_status = self.threadsafe_restriction_tracker.deferred_status()
+      output_watermark = None
+      if self.watermark_estimator:
+        output_watermark = self.watermark_estimator.current_watermark()
       if deferred_status:
         deferred_restriction, deferred_watermark = deferred_status
         element = windowed_value.value
         size = self.signature.get_restriction_provider().restriction_size(
             element, deferred_restriction)
-        return (
+        return ((
             windowed_value.with_value(((element, deferred_restriction), size)),
-            deferred_watermark)
+            output_watermark), deferred_watermark)
 
   def try_split(self, fraction):
-    restriction_tracker = self.restriction_tracker
+    restriction_tracker = self.threadsafe_restriction_tracker
     current_windowed_value = self.current_windowed_value
     if restriction_tracker and current_windowed_value:
       # Temporary workaround for [BEAM-7473]: get current_watermark before
       # split, in case watermark gets advanced before getting split results.
       # In worst case, current_watermark is always stale, which is ok.
-      current_watermark = restriction_tracker.current_watermark()
+      if self.watermark_estimator:
+        current_watermark = self.watermark_estimator.current_watermark()
+      else:
+        current_watermark = None
       split = restriction_tracker.try_split(fraction)
       if split:
         primary, residual = split
@@ -690,15 +720,13 @@
         primary_size = restriction_provider.restriction_size(element, primary)
         residual_size = restriction_provider.restriction_size(element, residual)
         return (
-            (self.current_windowed_value.with_value(
-                ((element, primary), primary_size)),
-             None),
-            (self.current_windowed_value.with_value(
-                ((element, residual), residual_size)),
-             current_watermark))
+            ((self.current_windowed_value.with_value((
+                (element, primary), primary_size)), None), None),
+            ((self.current_windowed_value.with_value((
+                (element, residual), residual_size)), current_watermark), None))
 
   def current_element_progress(self):
-    restriction_tracker = self.restriction_tracker
+    restriction_tracker = self.threadsafe_restriction_tracker
     if restriction_tracker:
       return restriction_tracker.current_progress()
 
diff --git a/sdks/python/apache_beam/runners/dataflow/internal/names.py b/sdks/python/apache_beam/runners/dataflow/internal/names.py
index ec6f988..5b2dd89 100644
--- a/sdks/python/apache_beam/runners/dataflow/internal/names.py
+++ b/sdks/python/apache_beam/runners/dataflow/internal/names.py
@@ -38,10 +38,10 @@
 
 # Update this version to the next version whenever there is a change that will
 # require changes to legacy Dataflow worker execution environment.
-BEAM_CONTAINER_VERSION = 'beam-master-20191107'
+BEAM_CONTAINER_VERSION = 'beam-master-20191112'
 # Update this version to the next version whenever there is a change that
 # requires changes to SDK harness container or SDK harness launcher.
-BEAM_FNAPI_CONTAINER_VERSION = 'beam-master-20191106'
+BEAM_FNAPI_CONTAINER_VERSION = 'beam-master-20191112'
 
 # TODO(BEAM-5939): Remove these shared names once Dataflow worker is updated.
 PICKLED_MAIN_SESSION_FILE = 'pickled_main_session'
diff --git a/sdks/python/apache_beam/runners/direct/sdf_direct_runner_test.py b/sdks/python/apache_beam/runners/direct/sdf_direct_runner_test.py
index 946ef34..fd04d4c 100644
--- a/sdks/python/apache_beam/runners/direct/sdf_direct_runner_test.py
+++ b/sdks/python/apache_beam/runners/direct/sdf_direct_runner_test.py
@@ -51,6 +51,9 @@
   def create_tracker(self, restriction):
     return OffsetRestrictionTracker(restriction)
 
+  def restriction_size(self, element, restriction):
+    return restriction.size()
+
 
 class ReadFiles(DoFn):
 
@@ -63,12 +66,11 @@
       restriction_tracker=DoFn.RestrictionParam(ReadFilesProvider()),
       *args, **kwargs):
     file_name = element
-    assert isinstance(restriction_tracker, OffsetRestrictionTracker)
 
     with open(file_name, 'rb') as file:
-      pos = restriction_tracker.start_position()
-      if restriction_tracker.start_position() > 0:
-        file.seek(restriction_tracker.start_position() - 1)
+      pos = restriction_tracker.current_restriction().start
+      if restriction_tracker.current_restriction().start > 0:
+        file.seek(restriction_tracker.current_restriction().start - 1)
         line = file.readline()
         pos = pos - 1 + len(line)
 
@@ -104,6 +106,9 @@
   def split(self, element, restriction):
     return [restriction,]
 
+  def restriction_size(self, element, restriction):
+    return restriction.size()
+
 
 class ExpandStrings(DoFn):
 
@@ -118,10 +123,9 @@
     side.extend(side1)
     side.extend(side2)
     side.extend(side3)
-    assert isinstance(restriction_tracker, OffsetRestrictionTracker)
     side = list(side)
-    for i in range(restriction_tracker.start_position(),
-                   restriction_tracker.stop_position()):
+    for i in range(restriction_tracker.current_restriction().start,
+                   restriction_tracker.current_restriction().stop):
       if restriction_tracker.try_claim(i):
         if not side:
           yield (
diff --git a/sdks/python/apache_beam/runners/portability/artifact_service_test.py b/sdks/python/apache_beam/runners/portability/artifact_service_test.py
index cd9d32b..f5da724 100644
--- a/sdks/python/apache_beam/runners/portability/artifact_service_test.py
+++ b/sdks/python/apache_beam/runners/portability/artifact_service_test.py
@@ -28,13 +28,13 @@
 import tempfile
 import time
 import unittest
-from concurrent import futures
 
 import grpc
 
 from apache_beam.portability.api import beam_artifact_api_pb2
 from apache_beam.portability.api import beam_artifact_api_pb2_grpc
 from apache_beam.runners.portability import artifact_service
+from apache_beam.utils.thread_pool_executor import UnboundedThreadPoolExecutor
 
 
 class AbstractArtifactServiceTest(unittest.TestCase):
@@ -76,7 +76,7 @@
     self._run_staging(self._service, self._service)
 
   def test_with_grpc(self):
-    server = grpc.server(futures.ThreadPoolExecutor(max_workers=2))
+    server = grpc.server(UnboundedThreadPoolExecutor())
     try:
       beam_artifact_api_pb2_grpc.add_ArtifactStagingServiceServicer_to_server(
           self._service, server)
@@ -208,7 +208,7 @@
               self._service, tokens[session(index)], name(index)))
 
     # pylint: disable=range-builtin-not-iterating
-    pool = futures.ThreadPoolExecutor(max_workers=10)
+    pool = UnboundedThreadPoolExecutor()
     sessions = set(pool.map(put, range(100)))
     tokens = dict(pool.map(commit, sessions))
     # List forces materialization.
diff --git a/sdks/python/apache_beam/runners/portability/expansion_service_test.py b/sdks/python/apache_beam/runners/portability/expansion_service_test.py
index 66b0fa3..7876246 100644
--- a/sdks/python/apache_beam/runners/portability/expansion_service_test.py
+++ b/sdks/python/apache_beam/runners/portability/expansion_service_test.py
@@ -17,7 +17,6 @@
 from __future__ import absolute_import
 
 import argparse
-import concurrent.futures as futures
 import logging
 import signal
 import sys
@@ -30,6 +29,7 @@
 from apache_beam.portability.api import beam_expansion_api_pb2_grpc
 from apache_beam.runners.portability import expansion_service
 from apache_beam.transforms import ptransform
+from apache_beam.utils.thread_pool_executor import UnboundedThreadPoolExecutor
 
 # This script provides an expansion service and example ptransforms for running
 # external transform test cases. See external_test.py for details.
@@ -163,7 +163,7 @@
                       help='port on which to serve the job api')
   options = parser.parse_args()
   global server
-  server = grpc.server(futures.ThreadPoolExecutor(max_workers=2))
+  server = grpc.server(UnboundedThreadPoolExecutor())
   beam_expansion_api_pb2_grpc.add_ExpansionServiceServicer_to_server(
       expansion_service.ExpansionServiceServicer(PipelineOptions()), server
   )
diff --git a/sdks/python/apache_beam/runners/portability/flink_runner_test.py b/sdks/python/apache_beam/runners/portability/flink_runner_test.py
index e97d65e..377ceb7 100644
--- a/sdks/python/apache_beam/runners/portability/flink_runner_test.py
+++ b/sdks/python/apache_beam/runners/portability/flink_runner_test.py
@@ -319,6 +319,9 @@
           line = f.readline()
       self.assertSetEqual(lines_actual, lines_expected)
 
+    def test_sdf_with_watermark_tracking(self):
+      raise unittest.SkipTest("BEAM-2939")
+
     def test_sdf_with_sdf_initiated_checkpointing(self):
       raise unittest.SkipTest("BEAM-2939")
 
diff --git a/sdks/python/apache_beam/runners/portability/fn_api_runner.py b/sdks/python/apache_beam/runners/portability/fn_api_runner.py
index 0735f30..b56c26a 100644
--- a/sdks/python/apache_beam/runners/portability/fn_api_runner.py
+++ b/sdks/python/apache_beam/runners/portability/fn_api_runner.py
@@ -33,7 +33,6 @@
 import time
 import uuid
 from builtins import object
-from concurrent import futures
 
 import grpc
 
@@ -78,6 +77,7 @@
 from apache_beam.utils import profiler
 from apache_beam.utils import proto_utils
 from apache_beam.utils import windowed_value
+from apache_beam.utils.thread_pool_executor import UnboundedThreadPoolExecutor
 
 # This module is experimental. No backwards-compatibility guarantees.
 
@@ -1224,12 +1224,10 @@
 
   _DEFAULT_SHUTDOWN_TIMEOUT_SECS = 5
 
-  def __init__(self, state, provision_info, max_workers):
+  def __init__(self, state, provision_info):
     self.state = state
     self.provision_info = provision_info
-    self.max_workers = max_workers
-    self.control_server = grpc.server(
-        futures.ThreadPoolExecutor(max_workers=self.max_workers))
+    self.control_server = grpc.server(UnboundedThreadPoolExecutor())
     self.control_port = self.control_server.add_insecure_port('[::]:0')
     self.control_address = 'localhost:%s' % self.control_port
 
@@ -1239,12 +1237,12 @@
     no_max_message_sizes = [("grpc.max_receive_message_length", -1),
                             ("grpc.max_send_message_length", -1)]
     self.data_server = grpc.server(
-        futures.ThreadPoolExecutor(max_workers=self.max_workers),
+        UnboundedThreadPoolExecutor(),
         options=no_max_message_sizes)
     self.data_port = self.data_server.add_insecure_port('[::]:0')
 
     self.state_server = grpc.server(
-        futures.ThreadPoolExecutor(max_workers=self.max_workers),
+        UnboundedThreadPoolExecutor(),
         options=no_max_message_sizes)
     self.state_port = self.state_server.add_insecure_port('[::]:0')
 
@@ -1280,7 +1278,7 @@
         self.state_server)
 
     self.logging_server = grpc.server(
-        futures.ThreadPoolExecutor(max_workers=2),
+        UnboundedThreadPoolExecutor(),
         options=no_max_message_sizes)
     self.logging_port = self.logging_server.add_insecure_port('[::]:0')
     beam_fn_api_pb2_grpc.add_BeamFnLoggingServicer_to_server(
@@ -1508,24 +1506,12 @@
       # Any environment will do, pick one arbitrarily.
       environment_id = next(iter(self._environments.keys()))
     environment = self._environments[environment_id]
-    max_total_workers = num_workers * len(self._environments)
 
     # assume all environments except EMBEDDED_PYTHON use gRPC.
     if environment.urn == python_urns.EMBEDDED_PYTHON:
       pass # no need for a gRPC server
     elif self._grpc_server is None:
-      self._grpc_server = GrpcServer(self._state, self._job_provision_info,
-                                     max_total_workers)
-    elif max_total_workers > self._grpc_server.max_workers:
-      # each gRPC server is running with fixed number of threads (
-      # max_total_workers), which is defined by the first call to
-      # get_worker_handlers(). Assumption here is a worker has a connection to a
-      # gRPC server. In case a stage tries to add more workers
-      # than the max_total_workers, some workers cannot connect to gRPC and
-      # pipeline will hang, hence raise an error here.
-      raise RuntimeError('gRPC servers are running with %s threads, we cannot '
-                         'attach %s workers.' % (self._grpc_server.max_workers,
-                                                 max_total_workers))
+      self._grpc_server = GrpcServer(self._state, self._job_provision_info)
 
     worker_handler_list = self._cached_handlers[environment_id]
     if len(worker_handler_list) < num_workers:
@@ -1801,7 +1787,7 @@
 
     merged_result = None
     split_result_list = []
-    with futures.ThreadPoolExecutor(max_workers=self._num_workers) as executor:
+    with UnboundedThreadPoolExecutor() as executor:
       for result, split_result in executor.map(lambda part: BundleManager(
           self._worker_handler_list, self._get_buffer,
           self._get_input_coder_impl, self._bundle_descriptor,
diff --git a/sdks/python/apache_beam/runners/portability/fn_api_runner_test.py b/sdks/python/apache_beam/runners/portability/fn_api_runner_test.py
index 2204a24..b7929cb 100644
--- a/sdks/python/apache_beam/runners/portability/fn_api_runner_test.py
+++ b/sdks/python/apache_beam/runners/portability/fn_api_runner_test.py
@@ -41,6 +41,7 @@
 from tenacity import stop_after_attempt
 
 import apache_beam as beam
+from apache_beam.io import iobase
 from apache_beam.io import restriction_trackers
 from apache_beam.metrics import monitoring_infos
 from apache_beam.metrics.execution import MetricKey
@@ -56,9 +57,11 @@
 from apache_beam.testing.util import assert_that
 from apache_beam.testing.util import equal_to
 from apache_beam.tools import utils
+from apache_beam.transforms import core
 from apache_beam.transforms import environments
 from apache_beam.transforms import userstate
 from apache_beam.transforms import window
+from apache_beam.utils import timestamp
 
 if statesampler.FAST_SAMPLER:
   DEFAULT_SAMPLING_PERIOD_MS = statesampler.DEFAULT_SAMPLING_PERIOD_MS
@@ -423,17 +426,14 @@
       assert_that(actual, is_buffered_correctly)
 
   def test_sdf(self):
-
     class ExpandingStringsDoFn(beam.DoFn):
       def process(
           self,
           element,
           restriction_tracker=beam.DoFn.RestrictionParam(
               ExpandStringsProvider())):
-        assert isinstance(
-            restriction_tracker,
-            restriction_trackers.OffsetRestrictionTracker), restriction_tracker
-        cur = restriction_tracker.start_position()
+        assert isinstance(restriction_tracker, iobase.RestrictionTrackerView)
+        cur = restriction_tracker.current_restriction().start
         while restriction_tracker.try_claim(cur):
           yield element[cur]
           cur += 1
@@ -446,6 +446,56 @@
           | beam.ParDo(ExpandingStringsDoFn()))
       assert_that(actual, equal_to(list(''.join(data))))
 
+  def test_sdf_with_check_done_failed(self):
+    class ExpandingStringsDoFn(beam.DoFn):
+      def process(
+          self,
+          element,
+          restriction_tracker=beam.DoFn.RestrictionParam(
+              ExpandStringsProvider())):
+        assert isinstance(restriction_tracker, iobase.RestrictionTrackerView)
+        cur = restriction_tracker.current_restriction().start
+        while restriction_tracker.try_claim(cur):
+          yield element[cur]
+          cur += 1
+          return
+    with self.assertRaises(Exception):
+      with self.create_pipeline() as p:
+        data = ['abc', 'defghijklmno', 'pqrstuv', 'wxyz']
+        _ = (
+            p
+            | beam.Create(data)
+            | beam.ParDo(ExpandingStringsDoFn()))
+
+  def test_sdf_with_watermark_tracking(self):
+
+    class ExpandingStringsDoFn(beam.DoFn):
+      def process(
+          self,
+          element,
+          restriction_tracker=beam.DoFn.RestrictionParam(
+              ExpandStringsProvider()),
+          watermark_estimator=beam.DoFn.WatermarkEstimatorParam(
+              core.WatermarkEstimator())):
+        cur = restriction_tracker.current_restriction().start
+        start = cur
+        while restriction_tracker.try_claim(cur):
+          watermark_estimator.set_watermark(timestamp.Timestamp(micros=cur))
+          assert watermark_estimator.current_watermark().micros == start
+          yield element[cur]
+          if cur % 2 == 1:
+            restriction_tracker.defer_remainder(timestamp.Duration(micros=5))
+            return
+          cur += 1
+
+    with self.create_pipeline() as p:
+      data = ['abc', 'defghijklmno', 'pqrstuv', 'wxyz']
+      actual = (
+          p
+          | beam.Create(data)
+          | beam.ParDo(ExpandingStringsDoFn()))
+      assert_that(actual, equal_to(list(''.join(data))))
+
   def test_sdf_with_sdf_initiated_checkpointing(self):
 
     counter = beam.metrics.Metrics.counter('ns', 'my_counter')
@@ -456,10 +506,8 @@
           element,
           restriction_tracker=beam.DoFn.RestrictionParam(
               ExpandStringsProvider())):
-        assert isinstance(
-            restriction_tracker,
-            restriction_trackers.OffsetRestrictionTracker), restriction_tracker
-        cur = restriction_tracker.start_position()
+        assert isinstance(restriction_tracker, iobase.RestrictionTrackerView)
+        cur = restriction_tracker.current_restriction().start
         while restriction_tracker.try_claim(cur):
           counter.inc()
           yield element[cur]
@@ -1123,6 +1171,9 @@
   def test_sdf_with_sdf_initiated_checkpointing(self):
     raise unittest.SkipTest("This test is for a single worker only.")
 
+  def test_sdf_with_watermark_tracking(self):
+    raise unittest.SkipTest("This test is for a single worker only.")
+
 
 class FnApiRunnerTestWithGrpcAndMultiWorkers(FnApiRunnerTest):
 
@@ -1142,6 +1193,9 @@
   def test_sdf_with_sdf_initiated_checkpointing(self):
     raise unittest.SkipTest("This test is for a single worker only.")
 
+  def test_sdf_with_watermark_tracking(self):
+    raise unittest.SkipTest("This test is for a single worker only.")
+
 
 class FnApiRunnerTestWithBundleRepeat(FnApiRunnerTest):
 
@@ -1172,6 +1226,9 @@
   def test_sdf_with_sdf_initiated_checkpointing(self):
     raise unittest.SkipTest("This test is for a single worker only.")
 
+  def test_sdf_with_watermark_tracking(self):
+    raise unittest.SkipTest("This test is for a single worker only.")
+
 
 class FnApiRunnerSplitTest(unittest.TestCase):
 
@@ -1340,7 +1397,7 @@
           element,
           restriction_tracker=beam.DoFn.RestrictionParam(EnumerateProvider())):
         to_emit = []
-        cur = restriction_tracker.start_position()
+        cur = restriction_tracker.current_restriction().start
         while restriction_tracker.try_claim(cur):
           to_emit.append((element, cur))
           element_counter.increment()
diff --git a/sdks/python/apache_beam/runners/portability/local_job_service.py b/sdks/python/apache_beam/runners/portability/local_job_service.py
index 6aad1af..b8f84ce 100644
--- a/sdks/python/apache_beam/runners/portability/local_job_service.py
+++ b/sdks/python/apache_beam/runners/portability/local_job_service.py
@@ -26,7 +26,6 @@
 import time
 import traceback
 from builtins import object
-from concurrent import futures
 
 import grpc
 from google.protobuf import text_format
@@ -42,6 +41,7 @@
 from apache_beam.runners.portability import abstract_job_service
 from apache_beam.runners.portability import artifact_service
 from apache_beam.runners.portability import fn_api_runner
+from apache_beam.utils.thread_pool_executor import UnboundedThreadPoolExecutor
 
 
 class LocalJobServicer(abstract_job_service.AbstractJobServiceServicer):
@@ -92,7 +92,7 @@
         self._artifact_staging_endpoint)
 
   def start_grpc_server(self, port=0):
-    self._server = grpc.server(futures.ThreadPoolExecutor(max_workers=3))
+    self._server = grpc.server(UnboundedThreadPoolExecutor())
     port = self._server.add_insecure_port('localhost:%d' % port)
     beam_job_api_pb2_grpc.add_JobServiceServicer_to_server(self, self._server)
     beam_artifact_api_pb2_grpc.add_ArtifactStagingServiceServicer_to_server(
@@ -139,7 +139,7 @@
     self._worker_id = worker_id
 
   def run(self):
-    logging_server = grpc.server(futures.ThreadPoolExecutor(max_workers=10))
+    logging_server = grpc.server(UnboundedThreadPoolExecutor())
     logging_port = logging_server.add_insecure_port('[::]:0')
     logging_server.start()
     logging_servicer = BeamFnLoggingServicer()
diff --git a/sdks/python/apache_beam/runners/portability/portable_runner.py b/sdks/python/apache_beam/runners/portability/portable_runner.py
index 6e10085..8b078a5 100644
--- a/sdks/python/apache_beam/runners/portability/portable_runner.py
+++ b/sdks/python/apache_beam/runners/portability/portable_runner.py
@@ -20,13 +20,11 @@
 import functools
 import itertools
 import logging
-import sys
 import threading
 import time
 
 import grpc
 
-from apache_beam import version as beam_version
 from apache_beam.metrics import metric
 from apache_beam.metrics.execution import MetricResult
 from apache_beam.options.pipeline_options import DebugOptions
@@ -79,22 +77,6 @@
     self._dockerized_job_server = None
 
   @staticmethod
-  def default_docker_image():
-    sdk_version = beam_version.__version__
-    version_suffix = '.'.join([str(i) for i in sys.version_info[0:2]])
-    logging.warning('Make sure that locally built Python SDK docker image '
-                    'has Python %d.%d interpreter.' % (
-                        sys.version_info[0], sys.version_info[1]))
-
-    image = ('apachebeam/python{version_suffix}_sdk:{tag}'.format(
-        version_suffix=version_suffix, tag=sdk_version))
-    logging.info(
-        'Using Python SDK docker image: %s. If the image is not '
-        'available at local, we will try to pull from hub.docker.com'
-        % (image))
-    return image
-
-  @staticmethod
   def _create_environment(options):
     portable_options = options.view_as(PortableOptions)
     # Do not set a Runner. Otherwise this can cause problems in Java's
diff --git a/sdks/python/apache_beam/runners/portability/portable_runner_test.py b/sdks/python/apache_beam/runners/portability/portable_runner_test.py
index 24c6b87..46dbad5 100644
--- a/sdks/python/apache_beam/runners/portability/portable_runner_test.py
+++ b/sdks/python/apache_beam/runners/portability/portable_runner_test.py
@@ -45,7 +45,10 @@
 from apache_beam.runners.portability.portable_runner import PortableRunner
 from apache_beam.runners.worker import worker_pool_main
 from apache_beam.runners.worker.channel_factory import GRPCChannelFactory
+from apache_beam.testing.util import assert_that
+from apache_beam.testing.util import equal_to
 from apache_beam.transforms import environments
+from apache_beam.transforms import userstate
 
 
 class PortableRunnerTest(fn_api_runner_test.FnApiRunnerTest):
@@ -185,8 +188,45 @@
   def create_pipeline(self):
     return beam.Pipeline(self.get_runner(), self.create_options())
 
-  def test_metrics(self):
-    self.skipTest('Metrics not supported.')
+  def test_pardo_state_with_custom_key_coder(self):
+    """Tests that state requests work correctly when the key coder is an
+    SDK-specific coder, i.e. non standard coder. This is additionally enforced
+    by Java's ProcessBundleDescriptorsTest and by Flink's
+    ExecutableStageDoFnOperator which detects invalid encoding by checking for
+    the correct key group of the encoded key."""
+    index_state_spec = userstate.CombiningValueStateSpec('index', sum)
+
+    # Test params
+    # Ensure decent amount of elements to serve all partitions
+    n = 200
+    duplicates = 1
+
+    split = n // (duplicates + 1)
+    inputs = [(i % split, str(i % split)) for i in range(0, n)]
+
+    # Use a DoFn which has to use FastPrimitivesCoder because the type cannot
+    # be inferred
+    class Input(beam.DoFn):
+      def process(self, impulse):
+        for i in inputs:
+          yield i
+
+    class AddIndex(beam.DoFn):
+      def process(self, kv,
+                  index=beam.DoFn.StateParam(index_state_spec)):
+        k, v = kv
+        index.add(1)
+        yield k, v, index.read()
+
+    expected = [(i % split, str(i % split), i // split + 1)
+                for i in range(0, n)]
+
+    with self.create_pipeline() as p:
+      assert_that(p
+                  | beam.Impulse()
+                  | beam.ParDo(Input())
+                  | beam.ParDo(AddIndex()),
+                  equal_to(expected))
 
   # Inherits all other tests from fn_api_runner_test.FnApiRunnerTest
 
@@ -256,7 +296,7 @@
 
 class PortableRunnerInternalTest(unittest.TestCase):
   def test__create_default_environment(self):
-    docker_image = PortableRunner.default_docker_image()
+    docker_image = environments.DockerEnvironment.default_docker_image()
     self.assertEqual(
         PortableRunner._create_environment(PipelineOptions.from_dictionary({})),
         environments.DockerEnvironment(container_image=docker_image))
@@ -313,7 +353,7 @@
 
 
 def hasDockerImage():
-  image = PortableRunner.default_docker_image()
+  image = environments.DockerEnvironment.default_docker_image()
   try:
     check_image = subprocess.check_output("docker images -q %s" % image,
                                           shell=True)
diff --git a/sdks/python/apache_beam/runners/portability/portable_stager_test.py b/sdks/python/apache_beam/runners/portability/portable_stager_test.py
index d65c404..fd86819 100644
--- a/sdks/python/apache_beam/runners/portability/portable_stager_test.py
+++ b/sdks/python/apache_beam/runners/portability/portable_stager_test.py
@@ -27,13 +27,13 @@
 import string
 import tempfile
 import unittest
-from concurrent import futures
 
 import grpc
 
 from apache_beam.portability.api import beam_artifact_api_pb2
 from apache_beam.portability.api import beam_artifact_api_pb2_grpc
 from apache_beam.runners.portability import portable_stager
+from apache_beam.utils.thread_pool_executor import UnboundedThreadPoolExecutor
 
 
 class PortableStagerTest(unittest.TestCase):
@@ -56,7 +56,7 @@
           describing the name of the artifacts in local temp folder and desired
           name in staging location.
     """
-    server = grpc.server(futures.ThreadPoolExecutor(max_workers=10))
+    server = grpc.server(UnboundedThreadPoolExecutor())
     staging_service = TestLocalFileSystemArtifactStagingServiceServicer(
         self._remote_dir)
     beam_artifact_api_pb2_grpc.add_ArtifactStagingServiceServicer_to_server(
diff --git a/sdks/python/apache_beam/runners/worker/bundle_processor.py b/sdks/python/apache_beam/runners/worker/bundle_processor.py
index 8439c8f..b3440df 100644
--- a/sdks/python/apache_beam/runners/worker/bundle_processor.py
+++ b/sdks/python/apache_beam/runners/worker/bundle_processor.py
@@ -32,6 +32,7 @@
 from builtins import object
 
 from future.utils import itervalues
+from google.protobuf import duration_pb2
 from google.protobuf import timestamp_pb2
 
 import apache_beam as beam
@@ -704,8 +705,7 @@
               ) = split
               if element_primary:
                 split_response.primary_roots.add().CopyFrom(
-                    self.delayed_bundle_application(
-                        *element_primary).application)
+                    self.bundle_application(*element_primary))
               if element_residual:
                 split_response.residual_roots.add().CopyFrom(
                     self.delayed_bundle_application(*element_residual))
@@ -718,22 +718,39 @@
     return split_response
 
   def delayed_bundle_application(self, op, deferred_remainder):
-    transform_id, main_input_tag, main_input_coder, outputs = op.input_info
     # TODO(SDF): For non-root nodes, need main_input_coder + residual_coder.
-    element_and_restriction, watermark = deferred_remainder
-    if watermark:
-      proto_watermark = timestamp_pb2.Timestamp()
-      proto_watermark.FromMicroseconds(watermark.micros)
-      output_watermarks = {output: proto_watermark for output in outputs}
+    ((element_and_restriction, output_watermark),
+     deferred_watermark) = deferred_remainder
+    if deferred_watermark:
+      assert isinstance(deferred_watermark, timestamp.Duration)
+      proto_deferred_watermark = duration_pb2.Duration()
+      proto_deferred_watermark.FromMicroseconds(deferred_watermark.micros)
+    else:
+      proto_deferred_watermark = None
+    return beam_fn_api_pb2.DelayedBundleApplication(
+        requested_time_delay=proto_deferred_watermark,
+        application=self.construct_bundle_application(
+            op, output_watermark, element_and_restriction))
+
+  def bundle_application(self, op, primary):
+    ((element_and_restriction, output_watermark),
+     _) = primary
+    return self.construct_bundle_application(
+        op, output_watermark, element_and_restriction)
+
+  def construct_bundle_application(self, op, output_watermark, element):
+    transform_id, main_input_tag, main_input_coder, outputs = op.input_info
+    if output_watermark:
+      proto_output_watermark = timestamp_pb2.Timestamp()
+      proto_output_watermark.FromMicroseconds(output_watermark.micros)
+      output_watermarks = {output: proto_output_watermark for output in outputs}
     else:
       output_watermarks = None
-    return beam_fn_api_pb2.DelayedBundleApplication(
-        application=beam_fn_api_pb2.BundleApplication(
-            transform_id=transform_id,
-            input_id=main_input_tag,
-            output_watermarks=output_watermarks,
-            element=main_input_coder.get_impl().encode_nested(
-                element_and_restriction)))
+    return beam_fn_api_pb2.BundleApplication(
+        transform_id=transform_id,
+        input_id=main_input_tag,
+        output_watermarks=output_watermarks,
+        element=main_input_coder.get_impl().encode_nested(element))
 
   def metrics(self):
     # DEPRECATED
diff --git a/sdks/python/apache_beam/runners/worker/data_plane_test.py b/sdks/python/apache_beam/runners/worker/data_plane_test.py
index d11390a..900532b 100644
--- a/sdks/python/apache_beam/runners/worker/data_plane_test.py
+++ b/sdks/python/apache_beam/runners/worker/data_plane_test.py
@@ -25,7 +25,6 @@
 import sys
 import threading
 import unittest
-from concurrent import futures
 
 import grpc
 from future.utils import raise_
@@ -34,6 +33,7 @@
 from apache_beam.portability.api import beam_fn_api_pb2_grpc
 from apache_beam.runners.worker import data_plane
 from apache_beam.runners.worker.worker_id_interceptor import WorkerIdInterceptor
+from apache_beam.utils.thread_pool_executor import UnboundedThreadPoolExecutor
 
 
 def timeout(timeout_secs):
@@ -67,7 +67,7 @@
     data_channel_service = \
       data_servicer.get_conn_by_worker_id(worker_id)
 
-    server = grpc.server(futures.ThreadPoolExecutor(max_workers=2))
+    server = grpc.server(UnboundedThreadPoolExecutor())
     beam_fn_api_pb2_grpc.add_BeamFnDataServicer_to_server(
         data_servicer, server)
     test_port = server.add_insecure_port('[::]:0')
diff --git a/sdks/python/apache_beam/runners/worker/log_handler_test.py b/sdks/python/apache_beam/runners/worker/log_handler_test.py
index ab042aa..6650ccd 100644
--- a/sdks/python/apache_beam/runners/worker/log_handler_test.py
+++ b/sdks/python/apache_beam/runners/worker/log_handler_test.py
@@ -20,7 +20,6 @@
 import logging
 import unittest
 from builtins import range
-from concurrent import futures
 
 import grpc
 
@@ -28,6 +27,7 @@
 from apache_beam.portability.api import beam_fn_api_pb2_grpc
 from apache_beam.portability.api import endpoints_pb2
 from apache_beam.runners.worker import log_handler
+from apache_beam.utils.thread_pool_executor import UnboundedThreadPoolExecutor
 
 
 class BeamFnLoggingServicer(beam_fn_api_pb2_grpc.BeamFnLoggingServicer):
@@ -47,7 +47,7 @@
 
   def setUp(self):
     self.test_logging_service = BeamFnLoggingServicer()
-    self.server = grpc.server(futures.ThreadPoolExecutor(max_workers=10))
+    self.server = grpc.server(UnboundedThreadPoolExecutor())
     beam_fn_api_pb2_grpc.add_BeamFnLoggingServicer_to_server(
         self.test_logging_service, self.server)
     self.test_port = self.server.add_insecure_port('[::]:0')
diff --git a/sdks/python/apache_beam/runners/worker/sdk_worker.py b/sdks/python/apache_beam/runners/worker/sdk_worker.py
index 74a3e99..2cbd196 100644
--- a/sdks/python/apache_beam/runners/worker/sdk_worker.py
+++ b/sdks/python/apache_beam/runners/worker/sdk_worker.py
@@ -31,7 +31,6 @@
 import traceback
 from builtins import object
 from builtins import range
-from concurrent import futures
 
 import grpc
 from future.utils import raise_
@@ -45,6 +44,7 @@
 from apache_beam.runners.worker.channel_factory import GRPCChannelFactory
 from apache_beam.runners.worker.statecache import StateCache
 from apache_beam.runners.worker.worker_id_interceptor import WorkerIdInterceptor
+from apache_beam.utils.thread_pool_executor import UnboundedThreadPoolExecutor
 
 # This SDK harness will (by default), log a "lull" in processing if it sees no
 # transitions in over 5 minutes.
@@ -97,15 +97,9 @@
     # one worker for progress/split request.
     self.progress_worker = SdkWorker(self._bundle_processor_cache,
                                      profiler_factory=self._profiler_factory)
-    # one thread is enough for getting the progress report.
-    # Assumption:
-    # Progress report generation should not do IO or wait on other resources.
-    #  Without wait, having multiple threads will not improve performance and
-    #  will only add complexity.
-    self._progress_thread_pool = futures.ThreadPoolExecutor(max_workers=1)
+    self._progress_thread_pool = UnboundedThreadPoolExecutor()
     # finalize and process share one thread pool.
-    self._process_thread_pool = futures.ThreadPoolExecutor(
-        max_workers=self._worker_count)
+    self._process_thread_pool = UnboundedThreadPoolExecutor()
     self._responses = queue.Queue()
     self._process_bundle_queue = queue.Queue()
     self._unscheduled_process_bundle = {}
@@ -202,7 +196,7 @@
     self._unscheduled_process_bundle[request.instruction_id] = time.time()
     self._process_thread_pool.submit(task)
     logging.debug(
-        "Currently using %s threads." % len(self._process_thread_pool._threads))
+        "Currently using %s threads." % len(self._process_thread_pool._workers))
 
   def _request_process_bundle_split(self, request):
     self._request_process_bundle_action(request)
diff --git a/sdks/python/apache_beam/runners/worker/sdk_worker_main.py b/sdks/python/apache_beam/runners/worker/sdk_worker_main.py
index c6cb8ed..ce2bb1f 100644
--- a/sdks/python/apache_beam/runners/worker/sdk_worker_main.py
+++ b/sdks/python/apache_beam/runners/worker/sdk_worker_main.py
@@ -190,7 +190,7 @@
   future releases.
 
   Returns:
-    an int containing the worker_threads to use. Default is 12
+    an int containing the worker_threads to use. Default is 12.
   """
   experiments = pipeline_options.view_as(DebugOptions).experiments
 
diff --git a/sdks/python/apache_beam/runners/worker/sdk_worker_test.py b/sdks/python/apache_beam/runners/worker/sdk_worker_test.py
index 71263a8..89047ef 100644
--- a/sdks/python/apache_beam/runners/worker/sdk_worker_test.py
+++ b/sdks/python/apache_beam/runners/worker/sdk_worker_test.py
@@ -23,7 +23,6 @@
 import logging
 import unittest
 from builtins import range
-from concurrent import futures
 
 import grpc
 
@@ -31,6 +30,7 @@
 from apache_beam.portability.api import beam_fn_api_pb2_grpc
 from apache_beam.portability.api import beam_runner_api_pb2
 from apache_beam.runners.worker import sdk_worker
+from apache_beam.utils.thread_pool_executor import UnboundedThreadPoolExecutor
 
 
 class BeamFnControlServicer(beam_fn_api_pb2_grpc.BeamFnControlServicer):
@@ -93,7 +93,7 @@
 
       test_controller = BeamFnControlServicer(requests)
 
-      server = grpc.server(futures.ThreadPoolExecutor(max_workers=10))
+      server = grpc.server(UnboundedThreadPoolExecutor())
       beam_fn_api_pb2_grpc.add_BeamFnControlServicer_to_server(
           test_controller, server)
       test_port = server.add_insecure_port("[::]:0")
diff --git a/sdks/python/apache_beam/runners/worker/worker_pool_main.py b/sdks/python/apache_beam/runners/worker/worker_pool_main.py
index 11db233..beacd30 100644
--- a/sdks/python/apache_beam/runners/worker/worker_pool_main.py
+++ b/sdks/python/apache_beam/runners/worker/worker_pool_main.py
@@ -35,13 +35,13 @@
 import sys
 import threading
 import time
-from concurrent import futures
 
 import grpc
 
 from apache_beam.portability.api import beam_fn_api_pb2
 from apache_beam.portability.api import beam_fn_api_pb2_grpc
 from apache_beam.runners.worker import sdk_worker
+from apache_beam.utils.thread_pool_executor import UnboundedThreadPoolExecutor
 
 
 class BeamFnExternalWorkerPoolServicer(
@@ -60,7 +60,7 @@
   @classmethod
   def start(cls, worker_threads=1, use_process=False, port=0,
             state_cache_size=0, container_executable=None):
-    worker_server = grpc.server(futures.ThreadPoolExecutor(max_workers=10))
+    worker_server = grpc.server(UnboundedThreadPoolExecutor())
     worker_address = 'localhost:%s' % worker_server.add_insecure_port(
         '[::]:%s' % port)
     worker_pool = cls(worker_threads,
diff --git a/sdks/python/apache_beam/testing/synthetic_pipeline.py b/sdks/python/apache_beam/testing/synthetic_pipeline.py
index 50740ba..fbef112 100644
--- a/sdks/python/apache_beam/testing/synthetic_pipeline.py
+++ b/sdks/python/apache_beam/testing/synthetic_pipeline.py
@@ -523,7 +523,7 @@
       element,
       restriction_tracker=beam.DoFn.RestrictionParam(
           SyntheticSDFSourceRestrictionProvider())):
-    cur = restriction_tracker.start_position()
+    cur = restriction_tracker.current_restriction().start
     while restriction_tracker.try_claim(cur):
       r = np.random.RandomState(cur)
       time.sleep(element['sleep_per_input_record_sec'])
diff --git a/sdks/python/apache_beam/transforms/core.py b/sdks/python/apache_beam/transforms/core.py
index 148caae..06fd201 100644
--- a/sdks/python/apache_beam/transforms/core.py
+++ b/sdks/python/apache_beam/transforms/core.py
@@ -63,6 +63,7 @@
 from apache_beam.typehints.decorators import get_type_hints
 from apache_beam.typehints.trivial_inference import element_type
 from apache_beam.typehints.typehints import is_consistent_with
+from apache_beam.utils import timestamp
 from apache_beam.utils import urns
 
 try:
@@ -91,7 +92,8 @@
     'Flatten',
     'Create',
     'Impulse',
-    'RestrictionProvider'
+    'RestrictionProvider',
+    'WatermarkEstimator'
     ]
 
 # Type variables
@@ -242,6 +244,8 @@
   def create_tracker(self, restriction):
     """Produces a new ``RestrictionTracker`` for the given restriction.
 
+    This API is required to be implemented.
+
     Args:
       restriction: an object that defines a restriction as identified by a
         Splittable ``DoFn`` that utilizes the current ``RestrictionProvider``.
@@ -252,7 +256,10 @@
     raise NotImplementedError
 
   def initial_restriction(self, element):
-    """Produces an initial restriction for the given element."""
+    """Produces an initial restriction for the given element.
+
+    This API is required to be implemented.
+    """
     raise NotImplementedError
 
   def split(self, element, restriction):
@@ -262,6 +269,9 @@
     reading input element for each of the returned restrictions should be the
     same as the total set of elements produced by reading the input element for
     the input restriction.
+
+    This API is optional if ``split_and_size`` has been implemented.
+
     """
     yield restriction
 
@@ -281,11 +291,16 @@
 
     By default, asks a newly-created restriction tracker for the default size
     of the restriction.
+
+    This API is required to be implemented.
     """
-    return self.create_tracker(restriction).default_size()
+    raise NotImplementedError
 
   def split_and_size(self, element, restriction):
     """Like split, but also does sizing, returning (restriction, size) pairs.
+
+    This API is optional if ``split`` and ``restriction_size`` have been
+    implemented.
     """
     for part in self.split(element, restriction):
       yield part, self.restriction_size(element, part)
@@ -379,6 +394,43 @@
     return None
 
 
+class WatermarkEstimator(object):
+  """A WatermarkEstimator which is used for tracking output_watermark in a
+  DoFn.process(), typically tracking per <element, restriction> pair in SDF in
+  streaming.
+
+  There are 3 APIs in this class: set_watermark, current_watermark and reset
+  with default implementations.
+
+  TODO(BEAM-8537): Create WatermarkEstimatorProvider to support different types.
+  """
+  def __init__(self):
+    self._watermark = None
+
+  def set_watermark(self, watermark):
+    """Update tracking output_watermark with latest output_watermark.
+    This function is called inside an SDF.Process() to track the watermark of
+    output element.
+
+    Args:
+      watermark: the `timestamp.Timestamp` of current output element.
+    """
+    if not isinstance(watermark, timestamp.Timestamp):
+      raise ValueError('watermark should be a object of timestamp.Timestamp')
+    if self._watermark is None:
+      self._watermark = watermark
+    else:
+      self._watermark = min(self._watermark, watermark)
+
+  def current_watermark(self):
+    """Get current output_watermark. This function is called by system."""
+    return self._watermark
+
+  def reset(self):
+    """ Reset current tracking watermark to None."""
+    self._watermark = None
+
+
 class _DoFnParam(object):
   """DoFn parameter."""
 
@@ -459,6 +511,17 @@
     del self._callbacks[:]
 
 
+class _WatermarkEstimatorParam(_DoFnParam):
+  """WatermarkEstomator DoFn parameter."""
+
+  def __init__(self, watermark_estimator):
+    if not isinstance(watermark_estimator, WatermarkEstimator):
+      raise ValueError('DoFn.WatermarkEstimatorParam expected'
+                       'WatermarkEstimator object.')
+    self.watermark_estimator = watermark_estimator
+    self.param_id = 'WatermarkEstimator'
+
+
 class DoFn(WithTypeHints, HasDisplayData, urns.RunnerApiFn):
   """A function object used by a transform with custom processing.
 
@@ -477,7 +540,7 @@
   TimestampParam = _DoFnParam('TimestampParam')
   WindowParam = _DoFnParam('WindowParam')
   PaneInfoParam = _DoFnParam('PaneInfoParam')
-  WatermarkReporterParam = _DoFnParam('WatermarkReporterParam')
+  WatermarkEstimatorParam = _WatermarkEstimatorParam
   BundleFinalizerParam = _BundleFinalizerParam
   KeyParam = _DoFnParam('KeyParam')
 
@@ -489,7 +552,7 @@
   TimerParam = _TimerDoFnParam
 
   DoFnProcessParams = [ElementParam, SideInputParam, TimestampParam,
-                       WindowParam, WatermarkReporterParam, PaneInfoParam,
+                       WindowParam, WatermarkEstimatorParam, PaneInfoParam,
                        BundleFinalizerParam, KeyParam, StateParam, TimerParam]
 
   RestrictionParam = _RestrictionDoFnParam
@@ -522,7 +585,7 @@
     ``DoFn.RestrictionParam``: an ``iobase.RestrictionTracker`` will be
     provided here to allow treatment as a Splittable ``DoFn``. The restriction
     tracker will be derived from the restriction provider in the parameter.
-    ``DoFn.WatermarkReporterParam``: a function that can be used to report
+    ``DoFn.WatermarkEstimatorParam``: a function that can be used to track
     output watermark of Splittable ``DoFn`` implementations.
 
     Args:
diff --git a/sdks/python/apache_beam/transforms/core_test.py b/sdks/python/apache_beam/transforms/core_test.py
new file mode 100644
index 0000000..1a27bd2
--- /dev/null
+++ b/sdks/python/apache_beam/transforms/core_test.py
@@ -0,0 +1,54 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+"""Unit tests for core module."""
+
+from __future__ import absolute_import
+
+import unittest
+
+from apache_beam.transforms.core import WatermarkEstimator
+from apache_beam.utils.timestamp import Timestamp
+
+
+class WatermarkEstimatorTest(unittest.TestCase):
+
+  def test_set_watermark(self):
+    watermark_estimator = WatermarkEstimator()
+    self.assertEqual(watermark_estimator.current_watermark(), None)
+    # set_watermark should only accept timestamp.Timestamp.
+    with self.assertRaises(ValueError):
+      watermark_estimator.set_watermark(0)
+
+    # watermark_estimator should always keep minimal timestamp.
+    watermark_estimator.set_watermark(Timestamp(100))
+    self.assertEqual(watermark_estimator.current_watermark(), 100)
+    watermark_estimator.set_watermark(Timestamp(150))
+    self.assertEqual(watermark_estimator.current_watermark(), 100)
+    watermark_estimator.set_watermark(Timestamp(50))
+    self.assertEqual(watermark_estimator.current_watermark(), 50)
+
+  def test_reset(self):
+    watermark_estimator = WatermarkEstimator()
+    watermark_estimator.set_watermark(Timestamp(100))
+    self.assertEqual(watermark_estimator.current_watermark(), 100)
+    watermark_estimator.reset()
+    self.assertEqual(watermark_estimator.current_watermark(), None)
+
+
+if __name__ == '__main__':
+  unittest.main()
diff --git a/sdks/python/apache_beam/transforms/environments.py b/sdks/python/apache_beam/transforms/environments.py
index 8758ab8..999647f 100644
--- a/sdks/python/apache_beam/transforms/environments.py
+++ b/sdks/python/apache_beam/transforms/environments.py
@@ -22,6 +22,8 @@
 from __future__ import absolute_import
 
 import json
+import logging
+import sys
 
 from google.protobuf import message
 
@@ -119,12 +121,10 @@
 class DockerEnvironment(Environment):
 
   def __init__(self, container_image=None):
-    from apache_beam.runners.portability.portable_runner import PortableRunner
-
     if container_image:
       self.container_image = container_image
     else:
-      self.container_image = PortableRunner.default_docker_image()
+      self.container_image = self.default_docker_image()
 
   def __eq__(self, other):
     return self.__class__ == other.__class__ \
@@ -153,6 +153,24 @@
   def from_options(cls, options):
     return cls(container_image=options.environment_config)
 
+  @staticmethod
+  def default_docker_image():
+    from apache_beam import version as beam_version
+
+    sdk_version = beam_version.__version__
+    version_suffix = '.'.join([str(i) for i in sys.version_info[0:2]])
+    logging.warning('Make sure that locally built Python SDK docker image '
+                    'has Python %d.%d interpreter.' % (
+                        sys.version_info[0], sys.version_info[1]))
+
+    image = ('apachebeam/python{version_suffix}_sdk:{tag}'.format(
+        version_suffix=version_suffix, tag=sdk_version))
+    logging.info(
+        'Using Python SDK docker image: %s. If the image is not '
+        'available at local, we will try to pull from hub.docker.com'
+        % (image))
+    return image
+
 
 @Environment.register_urn(common_urns.environments.PROCESS.urn,
                           beam_runner_api_pb2.ProcessPayload)
diff --git a/sdks/python/apache_beam/transforms/util.py b/sdks/python/apache_beam/transforms/util.py
index bb7e522..7a87e60 100644
--- a/sdks/python/apache_beam/transforms/util.py
+++ b/sdks/python/apache_beam/transforms/util.py
@@ -241,7 +241,8 @@
                target_batch_overhead=.1,
                target_batch_duration_secs=1,
                variance=0.25,
-               clock=time.time):
+               clock=time.time,
+               ignore_first_n_seen_per_batch_size=0):
     if min_batch_size > max_batch_size:
       raise ValueError("Minimum (%s) must not be greater than maximum (%s)" % (
           min_batch_size, max_batch_size))
@@ -254,6 +255,9 @@
     if not (target_batch_overhead or target_batch_duration_secs):
       raise ValueError("At least one of target_batch_overhead or "
                        "target_batch_duration_secs must be positive.")
+    if ignore_first_n_seen_per_batch_size < 0:
+      raise ValueError('ignore_first_n_seen_per_batch_size (%s) must be non '
+                       'negative' % (ignore_first_n_seen_per_batch_size))
     self._min_batch_size = min_batch_size
     self._max_batch_size = max_batch_size
     self._target_batch_overhead = target_batch_overhead
@@ -262,6 +266,10 @@
     self._clock = clock
     self._data = []
     self._ignore_next_timing = False
+    self._ignore_first_n_seen_per_batch_size = (
+        ignore_first_n_seen_per_batch_size)
+    self._batch_size_num_seen = {}
+    self._replay_last_batch_size = None
 
     self._size_distribution = Metrics.distribution(
         'BatchElements', 'batch_size')
@@ -279,7 +287,7 @@
     For example, the first emit of a ParDo operation is known to be anomalous
     due to setup that may occur.
     """
-    self._ignore_next_timing = False
+    self._ignore_next_timing = True
 
   @contextlib.contextmanager
   def record_time(self, batch_size):
@@ -290,8 +298,11 @@
     self._size_distribution.update(batch_size)
     self._time_distribution.update(int(elapsed_msec))
     self._remainder_msecs = elapsed_msec - int(elapsed_msec)
+    # If we ignore the next timing, replay the batch size to get accurate
+    # timing.
     if self._ignore_next_timing:
       self._ignore_next_timing = False
+      self._replay_last_batch_size = batch_size
     else:
       self._data.append((batch_size, elapsed))
       if len(self._data) >= self._MAX_DATA_POINTS:
@@ -364,7 +375,7 @@
   except ImportError:
     linear_regression = linear_regression_no_numpy
 
-  def next_batch_size(self):
+  def _calculate_next_batch_size(self):
     if self._min_batch_size == self._max_batch_size:
       return self._min_batch_size
     elif len(self._data) < 1:
@@ -414,6 +425,21 @@
 
     return int(max(self._min_batch_size + jitter, min(target, cap)))
 
+  def next_batch_size(self):
+    # Check if we should replay a previous batch size due to it not being
+    # recorded.
+    if self._replay_last_batch_size:
+      result = self._replay_last_batch_size
+      self._replay_last_batch_size = None
+    else:
+      result = self._calculate_next_batch_size()
+
+    seen_count = self._batch_size_num_seen.get(result, 0) + 1
+    if seen_count <= self._ignore_first_n_seen_per_batch_size:
+      self.ignore_next_timing()
+    self._batch_size_num_seen[result] = seen_count
+    return result
+
 
 class _GlobalWindowsBatchingDoFn(DoFn):
   def __init__(self, batch_size_estimator):
diff --git a/sdks/python/apache_beam/transforms/util_test.py b/sdks/python/apache_beam/transforms/util_test.py
index 4588c32..6ac05d0 100644
--- a/sdks/python/apache_beam/transforms/util_test.py
+++ b/sdks/python/apache_beam/transforms/util_test.py
@@ -157,6 +157,60 @@
     self.assertLess(
         max(stable_set), expected_target + expected_target * variance)
 
+  def test_ignore_first_n_batch_size(self):
+    clock = FakeClock()
+    batch_estimator = util._BatchSizeEstimator(
+        clock=clock, ignore_first_n_seen_per_batch_size=2)
+
+    expected_sizes = [
+        1, 1, 1, 2, 2, 2, 4, 4, 4, 8, 8, 8, 16, 16, 16, 32, 32, 32, 64, 64, 64
+    ]
+    actual_sizes = []
+    for i in range(len(expected_sizes)):
+      actual_sizes.append(batch_estimator.next_batch_size())
+      with batch_estimator.record_time(actual_sizes[-1]):
+        if i % 3 == 2:
+          clock.sleep(0.01)
+        else:
+          clock.sleep(1)
+
+    self.assertEqual(expected_sizes, actual_sizes)
+
+    # Check we only record the third timing.
+    expected_data_batch_sizes = [1, 2, 4, 8, 16, 32, 64]
+    actual_data_batch_sizes = [x[0] for x in batch_estimator._data]
+    self.assertEqual(expected_data_batch_sizes, actual_data_batch_sizes)
+    expected_data_timing = [0.01, 0.01, 0.01, 0.01, 0.01, 0.01, 0.01]
+    for i in range(len(expected_data_timing)):
+      self.assertAlmostEqual(
+          expected_data_timing[i], batch_estimator._data[i][1])
+
+  def test_ignore_next_timing(self):
+    clock = FakeClock()
+    batch_estimator = util._BatchSizeEstimator(clock=clock)
+    batch_estimator.ignore_next_timing()
+
+    expected_sizes = [1, 1, 2, 4, 8, 16]
+    actual_sizes = []
+    for i in range(len(expected_sizes)):
+      actual_sizes.append(batch_estimator.next_batch_size())
+      with batch_estimator.record_time(actual_sizes[-1]):
+        if i == 0:
+          clock.sleep(1)
+        else:
+          clock.sleep(0.01)
+
+    self.assertEqual(expected_sizes, actual_sizes)
+
+    # Check the first record_time was skipped.
+    expected_data_batch_sizes = [1, 2, 4, 8, 16]
+    actual_data_batch_sizes = [x[0] for x in batch_estimator._data]
+    self.assertEqual(expected_data_batch_sizes, actual_data_batch_sizes)
+    expected_data_timing = [0.01, 0.01, 0.01, 0.01, 0.01]
+    for i in range(len(expected_data_timing)):
+      self.assertAlmostEqual(
+          expected_data_timing[i], batch_estimator._data[i][1])
+
   def _run_regression_test(self, linear_regression_fn, test_outliers):
     xs = [random.random() for _ in range(10)]
     ys = [2*x + 1 for x in xs]
diff --git a/sdks/python/apache_beam/transforms/window_test.py b/sdks/python/apache_beam/transforms/window_test.py
index dda651d..3c45d83 100644
--- a/sdks/python/apache_beam/transforms/window_test.py
+++ b/sdks/python/apache_beam/transforms/window_test.py
@@ -22,6 +22,8 @@
 import unittest
 from builtins import range
 
+from nose.plugins.attrib import attr
+
 import apache_beam as beam
 from apache_beam.runners import pipeline_context
 from apache_beam.testing.test_pipeline import TestPipeline
@@ -281,6 +283,35 @@
       assert_that(mean_per_window, equal_to([(0, 2.0), (1, 7.0)]),
                   label='assert:mean')
 
+  @attr('ValidatesRunner')
+  def test_window_assignment_idempotency(self):
+    with TestPipeline() as p:
+      pcoll = self.timestamped_key_values(p, 'key', 0, 2, 4)
+      result = (pcoll
+                | 'window' >> WindowInto(FixedWindows(2))
+                | 'same window' >> WindowInto(FixedWindows(2))
+                | 'same window again' >> WindowInto(FixedWindows(2))
+                | GroupByKey())
+
+      assert_that(result, equal_to([('key', [0]),
+                                    ('key', [2]),
+                                    ('key', [4])]))
+
+  @attr('ValidatesRunner')
+  def test_window_assignment_through_multiple_gbk_idempotency(self):
+    with TestPipeline() as p:
+      pcoll = self.timestamped_key_values(p, 'key', 0, 2, 4)
+      result = (pcoll
+                | 'window' >> WindowInto(FixedWindows(2))
+                | 'gbk' >> GroupByKey()
+                | 'same window' >> WindowInto(FixedWindows(2))
+                | 'another gbk' >> GroupByKey()
+                | 'same window again' >> WindowInto(FixedWindows(2))
+                | 'gbk again' >> GroupByKey())
+
+      assert_that(result, equal_to([('key', [[[0]]]),
+                                    ('key', [[[2]]]),
+                                    ('key', [[[4]]])]))
 
 class RunnerApiTest(unittest.TestCase):
 
diff --git a/sdks/python/apache_beam/utils/thread_pool_executor.py b/sdks/python/apache_beam/utils/thread_pool_executor.py
new file mode 100644
index 0000000..aba8f5ad
--- /dev/null
+++ b/sdks/python/apache_beam/utils/thread_pool_executor.py
@@ -0,0 +1,170 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+from __future__ import absolute_import
+
+import sys
+import threading
+import weakref
+from concurrent.futures import _base
+
+try:  # Python3
+  import queue
+except Exception:  # Python2
+  import Queue as queue
+
+
+class _WorkItem(object):
+  def __init__(self, future, fn, args, kwargs):
+    self._future = future
+    self._fn = fn
+    self._fn_args = args
+    self._fn_kwargs = kwargs
+
+  def run(self):
+    if self._future.set_running_or_notify_cancel():
+      # If the future wasn't cancelled, then attempt to execute it.
+      try:
+        self._future.set_result(self._fn(*self._fn_args, **self._fn_kwargs))
+      except BaseException as exc:
+        # Even though Python 2 futures library has #set_exection(),
+        # the way it generates the traceback doesn't align with
+        # the way in which Python 3 does it so we provide alternative
+        # implementations that match our test expectations.
+        if sys.version_info.major >= 3:
+          self._future.set_exception(exc)
+        else:
+          e, tb = sys.exc_info()[1:]
+          self._future.set_exception_info(e, tb)
+
+
+class _Worker(threading.Thread):
+  def __init__(self, idle_worker_queue, permitted_thread_age_in_seconds,
+               work_item):
+    super(_Worker, self).__init__()
+    self._idle_worker_queue = idle_worker_queue
+    self._permitted_thread_age_in_seconds = permitted_thread_age_in_seconds
+    self._work_item = work_item
+    self._wake_event = threading.Event()
+    self._lock = threading.Lock()
+    self._shutdown = False
+
+  def run(self):
+    while True:
+      self._work_item.run()
+      self._work_item = None
+
+      # If we are explicitly awake then don't add ourselves back to the
+      # idle queue. This occurs in case 3 described below.
+      if not self._wake_event.is_set():
+        self._idle_worker_queue.put(self)
+
+      self._wake_event.wait(self._permitted_thread_age_in_seconds)
+      with self._lock:
+        # When we are awoken, we may be in one of three states:
+        #  1) _work_item is set and _shutdown is False.
+        #     This represents the case when we have accepted work.
+        #  2) _work_item is unset and _shutdown is True.
+        #     This represents the case where either we timed out before
+        #     accepting work or explicitly were shutdown without accepting
+        #     any work.
+        #  3) _work_item is set and _shutdown is True.
+        #     This represents a race where we accepted work and also
+        #     were shutdown before the worker thread started processing
+        #     that work. In this case we guarantee to process the work
+        #     but we don't clear the event ensuring that the next loop
+        #     around through to the wait() won't block and we will exit
+        #     since _work_item will be unset.
+
+        # We only exit when _work_item is unset to prevent dropping of
+        # submitted work.
+        if self._work_item is None:
+          self._shutdown = True
+          return
+        if not self._shutdown:
+          self._wake_event.clear()
+
+  def accepted_work(self, work_item):
+    """Returns True if the work was accepted.
+
+    This method must only be called while the worker is idle.
+    """
+    with self._lock:
+      if self._shutdown:
+        return False
+
+      self._work_item = work_item
+      self._wake_event.set()
+      return True
+
+  def shutdown(self):
+    """Marks this thread as shutdown possibly waking it up if it is idle."""
+    with self._lock:
+      if self._shutdown:
+        return
+      self._shutdown = True
+      self._wake_event.set()
+
+
+class UnboundedThreadPoolExecutor(_base.Executor):
+  def __init__(self, permitted_thread_age_in_seconds=30):
+    self._permitted_thread_age_in_seconds = permitted_thread_age_in_seconds
+    self._idle_worker_queue = queue.Queue()
+    self._workers = weakref.WeakSet()
+    self._shutdown = False
+    self._lock = threading.Lock() # Guards access to _workers and _shutdown
+
+  def submit(self, fn, *args, **kwargs):
+    """Attempts to submit the work item.
+
+    A runtime error is raised if the pool has been shutdown.
+    """
+    future = _base.Future()
+    work_item = _WorkItem(future, fn, args, kwargs)
+    try:
+      # Keep trying to get an idle worker from the queue until we find one
+      # that accepts the work.
+      while not self._idle_worker_queue.get(
+          block=False).accepted_work(work_item):
+        pass
+      return future
+    except queue.Empty:
+      with self._lock:
+        if self._shutdown:
+          raise RuntimeError('Cannot schedule new tasks after thread pool '
+                             'has been shutdown.')
+
+        worker = _Worker(
+            self._idle_worker_queue, self._permitted_thread_age_in_seconds,
+            work_item)
+        worker.daemon = True
+        worker.start()
+        self._workers.add(worker)
+        return future
+
+  def shutdown(self, wait=True):
+    with self._lock:
+      if self._shutdown:
+        return
+
+      self._shutdown = True
+      for worker in self._workers:
+        worker.shutdown()
+
+      if wait:
+        for worker in self._workers:
+          worker.join()
diff --git a/sdks/python/apache_beam/utils/thread_pool_executor_test.py b/sdks/python/apache_beam/utils/thread_pool_executor_test.py
new file mode 100644
index 0000000..3616409
--- /dev/null
+++ b/sdks/python/apache_beam/utils/thread_pool_executor_test.py
@@ -0,0 +1,106 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+"""Unit tests for UnboundedThreadPoolExecutor."""
+
+from __future__ import absolute_import
+
+import threading
+import time
+import traceback
+import unittest
+
+# patches unittest.TestCase to be python3 compatible
+import future.tests.base  # pylint: disable=unused-import
+
+from apache_beam.utils.thread_pool_executor import UnboundedThreadPoolExecutor
+
+
+class UnboundedThreadPoolExecutorTest(unittest.TestCase):
+  def setUp(self):
+    self._lock = threading.Lock()
+    self._worker_idents = []
+
+  def append_and_sleep(self, sleep_time):
+    with self._lock:
+      self._worker_idents.append(threading.current_thread().ident)
+    time.sleep(sleep_time)
+
+  def raise_error(self, message):
+    raise ValueError(message)
+
+  def test_shutdown_with_no_workers(self):
+    with UnboundedThreadPoolExecutor():
+      pass
+
+  def test_shutdown_with_fast_workers(self):
+    futures = []
+    with UnboundedThreadPoolExecutor() as executor:
+      for _ in range(0, 5):
+        futures.append(executor.submit(self.append_and_sleep, 0.01))
+
+    for future in futures:
+      future.result(timeout=10)
+
+    with self._lock:
+      self.assertEqual(5, len(self._worker_idents))
+
+  def test_shutdown_with_slow_workers(self):
+    futures = []
+    with UnboundedThreadPoolExecutor() as executor:
+      for _ in range(0, 5):
+        futures.append(executor.submit(self.append_and_sleep, 1))
+
+    for future in futures:
+      future.result(timeout=10)
+
+    with self._lock:
+      self.assertEqual(5, len(self._worker_idents))
+
+  def test_worker_reuse(self):
+    futures = []
+    with UnboundedThreadPoolExecutor() as executor:
+      for _ in range(0, 5):
+        futures.append(executor.submit(self.append_and_sleep, 0.01))
+      time.sleep(3)
+      for _ in range(0, 5):
+        futures.append(executor.submit(self.append_and_sleep, 0.01))
+
+    for future in futures:
+      future.result(timeout=10)
+
+    with self._lock:
+      self.assertEqual(10, len(self._worker_idents))
+      self.assertTrue(len(set(self._worker_idents)) < 10)
+
+  def test_exception_propagation(self):
+    with UnboundedThreadPoolExecutor() as executor:
+      future = executor.submit(self.raise_error, 'footest')
+
+    try:
+      future.result()
+    except Exception:
+      message = traceback.format_exc()
+    else:
+      raise AssertionError('expected exception not raised')
+
+    self.assertIn('footest', message)
+    self.assertIn('raise_error', message)
+
+
+if __name__ == '__main__':
+  unittest.main()
diff --git a/sdks/python/apache_beam/utils/timestamp.py b/sdks/python/apache_beam/utils/timestamp.py
index 9bccdfd..a3f3abf 100644
--- a/sdks/python/apache_beam/utils/timestamp.py
+++ b/sdks/python/apache_beam/utils/timestamp.py
@@ -25,6 +25,7 @@
 
 import datetime
 import functools
+import time
 from builtins import object
 
 import dateutil.parser
@@ -76,6 +77,10 @@
     return Timestamp(seconds)
 
   @staticmethod
+  def now():
+    return Timestamp(seconds=time.time())
+
+  @staticmethod
   def _epoch_datetime_utc():
     return datetime.datetime.fromtimestamp(0, pytz.utc)
 
@@ -173,6 +178,8 @@
     return self + other
 
   def __sub__(self, other):
+    if isinstance(other, Timestamp):
+      return Duration(micros=self.micros - other.micros)
     other = Duration.of(other)
     return Timestamp(micros=self.micros - other.micros)
 
diff --git a/sdks/python/apache_beam/utils/timestamp_test.py b/sdks/python/apache_beam/utils/timestamp_test.py
index d26d561..2a4d454 100644
--- a/sdks/python/apache_beam/utils/timestamp_test.py
+++ b/sdks/python/apache_beam/utils/timestamp_test.py
@@ -100,6 +100,7 @@
     self.assertEqual(Timestamp(123) - Duration(456), -333)
     self.assertEqual(Timestamp(1230) % 456, 318)
     self.assertEqual(Timestamp(1230) % Duration(456), 318)
+    self.assertEqual(Timestamp(123) - Timestamp(100), 23)
 
     # Check that direct comparison of Timestamp and Duration is allowed.
     self.assertTrue(Duration(123) == Timestamp(123))
@@ -116,6 +117,7 @@
     self.assertEqual((Timestamp(123) - Duration(456)).__class__, Timestamp)
     self.assertEqual((Timestamp(1230) % 456).__class__, Duration)
     self.assertEqual((Timestamp(1230) % Duration(456)).__class__, Duration)
+    self.assertEqual((Timestamp(123) - Timestamp(100)).__class__, Duration)
 
     # Unsupported operations.
     with self.assertRaises(TypeError):
@@ -159,6 +161,10 @@
     self.assertEqual('Timestamp(-999999999)',
                      str(Timestamp(-999999999)))
 
+  def test_now(self):
+    now = Timestamp.now()
+    self.assertTrue(isinstance(now, Timestamp))
+
 
 class DurationTest(unittest.TestCase):
 
diff --git a/sdks/python/scripts/generate_pydoc.sh b/sdks/python/scripts/generate_pydoc.sh
index e3794ba..3660b72 100755
--- a/sdks/python/scripts/generate_pydoc.sh
+++ b/sdks/python/scripts/generate_pydoc.sh
@@ -183,9 +183,11 @@
   '_TimerDoFnParam',
   '_BundleFinalizerParam',
   '_RestrictionDoFnParam',
+  '_WatermarkEstimatorParam',
 
   # Sphinx cannot find this py:class reference target
   'typing.Generic',
+  'concurrent.futures._base.Executor',
 ]
 
 # When inferring a base class it will use ':py:class'; if inferring a function
diff --git a/sdks/python/setup.py b/sdks/python/setup.py
index 7eea64c..9f1a9f3 100644
--- a/sdks/python/setup.py
+++ b/sdks/python/setup.py
@@ -228,7 +228,10 @@
     python_requires=python_requires,
     test_suite='nose.collector',
     setup_requires=['pytest_runner'],
-    tests_require=REQUIRED_TEST_PACKAGES,
+    tests_require= [
+        REQUIRED_TEST_PACKAGES,
+        INTERACTIVE_BEAM,
+    ],
     extras_require={
         'docs': ['Sphinx>=1.5.2,<2.0'],
         'test': REQUIRED_TEST_PACKAGES,
diff --git a/sdks/python/test-suites/tox/py37/build.gradle b/sdks/python/test-suites/tox/py37/build.gradle
index 2a57ca9..c9c99e6 100644
--- a/sdks/python/test-suites/tox/py37/build.gradle
+++ b/sdks/python/test-suites/tox/py37/build.gradle
@@ -41,9 +41,6 @@
 toxTask "testPy37Cython", "py37-cython"
 test.dependsOn testPy37Cython
 
-toxTask "testPy37Interactive", "py37-interactive"
-test.dependsOn testPy37Interactive
-
 // Ensure that testPy37Cython runs exclusively to other tests. This line is not
 // actually required, since gradle doesn't do parallel execution within a
 // project.
@@ -60,7 +57,6 @@
 task preCommitPy37() {
     dependsOn "testPy37Gcp"
     dependsOn "testPy37Cython"
-    dependsOn "testPy37Interactive"
 }
 
 task preCommitPy37Pytest {
diff --git a/sdks/python/tox.ini b/sdks/python/tox.ini
index fe3f65a..d227519 100644
--- a/sdks/python/tox.ini
+++ b/sdks/python/tox.ini
@@ -307,10 +307,3 @@
   coverage report --skip-covered
   # Generate report in xml format
   coverage xml
-
-[testenv:py37-interactive]
-setenv =
-  RUN_SKIPPED_PY3_TESTS=0
-extras = test,interactive
-commands =
-  python setup.py nosetests --ignore-files '.*py3[8-9]\.py$' {posargs}