Merge pull request #10069 Separate pydocs generation from py2 precommit tests.

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/.test-infra/jenkins/job_PreCommit_Java_Examples_Dataflow.groovy b/.test-infra/jenkins/job_PreCommit_Java_Examples_Dataflow.groovy
index 7855085..e4c17ca 100644
--- a/.test-infra/jenkins/job_PreCommit_Java_Examples_Dataflow.groovy
+++ b/.test-infra/jenkins/job_PreCommit_Java_Examples_Dataflow.groovy
@@ -30,7 +30,8 @@
       '^examples/java/.*$',
       '^examples/kotlin/.*$',
       '^release/.*$',
-    ]
+    ],
+    timeoutMins: 30,
 )
 builder.build {
   publishers {
diff --git a/.test-infra/jenkins/job_PreCommit_Python.groovy b/.test-infra/jenkins/job_PreCommit_Python.groovy
index 5605156..97f28ba 100644
--- a/.test-infra/jenkins/job_PreCommit_Python.groovy
+++ b/.test-infra/jenkins/job_PreCommit_Python.groovy
@@ -43,6 +43,7 @@
     nameBase: 'Python_pytest',
     gradleTask: ':pythonPreCommitPytest',
     commitTriggering: false,
+    timeoutMins: 180,
 )
 builderPytest.build {
   // Publish all test results to Jenkins.
diff --git a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy
index 3ea3643..557f45f 100644
--- a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy
+++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy
@@ -1905,6 +1905,7 @@
           mustRunAfter = [
             ':runners:flink:1.9:job-server-container:docker',
             ':runners:flink:1.9:job-server:shadowJar',
+            ':runners:spark:job-server:shadowJar',
             ':sdks:python:container:py2:docker',
             ':sdks:python:container:py35:docker',
             ':sdks:python:container:py36:docker',
@@ -1958,6 +1959,7 @@
         addPortableWordCountTask(true, "PortableRunner")
         addPortableWordCountTask(false, "FlinkRunner")
         addPortableWordCountTask(true, "FlinkRunner")
+        addPortableWordCountTask(false, "SparkRunner")
       }
     }
   }
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/job-management/src/main/proto/beam_job_api.proto b/model/job-management/src/main/proto/beam_job_api.proto
index e9f0eb9..d297d3b 100644
--- a/model/job-management/src/main/proto/beam_job_api.proto
+++ b/model/job-management/src/main/proto/beam_job_api.proto
@@ -213,17 +213,40 @@
 // without needing to pass through STARTING.
 message JobState {
   enum Enum {
+    // The job state reported by a runner cannot be interpreted by the SDK.
     UNSPECIFIED = 0;
+
+    // The job has not yet started.
     STOPPED = 1;
+
+    // The job is currently running.
     RUNNING = 2;
+
+    // The job has successfully completed. (terminal)
     DONE = 3;
+
+    // The job has failed. (terminal)
     FAILED = 4;
+
+    // The job has been explicitly cancelled. (terminal)
     CANCELLED = 5;
+
+    // The job has been updated. (terminal)
     UPDATED = 6;
+
+    // The job is draining its data. (optional)
     DRAINING = 7;
+
+    // The job has completed draining its data. (terminal)
     DRAINED = 8;
+
+    // The job is starting up.
     STARTING = 9;
+
+    // The job is cancelling. (optional)
     CANCELLING = 10;
+
+    // The job is in the process of being updated. (optional)
     UPDATING = 11;
   }
 }
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/flink_runner.gradle b/runners/flink/flink_runner.gradle
index 3254a85..6281b94 100644
--- a/runners/flink/flink_runner.gradle
+++ b/runners/flink/flink_runner.gradle
@@ -200,6 +200,7 @@
       excludeCategories 'org.apache.beam.sdk.testing.UsesCommittedMetrics'
       if (config.streaming) {
         excludeCategories 'org.apache.beam.sdk.testing.UsesImpulse'
+        excludeCategories 'org.apache.beam.sdk.testing.UsesTestStreamWithMultipleStages'  // BEAM-8598
         excludeCategories 'org.apache.beam.sdk.testing.UsesTestStreamWithProcessingTime'
       } else {
         excludeCategories 'org.apache.beam.sdk.testing.UsesUnboundedSplittableParDo'
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/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/FnApiWindowMappingFn.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/FnApiWindowMappingFn.java
index 7e298e7..dcf15f4 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/FnApiWindowMappingFn.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/FnApiWindowMappingFn.java
@@ -56,7 +56,6 @@
 import org.apache.beam.sdk.util.WindowedValue;
 import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.WindowingStrategy;
-import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Strings;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.cache.Cache;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.cache.CacheBuilder;
 import org.slf4j.Logger;
@@ -253,7 +252,7 @@
       }
 
       // Check to see if processing the request failed.
-      throwIfFailure(processResponse);
+      MoreFutures.get(processResponse);
 
       waitForInboundTermination.awaitCompletion();
       WindowedValue<KV<byte[], TargetWindowT>> sideInputWindow = outputValue.poll();
@@ -300,22 +299,10 @@
                               processBundleDescriptor.toBuilder().setId(descriptorId).build())
                           .build())
                   .build());
-      throwIfFailure(response);
+      // Check if the bundle descriptor is registered successfully.
+      MoreFutures.get(response);
       processBundleDescriptorId = descriptorId;
     }
     return processBundleDescriptorId;
   }
-
-  private static InstructionResponse throwIfFailure(
-      CompletionStage<InstructionResponse> responseFuture)
-      throws ExecutionException, InterruptedException {
-    InstructionResponse response = MoreFutures.get(responseFuture);
-    if (!Strings.isNullOrEmpty(response.getError())) {
-      throw new IllegalStateException(
-          String.format(
-              "Client failed to process %s with error [%s].",
-              response.getInstructionId(), response.getError()));
-    }
-    return response;
-  }
 }
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java
index a603d16..4def350 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java
@@ -1148,6 +1148,15 @@
     }
   }
 
+  private Windmill.WorkItemCommitRequest.Builder initializeOutputBuilder(
+      final ByteString key, final Windmill.WorkItem workItem) {
+    return Windmill.WorkItemCommitRequest.newBuilder()
+        .setKey(key)
+        .setShardingKey(workItem.getShardingKey())
+        .setWorkToken(workItem.getWorkToken())
+        .setCacheToken(workItem.getCacheToken());
+  }
+
   private void process(
       final SdkWorkerHarness worker,
       final ComputationState computationState,
@@ -1164,12 +1173,7 @@
     DataflowWorkerLoggingMDC.setStageName(computationId);
     LOG.debug("Starting processing for {}:\n{}", computationId, work);
 
-    Windmill.WorkItemCommitRequest.Builder outputBuilder =
-        Windmill.WorkItemCommitRequest.newBuilder()
-            .setKey(key)
-            .setShardingKey(workItem.getShardingKey())
-            .setWorkToken(workItem.getWorkToken())
-            .setCacheToken(workItem.getCacheToken());
+    Windmill.WorkItemCommitRequest.Builder outputBuilder = initializeOutputBuilder(key, workItem);
 
     // Before any processing starts, call any pending OnCommit callbacks.  Nothing that requires
     // cleanup should be done before this, since we might exit early here.
@@ -1334,20 +1338,22 @@
       WorkItemCommitRequest commitRequest = outputBuilder.build();
       int byteLimit = maxWorkItemCommitBytes;
       int commitSize = commitRequest.getSerializedSize();
+      int estimatedCommitSize = commitSize < 0 ? Integer.MAX_VALUE : commitSize;
+
       // Detect overflow of integer serialized size or if the byte limit was exceeded.
-      windmillMaxObservedWorkItemCommitBytes.addValue(
-          commitSize < 0 ? Integer.MAX_VALUE : commitSize);
-      if (commitSize < 0) {
-        throw KeyCommitTooLargeException.causedBy(computationId, byteLimit, commitRequest);
-      } else if (commitSize > byteLimit) {
-        // Once supported, we should communicate the desired truncation for the commit to the
-        // streaming engine. For now we report the error but attempt the commit so that it will be
-        // truncated by the streaming engine backend.
+      windmillMaxObservedWorkItemCommitBytes.addValue(estimatedCommitSize);
+      if (estimatedCommitSize > byteLimit) {
         KeyCommitTooLargeException e =
             KeyCommitTooLargeException.causedBy(computationId, byteLimit, commitRequest);
         reportFailure(computationId, workItem, e);
         LOG.error(e.toString());
+
+        // Drop the current request in favor of a new, minimal one requesting truncation.
+        // Messages, timers, counters, and other commit content will not be used by the service
+        // so we're purposefully dropping them here
+        commitRequest = buildWorkItemTruncationRequest(key, workItem, estimatedCommitSize);
       }
+
       commitQueue.put(new Commit(commitRequest, computationState, work));
 
       // Compute shuffle and state byte statistics these will be flushed asynchronously.
@@ -1442,6 +1448,14 @@
     }
   }
 
+  private WorkItemCommitRequest buildWorkItemTruncationRequest(
+      final ByteString key, final Windmill.WorkItem workItem, final int estimatedCommitSize) {
+    Windmill.WorkItemCommitRequest.Builder outputBuilder = initializeOutputBuilder(key, workItem);
+    outputBuilder.setExceedsMaxWorkItemCommitBytes(true);
+    outputBuilder.setEstimatedWorkItemCommitBytes(estimatedCommitSize);
+    return outputBuilder.build();
+  }
+
   private void commitLoop() {
     Map<ComputationState, Windmill.ComputationCommitWorkRequest.Builder> computationRequestMap =
         new HashMap<>();
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/fn/control/RegisterAndProcessBundleOperation.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/fn/control/RegisterAndProcessBundleOperation.java
index 0a3346c..bf42c4d 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/fn/control/RegisterAndProcessBundleOperation.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/fn/control/RegisterAndProcessBundleOperation.java
@@ -370,12 +370,8 @@
    * elements consumed from the upstream read operation.
    *
    * <p>May be called at any time, including before start() and after finish().
-   *
-   * @throws InterruptedException
-   * @throws ExecutionException
    */
-  public CompletionStage<BeamFnApi.ProcessBundleProgressResponse> getProcessBundleProgress()
-      throws InterruptedException, ExecutionException {
+  public CompletionStage<BeamFnApi.ProcessBundleProgressResponse> getProcessBundleProgress() {
     // processBundleId may be reset if this bundle finishes asynchronously.
     String processBundleId = this.processBundleId;
 
@@ -393,13 +389,7 @@
 
     return instructionRequestHandler
         .handle(processBundleRequest)
-        .thenApply(
-            response -> {
-              if (!response.getError().isEmpty()) {
-                throw new IllegalStateException(response.getError());
-              }
-              return response.getProcessBundleProgress();
-            });
+        .thenApply(InstructionResponse::getProcessBundleProgress);
   }
 
   /** Returns the final metrics returned by the SDK harness when it completes the bundle. */
@@ -636,53 +626,36 @@
     return true;
   }
 
-  private static CompletionStage<BeamFnApi.InstructionResponse> throwIfFailure(
+  private static CompletionStage<BeamFnApi.ProcessBundleResponse> getProcessBundleResponse(
       CompletionStage<InstructionResponse> responseFuture) {
     return responseFuture.thenApply(
         response -> {
-          if (!response.getError().isEmpty()) {
-            throw new IllegalStateException(
-                String.format(
-                    "Client failed to process %s with error [%s].",
-                    response.getInstructionId(), response.getError()));
+          switch (response.getResponseCase()) {
+            case PROCESS_BUNDLE:
+              return response.getProcessBundle();
+            default:
+              throw new IllegalStateException(
+                  String.format(
+                      "SDK harness returned wrong kind of response to ProcessBundleRequest: %s",
+                      TextFormat.printToString(response)));
           }
-          return response;
         });
   }
 
-  private static CompletionStage<BeamFnApi.ProcessBundleResponse> getProcessBundleResponse(
-      CompletionStage<InstructionResponse> responseFuture) {
-    return throwIfFailure(responseFuture)
-        .thenApply(
-            response -> {
-              switch (response.getResponseCase()) {
-                case PROCESS_BUNDLE:
-                  return response.getProcessBundle();
-                default:
-                  throw new IllegalStateException(
-                      String.format(
-                          "SDK harness returned wrong kind of response to ProcessBundleRequest: %s",
-                          TextFormat.printToString(response)));
-              }
-            });
-  }
-
   private static CompletionStage<BeamFnApi.RegisterResponse> getRegisterResponse(
-      CompletionStage<InstructionResponse> responseFuture)
-      throws ExecutionException, InterruptedException {
-    return throwIfFailure(responseFuture)
-        .thenApply(
-            response -> {
-              switch (response.getResponseCase()) {
-                case REGISTER:
-                  return response.getRegister();
-                default:
-                  throw new IllegalStateException(
-                      String.format(
-                          "SDK harness returned wrong kind of response to RegisterRequest: %s",
-                          TextFormat.printToString(response)));
-              }
-            });
+      CompletionStage<InstructionResponse> responseFuture) {
+    return responseFuture.thenApply(
+        response -> {
+          switch (response.getResponseCase()) {
+            case REGISTER:
+              return response.getRegister();
+            default:
+              throw new IllegalStateException(
+                  String.format(
+                      "SDK harness returned wrong kind of response to RegisterRequest: %s",
+                      TextFormat.printToString(response)));
+          }
+        });
   }
 
   private static void cancelIfNotNull(CompletionStage<?> future) {
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorkerTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorkerTest.java
index 356dddb..2cb5ada 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorkerTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorkerTest.java
@@ -562,37 +562,70 @@
 
   private WorkItemCommitRequest.Builder makeExpectedOutput(
       int index, long timestamp, String key, String outKey) throws Exception {
+    StringBuilder expectedCommitRequestBuilder = initializeExpectedCommitRequest(key, index);
+    appendCommitOutputMessages(expectedCommitRequestBuilder, index, timestamp, outKey);
+
     return setMessagesMetadata(
         PaneInfo.NO_FIRING,
         intervalWindowBytes(DEFAULT_WINDOW),
-        parseCommitRequest(
-            "key: \""
-                + key
-                + "\" "
-                + "sharding_key: 17 "
-                + "work_token: "
-                + index
-                + " "
-                + "cache_token: 3 "
-                + "output_messages {"
-                + "  destination_stream_id: \""
-                + DEFAULT_DESTINATION_STREAM_ID
-                + "\""
-                + "  bundles {"
-                + "    key: \""
-                + outKey
-                + "\""
-                + "    messages {"
-                + "      timestamp: "
-                + timestamp
-                + "      data: \""
-                + dataStringForIndex(index)
-                + "\""
-                + "      metadata: \"\""
-                + "    }"
-                + "    messages_ids: \"\""
-                + "  }"
-                + "}"));
+        parseCommitRequest(expectedCommitRequestBuilder.toString()));
+  }
+
+  private WorkItemCommitRequest.Builder makeExpectedTruncationRequestOutput(
+      int index, String key, long estimatedSize) throws Exception {
+    StringBuilder expectedCommitRequestBuilder = initializeExpectedCommitRequest(key, index);
+    appendCommitTruncationFields(expectedCommitRequestBuilder, estimatedSize);
+
+    return parseCommitRequest(expectedCommitRequestBuilder.toString());
+  }
+
+  private StringBuilder initializeExpectedCommitRequest(String key, int index) {
+    StringBuilder requestBuilder = new StringBuilder();
+
+    requestBuilder.append("key: \"");
+    requestBuilder.append(key);
+    requestBuilder.append("\" ");
+    requestBuilder.append("sharding_key: 17 ");
+    requestBuilder.append("work_token: ");
+    requestBuilder.append(index);
+    requestBuilder.append(" ");
+    requestBuilder.append("cache_token: 3 ");
+
+    return requestBuilder;
+  }
+
+  private StringBuilder appendCommitOutputMessages(
+      StringBuilder requestBuilder, int index, long timestamp, String outKey) {
+    requestBuilder.append("output_messages {");
+    requestBuilder.append("  destination_stream_id: \"");
+    requestBuilder.append(DEFAULT_DESTINATION_STREAM_ID);
+    requestBuilder.append("\"");
+    requestBuilder.append("  bundles {");
+    requestBuilder.append("    key: \"");
+    requestBuilder.append(outKey);
+    requestBuilder.append("\"");
+    requestBuilder.append("    messages {");
+    requestBuilder.append("      timestamp: ");
+    requestBuilder.append(timestamp);
+    requestBuilder.append("      data: \"");
+    requestBuilder.append(dataStringForIndex(index));
+    requestBuilder.append("\"");
+    requestBuilder.append("      metadata: \"\"");
+    requestBuilder.append("    }");
+    requestBuilder.append("    messages_ids: \"\"");
+    requestBuilder.append("  }");
+    requestBuilder.append("}");
+
+    return requestBuilder;
+  }
+
+  private StringBuilder appendCommitTruncationFields(
+      StringBuilder requestBuilder, long estimatedSize) {
+    requestBuilder.append("exceeds_max_work_item_commit_bytes: true ");
+    requestBuilder.append("estimated_work_item_commit_bytes: ");
+    requestBuilder.append(estimatedSize);
+
+    return requestBuilder;
   }
 
   private StreamingComputationConfig makeDefaultStreamingComputationConfig(
@@ -948,64 +981,19 @@
 
     assertEquals(2, result.size());
     assertEquals(makeExpectedOutput(2, 0, "key", "key").build(), result.get(2L));
+
     assertTrue(result.containsKey(1L));
-    assertEquals("large_key", result.get(1L).getKey().toStringUtf8());
-    assertTrue(result.get(1L).getSerializedSize() > 1000);
+    WorkItemCommitRequest largeCommit = result.get(1L);
+    assertEquals("large_key", largeCommit.getKey().toStringUtf8());
+    assertEquals(
+        makeExpectedTruncationRequestOutput(
+                1, "large_key", largeCommit.getEstimatedWorkItemCommitBytes())
+            .build(),
+        largeCommit);
 
-    // Spam worker updates a few times.
-    int maxTries = 10;
-    while (--maxTries > 0) {
-      worker.reportPeriodicWorkerUpdates();
-      Uninterruptibles.sleepUninterruptibly(1000, TimeUnit.MILLISECONDS);
-    }
-
-    // We should see an exception reported for the large commit but not the small one.
-    ArgumentCaptor<WorkItemStatus> workItemStatusCaptor =
-        ArgumentCaptor.forClass(WorkItemStatus.class);
-    verify(mockWorkUnitClient, atLeast(2)).reportWorkItemStatus(workItemStatusCaptor.capture());
-    List<WorkItemStatus> capturedStatuses = workItemStatusCaptor.getAllValues();
-    boolean foundErrors = false;
-    for (WorkItemStatus status : capturedStatuses) {
-      if (!status.getErrors().isEmpty()) {
-        assertFalse(foundErrors);
-        foundErrors = true;
-        String errorMessage = status.getErrors().get(0).getMessage();
-        assertThat(errorMessage, Matchers.containsString("KeyCommitTooLargeException"));
-      }
-    }
-    assertTrue(foundErrors);
-  }
-
-  @Test
-  public void testKeyCommitTooLargeException_StreamingEngine() throws Exception {
-    KvCoder<String, String> kvCoder = KvCoder.of(StringUtf8Coder.of(), StringUtf8Coder.of());
-
-    List<ParallelInstruction> instructions =
-        Arrays.asList(
-            makeSourceInstruction(kvCoder),
-            makeDoFnInstruction(new LargeCommitFn(), 0, kvCoder),
-            makeSinkInstruction(kvCoder, 1));
-
-    FakeWindmillServer server = new FakeWindmillServer(errorCollector);
-    server.setExpectedExceptionCount(1);
-
-    StreamingDataflowWorkerOptions options =
-        createTestingPipelineOptions(server, "--experiments=enable_streaming_engine");
-    StreamingDataflowWorker worker = makeWorker(instructions, options, true /* publishCounters */);
-    worker.setMaxWorkItemCommitBytes(1000);
-    worker.start();
-
-    server.addWorkToOffer(makeInput(1, 0, "large_key"));
-    server.addWorkToOffer(makeInput(2, 0, "key"));
-    server.waitForEmptyWorkQueue();
-
-    Map<Long, Windmill.WorkItemCommitRequest> result = server.waitForAndGetCommits(1);
-
-    assertEquals(2, result.size());
-    assertEquals(makeExpectedOutput(2, 0, "key", "key").build(), result.get(2L));
-    assertTrue(result.containsKey(1L));
-    assertEquals("large_key", result.get(1L).getKey().toStringUtf8());
-    assertTrue(result.get(1L).getSerializedSize() > 1000);
+    // Check this explicitly since the estimated commit bytes weren't actually
+    // checked against an expected value in the previous step
+    assertTrue(largeCommit.getEstimatedWorkItemCommitBytes() > 1000);
 
     // Spam worker updates a few times.
     int maxTries = 10;
diff --git a/runners/google-cloud-dataflow-java/worker/windmill/src/main/proto/windmill.proto b/runners/google-cloud-dataflow-java/worker/windmill/src/main/proto/windmill.proto
index 5310902..598164e 100644
--- a/runners/google-cloud-dataflow-java/worker/windmill/src/main/proto/windmill.proto
+++ b/runners/google-cloud-dataflow-java/worker/windmill/src/main/proto/windmill.proto
@@ -272,13 +272,16 @@
   optional string state_family = 3;
 }
 
-// next id: 19
+// next id: 24
 message WorkItemCommitRequest {
   required bytes key = 1;
   required fixed64 work_token = 2;
   optional fixed64 sharding_key = 15;
   optional fixed64 cache_token = 16;
 
+  optional bool exceeds_max_work_item_commit_bytes = 20;
+  optional int64 estimated_work_item_commit_bytes = 21;
+
   repeated OutputMessageBundle output_messages = 3;
   repeated PubSubMessageBundle pubsub_messages = 7;
   repeated Timer output_timers = 4;
@@ -290,12 +293,14 @@
   optional SourceState source_state_updates = 12;
   optional int64 source_watermark = 13 [default=-0x8000000000000000];
   optional int64 source_backlog_bytes = 17 [default=-1];
+  optional int64 source_bytes_processed = 22;
+
   repeated WatermarkHold watermark_holds = 14;
 
   // DEPRECATED
   repeated GlobalDataId global_data_id_requests = 9;
 
-  reserved 6;
+  reserved 6, 19, 23;
 }
 
 message ComputationCommitWorkRequest {
diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/BundleCheckpointHandler.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/BundleCheckpointHandler.java
new file mode 100644
index 0000000..1e5fa53
--- /dev/null
+++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/BundleCheckpointHandler.java
@@ -0,0 +1,33 @@
+/*
+ * 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 org.apache.beam.model.fnexecution.v1.BeamFnApi;
+
+/**
+ * A handler which is invoked when the SDK returns {@link BeamFnApi.DelayedBundleApplication}s as
+ * part of the bundle completion.
+ *
+ * <p>These bundle applications must be resumed otherwise data loss will occur.
+ *
+ * <p>See <a href="https://s.apache.org/beam-breaking-fusion">breaking the fusion barrier</a> for
+ * further details.
+ */
+public interface BundleCheckpointHandler {
+  void onCheckpoint(BeamFnApi.ProcessBundleResponse response);
+}
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
new file mode 100644
index 0000000..646cdf9
--- /dev/null
+++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/BundleFinalizationHandler.java
@@ -0,0 +1,38 @@
+/*
+ * 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;
+
+/**
+ * A handler for the runner when a finalization request has been received.
+ *
+ * <p>The runner is responsible for finalizing the bundle when all output from the bundle has been
+ * durably persisted.
+ *
+ * <p>See <a href="https://s.apache.org/beam-finalizing-bundles">finalizing bundles</a> for further
+ * details.
+ */
+public interface BundleFinalizationHandler {
+  /**
+   * 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/FnApiControlClient.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/FnApiControlClient.java
index f56e7f2..9051051 100644
--- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/FnApiControlClient.java
+++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/FnApiControlClient.java
@@ -148,16 +148,18 @@
       LOG.debug("Received InstructionResponse {}", response);
       CompletableFuture<BeamFnApi.InstructionResponse> responseFuture =
           outstandingRequests.remove(response.getInstructionId());
-      if (responseFuture != null) {
-        if (response.getError().isEmpty()) {
-          responseFuture.complete(response);
-        } else {
-          responseFuture.completeExceptionally(
-              new RuntimeException(
-                  String.format(
-                      "Error received from SDK harness for instruction %s: %s",
-                      response.getInstructionId(), response.getError())));
-        }
+      if (responseFuture == null) {
+        LOG.warn("Dropped unknown InstructionResponse {}", response);
+        return;
+      }
+      if (response.getError().isEmpty()) {
+        responseFuture.complete(response);
+      } else {
+        responseFuture.completeExceptionally(
+            new RuntimeException(
+                String.format(
+                    "Error received from SDK harness for instruction %s: %s",
+                    response.getInstructionId(), response.getError())));
       }
     }
 
diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/InstructionRequestHandler.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/InstructionRequestHandler.java
index b655732..8a9dc75 100644
--- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/InstructionRequestHandler.java
+++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/InstructionRequestHandler.java
@@ -20,7 +20,10 @@
 import java.util.concurrent.CompletionStage;
 import org.apache.beam.model.fnexecution.v1.BeamFnApi;
 
-/** Interface for any function that can handle a Fn API {@link BeamFnApi.InstructionRequest}. */
+/**
+ * Interface for any function that can handle a Fn API {@link BeamFnApi.InstructionRequest}. Any
+ * error responses will be converted to exceptionally completed futures.
+ */
 public interface InstructionRequestHandler extends AutoCloseable {
   CompletionStage<BeamFnApi.InstructionResponse> handle(BeamFnApi.InstructionRequest request);
 }
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 1ee5184..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
@@ -95,6 +95,9 @@
      *   // send all main input elements ...
      * }
      * }</pre>
+     *
+     * <p>An exception during {@link #close()} will be thrown if the bundle requests finalization or
+     * attempts to checkpoint by returning a {@link BeamFnApi.DelayedBundleApplication}.
      */
     public ActiveBundle newBundle(
         Map<String, RemoteOutputReceiver<?>> outputReceivers,
@@ -122,6 +125,47 @@
      * try (ActiveBundle bundle = SdkHarnessClient.newBundle(...)) {
      *   FnDataReceiver<InputT> inputReceiver =
      *       (FnDataReceiver) bundle.getInputReceivers().get(mainPCollectionId);
+     *   // send all main input elements ...
+     * }
+     * }</pre>
+     *
+     * <p>An exception during {@link #close()} will be thrown if the bundle requests finalization or
+     * attempts to checkpoint by returning a {@link BeamFnApi.DelayedBundleApplication}.
+     */
+    public ActiveBundle newBundle(
+        Map<String, RemoteOutputReceiver<?>> outputReceivers,
+        StateRequestHandler stateRequestHandler,
+        BundleProgressHandler progressHandler) {
+      return newBundle(
+          outputReceivers,
+          stateRequestHandler,
+          progressHandler,
+          request -> {
+            throw new UnsupportedOperationException(
+                String.format(
+                    "The %s does not have a registered bundle checkpoint handler.",
+                    ActiveBundle.class.getSimpleName()));
+          },
+          bundleId -> {
+            throw new UnsupportedOperationException(
+                String.format(
+                    "The %s does not have a registered bundle finalization handler.",
+                    ActiveBundle.class.getSimpleName()));
+          });
+    }
+
+    /**
+     * Start a new bundle for the given {@link BeamFnApi.ProcessBundleDescriptor} identifier.
+     *
+     * <p>The input channels for the returned {@link ActiveBundle} are derived from the instructions
+     * in the {@link BeamFnApi.ProcessBundleDescriptor}.
+     *
+     * <p>NOTE: It is important to {@link #close()} each bundle after all elements are emitted.
+     *
+     * <pre>{@code
+     * try (ActiveBundle bundle = SdkHarnessClient.newBundle(...)) {
+     *   FnDataReceiver<InputT> inputReceiver =
+     *       (FnDataReceiver) bundle.getInputReceivers().get(mainPCollectionId);
      *   // send all elements ...
      * }
      * }</pre>
@@ -129,7 +173,9 @@
     public ActiveBundle newBundle(
         Map<String, RemoteOutputReceiver<?>> outputReceivers,
         StateRequestHandler stateRequestHandler,
-        BundleProgressHandler progressHandler) {
+        BundleProgressHandler progressHandler,
+        BundleCheckpointHandler checkpointHandler,
+        BundleFinalizationHandler finalizationHandler) {
       String bundleId = idGenerator.getId();
 
       final CompletionStage<BeamFnApi.InstructionResponse> genericResponse =
@@ -175,7 +221,9 @@
           dataReceiversBuilder.build(),
           outputClients,
           stateDelegator.registerForProcessBundleInstructionId(bundleId, stateRequestHandler),
-          progressHandler);
+          progressHandler,
+          checkpointHandler,
+          finalizationHandler);
     }
 
     private <OutputT> InboundDataClient attachReceiver(
@@ -193,6 +241,8 @@
     private final Map<String, InboundDataClient> outputClients;
     private final StateDelegator.Registration stateRegistration;
     private final BundleProgressHandler progressHandler;
+    private final BundleCheckpointHandler checkpointHandler;
+    private final BundleFinalizationHandler finalizationHandler;
 
     private ActiveBundle(
         String bundleId,
@@ -200,13 +250,17 @@
         Map<String, CloseableFnDataReceiver> inputReceivers,
         Map<String, InboundDataClient> outputClients,
         StateDelegator.Registration stateRegistration,
-        BundleProgressHandler progressHandler) {
+        BundleProgressHandler progressHandler,
+        BundleCheckpointHandler checkpointHandler,
+        BundleFinalizationHandler finalizationHandler) {
       this.bundleId = bundleId;
       this.response = response;
       this.inputReceivers = inputReceivers;
       this.outputClients = outputClients;
       this.stateRegistration = stateRegistration;
       this.progressHandler = progressHandler;
+      this.checkpointHandler = checkpointHandler;
+      this.finalizationHandler = finalizationHandler;
     }
 
     /** Returns an id used to represent this bundle. */
@@ -256,13 +310,15 @@
           BeamFnApi.ProcessBundleResponse completedResponse = MoreFutures.get(response);
           progressHandler.onCompleted(completedResponse);
           if (completedResponse.getResidualRootsCount() > 0) {
-            throw new IllegalStateException(
-                "TODO: [BEAM-2939] residual roots in process bundle response not yet supported.");
+            checkpointHandler.onCheckpoint(completedResponse);
+          }
+          if (completedResponse.getRequiresFinalization()) {
+            finalizationHandler.requestsFinalization(bundleId);
           }
         } else {
           // TODO: [BEAM-3962] Handle aborting the bundle being processed.
           throw new IllegalStateException(
-              "Processing bundle failed, " + "TODO: [BEAM-3962] abort bundle.");
+              "Processing bundle failed, TODO: [BEAM-3962] abort bundle.");
         }
       } catch (Exception e) {
         if (exception == null) {
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 6a28735..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
@@ -31,6 +31,7 @@
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.verifyNoMoreInteractions;
+import static org.mockito.Mockito.verifyZeroInteractions;
 import static org.mockito.Mockito.when;
 
 import java.util.ArrayList;
@@ -41,6 +42,7 @@
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.ExecutionException;
 import org.apache.beam.model.fnexecution.v1.BeamFnApi;
+import org.apache.beam.model.fnexecution.v1.BeamFnApi.DelayedBundleApplication;
 import org.apache.beam.model.fnexecution.v1.BeamFnApi.InstructionResponse;
 import org.apache.beam.model.fnexecution.v1.BeamFnApi.ProcessBundleDescriptor;
 import org.apache.beam.model.fnexecution.v1.BeamFnApi.ProcessBundleResponse;
@@ -623,6 +625,104 @@
     assertThat(requests.get(1).getProcessBundle().getCacheTokensList(), is(cacheTokens));
   }
 
+  @Test
+  public void testBundleCheckpointCallback() throws Exception {
+    InboundDataClient mockOutputReceiver = mock(InboundDataClient.class);
+    CloseableFnDataReceiver mockInputSender = mock(CloseableFnDataReceiver.class);
+
+    CompletableFuture<InstructionResponse> processBundleResponseFuture = new CompletableFuture<>();
+    when(fnApiControlClient.handle(any(BeamFnApi.InstructionRequest.class)))
+        .thenReturn(createRegisterResponse())
+        .thenReturn(processBundleResponseFuture);
+
+    FullWindowedValueCoder<String> coder =
+        FullWindowedValueCoder.of(StringUtf8Coder.of(), Coder.INSTANCE);
+    BundleProcessor processor =
+        sdkHarnessClient.getProcessor(
+            descriptor,
+            Collections.singletonMap(
+                "inputPC",
+                RemoteInputDestination.of(
+                    (FullWindowedValueCoder) coder, SDK_GRPC_READ_TRANSFORM)));
+    when(dataService.receive(any(), any(), any())).thenReturn(mockOutputReceiver);
+    when(dataService.send(any(), eq(coder))).thenReturn(mockInputSender);
+
+    RemoteOutputReceiver mockRemoteOutputReceiver = mock(RemoteOutputReceiver.class);
+    BundleProgressHandler mockProgressHandler = mock(BundleProgressHandler.class);
+    BundleCheckpointHandler mockCheckpointHandler = mock(BundleCheckpointHandler.class);
+    BundleFinalizationHandler mockFinalizationHandler = mock(BundleFinalizationHandler.class);
+
+    ProcessBundleResponse response =
+        ProcessBundleResponse.newBuilder()
+            .addResidualRoots(DelayedBundleApplication.getDefaultInstance())
+            .build();
+    try (ActiveBundle activeBundle =
+        processor.newBundle(
+            ImmutableMap.of(SDK_GRPC_WRITE_TRANSFORM, mockRemoteOutputReceiver),
+            (request) -> {
+              throw new UnsupportedOperationException();
+            },
+            mockProgressHandler,
+            mockCheckpointHandler,
+            mockFinalizationHandler)) {
+      processBundleResponseFuture.complete(
+          InstructionResponse.newBuilder().setProcessBundle(response).build());
+    }
+
+    verify(mockProgressHandler).onCompleted(response);
+    verify(mockCheckpointHandler).onCheckpoint(response);
+    verifyZeroInteractions(mockFinalizationHandler);
+  }
+
+  @Test
+  public void testBundleFinalizationCallback() throws Exception {
+    InboundDataClient mockOutputReceiver = mock(InboundDataClient.class);
+    CloseableFnDataReceiver mockInputSender = mock(CloseableFnDataReceiver.class);
+
+    CompletableFuture<InstructionResponse> processBundleResponseFuture = new CompletableFuture<>();
+    when(fnApiControlClient.handle(any(BeamFnApi.InstructionRequest.class)))
+        .thenReturn(createRegisterResponse())
+        .thenReturn(processBundleResponseFuture);
+
+    FullWindowedValueCoder<String> coder =
+        FullWindowedValueCoder.of(StringUtf8Coder.of(), Coder.INSTANCE);
+    BundleProcessor processor =
+        sdkHarnessClient.getProcessor(
+            descriptor,
+            Collections.singletonMap(
+                "inputPC",
+                RemoteInputDestination.of(
+                    (FullWindowedValueCoder) coder, SDK_GRPC_READ_TRANSFORM)));
+    when(dataService.receive(any(), any(), any())).thenReturn(mockOutputReceiver);
+    when(dataService.send(any(), eq(coder))).thenReturn(mockInputSender);
+
+    RemoteOutputReceiver mockRemoteOutputReceiver = mock(RemoteOutputReceiver.class);
+    BundleProgressHandler mockProgressHandler = mock(BundleProgressHandler.class);
+    BundleCheckpointHandler mockCheckpointHandler = mock(BundleCheckpointHandler.class);
+    BundleFinalizationHandler mockFinalizationHandler = mock(BundleFinalizationHandler.class);
+
+    ProcessBundleResponse response =
+        ProcessBundleResponse.newBuilder().setRequiresFinalization(true).build();
+    String bundleId;
+    try (ActiveBundle activeBundle =
+        processor.newBundle(
+            ImmutableMap.of(SDK_GRPC_WRITE_TRANSFORM, mockRemoteOutputReceiver),
+            (request) -> {
+              throw new UnsupportedOperationException();
+            },
+            mockProgressHandler,
+            mockCheckpointHandler,
+            mockFinalizationHandler)) {
+      bundleId = activeBundle.getId();
+      processBundleResponseFuture.complete(
+          InstructionResponse.newBuilder().setProcessBundle(response).build());
+    }
+
+    verify(mockProgressHandler).onCompleted(response);
+    verify(mockFinalizationHandler).requestsFinalization(bundleId);
+    verifyZeroInteractions(mockCheckpointHandler);
+  }
+
   private CompletableFuture<InstructionResponse> createRegisterResponse() {
     return CompletableFuture.completedFuture(
         InstructionResponse.newBuilder()
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/testing/UsesTestStreamWithMultipleStages.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/UsesTestStreamWithMultipleStages.java
new file mode 100644
index 0000000..55999ce
--- /dev/null
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/UsesTestStreamWithMultipleStages.java
@@ -0,0 +1,25 @@
+/*
+ * 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.sdk.testing;
+
+/**
+ * Subcategory for {@link UsesTestStream} tests which use {@link TestStream} # across multiple
+ * stages. Some Runners do not properly support quiescence in a way that {@link TestStream} demands
+ * it.
+ */
+public interface UsesTestStreamWithMultipleStages extends UsesTestStream {}
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/testing/TestStreamTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestStreamTest.java
index 5e4cdcb..e48b6b2 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestStreamTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestStreamTest.java
@@ -17,6 +17,7 @@
  */
 package org.apache.beam.sdk.testing;
 
+import static org.apache.beam.sdk.transforms.windowing.Window.into;
 import static org.hamcrest.MatcherAssert.assertThat;
 import static org.hamcrest.Matchers.allOf;
 import static org.hamcrest.Matchers.greaterThanOrEqualTo;
@@ -28,12 +29,22 @@
 import org.apache.beam.sdk.coders.StringUtf8Coder;
 import org.apache.beam.sdk.coders.VarIntCoder;
 import org.apache.beam.sdk.coders.VarLongCoder;
+import org.apache.beam.sdk.state.StateSpec;
+import org.apache.beam.sdk.state.StateSpecs;
+import org.apache.beam.sdk.state.TimeDomain;
+import org.apache.beam.sdk.state.Timer;
+import org.apache.beam.sdk.state.TimerSpec;
+import org.apache.beam.sdk.state.TimerSpecs;
+import org.apache.beam.sdk.state.ValueState;
 import org.apache.beam.sdk.testing.TestStream.Builder;
 import org.apache.beam.sdk.transforms.Combine;
 import org.apache.beam.sdk.transforms.Count;
+import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.transforms.Flatten;
 import org.apache.beam.sdk.transforms.GroupByKey;
+import org.apache.beam.sdk.transforms.Keys;
 import org.apache.beam.sdk.transforms.MapElements;
+import org.apache.beam.sdk.transforms.ParDo;
 import org.apache.beam.sdk.transforms.Sum;
 import org.apache.beam.sdk.transforms.Values;
 import org.apache.beam.sdk.transforms.WithKeys;
@@ -44,6 +55,7 @@
 import org.apache.beam.sdk.transforms.windowing.DefaultTrigger;
 import org.apache.beam.sdk.transforms.windowing.FixedWindows;
 import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindows;
 import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
 import org.apache.beam.sdk.transforms.windowing.Never;
 import org.apache.beam.sdk.transforms.windowing.Window;
@@ -51,8 +63,10 @@
 import org.apache.beam.sdk.util.CoderUtils;
 import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionList;
 import org.apache.beam.sdk.values.TimestampedValue;
 import org.apache.beam.sdk.values.TypeDescriptors;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
 import org.joda.time.Duration;
 import org.joda.time.Instant;
 import org.junit.Rule;
@@ -263,7 +277,7 @@
     FixedWindows windows = FixedWindows.of(Duration.standardHours(6));
     PCollection<String> windowedValues =
         p.apply(stream)
-            .apply(Window.into(windows))
+            .apply(into(windows))
             .apply(WithKeys.of(1))
             .apply(GroupByKey.create())
             .apply(Values.create())
@@ -387,6 +401,74 @@
   }
 
   @Test
+  @Category({ValidatesRunner.class, UsesTestStream.class, UsesTestStreamWithMultipleStages.class})
+  public void testMultiStage() throws Exception {
+    TestStream<String> testStream =
+        TestStream.create(StringUtf8Coder.of())
+            .addElements("before") // before
+            .advanceWatermarkTo(Instant.ofEpochSecond(0)) // BEFORE
+            .addElements(TimestampedValue.of("after", Instant.ofEpochSecond(10))) // after
+            .advanceWatermarkToInfinity(); // AFTER
+
+    PCollection<String> input = p.apply(testStream);
+
+    PCollection<String> grouped =
+        input
+            .apply(Window.into(FixedWindows.of(Duration.standardSeconds(1))))
+            .apply(
+                MapElements.into(
+                        TypeDescriptors.kvs(TypeDescriptors.strings(), TypeDescriptors.strings()))
+                    .via(e -> KV.of(e, e)))
+            .apply(GroupByKey.create())
+            .apply(Keys.create())
+            .apply("Upper", MapElements.into(TypeDescriptors.strings()).via(String::toUpperCase))
+            .apply("Rewindow", Window.into(new GlobalWindows()));
+
+    PCollection<String> result =
+        PCollectionList.of(ImmutableList.of(input, grouped))
+            .apply(Flatten.pCollections())
+            .apply(
+                "Key",
+                MapElements.into(
+                        TypeDescriptors.kvs(TypeDescriptors.strings(), TypeDescriptors.strings()))
+                    .via(e -> KV.of("key", e)))
+            .apply(
+                ParDo.of(
+                    new DoFn<KV<String, String>, String>() {
+                      @StateId("seen")
+                      private final StateSpec<ValueState<String>> seenSpec =
+                          StateSpecs.value(StringUtf8Coder.of());
+
+                      @TimerId("emit")
+                      private final TimerSpec emitSpec = TimerSpecs.timer(TimeDomain.EVENT_TIME);
+
+                      @ProcessElement
+                      public void process(
+                          ProcessContext context,
+                          @StateId("seen") ValueState<String> seenState,
+                          @TimerId("emit") Timer emitTimer) {
+                        String element = context.element().getValue();
+                        if (seenState.read() == null) {
+                          seenState.write(element);
+                        } else {
+                          seenState.write(seenState.read() + "," + element);
+                        }
+                        emitTimer.set(Instant.ofEpochSecond(100));
+                      }
+
+                      @OnTimer("emit")
+                      public void onEmit(
+                          OnTimerContext context, @StateId("seen") ValueState<String> seenState) {
+                        context.output(seenState.read());
+                      }
+                    }));
+
+    PAssert.that(result).containsInAnyOrder("before,BEFORE,after,AFTER");
+
+    p.run().waitUntilFinish();
+  }
+
+  @Test
   @Category(UsesTestStreamWithProcessingTime.class)
   public void testCoder() throws Exception {
     TestStream<String> testStream =
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/bigquery/BeamBigQuerySqlDialect.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BeamBigQuerySqlDialect.java
new file mode 100644
index 0000000..ab847bb
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BeamBigQuerySqlDialect.java
@@ -0,0 +1,275 @@
+/*
+ * 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.sdk.extensions.sql.meta.provider.bigquery;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.regex.Pattern;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.avatica.util.Casing;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.config.NullCollation;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.type.RelDataType;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlCall;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlDataTypeSpec;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlDialect;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlIdentifier;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlKind;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlNode;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlOperator;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlSetOperator;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlSyntax;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlWriter;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.dialect.BigQuerySqlDialect;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.type.BasicSqlType;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
+
+// TODO(CALCITE-3381): BeamBigQuerySqlDialect can be deleted after updating vendor Calcite version.
+// Calcite v1_20_0 does not have type translation implemented, but later (unreleased) versions do.
+public class BeamBigQuerySqlDialect extends BigQuerySqlDialect {
+
+  public static final SqlDialect.Context DEFAULT_CONTEXT =
+      SqlDialect.EMPTY_CONTEXT
+          .withDatabaseProduct(SqlDialect.DatabaseProduct.BIG_QUERY)
+          .withIdentifierQuoteString("`")
+          .withNullCollation(NullCollation.LOW)
+          .withUnquotedCasing(Casing.UNCHANGED)
+          .withQuotedCasing(Casing.UNCHANGED)
+          .withCaseSensitive(false);
+
+  public static final SqlDialect DEFAULT = new BeamBigQuerySqlDialect(DEFAULT_CONTEXT);
+
+  /**
+   * An unquoted BigQuery identifier must start with a letter and be followed by zero or more
+   * letters, digits or _.
+   */
+  private static final Pattern IDENTIFIER_REGEX = Pattern.compile("[A-Za-z][A-Za-z0-9_]*");
+
+  /** List of BigQuery Specific Operators needed to form Syntactically Correct SQL. */
+  private static final SqlOperator UNION_DISTINCT =
+      new SqlSetOperator("UNION DISTINCT", SqlKind.UNION, 14, false);
+
+  private static final SqlSetOperator EXCEPT_DISTINCT =
+      new SqlSetOperator("EXCEPT DISTINCT", SqlKind.EXCEPT, 14, false);
+  private static final SqlSetOperator INTERSECT_DISTINCT =
+      new SqlSetOperator("INTERSECT DISTINCT", SqlKind.INTERSECT, 18, false);
+
+  private static final List<String> RESERVED_KEYWORDS =
+      ImmutableList.copyOf(
+          Arrays.asList(
+              "ALL",
+              "AND",
+              "ANY",
+              "ARRAY",
+              "AS",
+              "ASC",
+              "ASSERT_ROWS_MODIFIED",
+              "AT",
+              "BETWEEN",
+              "BY",
+              "CASE",
+              "CAST",
+              "COLLATE",
+              "CONTAINS",
+              "CREATE",
+              "CROSS",
+              "CUBE",
+              "CURRENT",
+              "DEFAULT",
+              "DEFINE",
+              "DESC",
+              "DISTINCT",
+              "ELSE",
+              "END",
+              "ENUM",
+              "ESCAPE",
+              "EXCEPT",
+              "EXCLUDE",
+              "EXISTS",
+              "EXTRACT",
+              "FALSE",
+              "FETCH",
+              "FOLLOWING",
+              "FOR",
+              "FROM",
+              "FULL",
+              "GROUP",
+              "GROUPING",
+              "GROUPS",
+              "HASH",
+              "HAVING",
+              "IF",
+              "IGNORE",
+              "IN",
+              "INNER",
+              "INTERSECT",
+              "INTERVAL",
+              "INTO",
+              "IS",
+              "JOIN",
+              "LATERAL",
+              "LEFT",
+              "LIKE",
+              "LIMIT",
+              "LOOKUP",
+              "MERGE",
+              "NATURAL",
+              "NEW",
+              "NO",
+              "NOT",
+              "NULL",
+              "NULLS",
+              "OF",
+              "ON",
+              "OR",
+              "ORDER",
+              "OUTER",
+              "OVER",
+              "PARTITION",
+              "PRECEDING",
+              "PROTO",
+              "RANGE",
+              "RECURSIVE",
+              "RESPECT",
+              "RIGHT",
+              "ROLLUP",
+              "ROWS",
+              "SELECT",
+              "SET",
+              "SOME",
+              "STRUCT",
+              "TABLESAMPLE",
+              "THEN",
+              "TO",
+              "TREAT",
+              "TRUE",
+              "UNBOUNDED",
+              "UNION",
+              "UNNEST",
+              "USING",
+              "WHEN",
+              "WHERE",
+              "WINDOW",
+              "WITH",
+              "WITHIN"));
+
+  public BeamBigQuerySqlDialect(Context context) {
+    super(context);
+  }
+
+  @Override
+  public String quoteIdentifier(String val) {
+    return quoteIdentifier(new StringBuilder(), val).toString();
+  }
+
+  @Override
+  public SqlNode emulateNullDirection(SqlNode node, boolean nullsFirst, boolean desc) {
+    return emulateNullDirectionWithIsNull(node, nullsFirst, desc);
+  }
+
+  @Override
+  public void unparseOffsetFetch(SqlWriter writer, SqlNode offset, SqlNode fetch) {
+    unparseFetchUsingLimit(writer, offset, fetch);
+  }
+
+  @Override
+  public void unparseCall(
+      final SqlWriter writer, final SqlCall call, final int leftPrec, final int rightPrec) {
+    switch (call.getKind()) {
+      case POSITION:
+        final SqlWriter.Frame frame = writer.startFunCall("STRPOS");
+        writer.sep(",");
+        call.operand(1).unparse(writer, leftPrec, rightPrec);
+        writer.sep(",");
+        call.operand(0).unparse(writer, leftPrec, rightPrec);
+        if (3 == call.operandCount()) {
+          throw new RuntimeException("3rd operand Not Supported for Function STRPOS in Big Query");
+        }
+        writer.endFunCall(frame);
+        break;
+      case UNION:
+        if (!((SqlSetOperator) call.getOperator()).isAll()) {
+          SqlSyntax.BINARY.unparse(writer, UNION_DISTINCT, call, leftPrec, rightPrec);
+        }
+        break;
+      case EXCEPT:
+        if (!((SqlSetOperator) call.getOperator()).isAll()) {
+          SqlSyntax.BINARY.unparse(writer, EXCEPT_DISTINCT, call, leftPrec, rightPrec);
+        }
+        break;
+      case INTERSECT:
+        if (!((SqlSetOperator) call.getOperator()).isAll()) {
+          SqlSyntax.BINARY.unparse(writer, INTERSECT_DISTINCT, call, leftPrec, rightPrec);
+        }
+        break;
+      default:
+        super.unparseCall(writer, call, leftPrec, rightPrec);
+    }
+  }
+
+  /**
+   * BigQuery data type reference: <a
+   * href="https://cloud.google.com/bigquery/docs/reference/standard-sql/data-types">Bigquery
+   * Standard SQL Data Types</a>.
+   */
+  @Override
+  public SqlNode getCastSpec(final RelDataType type) {
+    if (type instanceof BasicSqlType) {
+      switch (type.getSqlTypeName()) {
+          // BigQuery only supports INT64 for integer types.
+        case BIGINT:
+        case INTEGER:
+        case TINYINT:
+        case SMALLINT:
+          return typeFromName(type, "INT64");
+          // BigQuery only supports FLOAT64(aka. Double) for floating point types.
+        case FLOAT:
+        case DOUBLE:
+          return typeFromName(type, "FLOAT64");
+        case DECIMAL:
+          return typeFromName(type, "NUMERIC");
+        case BOOLEAN:
+          return typeFromName(type, "BOOL");
+        case CHAR:
+        case VARCHAR:
+          return typeFromName(type, "STRING");
+        case VARBINARY:
+        case BINARY:
+          return typeFromName(type, "BYTES");
+        case DATE:
+          return typeFromName(type, "DATE");
+        case TIME:
+          return typeFromName(type, "TIME");
+        case TIMESTAMP:
+          return typeFromName(type, "TIMESTAMP");
+        default:
+          break;
+      }
+    }
+    return super.getCastSpec(type);
+  }
+
+  private static SqlNode typeFromName(RelDataType type, String name) {
+    return new SqlDataTypeSpec(
+        new SqlIdentifier(name, SqlParserPos.ZERO),
+        type.getPrecision(),
+        -1,
+        null,
+        null,
+        SqlParserPos.ZERO);
+  }
+}
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BigQueryFilter.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BigQueryFilter.java
new file mode 100644
index 0000000..aaa7e41
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BigQueryFilter.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.sdk.extensions.sql.meta.provider.bigquery;
+
+import static org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlKind.AND;
+import static org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlKind.BETWEEN;
+import static org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlKind.CAST;
+import static org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlKind.COMPARISON;
+import static org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlKind.DIVIDE;
+import static org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlKind.LIKE;
+import static org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlKind.MINUS;
+import static org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlKind.MOD;
+import static org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlKind.OR;
+import static org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlKind.PLUS;
+import static org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlKind.TIMES;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.stream.Collectors;
+import org.apache.beam.repackaged.core.org.apache.commons.lang3.tuple.Pair;
+import org.apache.beam.sdk.extensions.sql.meta.BeamSqlTableFilter;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexCall;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexInputRef;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexLiteral;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexNode;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlKind;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableSet;
+
+public class BigQueryFilter implements BeamSqlTableFilter {
+  private static final ImmutableSet<SqlKind> SUPPORTED_OPS =
+      ImmutableSet.<SqlKind>builder()
+          .add(COMPARISON.toArray(new SqlKind[0]))
+          // TODO: Check what other functions are supported and add support for them (ex: trim).
+          .add(PLUS, MINUS, MOD, DIVIDE, TIMES, LIKE, BETWEEN, CAST, AND, OR)
+          .build();
+  private List<RexNode> supported;
+  private List<RexNode> unsupported;
+
+  public BigQueryFilter(List<RexNode> predicateCNF) {
+    supported = new ArrayList<>();
+    unsupported = new ArrayList<>();
+
+    for (RexNode node : predicateCNF) {
+      if (!node.getType().getSqlTypeName().equals(SqlTypeName.BOOLEAN)) {
+        throw new RuntimeException(
+            "Predicate node '"
+                + node.getClass().getSimpleName()
+                + "' should be a boolean expression, but was: "
+                + node.getType().getSqlTypeName());
+      }
+
+      if (isSupported(node).getLeft()) {
+        supported.add(node);
+      } else {
+        unsupported.add(node);
+      }
+    }
+  }
+
+  @Override
+  public List<RexNode> getNotSupported() {
+    return unsupported;
+  }
+
+  public List<RexNode> getSupported() {
+    return supported;
+  }
+
+  @Override
+  public String toString() {
+    String supStr =
+        "supported{"
+            + supported.stream().map(RexNode::toString).collect(Collectors.joining())
+            + "}";
+    String unsupStr =
+        "unsupported{"
+            + unsupported.stream().map(RexNode::toString).collect(Collectors.joining())
+            + "}";
+
+    return "[" + supStr + ", " + unsupStr + "]";
+  }
+
+  /**
+   * Check whether a {@code RexNode} is supported. As of right now BigQuery supports: 1. Complex
+   * predicates (both conjunction and disjunction). 2. Comparison between a column and a literal.
+   *
+   * <p>TODO: Check if comparison between two columns is supported. Also over a boolean field.
+   *
+   * @param node A node to check for predicate push-down support.
+   * @return A pair containing a boolean whether an expression is supported and the number of input
+   *     references used by the expression.
+   */
+  private Pair<Boolean, Integer> isSupported(RexNode node) {
+    int numberOfInputRefs = 0;
+    boolean isSupported = true;
+
+    if (node instanceof RexCall) {
+      RexCall compositeNode = (RexCall) node;
+
+      // Only support comparisons in a predicate, some sql functions such as:
+      //  CAST, TRIM? and REVERSE? should be supported as well.
+      if (!node.getKind().belongsTo(SUPPORTED_OPS)) {
+        isSupported = false;
+      } else {
+        for (RexNode operand : compositeNode.getOperands()) {
+          // All operands must be supported for a parent node to be supported.
+          Pair<Boolean, Integer> childSupported = isSupported(operand);
+          // BigQuery supports complex combinations of both conjunctions (AND) and disjunctions
+          // (OR).
+          if (!node.getKind().belongsTo(ImmutableSet.of(AND, OR))) {
+            numberOfInputRefs += childSupported.getRight();
+          }
+          // Predicate functions, where more than one field is involved are unsupported.
+          isSupported = numberOfInputRefs < 2 && childSupported.getLeft();
+        }
+      }
+    } else if (node instanceof RexInputRef) {
+      numberOfInputRefs = 1;
+    } else if (node instanceof RexLiteral) {
+      // RexLiterals are expected, but no action is needed.
+    } else {
+      throw new RuntimeException(
+          "Encountered an unexpected node type: " + node.getClass().getSimpleName());
+    }
+
+    return Pair.of(isSupported, numberOfInputRefs);
+  }
+}
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BigQueryTable.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BigQueryTable.java
index 121eab4..789d8ec 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BigQueryTable.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BigQueryTable.java
@@ -22,6 +22,7 @@
 import java.math.BigInteger;
 import java.util.Arrays;
 import java.util.List;
+import java.util.function.IntFunction;
 import java.util.stream.Collectors;
 import org.apache.beam.sdk.annotations.Experimental;
 import org.apache.beam.sdk.extensions.sql.impl.BeamTableStatistics;
@@ -47,6 +48,13 @@
 import org.apache.beam.sdk.values.POutput;
 import org.apache.beam.sdk.values.Row;
 import org.apache.beam.vendor.calcite.v1_20_0.com.google.common.annotations.VisibleForTesting;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.rel2sql.SqlImplementor;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexNode;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlIdentifier;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlNode;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -113,7 +121,7 @@
 
   @Override
   public PCollection<Row> buildIOReader(PBegin begin) {
-    return begin.apply("Read Input BQ Rows", getBigQueryReadBuilder(getSchema()));
+    return begin.apply("Read Input BQ Rows", getBigQueryTypedRead(getSchema()));
   }
 
   @Override
@@ -128,17 +136,24 @@
         FieldAccessDescriptor.withFieldNames(fieldNames).resolve(getSchema());
     final Schema newSchema = SelectHelpers.getOutputSchema(getSchema(), resolved);
 
-    TypedRead<Row> builder = getBigQueryReadBuilder(newSchema);
+    TypedRead<Row> typedRead = getBigQueryTypedRead(newSchema);
 
     if (!(filters instanceof DefaultTableFilter)) {
-      throw new RuntimeException("Unimplemented at the moment.");
+      BigQueryFilter bigQueryFilter = (BigQueryFilter) filters;
+      if (!bigQueryFilter.getSupported().isEmpty()) {
+        String rowRestriction = generateRowRestrictions(getSchema(), bigQueryFilter.getSupported());
+        if (!rowRestriction.isEmpty()) {
+          LOGGER.info("Pushing down the following filter: " + rowRestriction);
+          typedRead = typedRead.withRowRestriction(rowRestriction);
+        }
+      }
     }
 
     if (!fieldNames.isEmpty()) {
-      builder.withSelectedFields(fieldNames);
+      typedRead = typedRead.withSelectedFields(fieldNames);
     }
 
-    return begin.apply("Read Input BQ Rows with push-down", builder);
+    return begin.apply("Read Input BQ Rows with push-down", typedRead);
   }
 
   @Override
@@ -157,7 +172,43 @@
         : ProjectSupport.NONE;
   }
 
-  private TypedRead<Row> getBigQueryReadBuilder(Schema schema) {
+  @Override
+  public BeamSqlTableFilter constructFilter(List<RexNode> filter) {
+    if (method.equals(Method.DIRECT_READ)) {
+      return new BigQueryFilter(filter);
+    }
+
+    return super.constructFilter(filter);
+  }
+
+  private String generateRowRestrictions(Schema schema, List<RexNode> supported) {
+    assert !supported.isEmpty();
+    final IntFunction<SqlNode> field =
+        i -> new SqlIdentifier(schema.getField(i).getName(), SqlParserPos.ZERO);
+
+    // TODO: BigQuerySqlDialectWithTypeTranslation can be replaced with BigQuerySqlDialect after
+    // updating vendor Calcite version.
+    SqlImplementor.SimpleContext context =
+        new SqlImplementor.SimpleContext(BeamBigQuerySqlDialect.DEFAULT, field);
+
+    // Create a single SqlNode from a list of RexNodes
+    SqlNode andSqlNode = null;
+    for (RexNode node : supported) {
+      SqlNode sqlNode = context.toSql(null, node);
+      if (andSqlNode == null) {
+        andSqlNode = sqlNode;
+        continue;
+      }
+      // AND operator must have exactly 2 operands.
+      andSqlNode =
+          SqlStdOperatorTable.AND.createCall(
+              SqlParserPos.ZERO, ImmutableList.of(andSqlNode, sqlNode));
+    }
+
+    return andSqlNode.toSqlString(BeamBigQuerySqlDialect.DEFAULT).getSql();
+  }
+
+  private TypedRead<Row> getBigQueryTypedRead(Schema schema) {
     return BigQueryIO.read(
             record -> BigQueryUtils.toBeamRow(record.getRecord(), schema, conversionOptions))
         .withMethod(method)
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/bigquery/BigQueryFilterTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BigQueryFilterTest.java
new file mode 100644
index 0000000..fa84883
--- /dev/null
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BigQueryFilterTest.java
@@ -0,0 +1,121 @@
+/*
+ * 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.sdk.extensions.sql.meta.provider.bigquery;
+
+import static org.apache.beam.sdk.extensions.sql.meta.provider.test.TestTableProvider.PUSH_DOWN_OPTION;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.instanceOf;
+
+import com.alibaba.fastjson.JSON;
+import org.apache.beam.repackaged.core.org.apache.commons.lang3.tuple.Pair;
+import org.apache.beam.sdk.extensions.sql.impl.BeamSqlEnv;
+import org.apache.beam.sdk.extensions.sql.impl.rel.BeamCalcRel;
+import org.apache.beam.sdk.extensions.sql.impl.rel.BeamRelNode;
+import org.apache.beam.sdk.extensions.sql.meta.Table;
+import org.apache.beam.sdk.extensions.sql.meta.provider.test.TestTableProvider;
+import org.apache.beam.sdk.extensions.sql.meta.provider.test.TestTableProvider.PushDownOptions;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.values.Row;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+@RunWith(JUnit4.class)
+public class BigQueryFilterTest {
+  // TODO: add date, time, and datetime fields.
+  private static final Schema BASIC_SCHEMA =
+      Schema.builder()
+          .addInt32Field("unused1")
+          .addInt32Field("id")
+          .addStringField("name")
+          .addInt16Field("unused2")
+          .addBooleanField("b")
+          .build();
+
+  private BeamSqlEnv sqlEnv;
+
+  @Rule public TestPipeline pipeline = TestPipeline.create();
+
+  @Before
+  public void buildUp() {
+    TestTableProvider tableProvider = new TestTableProvider();
+    Table table = getTable("TEST", PushDownOptions.NONE);
+    tableProvider.createTable(table);
+    tableProvider.addRows(
+        table.getName(),
+        row(BASIC_SCHEMA, 100, 1, "one", (short) 100, true),
+        row(BASIC_SCHEMA, 200, 2, "two", (short) 200, false));
+
+    sqlEnv =
+        BeamSqlEnv.builder(tableProvider)
+            .setPipelineOptions(PipelineOptionsFactory.create())
+            .build();
+  }
+
+  @Test
+  public void testIsSupported() {
+    ImmutableList<Pair<String, Boolean>> sqlQueries =
+        ImmutableList.of(
+            Pair.of("select * from TEST where unused1=100", true),
+            Pair.of("select * from TEST where unused1 in (100, 200)", true),
+            Pair.of("select * from TEST where unused1+10=110", true),
+            Pair.of("select * from TEST where b", true),
+            Pair.of(
+                "select * from TEST where unused1>100 and unused1<=200 and id<>1 and (name='two' or id=2)",
+                true),
+            Pair.of("select * from TEST where unused2=200", true),
+            Pair.of("select * from TEST where name like 'o%e'", true),
+            // Functions involving more than one column are not supported yet.
+            Pair.of("select * from TEST where unused1=unused2 and id=2", false),
+            Pair.of("select * from TEST where unused1+unused2=10", false));
+
+    for (Pair<String, Boolean> query : sqlQueries) {
+      String sql = query.getLeft();
+      Boolean isSupported = query.getRight();
+
+      BeamRelNode beamRelNode = sqlEnv.parseQuery(sql);
+      assertThat(beamRelNode, instanceOf(BeamCalcRel.class));
+      BigQueryFilter filter =
+          new BigQueryFilter(((BeamCalcRel) beamRelNode).getProgram().split().right);
+
+      assertThat(
+          "Query: '" + sql + "' is expected to be " + (isSupported ? "supported." : "unsupported."),
+          filter.getNotSupported().isEmpty() == isSupported);
+    }
+  }
+
+  private static Table getTable(String name, PushDownOptions options) {
+    return Table.builder()
+        .name(name)
+        .comment(name + " table")
+        .schema(BASIC_SCHEMA)
+        .properties(
+            JSON.parseObject("{ " + PUSH_DOWN_OPTION + ": " + "\"" + options.toString() + "\" }"))
+        .type("test")
+        .build();
+  }
+
+  private static Row row(Schema schema, Object... objects) {
+    return Row.withSchema(schema).addValues(objects).build();
+  }
+}
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BigQueryReadWriteIT.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BigQueryReadWriteIT.java
index 9a14cab..bf32a48 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BigQueryReadWriteIT.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BigQueryReadWriteIT.java
@@ -17,6 +17,7 @@
  */
 package org.apache.beam.sdk.extensions.sql.meta.provider.bigquery;
 
+import static junit.framework.TestCase.assertNull;
 import static org.apache.beam.sdk.extensions.sql.meta.provider.bigquery.BigQueryTable.METHOD_PROPERTY;
 import static org.apache.beam.sdk.extensions.sql.utils.DateTimeUtils.parseTimestampWithUTCTimeZone;
 import static org.apache.beam.sdk.schemas.Schema.FieldType.BOOLEAN;
@@ -506,6 +507,82 @@
   }
 
   @Test
+  public void testSQLRead_withDirectRead_withProjectAndFilterPushDown() {
+    BeamSqlEnv sqlEnv = BeamSqlEnv.inMemory(new BigQueryTableProvider());
+
+    String createTableStatement =
+        "CREATE EXTERNAL TABLE TEST( \n"
+            + "   c_bigint BIGINT, \n"
+            + "   c_tinyint TINYINT, \n"
+            + "   c_smallint SMALLINT, \n"
+            + "   c_integer INTEGER, \n"
+            + "   c_float FLOAT, \n"
+            + "   c_double DOUBLE, \n"
+            + "   c_boolean BOOLEAN, \n"
+            + "   c_timestamp TIMESTAMP, \n"
+            + "   c_varchar VARCHAR, \n "
+            + "   c_char CHAR, \n"
+            + "   c_arr ARRAY<VARCHAR> \n"
+            + ") \n"
+            + "TYPE 'bigquery' \n"
+            + "LOCATION '"
+            + bigQueryTestingTypes.tableSpec()
+            + "' \n"
+            + "TBLPROPERTIES "
+            + "'{ "
+            + METHOD_PROPERTY
+            + ": \""
+            + Method.DIRECT_READ.toString()
+            + "\" }'";
+    sqlEnv.executeDdl(createTableStatement);
+
+    String insertStatement =
+        "INSERT INTO TEST VALUES ("
+            + "9223372036854775807, "
+            + "127, "
+            + "32767, "
+            + "2147483647, "
+            + "1.0, "
+            + "1.0, "
+            + "TRUE, "
+            + "TIMESTAMP '2018-05-28 20:17:40.123', "
+            + "'varchar', "
+            + "'char', "
+            + "ARRAY['123', '456']"
+            + ")";
+
+    sqlEnv.parseQuery(insertStatement);
+    BeamSqlRelUtils.toPCollection(pipeline, sqlEnv.parseQuery(insertStatement));
+    pipeline.run().waitUntilFinish(Duration.standardMinutes(5));
+
+    String selectTableStatement = "SELECT c_varchar, c_integer FROM TEST where c_tinyint=127";
+    BeamRelNode relNode = sqlEnv.parseQuery(selectTableStatement);
+    PCollection<Row> output = BeamSqlRelUtils.toPCollection(readPipeline, relNode);
+
+    assertThat(relNode, instanceOf(BeamCalcRel.class));
+    // Predicate should be pushed-down to IO level
+    assertNull(((BeamCalcRel) relNode).getProgram().getCondition());
+
+    assertThat(relNode.getInput(0), instanceOf(BeamIOSourceRel.class));
+    // Unused fields should not be projected by an IO
+    assertThat(
+        relNode.getInput(0).getRowType().getFieldNames(),
+        containsInAnyOrder("c_varchar", "c_integer"));
+
+    assertThat(
+        output.getSchema(),
+        equalTo(
+            Schema.builder()
+                .addNullableField("c_varchar", STRING)
+                .addNullableField("c_integer", INT32)
+                .build()));
+
+    PAssert.that(output).containsInAnyOrder(row(output.getSchema(), "varchar", 2147483647));
+    PipelineResult.State state = readPipeline.run().waitUntilFinish(Duration.standardMinutes(5));
+    assertThat(state, equalTo(State.DONE));
+  }
+
+  @Test
   public void testSQLTypes() {
     BeamSqlEnv sqlEnv = BeamSqlEnv.inMemory(new BigQueryTableProvider());
 
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/harness/src/main/java/org/apache/beam/fn/harness/state/BeamFnStateGrpcClientCache.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/BeamFnStateGrpcClientCache.java
index 1619c22..f85b3c8 100644
--- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/BeamFnStateGrpcClientCache.java
+++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/BeamFnStateGrpcClientCache.java
@@ -138,12 +138,14 @@
       public void onNext(StateResponse value) {
         LOG.debug("Received StateResponse {}", value);
         CompletableFuture<StateResponse> responseFuture = outstandingRequests.remove(value.getId());
-        if (responseFuture != null) {
-          if (value.getError().isEmpty()) {
-            responseFuture.complete(value);
-          } else {
-            responseFuture.completeExceptionally(new IllegalStateException(value.getError()));
-          }
+        if (responseFuture == null) {
+          LOG.warn("Dropped unknown StateResponse {}", value);
+          return;
+        }
+        if (value.getError().isEmpty()) {
+          responseFuture.complete(value);
+        } else {
+          responseFuture.completeExceptionally(new IllegalStateException(value.getError()));
         }
       }
 
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/internal/gcp/auth.py b/sdks/python/apache_beam/internal/gcp/auth.py
index 5142a34..8a94acf 100644
--- a/sdks/python/apache_beam/internal/gcp/auth.py
+++ b/sdks/python/apache_beam/internal/gcp/auth.py
@@ -40,6 +40,10 @@
 # information.
 executing_project = None
 
+
+_LOGGER = logging.getLogger(__name__)
+
+
 if GceAssertionCredentials is not None:
   class _GceAssertionCredentials(GceAssertionCredentials):
     """GceAssertionCredentials with retry wrapper.
@@ -101,9 +105,9 @@
       # apitools use urllib with the global timeout. Set it to 60 seconds
       # to prevent network related stuckness issues.
       if not socket.getdefaulttimeout():
-        logging.info("Setting socket default timeout to 60 seconds.")
+        _LOGGER.info("Setting socket default timeout to 60 seconds.")
         socket.setdefaulttimeout(60)
-      logging.info(
+      _LOGGER.info(
           "socket default timeout is %s seconds.", socket.getdefaulttimeout())
 
       cls._credentials = cls._get_service_credentials()
@@ -131,7 +135,7 @@
                       'Credentials.')
         return credentials
       except Exception as e:
-        logging.warning(
+        _LOGGER.warning(
             'Unable to find default credentials to use: %s\n'
             'Connecting anonymously.', e)
         return None
diff --git a/sdks/python/apache_beam/io/filebasedsink.py b/sdks/python/apache_beam/io/filebasedsink.py
index 3ff420f..76143dd 100644
--- a/sdks/python/apache_beam/io/filebasedsink.py
+++ b/sdks/python/apache_beam/io/filebasedsink.py
@@ -45,6 +45,9 @@
 __all__ = ['FileBasedSink']
 
 
+_LOGGER = logging.getLogger(__name__)
+
+
 class FileBasedSink(iobase.Sink):
   """A sink to a GCS or local files.
 
@@ -210,7 +213,7 @@
                       for file_metadata in mr.metadata_list]
 
     if dst_glob_files:
-      logging.warning('Deleting %d existing files in target path matching: %s',
+      _LOGGER.warning('Deleting %d existing files in target path matching: %s',
                       len(dst_glob_files), self.shard_name_glob_format)
       FileSystems.delete(dst_glob_files)
 
@@ -250,12 +253,12 @@
         raise BeamIOError('src and dst files do not exist. src: %s, dst: %s' % (
             src, dst))
       if not src_exists and dst_exists:
-        logging.debug('src: %s -> dst: %s already renamed, skipping', src, dst)
+        _LOGGER.debug('src: %s -> dst: %s already renamed, skipping', src, dst)
         num_skipped += 1
         continue
       if (src_exists and dst_exists and
           FileSystems.checksum(src) == FileSystems.checksum(dst)):
-        logging.debug('src: %s == dst: %s, deleting src', src, dst)
+        _LOGGER.debug('src: %s == dst: %s, deleting src', src, dst)
         delete_files.append(src)
         continue
 
@@ -284,7 +287,7 @@
                               for i in range(0, len(dst_files), chunk_size)]
 
     if num_shards_to_finalize:
-      logging.info(
+      _LOGGER.info(
           'Starting finalize_write threads with num_shards: %d (skipped: %d), '
           'batches: %d, num_threads: %d',
           num_shards_to_finalize, num_skipped, len(source_file_batch),
@@ -304,11 +307,11 @@
             raise
           for (src, dst), exception in iteritems(exp.exception_details):
             if exception:
-              logging.error(('Exception in _rename_batch. src: %s, '
+              _LOGGER.error(('Exception in _rename_batch. src: %s, '
                              'dst: %s, err: %s'), src, dst, exception)
               exceptions.append(exception)
             else:
-              logging.debug('Rename successful: %s -> %s', src, dst)
+              _LOGGER.debug('Rename successful: %s -> %s', src, dst)
           return exceptions
 
       exception_batches = util.run_using_threadpool(
@@ -324,10 +327,10 @@
       for final_name in dst_files:
         yield final_name
 
-      logging.info('Renamed %d shards in %.2f seconds.', num_shards_to_finalize,
+      _LOGGER.info('Renamed %d shards in %.2f seconds.', num_shards_to_finalize,
                    time.time() - start_time)
     else:
-      logging.warning(
+      _LOGGER.warning(
           'No shards found to finalize. num_shards: %d, skipped: %d',
           num_shards, num_skipped)
 
diff --git a/sdks/python/apache_beam/io/filebasedsink_test.py b/sdks/python/apache_beam/io/filebasedsink_test.py
index 07d0e8e..4c5ef6b 100644
--- a/sdks/python/apache_beam/io/filebasedsink_test.py
+++ b/sdks/python/apache_beam/io/filebasedsink_test.py
@@ -43,6 +43,8 @@
 from apache_beam.transforms.display import DisplayData
 from apache_beam.transforms.display_test import DisplayDataItemMatcher
 
+_LOGGER = logging.getLogger(__name__)
+
 
 # TODO: Refactor code so all io tests are using same library
 # TestCaseWithTempDirCleanup class.
@@ -247,7 +249,7 @@
           'gs://aaa/bbb', 'gs://aaa/bbb/', 'gs://aaa', 'gs://aaa/', 'gs://',
           '/')
     except ValueError:
-      logging.debug('Ignoring test since GCP module is not installed')
+      _LOGGER.debug('Ignoring test since GCP module is not installed')
 
   @mock.patch('apache_beam.io.localfilesystem.os')
   def test_temp_dir_local(self, filesystem_os_mock):
diff --git a/sdks/python/apache_beam/io/fileio.py b/sdks/python/apache_beam/io/fileio.py
index 1d9fcdd..14c35bc 100644
--- a/sdks/python/apache_beam/io/fileio.py
+++ b/sdks/python/apache_beam/io/fileio.py
@@ -114,6 +114,9 @@
            'ReadMatches']
 
 
+_LOGGER = logging.getLogger(__name__)
+
+
 class EmptyMatchTreatment(object):
   """How to treat empty matches in ``MatchAll`` and ``MatchFiles`` transforms.
 
@@ -479,7 +482,7 @@
           str,
           filesystems.FileSystems.join(temp_location,
                                        '.temp%s' % dir_uid))
-      logging.info('Added temporary directory %s', self._temp_directory.get())
+      _LOGGER.info('Added temporary directory %s', self._temp_directory.get())
 
     output = (pcoll
               | beam.ParDo(_WriteUnshardedRecordsFn(
@@ -557,7 +560,7 @@
                                             '',
                                             destination)
 
-      logging.info('Moving temporary file %s to dir: %s as %s. Res: %s',
+      _LOGGER.info('Moving temporary file %s to dir: %s as %s. Res: %s',
                    r.file_name, self.path.get(), final_file_name, r)
 
       final_full_path = filesystems.FileSystems.join(self.path.get(),
@@ -570,7 +573,7 @@
       except BeamIOError:
         # This error is not serious, because it may happen on a retry of the
         # bundle. We simply log it.
-        logging.debug('File %s failed to be copied. This may be due to a bundle'
+        _LOGGER.debug('File %s failed to be copied. This may be due to a bundle'
                       ' being retried.', r.file_name)
 
       yield FileResult(final_file_name,
@@ -580,7 +583,7 @@
                        r.pane,
                        destination)
 
-    logging.info('Cautiously removing temporary files for'
+    _LOGGER.info('Cautiously removing temporary files for'
                  ' destination %s and window %s', destination, w)
     writer_key = (destination, w)
     self._remove_temporary_files(writer_key)
@@ -592,10 +595,10 @@
       match_result = filesystems.FileSystems.match(['%s*' % prefix])
       orphaned_files = [m.path for m in match_result[0].metadata_list]
 
-      logging.debug('Deleting orphaned files: %s', orphaned_files)
+      _LOGGER.debug('Deleting orphaned files: %s', orphaned_files)
       filesystems.FileSystems.delete(orphaned_files)
     except BeamIOError as e:
-      logging.debug('Exceptions when deleting files: %s', e)
+      _LOGGER.debug('Exceptions when deleting files: %s', e)
 
 
 class _WriteShardedRecordsFn(beam.DoFn):
@@ -625,7 +628,7 @@
     sink.flush()
     writer.close()
 
-    logging.info('Writing file %s for destination %s and shard %s',
+    _LOGGER.info('Writing file %s for destination %s and shard %s',
                  full_file_name, destination, repr(shard))
 
     yield FileResult(full_file_name,
diff --git a/sdks/python/apache_beam/io/filesystemio_test.py b/sdks/python/apache_beam/io/filesystemio_test.py
index 72e7f0d..7797eb8 100644
--- a/sdks/python/apache_beam/io/filesystemio_test.py
+++ b/sdks/python/apache_beam/io/filesystemio_test.py
@@ -28,6 +28,8 @@
 
 from apache_beam.io import filesystemio
 
+_LOGGER = logging.getLogger(__name__)
+
 
 class FakeDownloader(filesystemio.Downloader):
 
@@ -206,7 +208,7 @@
 
     for buffer_size in buffer_sizes:
       for target in [self._read_and_verify, self._read_and_seek]:
-        logging.info('buffer_size=%s, target=%s' % (buffer_size, target))
+        _LOGGER.info('buffer_size=%s, target=%s' % (buffer_size, target))
         parent_conn, child_conn = multiprocessing.Pipe()
         stream = filesystemio.PipeStream(child_conn)
         success = [False]
diff --git a/sdks/python/apache_beam/io/gcp/big_query_query_to_table_it_test.py b/sdks/python/apache_beam/io/gcp/big_query_query_to_table_it_test.py
index 21de828..d357946 100644
--- a/sdks/python/apache_beam/io/gcp/big_query_query_to_table_it_test.py
+++ b/sdks/python/apache_beam/io/gcp/big_query_query_to_table_it_test.py
@@ -45,6 +45,9 @@
 except ImportError:
   pass
 
+
+_LOGGER = logging.getLogger(__name__)
+
 WAIT_UNTIL_FINISH_DURATION_MS = 15 * 60 * 1000
 
 BIG_QUERY_DATASET_ID = 'python_query_to_table_'
@@ -90,7 +93,7 @@
     try:
       self.bigquery_client.client.datasets.Delete(request)
     except HttpError:
-      logging.debug('Failed to clean up dataset %s' % self.dataset_id)
+      _LOGGER.debug('Failed to clean up dataset %s' % self.dataset_id)
 
   def _setup_new_types_env(self):
     table_schema = bigquery.TableSchema()
diff --git a/sdks/python/apache_beam/io/gcp/bigquery.py b/sdks/python/apache_beam/io/gcp/bigquery.py
index d3ac5ca..0280c61 100644
--- a/sdks/python/apache_beam/io/gcp/bigquery.py
+++ b/sdks/python/apache_beam/io/gcp/bigquery.py
@@ -273,6 +273,9 @@
     ]
 
 
+_LOGGER = logging.getLogger(__name__)
+
+
 @deprecated(since='2.11.0', current="bigquery_tools.parse_table_reference")
 def _parse_table_reference(table, dataset=None, project=None):
   return bigquery_tools.parse_table_reference(table, dataset, project)
@@ -787,7 +790,7 @@
       # and avoid the get-or-create step.
       return
 
-    logging.debug('Creating or getting table %s with schema %s.',
+    _LOGGER.debug('Creating or getting table %s with schema %s.',
                   table_reference, schema)
 
     table_schema = self.get_table_schema(schema)
@@ -833,7 +836,7 @@
     return self._flush_all_batches()
 
   def _flush_all_batches(self):
-    logging.debug('Attempting to flush to all destinations. Total buffered: %s',
+    _LOGGER.debug('Attempting to flush to all destinations. Total buffered: %s',
                   self._total_buffered_rows)
 
     return itertools.chain(*[self._flush_batch(destination)
@@ -850,7 +853,7 @@
       table_reference.projectId = vp.RuntimeValueProvider.get_value(
           'project', str, '')
 
-    logging.debug('Flushing data to %s. Total %s rows.',
+    _LOGGER.debug('Flushing data to %s. Total %s rows.',
                   destination, len(rows_and_insert_ids))
 
     rows = [r[0] for r in rows_and_insert_ids]
@@ -865,7 +868,7 @@
           insert_ids=insert_ids,
           skip_invalid_rows=True)
 
-      logging.debug("Passed: %s. Errors are %s", passed, errors)
+      _LOGGER.debug("Passed: %s. Errors are %s", passed, errors)
       failed_rows = [rows[entry.index] for entry in errors]
       should_retry = any(
           bigquery_tools.RetryStrategy.should_retry(
@@ -877,7 +880,7 @@
         break
       else:
         retry_backoff = next(self._backoff_calculator)
-        logging.info('Sleeping %s seconds before retrying insertion.',
+        _LOGGER.info('Sleeping %s seconds before retrying insertion.',
                      retry_backoff)
         time.sleep(retry_backoff)
 
diff --git a/sdks/python/apache_beam/io/gcp/bigquery_file_loads.py b/sdks/python/apache_beam/io/gcp/bigquery_file_loads.py
index cb285ea..06525cd 100644
--- a/sdks/python/apache_beam/io/gcp/bigquery_file_loads.py
+++ b/sdks/python/apache_beam/io/gcp/bigquery_file_loads.py
@@ -46,6 +46,8 @@
 from apache_beam.transforms import trigger
 from apache_beam.transforms.window import GlobalWindows
 
+_LOGGER = logging.getLogger(__name__)
+
 ONE_TERABYTE = (1 << 40)
 
 # The maximum file size for imports is 5TB. We keep our files under that.
@@ -320,7 +322,7 @@
                                copy_to_reference.datasetId,
                                copy_to_reference.tableId)))
 
-    logging.info("Triggering copy job from %s to %s",
+    _LOGGER.info("Triggering copy job from %s to %s",
                  copy_from_reference, copy_to_reference)
     job_reference = self.bq_wrapper._insert_copy_job(
         copy_to_reference.projectId,
@@ -407,7 +409,7 @@
     uid = _bq_uuid()
     job_name = '%s_%s_%s' % (
         load_job_name_prefix, destination_hash, uid)
-    logging.debug('Load job has %s files. Job name is %s.',
+    _LOGGER.debug('Load job has %s files. Job name is %s.',
                   len(files), job_name)
 
     if self.temporary_tables:
@@ -415,7 +417,7 @@
       table_reference.tableId = job_name
       yield pvalue.TaggedOutput(TriggerLoadJobs.TEMP_TABLES, table_reference)
 
-    logging.info('Triggering job %s to load data to BigQuery table %s.'
+    _LOGGER.info('Triggering job %s to load data to BigQuery table %s.'
                  'Schema: %s. Additional parameters: %s',
                  job_name, table_reference,
                  schema, additional_parameters)
@@ -519,9 +521,9 @@
                                     ref.jobId,
                                     ref.location)
 
-      logging.info("Job status: %s", job.status)
+      _LOGGER.info("Job status: %s", job.status)
       if job.status.state == 'DONE' and job.status.errorResult:
-        logging.warning("Job %s seems to have failed. Error Result: %s",
+        _LOGGER.warning("Job %s seems to have failed. Error Result: %s",
                         ref.jobId, job.status.errorResult)
         self._latest_error = job.status
         return WaitForBQJobs.FAILED
@@ -541,7 +543,7 @@
     self.bq_wrapper = bigquery_tools.BigQueryWrapper(client=self.test_client)
 
   def process(self, table_reference):
-    logging.info("Deleting table %s", table_reference)
+    _LOGGER.info("Deleting table %s", table_reference)
     table_reference = bigquery_tools.parse_table_reference(table_reference)
     self.bq_wrapper._delete_table(
         table_reference.projectId,
diff --git a/sdks/python/apache_beam/io/gcp/bigquery_file_loads_test.py b/sdks/python/apache_beam/io/gcp/bigquery_file_loads_test.py
index 035be18..bbf8d3a 100644
--- a/sdks/python/apache_beam/io/gcp/bigquery_file_loads_test.py
+++ b/sdks/python/apache_beam/io/gcp/bigquery_file_loads_test.py
@@ -58,6 +58,8 @@
   HttpError = None
 
 
+_LOGGER = logging.getLogger(__name__)
+
 _DESTINATION_ELEMENT_PAIRS = [
     # DESTINATION 1
     ('project1:dataset1.table1', '{"name":"beam", "language":"py"}'),
@@ -609,7 +611,7 @@
     self.bigquery_client = bigquery_tools.BigQueryWrapper()
     self.bigquery_client.get_or_create_dataset(self.project, self.dataset_id)
     self.output_table = "%s.output_table" % (self.dataset_id)
-    logging.info("Created dataset %s in project %s",
+    _LOGGER.info("Created dataset %s in project %s",
                  self.dataset_id, self.project)
 
   @attr('IT')
@@ -794,11 +796,11 @@
         projectId=self.project, datasetId=self.dataset_id,
         deleteContents=True)
     try:
-      logging.info("Deleting dataset %s in project %s",
+      _LOGGER.info("Deleting dataset %s in project %s",
                    self.dataset_id, self.project)
       self.bigquery_client.client.datasets.Delete(request)
     except HttpError:
-      logging.debug('Failed to clean up dataset %s in project %s',
+      _LOGGER.debug('Failed to clean up dataset %s in project %s',
                     self.dataset_id, self.project)
 
 
diff --git a/sdks/python/apache_beam/io/gcp/bigquery_read_it_test.py b/sdks/python/apache_beam/io/gcp/bigquery_read_it_test.py
index 246d2ce..ff63eda 100644
--- a/sdks/python/apache_beam/io/gcp/bigquery_read_it_test.py
+++ b/sdks/python/apache_beam/io/gcp/bigquery_read_it_test.py
@@ -46,6 +46,9 @@
 # pylint: enable=wrong-import-order, wrong-import-position
 
 
+_LOGGER = logging.getLogger(__name__)
+
+
 class BigQueryReadIntegrationTests(unittest.TestCase):
   BIG_QUERY_DATASET_ID = 'python_read_table_'
 
@@ -59,7 +62,7 @@
                                   str(int(time.time())),
                                   random.randint(0, 10000))
     self.bigquery_client.get_or_create_dataset(self.project, self.dataset_id)
-    logging.info("Created dataset %s in project %s",
+    _LOGGER.info("Created dataset %s in project %s",
                  self.dataset_id, self.project)
 
   def tearDown(self):
@@ -67,11 +70,11 @@
         projectId=self.project, datasetId=self.dataset_id,
         deleteContents=True)
     try:
-      logging.info("Deleting dataset %s in project %s",
+      _LOGGER.info("Deleting dataset %s in project %s",
                    self.dataset_id, self.project)
       self.bigquery_client.client.datasets.Delete(request)
     except HttpError:
-      logging.debug('Failed to clean up dataset %s in project %s',
+      _LOGGER.debug('Failed to clean up dataset %s in project %s',
                     self.dataset_id, self.project)
 
   def create_table(self, tablename):
diff --git a/sdks/python/apache_beam/io/gcp/bigquery_test.py b/sdks/python/apache_beam/io/gcp/bigquery_test.py
index b8c8c1c..6cf4529 100644
--- a/sdks/python/apache_beam/io/gcp/bigquery_test.py
+++ b/sdks/python/apache_beam/io/gcp/bigquery_test.py
@@ -71,6 +71,9 @@
 # pylint: enable=wrong-import-order, wrong-import-position
 
 
+_LOGGER = logging.getLogger(__name__)
+
+
 @unittest.skipIf(HttpError is None, 'GCP dependencies are not installed')
 class TestTableRowJsonCoder(unittest.TestCase):
 
@@ -579,7 +582,7 @@
     self.bigquery_client = bigquery_tools.BigQueryWrapper()
     self.bigquery_client.get_or_create_dataset(self.project, self.dataset_id)
     self.output_table = "%s.output_table" % (self.dataset_id)
-    logging.info("Created dataset %s in project %s",
+    _LOGGER.info("Created dataset %s in project %s",
                  self.dataset_id, self.project)
 
   @attr('IT')
@@ -741,11 +744,11 @@
         projectId=self.project, datasetId=self.dataset_id,
         deleteContents=True)
     try:
-      logging.info("Deleting dataset %s in project %s",
+      _LOGGER.info("Deleting dataset %s in project %s",
                    self.dataset_id, self.project)
       self.bigquery_client.client.datasets.Delete(request)
     except HttpError:
-      logging.debug('Failed to clean up dataset %s in project %s',
+      _LOGGER.debug('Failed to clean up dataset %s in project %s',
                     self.dataset_id, self.project)
 
 
diff --git a/sdks/python/apache_beam/io/gcp/bigquery_tools.py b/sdks/python/apache_beam/io/gcp/bigquery_tools.py
index 0649703..f2763ca 100644
--- a/sdks/python/apache_beam/io/gcp/bigquery_tools.py
+++ b/sdks/python/apache_beam/io/gcp/bigquery_tools.py
@@ -62,8 +62,9 @@
 # pylint: enable=wrong-import-order, wrong-import-position
 
 
-MAX_RETRIES = 3
+_LOGGER = logging.getLogger(__name__)
 
+MAX_RETRIES = 3
 
 JSON_COMPLIANCE_ERROR = 'NAN, INF and -INF values are not JSON compliant.'
 
@@ -262,7 +263,7 @@
 
     if response.statistics is None:
       # This behavior is only expected in tests
-      logging.warning(
+      _LOGGER.warning(
           "Unable to get location, missing response.statistics. Query: %s",
           query)
       return None
@@ -274,11 +275,11 @@
           table.projectId,
           table.datasetId,
           table.tableId)
-      logging.info("Using location %r from table %r referenced by query %s",
+      _LOGGER.info("Using location %r from table %r referenced by query %s",
                    location, table, query)
       return location
 
-    logging.debug("Query %s does not reference any tables.", query)
+    _LOGGER.debug("Query %s does not reference any tables.", query)
     return None
 
   @retry.with_exponential_backoff(
@@ -309,9 +310,9 @@
         )
     )
 
-    logging.info("Inserting job request: %s", request)
+    _LOGGER.info("Inserting job request: %s", request)
     response = self.client.jobs.Insert(request)
-    logging.info("Response was %s", response)
+    _LOGGER.info("Response was %s", response)
     return response.jobReference
 
   @retry.with_exponential_backoff(
@@ -442,7 +443,7 @@
     request = bigquery.BigqueryTablesInsertRequest(
         projectId=project_id, datasetId=dataset_id, table=table)
     response = self.client.tables.Insert(request)
-    logging.debug("Created the table with id %s", table_id)
+    _LOGGER.debug("Created the table with id %s", table_id)
     # The response is a bigquery.Table instance.
     return response
 
@@ -491,7 +492,7 @@
       self.client.tables.Delete(request)
     except HttpError as exn:
       if exn.status_code == 404:
-        logging.warning('Table %s:%s.%s does not exist', project_id,
+        _LOGGER.warning('Table %s:%s.%s does not exist', project_id,
                         dataset_id, table_id)
         return
       else:
@@ -508,7 +509,7 @@
       self.client.datasets.Delete(request)
     except HttpError as exn:
       if exn.status_code == 404:
-        logging.warning('Dataset %s:%s does not exist', project_id,
+        _LOGGER.warning('Dataset %s:%s does not exist', project_id,
                         dataset_id)
         return
       else:
@@ -537,7 +538,7 @@
             % (project_id, dataset_id))
     except HttpError as exn:
       if exn.status_code == 404:
-        logging.warning(
+        _LOGGER.warning(
             'Dataset %s:%s does not exist so we will create it as temporary '
             'with location=%s',
             project_id, dataset_id, location)
@@ -555,7 +556,7 @@
           projectId=project_id, datasetId=temp_table.datasetId))
     except HttpError as exn:
       if exn.status_code == 404:
-        logging.warning('Dataset %s:%s does not exist', project_id,
+        _LOGGER.warning('Dataset %s:%s does not exist', project_id,
                         temp_table.datasetId)
         return
       else:
@@ -669,12 +670,12 @@
             additional_parameters=additional_create_parameters)
       except HttpError as exn:
         if exn.status_code == 409:
-          logging.debug('Skipping Creation. Table %s:%s.%s already exists.'
+          _LOGGER.debug('Skipping Creation. Table %s:%s.%s already exists.'
                         % (project_id, dataset_id, table_id))
           created_table = self.get_table(project_id, dataset_id, table_id)
         else:
           raise
-      logging.info('Created table %s.%s.%s with schema %s. '
+      _LOGGER.info('Created table %s.%s.%s with schema %s. '
                    'Result: %s.',
                    project_id, dataset_id, table_id,
                    schema or found_table.schema,
@@ -684,7 +685,7 @@
       if write_disposition == BigQueryDisposition.WRITE_TRUNCATE:
         # BigQuery can route data to the old table for 2 mins max so wait
         # that much time before creating the table and writing it
-        logging.warning('Sleeping for 150 seconds before the write as ' +
+        _LOGGER.warning('Sleeping for 150 seconds before the write as ' +
                         'BigQuery inserts can be routed to deleted table ' +
                         'for 2 mins after the delete and create.')
         # TODO(BEAM-2673): Remove this sleep by migrating to load api
@@ -713,7 +714,7 @@
         # request not for the actual execution of the query in the service.  If
         # the request times out we keep trying. This situation is quite possible
         # if the query will return a large number of rows.
-        logging.info('Waiting on response from query: %s ...', query)
+        _LOGGER.info('Waiting on response from query: %s ...', query)
         time.sleep(1.0)
         continue
       # We got some results. The last page is signalled by a missing pageToken.
@@ -975,7 +976,7 @@
 
   def _flush_rows_buffer(self):
     if self.rows_buffer:
-      logging.info('Writing %d rows to %s:%s.%s table.', len(self.rows_buffer),
+      _LOGGER.info('Writing %d rows to %s:%s.%s table.', len(self.rows_buffer),
                    self.project_id, self.dataset_id, self.table_id)
       passed, errors = self.client.insert_rows(
           project_id=self.project_id, dataset_id=self.dataset_id,
diff --git a/sdks/python/apache_beam/io/gcp/bigquery_write_it_test.py b/sdks/python/apache_beam/io/gcp/bigquery_write_it_test.py
index 3658b9c..ae56e35 100644
--- a/sdks/python/apache_beam/io/gcp/bigquery_write_it_test.py
+++ b/sdks/python/apache_beam/io/gcp/bigquery_write_it_test.py
@@ -48,6 +48,9 @@
 # pylint: enable=wrong-import-order, wrong-import-position
 
 
+_LOGGER = logging.getLogger(__name__)
+
+
 class BigQueryWriteIntegrationTests(unittest.TestCase):
   BIG_QUERY_DATASET_ID = 'python_write_to_table_'
 
@@ -61,7 +64,7 @@
                                   str(int(time.time())),
                                   random.randint(0, 10000))
     self.bigquery_client.get_or_create_dataset(self.project, self.dataset_id)
-    logging.info("Created dataset %s in project %s",
+    _LOGGER.info("Created dataset %s in project %s",
                  self.dataset_id, self.project)
 
   def tearDown(self):
@@ -69,11 +72,11 @@
         projectId=self.project, datasetId=self.dataset_id,
         deleteContents=True)
     try:
-      logging.info("Deleting dataset %s in project %s",
+      _LOGGER.info("Deleting dataset %s in project %s",
                    self.dataset_id, self.project)
       self.bigquery_client.client.datasets.Delete(request)
     except HttpError:
-      logging.debug('Failed to clean up dataset %s in project %s',
+      _LOGGER.debug('Failed to clean up dataset %s in project %s',
                     self.dataset_id, self.project)
 
   def create_table(self, table_name):
diff --git a/sdks/python/apache_beam/io/gcp/datastore/v1/datastoreio.py b/sdks/python/apache_beam/io/gcp/datastore/v1/datastoreio.py
index 3d32611..9af7674 100644
--- a/sdks/python/apache_beam/io/gcp/datastore/v1/datastoreio.py
+++ b/sdks/python/apache_beam/io/gcp/datastore/v1/datastoreio.py
@@ -45,12 +45,15 @@
 from apache_beam.transforms import PTransform
 from apache_beam.transforms.util import Values
 
+_LOGGER = logging.getLogger(__name__)
+
+
 # Protect against environments where datastore library is not available.
 # pylint: disable=wrong-import-order, wrong-import-position
 try:
   from google.cloud.proto.datastore.v1 import datastore_pb2
   from googledatastore import helper as datastore_helper
-  logging.warning(
+  _LOGGER.warning(
       'Using deprecated Datastore client.\n'
       'This client will be removed in Beam 3.0 (next Beam major release).\n'
       'Please migrate to apache_beam.io.gcp.datastore.v1new.datastoreio.')
@@ -125,7 +128,7 @@
           'Google Cloud IO not available, '
           'please install apache_beam[gcp]')
 
-    logging.warning('datastoreio read transform is experimental.')
+    _LOGGER.warning('datastoreio read transform is experimental.')
     super(ReadFromDatastore, self).__init__()
 
     if not project:
@@ -213,13 +216,13 @@
       else:
         estimated_num_splits = self._num_splits
 
-      logging.info("Splitting the query into %d splits", estimated_num_splits)
+      _LOGGER.info("Splitting the query into %d splits", estimated_num_splits)
       try:
         query_splits = query_splitter.get_splits(
             self._datastore, query, estimated_num_splits,
             helper.make_partition(self._project, self._datastore_namespace))
       except Exception:
-        logging.warning("Unable to parallelize the given query: %s", query,
+        _LOGGER.warning("Unable to parallelize the given query: %s", query,
                         exc_info=True)
         query_splits = [query]
 
@@ -296,7 +299,7 @@
     kind = query.kind[0].name
     latest_timestamp = ReadFromDatastore.query_latest_statistics_timestamp(
         project, namespace, datastore)
-    logging.info('Latest stats timestamp for kind %s is %s',
+    _LOGGER.info('Latest stats timestamp for kind %s is %s',
                  kind, latest_timestamp)
 
     kind_stats_query = (
@@ -316,13 +319,13 @@
     try:
       estimated_size_bytes = ReadFromDatastore.get_estimated_size_bytes(
           project, namespace, query, datastore)
-      logging.info('Estimated size bytes for query: %s', estimated_size_bytes)
+      _LOGGER.info('Estimated size bytes for query: %s', estimated_size_bytes)
       num_splits = int(min(ReadFromDatastore._NUM_QUERY_SPLITS_MAX, round(
           (float(estimated_size_bytes) /
            ReadFromDatastore._DEFAULT_BUNDLE_SIZE_BYTES))))
 
     except Exception as e:
-      logging.warning('Failed to fetch estimated size bytes: %s', e)
+      _LOGGER.warning('Failed to fetch estimated size bytes: %s', e)
       # Fallback in case estimated size is unavailable.
       num_splits = ReadFromDatastore._NUM_QUERY_SPLITS_MIN
 
@@ -346,7 +349,7 @@
      """
     self._project = project
     self._mutation_fn = mutation_fn
-    logging.warning('datastoreio write transform is experimental.')
+    _LOGGER.warning('datastoreio write transform is experimental.')
 
   def expand(self, pcoll):
     return (pcoll
@@ -424,7 +427,7 @@
           self._datastore, self._project, self._mutations,
           self._throttler, self._update_rpc_stats,
           throttle_delay=util.WRITE_BATCH_TARGET_LATENCY_MS//1000)
-      logging.debug("Successfully wrote %d mutations in %dms.",
+      _LOGGER.debug("Successfully wrote %d mutations in %dms.",
                     len(self._mutations), latency_ms)
 
       if not self._fixed_batch_size:
diff --git a/sdks/python/apache_beam/io/gcp/datastore/v1/helper.py b/sdks/python/apache_beam/io/gcp/datastore/v1/helper.py
index a2bc521..4ea2898 100644
--- a/sdks/python/apache_beam/io/gcp/datastore/v1/helper.py
+++ b/sdks/python/apache_beam/io/gcp/datastore/v1/helper.py
@@ -54,6 +54,9 @@
 # pylint: enable=ungrouped-imports
 
 
+_LOGGER = logging.getLogger(__name__)
+
+
 def key_comparator(k1, k2):
   """A comparator for Datastore keys.
 
@@ -216,7 +219,7 @@
   def commit(request):
     # Client-side throttling.
     while throttler.throttle_request(time.time()*1000):
-      logging.info("Delaying request for %ds due to previous failures",
+      _LOGGER.info("Delaying request for %ds due to previous failures",
                    throttle_delay)
       time.sleep(throttle_delay)
       rpc_stats_callback(throttled_secs=throttle_delay)
diff --git a/sdks/python/apache_beam/io/gcp/datastore/v1new/datastore_write_it_pipeline.py b/sdks/python/apache_beam/io/gcp/datastore/v1new/datastore_write_it_pipeline.py
index f5b1157..efe80c8 100644
--- a/sdks/python/apache_beam/io/gcp/datastore/v1new/datastore_write_it_pipeline.py
+++ b/sdks/python/apache_beam/io/gcp/datastore/v1new/datastore_write_it_pipeline.py
@@ -47,6 +47,8 @@
 from apache_beam.testing.util import assert_that
 from apache_beam.testing.util import equal_to
 
+_LOGGER = logging.getLogger(__name__)
+
 
 def new_pipeline_with_job_name(pipeline_options, job_name, suffix):
   """Create a pipeline with the given job_name and a suffix."""
@@ -108,7 +110,7 @@
   # Pipeline 1: Create and write the specified number of Entities to the
   # Cloud Datastore.
   ancestor_key = Key([kind, str(uuid.uuid4())], project=project)
-  logging.info('Writing %s entities to %s', num_entities, project)
+  _LOGGER.info('Writing %s entities to %s', num_entities, project)
   p = new_pipeline_with_job_name(pipeline_options, job_name, '-write')
   _ = (p
        | 'Input' >> beam.Create(list(range(num_entities)))
@@ -121,7 +123,7 @@
   # Optional Pipeline 2: If a read limit was provided, read it and confirm
   # that the expected entities were read.
   if known_args.limit is not None:
-    logging.info('Querying a limited set of %s entities and verifying count.',
+    _LOGGER.info('Querying a limited set of %s entities and verifying count.',
                  known_args.limit)
     p = new_pipeline_with_job_name(pipeline_options, job_name, '-verify-limit')
     query.limit = known_args.limit
@@ -134,7 +136,7 @@
     query.limit = None
 
   # Pipeline 3: Query the written Entities and verify result.
-  logging.info('Querying entities, asserting they match.')
+  _LOGGER.info('Querying entities, asserting they match.')
   p = new_pipeline_with_job_name(pipeline_options, job_name, '-verify')
   entities = p | 'read from datastore' >> ReadFromDatastore(query)
 
@@ -145,7 +147,7 @@
   p.run()
 
   # Pipeline 4: Delete Entities.
-  logging.info('Deleting entities.')
+  _LOGGER.info('Deleting entities.')
   p = new_pipeline_with_job_name(pipeline_options, job_name, '-delete')
   entities = p | 'read from datastore' >> ReadFromDatastore(query)
   _ = (entities
@@ -155,7 +157,7 @@
   p.run()
 
   # Pipeline 5: Query the written Entities, verify no results.
-  logging.info('Querying for the entities to make sure there are none present.')
+  _LOGGER.info('Querying for the entities to make sure there are none present.')
   p = new_pipeline_with_job_name(pipeline_options, job_name, '-verify-deleted')
   entities = p | 'read from datastore' >> ReadFromDatastore(query)
 
diff --git a/sdks/python/apache_beam/io/gcp/datastore/v1new/datastoreio.py b/sdks/python/apache_beam/io/gcp/datastore/v1new/datastoreio.py
index f71a801..a70ea95 100644
--- a/sdks/python/apache_beam/io/gcp/datastore/v1new/datastoreio.py
+++ b/sdks/python/apache_beam/io/gcp/datastore/v1new/datastoreio.py
@@ -51,6 +51,9 @@
 __all__ = ['ReadFromDatastore', 'WriteToDatastore', 'DeleteFromDatastore']
 
 
+_LOGGER = logging.getLogger(__name__)
+
+
 @typehints.with_output_types(types.Entity)
 class ReadFromDatastore(PTransform):
   """A ``PTransform`` for querying Google Cloud Datastore.
@@ -173,11 +176,11 @@
         else:
           estimated_num_splits = self._num_splits
 
-        logging.info("Splitting the query into %d splits", estimated_num_splits)
+        _LOGGER.info("Splitting the query into %d splits", estimated_num_splits)
         query_splits = query_splitter.get_splits(
             client, query, estimated_num_splits)
       except query_splitter.QuerySplitterError:
-        logging.info("Unable to parallelize the given query: %s", query,
+        _LOGGER.info("Unable to parallelize the given query: %s", query,
                      exc_info=True)
         query_splits = [query]
 
@@ -219,7 +222,7 @@
       latest_timestamp = (
           ReadFromDatastore._SplitQueryFn
           .query_latest_statistics_timestamp(client))
-      logging.info('Latest stats timestamp for kind %s is %s',
+      _LOGGER.info('Latest stats timestamp for kind %s is %s',
                    kind_name, latest_timestamp)
 
       if client.namespace is None:
@@ -243,12 +246,12 @@
         estimated_size_bytes = (
             ReadFromDatastore._SplitQueryFn
             .get_estimated_size_bytes(client, query))
-        logging.info('Estimated size bytes for query: %s', estimated_size_bytes)
+        _LOGGER.info('Estimated size bytes for query: %s', estimated_size_bytes)
         num_splits = int(min(ReadFromDatastore._NUM_QUERY_SPLITS_MAX, round(
             (float(estimated_size_bytes) /
              ReadFromDatastore._DEFAULT_BUNDLE_SIZE_BYTES))))
       except Exception as e:
-        logging.warning('Failed to fetch estimated size bytes: %s', e)
+        _LOGGER.warning('Failed to fetch estimated size bytes: %s', e)
         # Fallback in case estimated size is unavailable.
         num_splits = ReadFromDatastore._NUM_QUERY_SPLITS_MIN
 
@@ -360,7 +363,7 @@
       """
       # Client-side throttling.
       while throttler.throttle_request(time.time() * 1000):
-        logging.info("Delaying request for %ds due to previous failures",
+        _LOGGER.info("Delaying request for %ds due to previous failures",
                      throttle_delay)
         time.sleep(throttle_delay)
         rpc_stats_callback(throttled_secs=throttle_delay)
@@ -412,7 +415,7 @@
           self._throttler,
           rpc_stats_callback=self._update_rpc_stats,
           throttle_delay=util.WRITE_BATCH_TARGET_LATENCY_MS // 1000)
-      logging.debug("Successfully wrote %d mutations in %dms.",
+      _LOGGER.debug("Successfully wrote %d mutations in %dms.",
                     len(self._batch.mutations), latency_ms)
 
       now = time.time() * 1000
diff --git a/sdks/python/apache_beam/io/gcp/datastore/v1new/types_test.py b/sdks/python/apache_beam/io/gcp/datastore/v1new/types_test.py
index 21633d9..c3bf8ef 100644
--- a/sdks/python/apache_beam/io/gcp/datastore/v1new/types_test.py
+++ b/sdks/python/apache_beam/io/gcp/datastore/v1new/types_test.py
@@ -40,6 +40,9 @@
   client = None
 
 
+_LOGGER = logging.getLogger(__name__)
+
+
 @unittest.skipIf(client is None, 'Datastore dependencies are not installed')
 class TypesTest(unittest.TestCase):
   _PROJECT = 'project'
@@ -168,7 +171,7 @@
     self.assertEqual(order, cq.order)
     self.assertEqual(distinct_on, cq.distinct_on)
 
-    logging.info('query: %s', q)  # Test __repr__()
+    _LOGGER.info('query: %s', q)  # Test __repr__()
 
   def testValueProviderFilters(self):
     self.vp_filters = [
@@ -193,7 +196,7 @@
       cq = q._to_client_query(self._test_client)
       self.assertEqual(exp_filter, cq.filters)
 
-      logging.info('query: %s', q)  # Test __repr__()
+      _LOGGER.info('query: %s', q)  # Test __repr__()
 
   def testQueryEmptyNamespace(self):
     # Test that we can pass a namespace of None.
diff --git a/sdks/python/apache_beam/io/gcp/datastore_write_it_pipeline.py b/sdks/python/apache_beam/io/gcp/datastore_write_it_pipeline.py
index 67e375f..2d0be8f 100644
--- a/sdks/python/apache_beam/io/gcp/datastore_write_it_pipeline.py
+++ b/sdks/python/apache_beam/io/gcp/datastore_write_it_pipeline.py
@@ -57,6 +57,9 @@
 # pylint: enable=ungrouped-imports
 
 
+_LOGGER = logging.getLogger(__name__)
+
+
 def new_pipeline_with_job_name(pipeline_options, job_name, suffix):
   """Create a pipeline with the given job_name and a suffix."""
   gcp_options = pipeline_options.view_as(GoogleCloudOptions)
@@ -137,7 +140,7 @@
 
   # Pipeline 1: Create and write the specified number of Entities to the
   # Cloud Datastore.
-  logging.info('Writing %s entities to %s', num_entities, project)
+  _LOGGER.info('Writing %s entities to %s', num_entities, project)
   p = new_pipeline_with_job_name(pipeline_options, job_name, '-write')
 
   # pylint: disable=expression-not-assigned
@@ -152,7 +155,7 @@
   # Optional Pipeline 2: If a read limit was provided, read it and confirm
   # that the expected entities were read.
   if known_args.limit is not None:
-    logging.info('Querying a limited set of %s entities and verifying count.',
+    _LOGGER.info('Querying a limited set of %s entities and verifying count.',
                  known_args.limit)
     p = new_pipeline_with_job_name(pipeline_options, job_name, '-verify-limit')
     query_with_limit = query_pb2.Query()
@@ -167,7 +170,7 @@
     p.run()
 
   # Pipeline 3: Query the written Entities and verify result.
-  logging.info('Querying entities, asserting they match.')
+  _LOGGER.info('Querying entities, asserting they match.')
   p = new_pipeline_with_job_name(pipeline_options, job_name, '-verify')
   entities = p | 'read from datastore' >> ReadFromDatastore(project, query)
 
@@ -178,7 +181,7 @@
   p.run()
 
   # Pipeline 4: Delete Entities.
-  logging.info('Deleting entities.')
+  _LOGGER.info('Deleting entities.')
   p = new_pipeline_with_job_name(pipeline_options, job_name, '-delete')
   entities = p | 'read from datastore' >> ReadFromDatastore(project, query)
   # pylint: disable=expression-not-assigned
@@ -189,7 +192,7 @@
   p.run()
 
   # Pipeline 5: Query the written Entities, verify no results.
-  logging.info('Querying for the entities to make sure there are none present.')
+  _LOGGER.info('Querying for the entities to make sure there are none present.')
   p = new_pipeline_with_job_name(pipeline_options, job_name, '-verify-deleted')
   entities = p | 'read from datastore' >> ReadFromDatastore(project, query)
 
diff --git a/sdks/python/apache_beam/io/gcp/gcsio.py b/sdks/python/apache_beam/io/gcp/gcsio.py
index dfdc29d..c1e0314 100644
--- a/sdks/python/apache_beam/io/gcp/gcsio.py
+++ b/sdks/python/apache_beam/io/gcp/gcsio.py
@@ -44,6 +44,9 @@
 __all__ = ['GcsIO']
 
 
+_LOGGER = logging.getLogger(__name__)
+
+
 # Issue a friendlier error message if the storage library is not available.
 # TODO(silviuc): Remove this guard when storage is available everywhere.
 try:
@@ -250,7 +253,7 @@
         maxBytesRewrittenPerCall=max_bytes_rewritten_per_call)
     response = self.client.objects.Rewrite(request)
     while not response.done:
-      logging.debug(
+      _LOGGER.debug(
           'Rewrite progress: %d of %d bytes, %s to %s',
           response.totalBytesRewritten, response.objectSize, src, dest)
       request.rewriteToken = response.rewriteToken
@@ -258,7 +261,7 @@
       if self._rewrite_cb is not None:
         self._rewrite_cb(response)
 
-    logging.debug('Rewrite done: %s to %s', src, dest)
+    _LOGGER.debug('Rewrite done: %s to %s', src, dest)
 
   # We intentionally do not decorate this method with a retry, as retrying is
   # handled in BatchApiRequest.Execute().
@@ -320,12 +323,12 @@
                 GcsIOError(errno.ENOENT, 'Source file not found: %s' % src))
           pair_to_status[pair] = exception
         elif not response.done:
-          logging.debug(
+          _LOGGER.debug(
               'Rewrite progress: %d of %d bytes, %s to %s',
               response.totalBytesRewritten, response.objectSize, src, dest)
           pair_to_request[pair].rewriteToken = response.rewriteToken
         else:
-          logging.debug('Rewrite done: %s to %s', src, dest)
+          _LOGGER.debug('Rewrite done: %s to %s', src, dest)
           pair_to_status[pair] = None
 
     return [(pair[0], pair[1], pair_to_status[pair]) for pair in src_dest_pairs]
@@ -458,7 +461,7 @@
     file_sizes = {}
     counter = 0
     start_time = time.time()
-    logging.info("Starting the size estimation of the input")
+    _LOGGER.info("Starting the size estimation of the input")
     while True:
       response = self.client.objects.List(request)
       for item in response.items:
@@ -466,12 +469,12 @@
         file_sizes[file_name] = item.size
         counter += 1
         if counter % 10000 == 0:
-          logging.info("Finished computing size of: %s files", len(file_sizes))
+          _LOGGER.info("Finished computing size of: %s files", len(file_sizes))
       if response.nextPageToken:
         request.pageToken = response.nextPageToken
       else:
         break
-    logging.info("Finished listing %s files in %s seconds.",
+    _LOGGER.info("Finished listing %s files in %s seconds.",
                  counter, time.time() - start_time)
     return file_sizes
 
@@ -492,7 +495,7 @@
       if http_error.status_code == 404:
         raise IOError(errno.ENOENT, 'Not found: %s' % self._path)
       else:
-        logging.error('HTTP error while requesting file %s: %s', self._path,
+        _LOGGER.error('HTTP error while requesting file %s: %s', self._path,
                       http_error)
         raise
     self._size = metadata.size
@@ -564,7 +567,7 @@
     try:
       self._client.objects.Insert(self._insert_request, upload=self._upload)
     except Exception as e:  # pylint: disable=broad-except
-      logging.error('Error in _start_upload while inserting file %s: %s',
+      _LOGGER.error('Error in _start_upload while inserting file %s: %s',
                     self._path, traceback.format_exc())
       self._upload_thread.last_error = e
     finally:
diff --git a/sdks/python/apache_beam/io/gcp/gcsio_overrides.py b/sdks/python/apache_beam/io/gcp/gcsio_overrides.py
index a5fc749..1be587d 100644
--- a/sdks/python/apache_beam/io/gcp/gcsio_overrides.py
+++ b/sdks/python/apache_beam/io/gcp/gcsio_overrides.py
@@ -26,6 +26,8 @@
 from apitools.base.py import http_wrapper
 from apitools.base.py import util
 
+_LOGGER = logging.getLogger(__name__)
+
 
 class GcsIOOverrides(object):
   """Functions for overriding Google Cloud Storage I/O client."""
@@ -37,13 +39,13 @@
     # handling GCS download throttling errors (BEAM-7424)
     if (isinstance(retry_args.exc, exceptions.BadStatusCodeError) and
         retry_args.exc.status_code == http_wrapper.TOO_MANY_REQUESTS):
-      logging.debug(
+      _LOGGER.debug(
           'Caught GCS quota error (%s), retrying.', retry_args.exc.status_code)
     else:
       return http_wrapper.HandleExceptionsAndRebuildHttpConnections(retry_args)
 
     http_wrapper.RebuildHttpConnections(retry_args.http)
-    logging.debug('Retrying request to url %s after exception %s',
+    _LOGGER.debug('Retrying request to url %s after exception %s',
                   retry_args.http_request.url, retry_args.exc)
     sleep_seconds = util.CalculateWaitForRetry(
         retry_args.num_retries, max_wait=retry_args.max_retry_wait)
diff --git a/sdks/python/apache_beam/io/gcp/tests/bigquery_matcher.py b/sdks/python/apache_beam/io/gcp/tests/bigquery_matcher.py
index c3394a1..2e50763 100644
--- a/sdks/python/apache_beam/io/gcp/tests/bigquery_matcher.py
+++ b/sdks/python/apache_beam/io/gcp/tests/bigquery_matcher.py
@@ -45,6 +45,8 @@
 
 MAX_RETRIES = 5
 
+_LOGGER = logging.getLogger(__name__)
+
 
 def retry_on_http_and_value_error(exception):
   """Filter allowing retries on Bigquery errors and value error."""
@@ -83,10 +85,10 @@
   def _matches(self, _):
     if self.checksum is None:
       response = self._query_with_retry()
-      logging.info('Read from given query (%s), total rows %d',
+      _LOGGER.info('Read from given query (%s), total rows %d',
                    self.query, len(response))
       self.checksum = compute_hash(response)
-      logging.info('Generate checksum: %s', self.checksum)
+      _LOGGER.info('Generate checksum: %s', self.checksum)
 
     return self.checksum == self.expected_checksum
 
@@ -95,7 +97,7 @@
       retry_filter=retry_on_http_and_value_error)
   def _query_with_retry(self):
     """Run Bigquery query with retry if got error http response"""
-    logging.info('Attempting to perform query %s to BQ', self.query)
+    _LOGGER.info('Attempting to perform query %s to BQ', self.query)
     # Create client here since it throws an exception if pickled.
     bigquery_client = bigquery.Client(self.project)
     query_job = bigquery_client.query(self.query)
@@ -134,7 +136,7 @@
   def _matches(self, _):
     if self.actual_data is None:
       self.actual_data = self._get_query_result()
-      logging.info('Result of query is: %r', self.actual_data)
+      _LOGGER.info('Result of query is: %r', self.actual_data)
 
     try:
       equal_to(self.expected_data)(self.actual_data)
@@ -179,7 +181,7 @@
       response = self._query_with_retry()
       if len(response) >= len(self.expected_data):
         return response
-      logging.debug('Query result contains %d rows' % len(response))
+      _LOGGER.debug('Query result contains %d rows' % len(response))
       time.sleep(1)
     if sys.version_info >= (3,):
       raise TimeoutError('Timeout exceeded for matcher.') # noqa: F821
@@ -207,13 +209,13 @@
     return bigquery_wrapper.get_table(self.project, self.dataset, self.table)
 
   def _matches(self, _):
-    logging.info('Start verify Bigquery table properties.')
+    _LOGGER.info('Start verify Bigquery table properties.')
     # Run query
     bigquery_wrapper = bigquery_tools.BigQueryWrapper()
 
     self.actual_table = self._get_table_with_retry(bigquery_wrapper)
 
-    logging.info('Table proto is %s', self.actual_table)
+    _LOGGER.info('Table proto is %s', self.actual_table)
 
     return all(
         self._match_property(v, self._get_or_none(self.actual_table, k))
@@ -231,7 +233,7 @@
 
   @staticmethod
   def _match_property(expected, actual):
-    logging.info("Matching %s to %s", expected, actual)
+    _LOGGER.info("Matching %s to %s", expected, actual)
     if isinstance(expected, dict):
       return all(
           BigQueryTableMatcher._match_property(
diff --git a/sdks/python/apache_beam/io/gcp/tests/pubsub_matcher.py b/sdks/python/apache_beam/io/gcp/tests/pubsub_matcher.py
index 7a0b5c8..af94e02 100644
--- a/sdks/python/apache_beam/io/gcp/tests/pubsub_matcher.py
+++ b/sdks/python/apache_beam/io/gcp/tests/pubsub_matcher.py
@@ -39,6 +39,8 @@
 DEFAULT_TIMEOUT = 5 * 60
 MAX_MESSAGES_IN_ONE_PULL = 50
 
+_LOGGER = logging.getLogger(__name__)
+
 
 class PubSubMessageMatcher(BaseMatcher):
   """Matcher that verifies messages from given subscription.
@@ -123,7 +125,7 @@
       time.sleep(1)
 
     if time.time() - start_time > timeout:
-      logging.error('Timeout after %d sec. Received %d messages from %s.',
+      _LOGGER.error('Timeout after %d sec. Received %d messages from %s.',
                     timeout, len(total_messages), self.sub_name)
     return total_messages
 
diff --git a/sdks/python/apache_beam/io/gcp/tests/utils.py b/sdks/python/apache_beam/io/gcp/tests/utils.py
index 4ed9af3..dbf8ac9 100644
--- a/sdks/python/apache_beam/io/gcp/tests/utils.py
+++ b/sdks/python/apache_beam/io/gcp/tests/utils.py
@@ -37,6 +37,9 @@
   bigquery = None
 
 
+_LOGGER = logging.getLogger(__name__)
+
+
 class GcpTestIOError(retry.PermanentException):
   """Basic GCP IO error for testing. Function that raises this error should
   not be retried."""
@@ -93,7 +96,7 @@
     dataset_id: Name of the dataset where table is.
     table_id: Name of the table.
   """
-  logging.info('Clean up a BigQuery table with project: %s, dataset: %s, '
+  _LOGGER.info('Clean up a BigQuery table with project: %s, dataset: %s, '
                'table: %s.', project, dataset_id, table_id)
   client = bigquery.Client(project=project)
   table_ref = client.dataset(dataset_id).table(table_id)
diff --git a/sdks/python/apache_beam/io/hadoopfilesystem.py b/sdks/python/apache_beam/io/hadoopfilesystem.py
index 71d74e8..0abdbaf 100644
--- a/sdks/python/apache_beam/io/hadoopfilesystem.py
+++ b/sdks/python/apache_beam/io/hadoopfilesystem.py
@@ -55,6 +55,8 @@
 _FILE_STATUS_TYPE_DIRECTORY = 'DIRECTORY'
 _FILE_STATUS_TYPE_FILE = 'FILE'
 
+_LOGGER = logging.getLogger(__name__)
+
 
 class HdfsDownloader(filesystemio.Downloader):
 
@@ -196,7 +198,7 @@
   @staticmethod
   def _add_compression(stream, path, mime_type, compression_type):
     if mime_type != 'application/octet-stream':
-      logging.warning('Mime types are not supported. Got non-default mime_type:'
+      _LOGGER.warning('Mime types are not supported. Got non-default mime_type:'
                       ' %s', mime_type)
     if compression_type == CompressionTypes.AUTO:
       compression_type = CompressionTypes.detect_compression_type(path)
diff --git a/sdks/python/apache_beam/io/iobase.py b/sdks/python/apache_beam/io/iobase.py
index 5b66730..dd97b7f 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
@@ -60,6 +61,9 @@
            'Sink', 'Write', 'Writer']
 
 
+_LOGGER = logging.getLogger(__name__)
+
+
 # Encapsulates information about a bundle of a source generated when method
 # BoundedSource.split() is invoked.
 # This is a named 4-tuple that has following fields.
@@ -1074,7 +1078,7 @@
   write_results = list(write_results)
   extra_shards = []
   if len(write_results) < min_shards:
-    logging.debug(
+    _LOGGER.debug(
         'Creating %s empty shard(s).', min_shards - len(write_results))
     for _ in range(min_shards - len(write_results)):
       writer = sink.open_writer(init_result, str(uuid.uuid4()))
@@ -1104,13 +1108,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 +1129,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 +1155,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 +1186,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 +1201,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 +1214,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 +1464,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 +1518,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/mongodbio.py b/sdks/python/apache_beam/io/mongodbio.py
index 6004ca1..a89ccb1 100644
--- a/sdks/python/apache_beam/io/mongodbio.py
+++ b/sdks/python/apache_beam/io/mongodbio.py
@@ -66,6 +66,9 @@
 from apache_beam.transforms import Reshuffle
 from apache_beam.utils.annotations import experimental
 
+_LOGGER = logging.getLogger(__name__)
+
+
 try:
   # Mongodb has its own bundled bson, which is not compatible with bson pakcage.
   # (https://github.com/py-bson/bson/issues/82). Try to import objectid and if
@@ -80,7 +83,7 @@
   from pymongo import ReplaceOne
 except ImportError:
   objectid = None
-  logging.warning("Could not find a compatible bson package.")
+  _LOGGER.warning("Could not find a compatible bson package.")
 
 __all__ = ['ReadFromMongoDB', 'WriteToMongoDB']
 
@@ -497,7 +500,7 @@
                      replacement=doc,
                      upsert=True))
     resp = self.client[self.db][self.coll].bulk_write(requests)
-    logging.debug('BulkWrite to MongoDB result in nModified:%d, nUpserted:%d, '
+    _LOGGER.debug('BulkWrite to MongoDB result in nModified:%d, nUpserted:%d, '
                   'nMatched:%d, Errors:%s' %
                   (resp.modified_count, resp.upserted_count, resp.matched_count,
                    resp.bulk_api_result.get('writeErrors')))
diff --git a/sdks/python/apache_beam/io/mongodbio_it_test.py b/sdks/python/apache_beam/io/mongodbio_it_test.py
index bfc6099..b315562 100644
--- a/sdks/python/apache_beam/io/mongodbio_it_test.py
+++ b/sdks/python/apache_beam/io/mongodbio_it_test.py
@@ -27,6 +27,8 @@
 from apache_beam.testing.util import assert_that
 from apache_beam.testing.util import equal_to
 
+_LOGGER = logging.getLogger(__name__)
+
 
 def run(argv=None):
   default_db = 'beam_mongodbio_it_db'
@@ -54,7 +56,7 @@
   # Test Write to MongoDB
   with TestPipeline(options=PipelineOptions(pipeline_args)) as p:
     start_time = time.time()
-    logging.info('Writing %d documents to mongodb' % known_args.num_documents)
+    _LOGGER.info('Writing %d documents to mongodb' % known_args.num_documents)
     docs = [{
         'number': x,
         'number_mod_2': x % 2,
@@ -67,13 +69,13 @@
                                                        known_args.mongo_coll,
                                                        known_args.batch_size)
   elapsed = time.time() - start_time
-  logging.info('Writing %d documents to mongodb finished in %.3f seconds' %
+  _LOGGER.info('Writing %d documents to mongodb finished in %.3f seconds' %
                (known_args.num_documents, elapsed))
 
   # Test Read from MongoDB
   with TestPipeline(options=PipelineOptions(pipeline_args)) as p:
     start_time = time.time()
-    logging.info('Reading from mongodb %s:%s' %
+    _LOGGER.info('Reading from mongodb %s:%s' %
                  (known_args.mongo_db, known_args.mongo_coll))
     r = p | 'ReadFromMongoDB' >> \
                 beam.io.ReadFromMongoDB(known_args.mongo_uri,
@@ -85,7 +87,7 @@
         r, equal_to([number for number in range(known_args.num_documents)]))
 
   elapsed = time.time() - start_time
-  logging.info('Read %d documents from mongodb finished in %.3f seconds' %
+  _LOGGER.info('Read %d documents from mongodb finished in %.3f seconds' %
                (known_args.num_documents, elapsed))
 
 
diff --git a/sdks/python/apache_beam/io/range_trackers.py b/sdks/python/apache_beam/io/range_trackers.py
index c46f801..d4845fb 100644
--- a/sdks/python/apache_beam/io/range_trackers.py
+++ b/sdks/python/apache_beam/io/range_trackers.py
@@ -34,6 +34,9 @@
            'OrderedPositionRangeTracker', 'UnsplittableRangeTracker']
 
 
+_LOGGER = logging.getLogger(__name__)
+
+
 class OffsetRangeTracker(iobase.RangeTracker):
   """A 'RangeTracker' for non-negative positions of type 'long'."""
 
@@ -137,27 +140,27 @@
     assert isinstance(split_offset, (int, long))
     with self._lock:
       if self._stop_offset == OffsetRangeTracker.OFFSET_INFINITY:
-        logging.debug('refusing to split %r at %d: stop position unspecified',
+        _LOGGER.debug('refusing to split %r at %d: stop position unspecified',
                       self, split_offset)
         return
       if self._last_record_start == -1:
-        logging.debug('Refusing to split %r at %d: unstarted', self,
+        _LOGGER.debug('Refusing to split %r at %d: unstarted', self,
                       split_offset)
         return
 
       if split_offset <= self._last_record_start:
-        logging.debug(
+        _LOGGER.debug(
             'Refusing to split %r at %d: already past proposed stop offset',
             self, split_offset)
         return
       if (split_offset < self.start_position()
           or split_offset >= self.stop_position()):
-        logging.debug(
+        _LOGGER.debug(
             'Refusing to split %r at %d: proposed split position out of range',
             self, split_offset)
         return
 
-      logging.debug('Agreeing to split %r at %d', self, split_offset)
+      _LOGGER.debug('Agreeing to split %r at %d', self, split_offset)
 
       split_fraction = (float(split_offset - self._start_offset) / (
           self._stop_offset - self._start_offset))
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/io/source_test_utils.py b/sdks/python/apache_beam/io/source_test_utils.py
index d83a62a..7291786 100644
--- a/sdks/python/apache_beam/io/source_test_utils.py
+++ b/sdks/python/apache_beam/io/source_test_utils.py
@@ -68,6 +68,9 @@
            'assert_split_at_fraction_succeeds_and_consistent']
 
 
+_LOGGER = logging.getLogger(__name__)
+
+
 class ExpectedSplitOutcome(object):
   MUST_SUCCEED_AND_BE_CONSISTENT = 1
   MUST_FAIL = 2
@@ -588,7 +591,7 @@
         num_trials += 1
         if (num_trials >
             MAX_CONCURRENT_SPLITTING_TRIALS_PER_ITEM):
-          logging.warning(
+          _LOGGER.warning(
               'After %d concurrent splitting trials at item #%d, observed '
               'only %s, giving up on this item',
               num_trials,
@@ -604,7 +607,7 @@
           have_failure = True
 
         if have_success and have_failure:
-          logging.info('%d trials to observe both success and failure of '
+          _LOGGER.info('%d trials to observe both success and failure of '
                        'concurrent splitting at item #%d', num_trials, i)
           break
     finally:
@@ -613,11 +616,11 @@
     num_total_trials += num_trials
 
     if num_total_trials > MAX_CONCURRENT_SPLITTING_TRIALS_TOTAL:
-      logging.warning('After %d total concurrent splitting trials, considered '
+      _LOGGER.warning('After %d total concurrent splitting trials, considered '
                       'only %d items, giving up.', num_total_trials, i)
       break
 
-  logging.info('%d total concurrent splitting trials for %d items',
+  _LOGGER.info('%d total concurrent splitting trials for %d items',
                num_total_trials, len(expected_items))
 
 
diff --git a/sdks/python/apache_beam/io/textio.py b/sdks/python/apache_beam/io/textio.py
index 340449f..3b426cc 100644
--- a/sdks/python/apache_beam/io/textio.py
+++ b/sdks/python/apache_beam/io/textio.py
@@ -42,6 +42,9 @@
            'WriteToText']
 
 
+_LOGGER = logging.getLogger(__name__)
+
+
 class _TextSource(filebasedsource.FileBasedSource):
   r"""A source for reading text files.
 
@@ -127,7 +130,7 @@
       raise ValueError('Cannot skip negative number of header lines: %d'
                        % skip_header_lines)
     elif skip_header_lines > 10:
-      logging.warning(
+      _LOGGER.warning(
           'Skipping %d header lines. Skipping large number of header '
           'lines might significantly slow down processing.')
     self._skip_header_lines = skip_header_lines
diff --git a/sdks/python/apache_beam/io/tfrecordio.py b/sdks/python/apache_beam/io/tfrecordio.py
index 7b0bd87..ab7d2f5 100644
--- a/sdks/python/apache_beam/io/tfrecordio.py
+++ b/sdks/python/apache_beam/io/tfrecordio.py
@@ -38,6 +38,9 @@
 __all__ = ['ReadFromTFRecord', 'WriteToTFRecord']
 
 
+_LOGGER = logging.getLogger(__name__)
+
+
 def _default_crc32c_fn(value):
   """Calculates crc32c of a bytes object using either snappy or crcmod."""
 
@@ -54,7 +57,7 @@
       pass
 
     if not _default_crc32c_fn.fn:
-      logging.warning('Couldn\'t find python-snappy so the implementation of '
+      _LOGGER.warning('Couldn\'t find python-snappy so the implementation of '
                       '_TFRecordUtil._masked_crc32c is not as fast as it could '
                       'be.')
       _default_crc32c_fn.fn = crcmod.predefined.mkPredefinedCrcFun('crc-32c')
diff --git a/sdks/python/apache_beam/io/vcfio.py b/sdks/python/apache_beam/io/vcfio.py
index 2a13bf8..aed3579 100644
--- a/sdks/python/apache_beam/io/vcfio.py
+++ b/sdks/python/apache_beam/io/vcfio.py
@@ -51,6 +51,9 @@
 __all__ = ['ReadFromVcf', 'Variant', 'VariantCall', 'VariantInfo',
            'MalformedVcfRecord']
 
+
+_LOGGER = logging.getLogger(__name__)
+
 # Stores data about variant INFO fields. The type of 'data' is specified in the
 # VCF headers. 'field_count' is a string that specifies the number of fields
 # that the data type contains. Its value can either be a number representing a
@@ -346,7 +349,7 @@
                                                self._vcf_reader.formats)
       except (LookupError, ValueError):
         if self._allow_malformed_records:
-          logging.warning(
+          _LOGGER.warning(
               'An exception was raised when reading record from VCF file '
               '%s. Invalid record was %s: %s',
               self._file_name, self._last_record, traceback.format_exc())
diff --git a/sdks/python/apache_beam/metrics/cells.pxd b/sdks/python/apache_beam/metrics/cells.pxd
new file mode 100644
index 0000000..0204da8
--- /dev/null
+++ b/sdks/python/apache_beam/metrics/cells.pxd
@@ -0,0 +1,49 @@
+#
+# 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.
+#
+
+cimport cython
+cimport libc.stdint
+
+
+cdef class MetricCell(object):
+  cdef object _lock
+  cpdef bint update(self, value) except -1
+
+
+cdef class CounterCell(MetricCell):
+  cdef readonly libc.stdint.int64_t value
+
+  @cython.locals(ivalue=libc.stdint.int64_t)
+  cpdef bint update(self, value) except -1
+
+
+cdef class DistributionCell(MetricCell):
+  cdef readonly DistributionData data
+
+  @cython.locals(ivalue=libc.stdint.int64_t)
+  cdef inline bint _update(self, value) except -1
+
+
+cdef class GaugeCell(MetricCell):
+  cdef readonly object data
+
+
+cdef class DistributionData(object):
+  cdef readonly libc.stdint.int64_t sum
+  cdef readonly libc.stdint.int64_t count
+  cdef readonly libc.stdint.int64_t min
+  cdef readonly libc.stdint.int64_t max
diff --git a/sdks/python/apache_beam/metrics/cells.py b/sdks/python/apache_beam/metrics/cells.py
index e7336e4..1df52d9 100644
--- a/sdks/python/apache_beam/metrics/cells.py
+++ b/sdks/python/apache_beam/metrics/cells.py
@@ -30,12 +30,16 @@
 
 from google.protobuf import timestamp_pb2
 
-from apache_beam.metrics.metricbase import Counter
-from apache_beam.metrics.metricbase import Distribution
-from apache_beam.metrics.metricbase import Gauge
 from apache_beam.portability.api import beam_fn_api_pb2
 from apache_beam.portability.api import metrics_pb2
 
+try:
+  import cython
+except ImportError:
+  class fake_cython:
+    compiled = False
+  globals()['cython'] = fake_cython
+
 __all__ = ['DistributionResult', 'GaugeResult']
 
 
@@ -52,11 +56,17 @@
   def __init__(self):
     self._lock = threading.Lock()
 
+  def update(self, value):
+    raise NotImplementedError
+
   def get_cumulative(self):
     raise NotImplementedError
 
+  def __reduce__(self):
+    raise NotImplementedError
 
-class CounterCell(Counter, MetricCell):
+
+class CounterCell(MetricCell):
   """For internal use only; no backwards-compatibility guarantees.
 
   Tracks the current value and delta of a counter metric.
@@ -80,27 +90,41 @@
     return result
 
   def inc(self, n=1):
-    with self._lock:
-      self.value += n
+    self.update(n)
+
+  def dec(self, n=1):
+    self.update(-n)
+
+  def update(self, value):
+    if cython.compiled:
+      ivalue = value
+      # We hold the GIL, no need for another lock.
+      self.value += ivalue
+    else:
+      with self._lock:
+        self.value += value
 
   def get_cumulative(self):
     with self._lock:
       return self.value
 
-  def to_runner_api_monitoring_info(self):
-    """Returns a Metric with this counter value for use in a MonitoringInfo."""
-    # TODO(ajamato): Update this code to be consistent with Gauges
-    # and Distributions. Since there is no CounterData class this method
-    # was added to CounterCell. Consider adding a CounterData class or
-    # removing the GaugeData and DistributionData classes.
-    return metrics_pb2.Metric(
-        counter_data=metrics_pb2.CounterData(
-            int64_value=self.get_cumulative()
-        )
-    )
+  def to_runner_api_user_metric(self, metric_name):
+    return beam_fn_api_pb2.Metrics.User(
+        metric_name=metric_name.to_runner_api(),
+        counter_data=beam_fn_api_pb2.Metrics.User.CounterData(
+            value=self.value))
+
+  def to_runner_api_monitoring_info(self, name, transform_id):
+    from apache_beam.metrics import monitoring_infos
+    return monitoring_infos.int64_user_counter(
+        name.namespace, name.name,
+        metrics_pb2.Metric(
+            counter_data=metrics_pb2.CounterData(
+                int64_value=self.get_cumulative())),
+        ptransform=transform_id)
 
 
-class DistributionCell(Distribution, MetricCell):
+class DistributionCell(MetricCell):
   """For internal use only; no backwards-compatibility guarantees.
 
   Tracks the current value and delta for a distribution metric.
@@ -124,26 +148,43 @@
     return result
 
   def update(self, value):
-    with self._lock:
+    if cython.compiled:
+      # We will hold the GIL throughout the entire _update.
       self._update(value)
+    else:
+      with self._lock:
+        self._update(value)
 
   def _update(self, value):
-    value = int(value)
-    self.data.count += 1
-    self.data.sum += value
-    self.data.min = (value
-                     if self.data.min is None or self.data.min > value
-                     else self.data.min)
-    self.data.max = (value
-                     if self.data.max is None or self.data.max < value
-                     else self.data.max)
+    if cython.compiled:
+      ivalue = value
+    else:
+      ivalue = int(value)
+    self.data.count = self.data.count + 1
+    self.data.sum = self.data.sum + ivalue
+    if ivalue < self.data.min:
+      self.data.min = ivalue
+    if ivalue > self.data.max:
+      self.data.max = ivalue
 
   def get_cumulative(self):
     with self._lock:
       return self.data.get_cumulative()
 
+  def to_runner_api_user_metric(self, metric_name):
+    return beam_fn_api_pb2.Metrics.User(
+        metric_name=metric_name.to_runner_api(),
+        distribution_data=self.get_cumulative().to_runner_api())
 
-class GaugeCell(Gauge, MetricCell):
+  def to_runner_api_monitoring_info(self, name, transform_id):
+    from apache_beam.metrics import monitoring_infos
+    return monitoring_infos.int64_user_distribution(
+        name.namespace, name.name,
+        self.get_cumulative().to_runner_api_monitoring_info(),
+        ptransform=transform_id)
+
+
+class GaugeCell(MetricCell):
   """For internal use only; no backwards-compatibility guarantees.
 
   Tracks the current value and delta for a gauge metric.
@@ -167,6 +208,9 @@
     return result
 
   def set(self, value):
+    self.update(value)
+
+  def update(self, value):
     value = int(value)
     with self._lock:
       # Set the value directly without checking timestamp, because
@@ -178,6 +222,18 @@
     with self._lock:
       return self.data.get_cumulative()
 
+  def to_runner_api_user_metric(self, metric_name):
+    return beam_fn_api_pb2.Metrics.User(
+        metric_name=metric_name.to_runner_api(),
+        gauge_data=self.get_cumulative().to_runner_api())
+
+  def to_runner_api_monitoring_info(self, name, transform_id):
+    from apache_beam.metrics import monitoring_infos
+    return monitoring_infos.int64_user_gauge(
+        name.namespace, name.name,
+        self.get_cumulative().to_runner_api_monitoring_info(),
+        ptransform=transform_id)
+
 
 class DistributionResult(object):
   """The result of a Distribution metric."""
@@ -198,7 +254,7 @@
     return not self == other
 
   def __repr__(self):
-    return '<DistributionResult(sum={}, count={}, min={}, max={})>'.format(
+    return 'DistributionResult(sum={}, count={}, min={}, max={})'.format(
         self.sum,
         self.count,
         self.min,
@@ -206,11 +262,11 @@
 
   @property
   def max(self):
-    return self.data.max
+    return self.data.max if self.data.count else None
 
   @property
   def min(self):
-    return self.data.min
+    return self.data.min if self.data.count else None
 
   @property
   def count(self):
@@ -340,10 +396,16 @@
   by other than the DistributionCell that contains it.
   """
   def __init__(self, sum, count, min, max):
-    self.sum = sum
-    self.count = count
-    self.min = min
-    self.max = max
+    if count:
+      self.sum = sum
+      self.count = count
+      self.min = min
+      self.max = max
+    else:
+      self.sum = self.count = 0
+      self.min = 2**63 - 1
+      # Avoid Wimplicitly-unsigned-literal caused by -2**63.
+      self.max = -self.min - 1
 
   def __eq__(self, other):
     return (self.sum == other.sum and
@@ -359,7 +421,7 @@
     return not self == other
 
   def __repr__(self):
-    return '<DistributionData(sum={}, count={}, min={}, max={})>'.format(
+    return 'DistributionData(sum={}, count={}, min={}, max={})'.format(
         self.sum,
         self.count,
         self.min,
@@ -372,15 +434,11 @@
     if other is None:
       return self
 
-    new_min = (None if self.min is None and other.min is None else
-               min(x for x in (self.min, other.min) if x is not None))
-    new_max = (None if self.max is None and other.max is None else
-               max(x for x in (self.max, other.max) if x is not None))
     return DistributionData(
         self.sum + other.sum,
         self.count + other.count,
-        new_min,
-        new_max)
+        self.min if self.min < other.min else other.min,
+        self.max if self.max > other.max else other.max)
 
   @staticmethod
   def singleton(value):
@@ -449,7 +507,7 @@
   """
   @staticmethod
   def identity_element():
-    return DistributionData(0, 0, None, None)
+    return DistributionData(0, 0, 2**63 - 1, -2**63)
 
   def combine(self, x, y):
     return x.combine(y)
diff --git a/sdks/python/apache_beam/metrics/execution.pxd b/sdks/python/apache_beam/metrics/execution.pxd
index 74b34fb..6e1cbb0 100644
--- a/sdks/python/apache_beam/metrics/execution.pxd
+++ b/sdks/python/apache_beam/metrics/execution.pxd
@@ -16,10 +16,30 @@
 #
 
 cimport cython
+cimport libc.stdint
+
+from apache_beam.metrics.cells cimport MetricCell
+
+
+cdef object get_current_tracker
+
+
+cdef class _TypedMetricName(object):
+  cdef readonly object cell_type
+  cdef readonly object metric_name
+  cdef readonly object fast_name
+  cdef libc.stdint.int64_t _hash
+
+
+cdef object _DEFAULT
+
+
+cdef class MetricUpdater(object):
+  cdef _TypedMetricName typed_metric_name
+  cdef object default
 
 
 cdef class MetricsContainer(object):
   cdef object step_name
-  cdef public object counters
-  cdef public object distributions
-  cdef public object gauges
+  cdef public dict metrics
+  cpdef MetricCell get_metric_cell(self, metric_key)
diff --git a/sdks/python/apache_beam/metrics/execution.py b/sdks/python/apache_beam/metrics/execution.py
index 91fe2f8..6918914 100644
--- a/sdks/python/apache_beam/metrics/execution.py
+++ b/sdks/python/apache_beam/metrics/execution.py
@@ -33,14 +33,13 @@
 from __future__ import absolute_import
 
 from builtins import object
-from collections import defaultdict
 
 from apache_beam.metrics import monitoring_infos
 from apache_beam.metrics.cells import CounterCell
 from apache_beam.metrics.cells import DistributionCell
 from apache_beam.metrics.cells import GaugeCell
-from apache_beam.portability.api import beam_fn_api_pb2
 from apache_beam.runners.worker import statesampler
+from apache_beam.runners.worker.statesampler import get_current_tracker
 
 
 class MetricKey(object):
@@ -150,88 +149,117 @@
 MetricsEnvironment = _MetricsEnvironment()
 
 
+class _TypedMetricName(object):
+  """Like MetricName, but also stores the cell type of the metric."""
+  def __init__(self, cell_type, metric_name):
+    self.cell_type = cell_type
+    self.metric_name = metric_name
+    if isinstance(metric_name, str):
+      self.fast_name = metric_name
+    else:
+      self.fast_name = '%d_%s%s' % (
+          len(metric_name.name), metric_name.name, metric_name.namespace)
+    # Cached for speed, as this is used as a key for every counter update.
+    self._hash = hash((cell_type, self.fast_name))
+
+  def __eq__(self, other):
+    return self is other or (
+        self.cell_type == other.cell_type and self.fast_name == other.fast_name)
+
+  def __ne__(self, other):
+    return not self == other
+
+  def __hash__(self):
+    return self._hash
+
+  def __reduce__(self):
+    return _TypedMetricName, (self.cell_type, self.metric_name)
+
+
+_DEFAULT = None
+
+
+class MetricUpdater(object):
+  """A callable that updates the metric as quickly as possible."""
+  def __init__(self, cell_type, metric_name, default=None):
+    self.typed_metric_name = _TypedMetricName(cell_type, metric_name)
+    self.default = default
+
+  def __call__(self, value=_DEFAULT):
+    if value is _DEFAULT:
+      if self.default is _DEFAULT:
+        raise ValueError(
+            'Missing value for update of %s' % self.metric_name)
+      value = self.default
+    tracker = get_current_tracker()
+    if tracker is not None:
+      tracker.update_metric(self.typed_metric_name, value)
+
+  def __reduce__(self):
+    return MetricUpdater, (
+        self.typed_metric_name.cell_type,
+        self.typed_metric_name.metric_name,
+        self.default)
+
+
 class MetricsContainer(object):
   """Holds the metrics of a single step and a single bundle."""
   def __init__(self, step_name):
     self.step_name = step_name
-    self.counters = defaultdict(lambda: CounterCell())
-    self.distributions = defaultdict(lambda: DistributionCell())
-    self.gauges = defaultdict(lambda: GaugeCell())
+    self.metrics = dict()
 
   def get_counter(self, metric_name):
-    return self.counters[metric_name]
+    return self.get_metric_cell(_TypedMetricName(CounterCell, metric_name))
 
   def get_distribution(self, metric_name):
-    return self.distributions[metric_name]
+    return self.get_metric_cell(_TypedMetricName(DistributionCell, metric_name))
 
   def get_gauge(self, metric_name):
-    return self.gauges[metric_name]
+    return self.get_metric_cell(_TypedMetricName(GaugeCell, metric_name))
+
+  def get_metric_cell(self, typed_metric_name):
+    cell = self.metrics.get(typed_metric_name, None)
+    if cell is None:
+      cell = self.metrics[typed_metric_name] = typed_metric_name.cell_type()
+    return cell
 
   def get_cumulative(self):
     """Return MetricUpdates with cumulative values of all metrics in container.
 
     This returns all the cumulative values for all metrics.
     """
-    counters = {MetricKey(self.step_name, k): v.get_cumulative()
-                for k, v in self.counters.items()}
+    counters = {MetricKey(self.step_name, k.metric_name): v.get_cumulative()
+                for k, v in self.metrics.items()
+                if k.cell_type == CounterCell}
 
-    distributions = {MetricKey(self.step_name, k): v.get_cumulative()
-                     for k, v in self.distributions.items()}
+    distributions = {
+        MetricKey(self.step_name, k.metric_name): v.get_cumulative()
+        for k, v in self.metrics.items()
+        if k.cell_type == DistributionCell}
 
-    gauges = {MetricKey(self.step_name, k): v.get_cumulative()
-              for k, v in self.gauges.items()}
+    gauges = {MetricKey(self.step_name, k.metric_name): v.get_cumulative()
+              for k, v in self.metrics.items()
+              if k.cell_type == GaugeCell}
 
     return MetricUpdates(counters, distributions, gauges)
 
   def to_runner_api(self):
-    return (
-        [beam_fn_api_pb2.Metrics.User(
-            metric_name=k.to_runner_api(),
-            counter_data=beam_fn_api_pb2.Metrics.User.CounterData(
-                value=v.get_cumulative()))
-         for k, v in self.counters.items()] +
-        [beam_fn_api_pb2.Metrics.User(
-            metric_name=k.to_runner_api(),
-            distribution_data=v.get_cumulative().to_runner_api())
-         for k, v in self.distributions.items()] +
-        [beam_fn_api_pb2.Metrics.User(
-            metric_name=k.to_runner_api(),
-            gauge_data=v.get_cumulative().to_runner_api())
-         for k, v in self.gauges.items()]
-    )
+    return [cell.to_runner_api_user_metric(key.metric_name)
+            for key, cell in self.metrics.items()]
 
   def to_runner_api_monitoring_infos(self, transform_id):
     """Returns a list of MonitoringInfos for the metrics in this container."""
-    all_user_metrics = []
-    for k, v in self.counters.items():
-      all_user_metrics.append(monitoring_infos.int64_user_counter(
-          k.namespace, k.name,
-          v.to_runner_api_monitoring_info(),
-          ptransform=transform_id
-      ))
-
-    for k, v in self.distributions.items():
-      all_user_metrics.append(monitoring_infos.int64_user_distribution(
-          k.namespace, k.name,
-          v.get_cumulative().to_runner_api_monitoring_info(),
-          ptransform=transform_id
-      ))
-
-    for k, v in self.gauges.items():
-      all_user_metrics.append(monitoring_infos.int64_user_gauge(
-          k.namespace, k.name,
-          v.get_cumulative().to_runner_api_monitoring_info(),
-          ptransform=transform_id
-      ))
+    all_user_metrics = [
+        cell.to_runner_api_monitoring_info(key.metric_name, transform_id)
+        for key, cell in self.metrics.items()]
     return {monitoring_infos.to_key(mi) : mi for mi in all_user_metrics}
 
   def reset(self):
-    for counter in self.counters.values():
-      counter.reset()
-    for distribution in self.distributions.values():
-      distribution.reset()
-    for gauge in self.gauges.values():
-      gauge.reset()
+    for metric in self.metrics.values():
+      metric.reset()
+
+  def __reduce__(self):
+    raise NotImplementedError
 
 
 class MetricUpdates(object):
diff --git a/sdks/python/apache_beam/metrics/execution_test.py b/sdks/python/apache_beam/metrics/execution_test.py
index 9af1696..fc363a4 100644
--- a/sdks/python/apache_beam/metrics/execution_test.py
+++ b/sdks/python/apache_beam/metrics/execution_test.py
@@ -73,12 +73,6 @@
 
 
 class TestMetricsContainer(unittest.TestCase):
-  def test_create_new_counter(self):
-    mc = MetricsContainer('astep')
-    self.assertFalse(MetricName('namespace', 'name') in mc.counters)
-    mc.get_counter(MetricName('namespace', 'name'))
-    self.assertTrue(MetricName('namespace', 'name') in mc.counters)
-
   def test_add_to_counter(self):
     mc = MetricsContainer('astep')
     counter = mc.get_counter(MetricName('namespace', 'name'))
diff --git a/sdks/python/apache_beam/metrics/metric.py b/sdks/python/apache_beam/metrics/metric.py
index acd4771..8bbe191 100644
--- a/sdks/python/apache_beam/metrics/metric.py
+++ b/sdks/python/apache_beam/metrics/metric.py
@@ -29,7 +29,8 @@
 import inspect
 from builtins import object
 
-from apache_beam.metrics.execution import MetricsEnvironment
+from apache_beam.metrics import cells
+from apache_beam.metrics.execution import MetricUpdater
 from apache_beam.metrics.metricbase import Counter
 from apache_beam.metrics.metricbase import Distribution
 from apache_beam.metrics.metricbase import Gauge
@@ -101,11 +102,7 @@
     def __init__(self, metric_name):
       super(Metrics.DelegatingCounter, self).__init__()
       self.metric_name = metric_name
-
-    def inc(self, n=1):
-      container = MetricsEnvironment.current_container()
-      if container is not None:
-        container.get_counter(self.metric_name).inc(n)
+      self.inc = MetricUpdater(cells.CounterCell, metric_name, default=1)
 
   class DelegatingDistribution(Distribution):
     """Metrics Distribution Delegates functionality to MetricsEnvironment."""
@@ -113,11 +110,7 @@
     def __init__(self, metric_name):
       super(Metrics.DelegatingDistribution, self).__init__()
       self.metric_name = metric_name
-
-    def update(self, value):
-      container = MetricsEnvironment.current_container()
-      if container is not None:
-        container.get_distribution(self.metric_name).update(value)
+      self.update = MetricUpdater(cells.DistributionCell, metric_name)
 
   class DelegatingGauge(Gauge):
     """Metrics Gauge that Delegates functionality to MetricsEnvironment."""
@@ -125,11 +118,7 @@
     def __init__(self, metric_name):
       super(Metrics.DelegatingGauge, self).__init__()
       self.metric_name = metric_name
-
-    def set(self, value):
-      container = MetricsEnvironment.current_container()
-      if container is not None:
-        container.get_gauge(self.metric_name).set(value)
+      self.set = MetricUpdater(cells.GaugeCell, metric_name)
 
 
 class MetricResults(object):
diff --git a/sdks/python/apache_beam/metrics/metric_test.py b/sdks/python/apache_beam/metrics/metric_test.py
index 6e8ee08..cb18dc7 100644
--- a/sdks/python/apache_beam/metrics/metric_test.py
+++ b/sdks/python/apache_beam/metrics/metric_test.py
@@ -130,31 +130,36 @@
     statesampler.set_current_tracker(sampler)
     state1 = sampler.scoped_state('mystep', 'myState',
                                   metrics_container=MetricsContainer('mystep'))
-    sampler.start()
-    with state1:
-      counter_ns = 'aCounterNamespace'
-      distro_ns = 'aDistributionNamespace'
-      name = 'a_name'
-      counter = Metrics.counter(counter_ns, name)
-      distro = Metrics.distribution(distro_ns, name)
-      counter.inc(10)
-      counter.dec(3)
-      distro.update(10)
-      distro.update(2)
-      self.assertTrue(isinstance(counter, Metrics.DelegatingCounter))
-      self.assertTrue(isinstance(distro, Metrics.DelegatingDistribution))
 
-      del distro
-      del counter
+    try:
+      sampler.start()
+      with state1:
+        counter_ns = 'aCounterNamespace'
+        distro_ns = 'aDistributionNamespace'
+        name = 'a_name'
+        counter = Metrics.counter(counter_ns, name)
+        distro = Metrics.distribution(distro_ns, name)
+        counter.inc(10)
+        counter.dec(3)
+        distro.update(10)
+        distro.update(2)
+        self.assertTrue(isinstance(counter, Metrics.DelegatingCounter))
+        self.assertTrue(isinstance(distro, Metrics.DelegatingDistribution))
 
-      container = MetricsEnvironment.current_container()
-      self.assertEqual(
-          container.counters[MetricName(counter_ns, name)].get_cumulative(),
-          7)
-      self.assertEqual(
-          container.distributions[MetricName(distro_ns, name)].get_cumulative(),
-          DistributionData(12, 2, 2, 10))
-    sampler.stop()
+        del distro
+        del counter
+
+        container = MetricsEnvironment.current_container()
+        self.assertEqual(
+            container.get_counter(
+                MetricName(counter_ns, name)).get_cumulative(),
+            7)
+        self.assertEqual(
+            container.get_distribution(
+                MetricName(distro_ns, name)).get_cumulative(),
+            DistributionData(12, 2, 2, 10))
+    finally:
+      sampler.stop()
 
 
 if __name__ == '__main__':
diff --git a/sdks/python/apache_beam/options/pipeline_options.py b/sdks/python/apache_beam/options/pipeline_options.py
index 5ade96f..4de4b51 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
 
@@ -47,6 +48,9 @@
     ]
 
 
+_LOGGER = logging.getLogger(__name__)
+
+
 def _static_value_provider_of(value_type):
   """"Helper function to plug a ValueProvider into argparse.
 
@@ -261,7 +265,7 @@
       add_extra_args_fn(parser)
     known_args, unknown_args = parser.parse_known_args(self._flags)
     if unknown_args:
-      logging.warning("Discarding unparseable args: %s", unknown_args)
+      _LOGGER.warning("Discarding unparseable args: %s", unknown_args)
     result = vars(known_args)
 
     # Apply the overrides if any
@@ -509,19 +513,25 @@
     """
     environment_region = os.environ.get('CLOUDSDK_COMPUTE_REGION')
     if environment_region:
-      logging.info('Using default GCP region %s from $CLOUDSDK_COMPUTE_REGION',
+      _LOGGER.info('Using default GCP region %s from $CLOUDSDK_COMPUTE_REGION',
                    environment_region)
       return environment_region
     try:
       cmd = ['gcloud', 'config', 'get-value', 'compute/region']
-      output = processes.check_output(cmd).decode('utf-8').strip()
-      if output:
-        logging.info('Using default GCP region %s from `%s`',
-                     output, ' '.join(cmd))
-        return 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:
+        _LOGGER.info('Using default GCP region %s from `%s`',
+                     formatted_output, ' '.join(cmd))
+        return formatted_output
     except RuntimeError:
       pass
-    logging.warning(
+    _LOGGER.warning(
         '--region not set; will default to us-central1. Future releases of '
         'Beam will require the user to set --region explicitly, or else have a '
         'default set via the gcloud tool. '
@@ -865,8 +875,13 @@
   def _add_argparse_args(cls, parser):
     parser.add_argument(
         '--job_endpoint', default=None,
-        help=('Job service endpoint to use. Should be in the form of address '
-              'and port, e.g. localhost:3000'))
+        help=('Job service endpoint to use. Should be in the form of host '
+              'and port, e.g. localhost:8099.'))
+    parser.add_argument(
+        '--artifact_endpoint', default=None,
+        help=('Artifact staging endpoint to use. Should be in the form of host '
+              'and port, e.g. localhost:8098. If none is specified, the '
+              'artifact endpoint sent from the job server is used.'))
     parser.add_argument(
         '--job-server-timeout', default=60, type=int,
         help=('Job service request timeout in seconds. The timeout '
diff --git a/sdks/python/apache_beam/pipeline_test.py b/sdks/python/apache_beam/pipeline_test.py
index 201324a..c1c25d1 100644
--- a/sdks/python/apache_beam/pipeline_test.py
+++ b/sdks/python/apache_beam/pipeline_test.py
@@ -24,7 +24,6 @@
 import unittest
 from builtins import object
 from builtins import range
-from collections import defaultdict
 
 import mock
 
@@ -40,9 +39,6 @@
 from apache_beam.pvalue import AsSingleton
 from apache_beam.pvalue import TaggedOutput
 from apache_beam.runners.dataflow.native_io.iobase import NativeSource
-from apache_beam.runners.direct.evaluation_context import _ExecutionContext
-from apache_beam.runners.direct.transform_evaluator import _GroupByKeyOnlyEvaluator
-from apache_beam.runners.direct.transform_evaluator import _TransformEvaluator
 from apache_beam.testing.test_pipeline import TestPipeline
 from apache_beam.testing.util import assert_that
 from apache_beam.testing.util import equal_to
@@ -812,84 +808,5 @@
                      p.transforms_stack[0])
 
 
-class DirectRunnerRetryTests(unittest.TestCase):
-
-  def test_retry_fork_graph(self):
-    # TODO(BEAM-3642): The FnApiRunner currently does not currently support
-    # retries.
-    p = beam.Pipeline(runner='BundleBasedDirectRunner')
-
-    # TODO(mariagh): Remove the use of globals from the test.
-    global count_b, count_c # pylint: disable=global-variable-undefined
-    count_b, count_c = 0, 0
-
-    def f_b(x):
-      global count_b  # pylint: disable=global-variable-undefined
-      count_b += 1
-      raise Exception('exception in f_b')
-
-    def f_c(x):
-      global count_c  # pylint: disable=global-variable-undefined
-      count_c += 1
-      raise Exception('exception in f_c')
-
-    names = p | 'CreateNodeA' >> beam.Create(['Ann', 'Joe'])
-
-    fork_b = names | 'SendToB' >> beam.Map(f_b) # pylint: disable=unused-variable
-    fork_c = names | 'SendToC' >> beam.Map(f_c) # pylint: disable=unused-variable
-
-    with self.assertRaises(Exception):
-      p.run().wait_until_finish()
-    assert count_b == count_c == 4
-
-  def test_no_partial_writeouts(self):
-
-    class TestTransformEvaluator(_TransformEvaluator):
-
-      def __init__(self):
-        self._execution_context = _ExecutionContext(None, {})
-
-      def start_bundle(self):
-        self.step_context = self._execution_context.get_step_context()
-
-      def process_element(self, element):
-        k, v = element
-        state = self.step_context.get_keyed_state(k)
-        state.add_state(None, _GroupByKeyOnlyEvaluator.ELEMENTS_TAG, v)
-
-    # Create instance and add key/value, key/value2
-    evaluator = TestTransformEvaluator()
-    evaluator.start_bundle()
-    self.assertIsNone(evaluator.step_context.existing_keyed_state.get('key'))
-    self.assertIsNone(evaluator.step_context.partial_keyed_state.get('key'))
-
-    evaluator.process_element(['key', 'value'])
-    self.assertEqual(
-        evaluator.step_context.existing_keyed_state['key'].state,
-        defaultdict(lambda: defaultdict(list)))
-    self.assertEqual(
-        evaluator.step_context.partial_keyed_state['key'].state,
-        {None: {'elements':['value']}})
-
-    evaluator.process_element(['key', 'value2'])
-    self.assertEqual(
-        evaluator.step_context.existing_keyed_state['key'].state,
-        defaultdict(lambda: defaultdict(list)))
-    self.assertEqual(
-        evaluator.step_context.partial_keyed_state['key'].state,
-        {None: {'elements':['value', 'value2']}})
-
-    # Simulate an exception (redo key/value)
-    evaluator._execution_context.reset()
-    evaluator.start_bundle()
-    evaluator.process_element(['key', 'value'])
-    self.assertEqual(
-        evaluator.step_context.existing_keyed_state['key'].state,
-        defaultdict(lambda: defaultdict(list)))
-    self.assertEqual(
-        evaluator.step_context.partial_keyed_state['key'].state,
-        {None: {'elements':['value']}})
-
-
 if __name__ == '__main__':
   unittest.main()
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/bundle_factory.py b/sdks/python/apache_beam/runners/direct/bundle_factory.py
index 558e925..382cf52 100644
--- a/sdks/python/apache_beam/runners/direct/bundle_factory.py
+++ b/sdks/python/apache_beam/runners/direct/bundle_factory.py
@@ -99,6 +99,10 @@
     def windows(self):
       return self._initial_windowed_value.windows
 
+    @property
+    def pane_info(self):
+      return self._initial_windowed_value.pane_info
+
     def add_value(self, value):
       self._appended_values.append(value)
 
@@ -107,8 +111,7 @@
       # _appended_values to yield WindowedValue on the fly.
       yield self._initial_windowed_value
       for v in self._appended_values:
-        yield WindowedValue(v, self._initial_windowed_value.timestamp,
-                            self._initial_windowed_value.windows)
+        yield self._initial_windowed_value.with_value(v)
 
   def __init__(self, pcollection, stacked=True):
     assert isinstance(pcollection, (pvalue.PBegin, pvalue.PCollection))
@@ -178,7 +181,8 @@
         (isinstance(self._elements[-1], (WindowedValue,
                                          _Bundle._StackedWindowedValues))) and
         self._elements[-1].timestamp == element.timestamp and
-        self._elements[-1].windows == element.windows):
+        self._elements[-1].windows == element.windows and
+        self._elements[-1].pane_info == element.pane_info):
       if isinstance(self._elements[-1], WindowedValue):
         self._elements[-1] = _Bundle._StackedWindowedValues(self._elements[-1])
       self._elements[-1].add_value(element.value)
diff --git a/sdks/python/apache_beam/runners/direct/direct_runner.py b/sdks/python/apache_beam/runners/direct/direct_runner.py
index 73063a1..3332e39 100644
--- a/sdks/python/apache_beam/runners/direct/direct_runner.py
+++ b/sdks/python/apache_beam/runners/direct/direct_runner.py
@@ -187,7 +187,7 @@
   class CombinePerKeyOverride(PTransformOverride):
     def matches(self, applied_ptransform):
       if isinstance(applied_ptransform.transform, CombinePerKey):
-        return True
+        return applied_ptransform.inputs[0].windowing.is_default()
 
     def get_replacement_transform(self, transform):
       # TODO: Move imports to top. Pipeline <-> Runner dependency cause problems
diff --git a/sdks/python/apache_beam/runners/direct/direct_runner_test.py b/sdks/python/apache_beam/runners/direct/direct_runner_test.py
index 66f1845..22a930c 100644
--- a/sdks/python/apache_beam/runners/direct/direct_runner_test.py
+++ b/sdks/python/apache_beam/runners/direct/direct_runner_test.py
@@ -19,6 +19,7 @@
 
 import threading
 import unittest
+from collections import defaultdict
 
 import hamcrest as hc
 
@@ -33,6 +34,9 @@
 from apache_beam.runners import DirectRunner
 from apache_beam.runners import TestDirectRunner
 from apache_beam.runners import create_runner
+from apache_beam.runners.direct.evaluation_context import _ExecutionContext
+from apache_beam.runners.direct.transform_evaluator import _GroupByKeyOnlyEvaluator
+from apache_beam.runners.direct.transform_evaluator import _TransformEvaluator
 from apache_beam.testing import test_pipeline
 from apache_beam.testing.util import assert_that
 from apache_beam.testing.util import equal_to
@@ -129,5 +133,84 @@
            | beam.combiners.Count.Globally())
 
 
+class DirectRunnerRetryTests(unittest.TestCase):
+
+  def test_retry_fork_graph(self):
+    # TODO(BEAM-3642): The FnApiRunner currently does not currently support
+    # retries.
+    p = beam.Pipeline(runner='BundleBasedDirectRunner')
+
+    # TODO(mariagh): Remove the use of globals from the test.
+    global count_b, count_c # pylint: disable=global-variable-undefined
+    count_b, count_c = 0, 0
+
+    def f_b(x):
+      global count_b  # pylint: disable=global-variable-undefined
+      count_b += 1
+      raise Exception('exception in f_b')
+
+    def f_c(x):
+      global count_c  # pylint: disable=global-variable-undefined
+      count_c += 1
+      raise Exception('exception in f_c')
+
+    names = p | 'CreateNodeA' >> beam.Create(['Ann', 'Joe'])
+
+    fork_b = names | 'SendToB' >> beam.Map(f_b) # pylint: disable=unused-variable
+    fork_c = names | 'SendToC' >> beam.Map(f_c) # pylint: disable=unused-variable
+
+    with self.assertRaises(Exception):
+      p.run().wait_until_finish()
+    assert count_b == count_c == 4
+
+  def test_no_partial_writeouts(self):
+
+    class TestTransformEvaluator(_TransformEvaluator):
+
+      def __init__(self):
+        self._execution_context = _ExecutionContext(None, {})
+
+      def start_bundle(self):
+        self.step_context = self._execution_context.get_step_context()
+
+      def process_element(self, element):
+        k, v = element
+        state = self.step_context.get_keyed_state(k)
+        state.add_state(None, _GroupByKeyOnlyEvaluator.ELEMENTS_TAG, v)
+
+    # Create instance and add key/value, key/value2
+    evaluator = TestTransformEvaluator()
+    evaluator.start_bundle()
+    self.assertIsNone(evaluator.step_context.existing_keyed_state.get('key'))
+    self.assertIsNone(evaluator.step_context.partial_keyed_state.get('key'))
+
+    evaluator.process_element(['key', 'value'])
+    self.assertEqual(
+        evaluator.step_context.existing_keyed_state['key'].state,
+        defaultdict(lambda: defaultdict(list)))
+    self.assertEqual(
+        evaluator.step_context.partial_keyed_state['key'].state,
+        {None: {'elements':['value']}})
+
+    evaluator.process_element(['key', 'value2'])
+    self.assertEqual(
+        evaluator.step_context.existing_keyed_state['key'].state,
+        defaultdict(lambda: defaultdict(list)))
+    self.assertEqual(
+        evaluator.step_context.partial_keyed_state['key'].state,
+        {None: {'elements':['value', 'value2']}})
+
+    # Simulate an exception (redo key/value)
+    evaluator._execution_context.reset()
+    evaluator.start_bundle()
+    evaluator.process_element(['key', 'value'])
+    self.assertEqual(
+        evaluator.step_context.existing_keyed_state['key'].state,
+        defaultdict(lambda: defaultdict(list)))
+    self.assertEqual(
+        evaluator.step_context.partial_keyed_state['key'].state,
+        {None: {'elements':['value']}})
+
+
 if __name__ == '__main__':
   unittest.main()
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/interactive/display/pcoll_visualization_test.py b/sdks/python/apache_beam/runners/interactive/display/pcoll_visualization_test.py
index 4628c25..8eefec7 100644
--- a/sdks/python/apache_beam/runners/interactive/display/pcoll_visualization_test.py
+++ b/sdks/python/apache_beam/runners/interactive/display/pcoll_visualization_test.py
@@ -42,6 +42,8 @@
 
 @unittest.skipIf(not ie.current_env().is_interactive_ready,
                  '[interactive] dependency is not installed.')
+@unittest.skipIf(sys.version_info < (3, 6),
+                 'The tests require at least Python 3.6 to work.')
 class PCollectionVisualizationTest(unittest.TestCase):
 
   def setUp(self):
@@ -56,8 +58,6 @@
     # pylint: disable=range-builtin-not-iterating
     self._pcoll = self._p | 'Create' >> beam.Create(range(1000))
 
-  @unittest.skipIf(sys.version_info < (3, 5, 3),
-                   'PCollectionVisualization is supported on Python 3.5.3+.')
   def test_raise_error_for_non_pcoll_input(self):
     class Foo(object):
       pass
@@ -67,8 +67,6 @@
       self.assertTrue('pcoll should be apache_beam.pvalue.PCollection' in
                       ctx.exception)
 
-  @unittest.skipIf(sys.version_info < (3, 5, 3),
-                   'PCollectionVisualization is supported on Python 3.5.3+.')
   def test_pcoll_visualization_generate_unique_display_id(self):
     pv_1 = pv.PCollectionVisualization(self._pcoll)
     pv_2 = pv.PCollectionVisualization(self._pcoll)
@@ -76,8 +74,6 @@
     self.assertNotEqual(pv_1._overview_display_id, pv_2._overview_display_id)
     self.assertNotEqual(pv_1._df_display_id, pv_2._df_display_id)
 
-  @unittest.skipIf(sys.version_info < (3, 5, 3),
-                   'PCollectionVisualization is supported on Python 3.5.3+.')
   @patch('apache_beam.runners.interactive.display.pcoll_visualization'
          '.PCollectionVisualization._to_element_list', lambda x: [1, 2, 3])
   def test_one_shot_visualization_not_return_handle(self):
@@ -91,8 +87,6 @@
     yield [1, 2, 3, 4, 5, 6, 7]
     yield [1, 2, 3, 4, 5, 6, 7, 8]
 
-  @unittest.skipIf(sys.version_info < (3, 5, 3),
-                   'PCollectionVisualization is supported on Python 3.5.3+.')
   @patch('apache_beam.runners.interactive.display.pcoll_visualization'
          '.PCollectionVisualization._to_element_list', _mock_to_element_list)
   def test_dynamic_plotting_return_handle(self):
@@ -100,8 +94,6 @@
     self.assertIsInstance(h, timeloop.Timeloop)
     h.stop()
 
-  @unittest.skipIf(sys.version_info < (3, 5, 3),
-                   'PCollectionVisualization is supported on Python 3.5.3+.')
   @patch('apache_beam.runners.interactive.display.pcoll_visualization'
          '.PCollectionVisualization._to_element_list', _mock_to_element_list)
   @patch('apache_beam.runners.interactive.display.pcoll_visualization'
@@ -126,10 +118,6 @@
       self.assertIs(kwargs['updating_pv'], updating_pv)
     h.stop()
 
-  # The code being tested supports 3.5.3+. This specific test has assertion
-  # feature that was introduced in 3.6.
-  @unittest.skipIf(sys.version_info < (3, 6),
-                   'The test requires Python 3.6+.')
   @patch('apache_beam.runners.interactive.display.pcoll_visualization'
          '.PCollectionVisualization._to_element_list', _mock_to_element_list)
   @patch('timeloop.Timeloop.stop')
@@ -150,8 +138,6 @@
     # "assert_called" is new in Python 3.6.
     mocked_timeloop.assert_called()
 
-  @unittest.skipIf(sys.version_info < (3, 5, 3),
-                   'PCollectionVisualization is supported on Python 3.5.3+.')
   @patch('apache_beam.runners.interactive.display.pcoll_visualization'
          '.PCollectionVisualization._to_element_list', lambda x: [1, 2, 3])
   @patch('pandas.DataFrame.sample')
diff --git a/sdks/python/apache_beam/runners/interactive/interactive_environment.py b/sdks/python/apache_beam/runners/interactive/interactive_environment.py
index 2dbc102..414d564 100644
--- a/sdks/python/apache_beam/runners/interactive/interactive_environment.py
+++ b/sdks/python/apache_beam/runners/interactive/interactive_environment.py
@@ -24,6 +24,7 @@
 """
 from __future__ import absolute_import
 
+import atexit
 import importlib
 import logging
 import sys
@@ -32,6 +33,9 @@
 from apache_beam.runners import runner
 from apache_beam.runners.utils import is_interactive
 
+# Interactive Beam user flow is data-centric rather than pipeline-centric, so
+# there is only one global interactive environment instance that manages
+# implementation that enables interactivity.
 _interactive_beam_env = None
 
 
@@ -46,6 +50,8 @@
 def new_env(cache_manager=None):
   """Creates a new Interactive Beam environment to replace current one."""
   global _interactive_beam_env
+  if _interactive_beam_env:
+    _interactive_beam_env.cleanup()
   _interactive_beam_env = None
   return current_env(cache_manager)
 
@@ -63,6 +69,9 @@
 
   def __init__(self, cache_manager=None):
     self._cache_manager = cache_manager
+    # Register a cleanup routine when kernel is restarted or terminated.
+    if cache_manager:
+      atexit.register(self.cleanup)
     # Holds class instances, module object, string of module names.
     self._watching_set = set()
     # Holds variables list of (Dict[str, object]).
@@ -74,10 +83,10 @@
     self._pipeline_results = {}
     # Always watch __main__ module.
     self.watch('__main__')
-    # Do a warning level logging if current python version is below 3.5.3.
-    if sys.version_info < (3, 5, 3):
+    # Do a warning level logging if current python version is below 3.6.
+    if sys.version_info < (3, 6):
       self._is_py_version_ready = False
-      logging.warning('Interactive Beam requires Python 3.5.3+.')
+      logging.warning('Interactive Beam requires Python 3.6+.')
     else:
       self._is_py_version_ready = True
     # Check if [interactive] dependencies are installed.
@@ -127,6 +136,11 @@
     """
     return self._is_in_notebook
 
+  def cleanup(self):
+    # Utilizes cache manager to clean up cache from everywhere.
+    if self.cache_manager():
+      self.cache_manager().cleanup()
+
   def watch(self, watchable):
     """Watches a watchable.
 
@@ -163,7 +177,18 @@
 
   def set_cache_manager(self, cache_manager):
     """Sets the cache manager held by current Interactive Environment."""
+    if self._cache_manager is cache_manager:
+      # NOOP if setting to the same cache_manager.
+      return
+    if self._cache_manager:
+      # Invoke cleanup routine when a new cache_manager is forcefully set and
+      # current cache_manager is not None.
+      self.cleanup()
+      atexit.unregister(self.cleanup)
     self._cache_manager = cache_manager
+    if self._cache_manager:
+      # Re-register cleanup routine for the new cache_manager if it's not None.
+      atexit.register(self.cleanup)
 
   def cache_manager(self):
     """Gets the cache manager held by current Interactive Environment."""
diff --git a/sdks/python/apache_beam/runners/interactive/interactive_environment_test.py b/sdks/python/apache_beam/runners/interactive/interactive_environment_test.py
index 342f400..76c29b8 100644
--- a/sdks/python/apache_beam/runners/interactive/interactive_environment_test.py
+++ b/sdks/python/apache_beam/runners/interactive/interactive_environment_test.py
@@ -19,16 +19,27 @@
 from __future__ import absolute_import
 
 import importlib
+import sys
 import unittest
 
 import apache_beam as beam
 from apache_beam.runners import runner
+from apache_beam.runners.interactive import cache_manager as cache
 from apache_beam.runners.interactive import interactive_environment as ie
 
+# TODO(BEAM-8288): clean up the work-around of nose tests using Python2 without
+# unittest.mock module.
+try:
+  from unittest.mock import call, patch
+except ImportError:
+  from mock import call, patch
+
 # The module name is also a variable in module.
 _module_name = 'apache_beam.runners.interactive.interactive_environment_test'
 
 
+@unittest.skipIf(sys.version_info < (3, 6),
+                 'The tests require at least Python 3.6 to work.')
 class InteractiveEnvironmentTest(unittest.TestCase):
 
   def setUp(self):
@@ -126,7 +137,6 @@
 
   def test_determine_terminal_state(self):
     for state in (runner.PipelineState.DONE,
-                  runner.PipelineState.STOPPED,
                   runner.PipelineState.FAILED,
                   runner.PipelineState.CANCELLED,
                   runner.PipelineState.UPDATED,
@@ -136,7 +146,7 @@
       self.assertTrue(ie.current_env().is_terminated(self._p))
     for state in (runner.PipelineState.UNKNOWN,
                   runner.PipelineState.STARTING,
-
+                  runner.PipelineState.STOPPED,
                   runner.PipelineState.RUNNING,
                   runner.PipelineState.DRAINING,
                   runner.PipelineState.PENDING,
@@ -153,11 +163,91 @@
                   pipeline_result)
     self.assertIs(ie.current_env().pipeline_result(self._p), None)
 
-  def test_is_none_when_pipeline_absent(self):
+  def test_pipeline_result_is_none_when_pipeline_absent(self):
     self.assertIs(ie.current_env().pipeline_result(self._p), None)
     self.assertIs(ie.current_env().is_terminated(self._p), True)
     self.assertIs(ie.current_env().evict_pipeline_result(self._p), None)
 
+  @patch('atexit.register')
+  def test_no_cleanup_when_cm_none(self,
+                                   mocked_atexit):
+    ie.new_env(None)
+    mocked_atexit.assert_not_called()
+
+  @patch('atexit.register')
+  def test_cleanup_when_cm_not_none(self,
+                                    mocked_atexit):
+    ie.new_env(cache.FileBasedCacheManager())
+    mocked_atexit.assert_called_once()
+
+  @patch('atexit.register')
+  @patch('atexit.unregister')
+  def test_cleanup_unregistered_when_not_none_cm_cleared(self,
+                                                         mocked_unreg,
+                                                         mocked_reg):
+    ie.new_env(cache.FileBasedCacheManager())
+    mocked_reg.assert_called_once()
+    mocked_unreg.assert_not_called()
+    ie.current_env().set_cache_manager(None)
+    mocked_reg.assert_called_once()
+    mocked_unreg.assert_called_once()
+
+  @patch('atexit.register')
+  @patch('atexit.unregister')
+  def test_cleanup_reregistered_when_cm_changed(self,
+                                                mocked_unreg,
+                                                mocked_reg):
+    ie.new_env(cache.FileBasedCacheManager())
+    mocked_unreg.assert_not_called()
+    ie.current_env().set_cache_manager(cache.FileBasedCacheManager())
+    mocked_unreg.assert_called_once()
+    mocked_reg.assert_has_calls([call(ie.current_env().cleanup),
+                                 call(ie.current_env().cleanup)])
+
+  @patch('apache_beam.runners.interactive.interactive_environment'
+         '.InteractiveEnvironment.cleanup')
+  def test_cleanup_invoked_when_new_env_replace_not_none_env(self,
+                                                             mocked_cleanup):
+    ie._interactive_beam_env = None
+    ie.new_env(cache.FileBasedCacheManager())
+    mocked_cleanup.assert_not_called()
+    ie.new_env(cache.FileBasedCacheManager())
+    mocked_cleanup.assert_called_once()
+
+  @patch('apache_beam.runners.interactive.interactive_environment'
+         '.InteractiveEnvironment.cleanup')
+  def test_cleanup_invoked_when_cm_changed(self,
+                                           mocked_cleanup):
+    ie._interactive_beam_env = None
+    ie.new_env(cache.FileBasedCacheManager())
+    ie.current_env().set_cache_manager(cache.FileBasedCacheManager())
+    mocked_cleanup.assert_called_once()
+
+  @patch('atexit.register')
+  @patch('atexit.unregister')
+  def test_cleanup_registered_when_none_cm_changed(self,
+                                                   mocked_unreg,
+                                                   mocked_reg):
+    ie.new_env(None)
+    mocked_reg.assert_not_called()
+    mocked_unreg.assert_not_called()
+    ie.current_env().set_cache_manager(cache.FileBasedCacheManager())
+    mocked_reg.assert_called_once()
+    mocked_unreg.assert_not_called()
+
+  @patch('atexit.register')
+  @patch('atexit.unregister')
+  def test_noop_when_cm_is_not_changed(self,
+                                       mocked_unreg,
+                                       mocked_reg):
+    cache_manager = cache.FileBasedCacheManager()
+    ie.new_env(cache_manager)
+    mocked_unreg.assert_not_called()
+    mocked_reg.assert_called_once()
+    ie.current_env().set_cache_manager(cache_manager)
+    mocked_unreg.assert_not_called()
+    mocked_reg.assert_called_once()
+
 
 if __name__ == '__main__':
   unittest.main()
diff --git a/sdks/python/apache_beam/runners/portability/abstract_job_service.py b/sdks/python/apache_beam/runners/portability/abstract_job_service.py
index 982fad1..5dd497a 100644
--- a/sdks/python/apache_beam/runners/portability/abstract_job_service.py
+++ b/sdks/python/apache_beam/runners/portability/abstract_job_service.py
@@ -23,13 +23,6 @@
 from apache_beam.portability.api import beam_job_api_pb2
 from apache_beam.portability.api import beam_job_api_pb2_grpc
 
-TERMINAL_STATES = [
-    beam_job_api_pb2.JobState.DONE,
-    beam_job_api_pb2.JobState.STOPPED,
-    beam_job_api_pb2.JobState.FAILED,
-    beam_job_api_pb2.JobState.CANCELLED,
-]
-
 
 class AbstractJobServiceServicer(beam_job_api_pb2_grpc.JobServiceServicer):
   """Manages one or more pipelines, possibly concurrently.
@@ -131,6 +124,11 @@
   def get_pipeline(self):
     return self._pipeline_proto
 
+  @staticmethod
+  def is_terminal_state(state):
+    from apache_beam.runners.portability import portable_runner
+    return state in portable_runner.TERMINAL_STATES
+
   def to_runner_api(self):
     return beam_job_api_pb2.JobInfo(
         job_id=self._job_id,
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/flink_uber_jar_job_server.py b/sdks/python/apache_beam/runners/portability/flink_uber_jar_job_server.py
index b69da66..d2a890f 100644
--- a/sdks/python/apache_beam/runners/portability/flink_uber_jar_job_server.py
+++ b/sdks/python/apache_beam/runners/portability/flink_uber_jar_job_server.py
@@ -216,7 +216,7 @@
         'IN_PROGRESS': beam_job_api_pb2.JobState.RUNNING,
         'COMPLETED': beam_job_api_pb2.JobState.DONE,
     }.get(flink_status, beam_job_api_pb2.JobState.UNSPECIFIED)
-    if beam_state in abstract_job_service.TERMINAL_STATES:
+    if self.is_terminal_state(beam_state):
       self.delete_jar()
     return beam_state
 
@@ -224,7 +224,7 @@
     sleep_secs = 1.0
     current_state = self.get_state()
     yield current_state
-    while current_state not in abstract_job_service.TERMINAL_STATES:
+    while not self.is_terminal_state(current_state):
       sleep_secs = min(60, sleep_secs * 1.2)
       time.sleep(sleep_secs)
       previous_state, current_state = current_state, self.get_state()
@@ -233,7 +233,7 @@
 
   def get_message_stream(self):
     for state in self.get_state_stream():
-      if state in abstract_job_service.TERMINAL_STATES:
+      if self.is_terminal_state(state):
         response = self.get('v1/jobs/%s/exceptions' % self._flink_job_id)
         for ix, exc in enumerate(response['all-exceptions']):
           yield beam_job_api_pb2.JobMessage(
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..e479708 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(
@@ -1385,17 +1383,15 @@
     super(EmbeddedGrpcWorkerHandler, self).__init__(state, provision_info,
                                                     grpc_server)
     if payload:
-      num_workers, state_cache_size = payload.decode('ascii').split(',')
-      self._num_threads = int(num_workers)
+      state_cache_size = payload.decode('ascii')
       self._state_cache_size = int(state_cache_size)
     else:
-      self._num_threads = 1
       self._state_cache_size = STATE_CACHE_SIZE
 
   def start_worker(self):
     self.worker = sdk_worker.SdkHarness(
-        self.control_address, worker_count=self._num_threads,
-        state_cache_size=self._state_cache_size, worker_id=self.worker_id)
+        self.control_address, state_cache_size=self._state_cache_size,
+        worker_id=self.worker_id)
     self.worker_thread = threading.Thread(
         name='run_worker', target=self.worker.run)
     self.worker_thread.daemon = True
@@ -1508,24 +1504,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 +1785,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..e31e0a5 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]
@@ -608,7 +656,7 @@
       pcoll_b = p | 'b' >> beam.Create(['b'])
       assert_that((pcoll_a, pcoll_b) | First(), equal_to(['a']))
 
-  def test_metrics(self):
+  def test_metrics(self, check_gauge=True):
     p = self.create_pipeline()
 
     counter = beam.metrics.Metrics.counter('ns', 'counter')
@@ -630,14 +678,17 @@
     c2, = res.metrics().query(beam.metrics.MetricsFilter().with_step('count2'))[
         'counters']
     self.assertEqual(c2.committed, 4)
+
     dist, = res.metrics().query(beam.metrics.MetricsFilter().with_step('dist'))[
         'distributions']
-    gaug, = res.metrics().query(
-        beam.metrics.MetricsFilter().with_step('gauge'))['gauges']
     self.assertEqual(
         dist.committed.data, beam.metrics.cells.DistributionData(4, 2, 1, 3))
     self.assertEqual(dist.committed.mean, 2.0)
-    self.assertEqual(gaug.committed.value, 3)
+
+    if check_gauge:
+      gaug, = res.metrics().query(
+          beam.metrics.MetricsFilter().with_step('gauge'))['gauges']
+      self.assertEqual(gaug.committed.value, 3)
 
   def test_callbacks_with_exception(self):
     elements_list = ['1', '2']
@@ -1087,23 +1138,13 @@
             default_environment=environments.EmbeddedPythonGrpcEnvironment()))
 
 
-class FnApiRunnerTestWithGrpcMultiThreaded(FnApiRunnerTest):
-
-  def create_pipeline(self):
-    return beam.Pipeline(
-        runner=fn_api_runner.FnApiRunner(
-            default_environment=environments.EmbeddedPythonGrpcEnvironment(
-                num_workers=2,
-                state_cache_size=fn_api_runner.STATE_CACHE_SIZE)))
-
-
 class FnApiRunnerTestWithDisabledCaching(FnApiRunnerTest):
 
   def create_pipeline(self):
     return beam.Pipeline(
         runner=fn_api_runner.FnApiRunner(
             default_environment=environments.EmbeddedPythonGrpcEnvironment(
-                num_workers=2, state_cache_size=0)))
+                state_cache_size=0)))
 
 
 class FnApiRunnerTestWithMultiWorkers(FnApiRunnerTest):
@@ -1123,6 +1164,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 +1186,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 +1219,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 +1390,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/fn_api_runner_transforms.py b/sdks/python/apache_beam/runners/portability/fn_api_runner_transforms.py
index 4f3e2f9..91f106f 100644
--- a/sdks/python/apache_beam/runners/portability/fn_api_runner_transforms.py
+++ b/sdks/python/apache_beam/runners/portability/fn_api_runner_transforms.py
@@ -588,122 +588,175 @@
 
   ... -> PreCombine -> GBK -> MergeAccumulators -> ExtractOutput -> ...
   """
+  def is_compatible_with_combiner_lifting(trigger):
+    '''Returns whether this trigger is compatible with combiner lifting.
+
+    Certain triggers, such as those that fire after a certain number of
+    elements, need to observe every element, and as such are incompatible
+    with combiner lifting (which may aggregate several elements into one
+    before they reach the triggering code after shuffle).
+    '''
+    if trigger is None:
+      return True
+    elif trigger.WhichOneof('trigger') in (
+        'default', 'always', 'never', 'after_processing_time',
+        'after_synchronized_processing_time'):
+      return True
+    elif trigger.HasField('element_count'):
+      return trigger.element_count.element_count == 1
+    elif trigger.HasField('after_end_of_window'):
+      return is_compatible_with_combiner_lifting(
+          trigger.after_end_of_window.early_firings
+          ) and is_compatible_with_combiner_lifting(
+              trigger.after_end_of_window.late_firings)
+    elif trigger.HasField('after_any'):
+      return all(
+          is_compatible_with_combiner_lifting(t)
+          for t in trigger.after_any.subtriggers)
+    elif trigger.HasField('repeat'):
+      return is_compatible_with_combiner_lifting(trigger.repeat.subtrigger)
+    else:
+      return False
+
+  def can_lift(combine_per_key_transform):
+    windowing = context.components.windowing_strategies[
+        context.components.pcollections[
+            only_element(list(combine_per_key_transform.inputs.values()))
+        ].windowing_strategy_id]
+    if windowing.output_time != beam_runner_api_pb2.OutputTime.END_OF_WINDOW:
+      # This depends on the spec of PartialGroupByKey.
+      return False
+    elif not is_compatible_with_combiner_lifting(windowing.trigger):
+      return False
+    else:
+      return True
+
+  def make_stage(base_stage, transform):
+    return Stage(
+        transform.unique_name,
+        [transform],
+        downstream_side_inputs=base_stage.downstream_side_inputs,
+        must_follow=base_stage.must_follow,
+        parent=base_stage,
+        environment=base_stage.environment)
+
+  def lifted_stages(stage):
+    transform = stage.transforms[0]
+    combine_payload = proto_utils.parse_Bytes(
+        transform.spec.payload, beam_runner_api_pb2.CombinePayload)
+
+    input_pcoll = context.components.pcollections[only_element(
+        list(transform.inputs.values()))]
+    output_pcoll = context.components.pcollections[only_element(
+        list(transform.outputs.values()))]
+
+    element_coder_id = input_pcoll.coder_id
+    element_coder = context.components.coders[element_coder_id]
+    key_coder_id, _ = element_coder.component_coder_ids
+    accumulator_coder_id = combine_payload.accumulator_coder_id
+
+    key_accumulator_coder = beam_runner_api_pb2.Coder(
+        spec=beam_runner_api_pb2.FunctionSpec(
+            urn=common_urns.coders.KV.urn),
+        component_coder_ids=[key_coder_id, accumulator_coder_id])
+    key_accumulator_coder_id = context.add_or_get_coder_id(
+        key_accumulator_coder)
+
+    accumulator_iter_coder = beam_runner_api_pb2.Coder(
+        spec=beam_runner_api_pb2.FunctionSpec(
+            urn=common_urns.coders.ITERABLE.urn),
+        component_coder_ids=[accumulator_coder_id])
+    accumulator_iter_coder_id = context.add_or_get_coder_id(
+        accumulator_iter_coder)
+
+    key_accumulator_iter_coder = beam_runner_api_pb2.Coder(
+        spec=beam_runner_api_pb2.FunctionSpec(
+            urn=common_urns.coders.KV.urn),
+        component_coder_ids=[key_coder_id, accumulator_iter_coder_id])
+    key_accumulator_iter_coder_id = context.add_or_get_coder_id(
+        key_accumulator_iter_coder)
+
+    precombined_pcoll_id = unique_name(
+        context.components.pcollections, 'pcollection')
+    context.components.pcollections[precombined_pcoll_id].CopyFrom(
+        beam_runner_api_pb2.PCollection(
+            unique_name=transform.unique_name + '/Precombine.out',
+            coder_id=key_accumulator_coder_id,
+            windowing_strategy_id=input_pcoll.windowing_strategy_id,
+            is_bounded=input_pcoll.is_bounded))
+
+    grouped_pcoll_id = unique_name(
+        context.components.pcollections, 'pcollection')
+    context.components.pcollections[grouped_pcoll_id].CopyFrom(
+        beam_runner_api_pb2.PCollection(
+            unique_name=transform.unique_name + '/Group.out',
+            coder_id=key_accumulator_iter_coder_id,
+            windowing_strategy_id=output_pcoll.windowing_strategy_id,
+            is_bounded=output_pcoll.is_bounded))
+
+    merged_pcoll_id = unique_name(
+        context.components.pcollections, 'pcollection')
+    context.components.pcollections[merged_pcoll_id].CopyFrom(
+        beam_runner_api_pb2.PCollection(
+            unique_name=transform.unique_name + '/Merge.out',
+            coder_id=key_accumulator_coder_id,
+            windowing_strategy_id=output_pcoll.windowing_strategy_id,
+            is_bounded=output_pcoll.is_bounded))
+
+    yield make_stage(
+        stage,
+        beam_runner_api_pb2.PTransform(
+            unique_name=transform.unique_name + '/Precombine',
+            spec=beam_runner_api_pb2.FunctionSpec(
+                urn=common_urns.combine_components
+                .COMBINE_PER_KEY_PRECOMBINE.urn,
+                payload=transform.spec.payload),
+            inputs=transform.inputs,
+            outputs={'out': precombined_pcoll_id}))
+
+    yield make_stage(
+        stage,
+        beam_runner_api_pb2.PTransform(
+            unique_name=transform.unique_name + '/Group',
+            spec=beam_runner_api_pb2.FunctionSpec(
+                urn=common_urns.primitives.GROUP_BY_KEY.urn),
+            inputs={'in': precombined_pcoll_id},
+            outputs={'out': grouped_pcoll_id}))
+
+    yield make_stage(
+        stage,
+        beam_runner_api_pb2.PTransform(
+            unique_name=transform.unique_name + '/Merge',
+            spec=beam_runner_api_pb2.FunctionSpec(
+                urn=common_urns.combine_components
+                .COMBINE_PER_KEY_MERGE_ACCUMULATORS.urn,
+                payload=transform.spec.payload),
+            inputs={'in': grouped_pcoll_id},
+            outputs={'out': merged_pcoll_id}))
+
+    yield make_stage(
+        stage,
+        beam_runner_api_pb2.PTransform(
+            unique_name=transform.unique_name + '/ExtractOutputs',
+            spec=beam_runner_api_pb2.FunctionSpec(
+                urn=common_urns.combine_components
+                .COMBINE_PER_KEY_EXTRACT_OUTPUTS.urn,
+                payload=transform.spec.payload),
+            inputs={'in': merged_pcoll_id},
+            outputs=transform.outputs))
+
+  def unlifted_stages(stage):
+    transform = stage.transforms[0]
+    for sub in transform.subtransforms:
+      yield make_stage(stage, context.components.transforms[sub])
+
   for stage in stages:
     assert len(stage.transforms) == 1
     transform = stage.transforms[0]
     if transform.spec.urn == common_urns.composites.COMBINE_PER_KEY.urn:
-      combine_payload = proto_utils.parse_Bytes(
-          transform.spec.payload, beam_runner_api_pb2.CombinePayload)
-
-      input_pcoll = context.components.pcollections[only_element(
-          list(transform.inputs.values()))]
-      output_pcoll = context.components.pcollections[only_element(
-          list(transform.outputs.values()))]
-
-      element_coder_id = input_pcoll.coder_id
-      element_coder = context.components.coders[element_coder_id]
-      key_coder_id, _ = element_coder.component_coder_ids
-      accumulator_coder_id = combine_payload.accumulator_coder_id
-
-      key_accumulator_coder = beam_runner_api_pb2.Coder(
-          spec=beam_runner_api_pb2.FunctionSpec(
-              urn=common_urns.coders.KV.urn),
-          component_coder_ids=[key_coder_id, accumulator_coder_id])
-      key_accumulator_coder_id = context.add_or_get_coder_id(
-          key_accumulator_coder)
-
-      accumulator_iter_coder = beam_runner_api_pb2.Coder(
-          spec=beam_runner_api_pb2.FunctionSpec(
-              urn=common_urns.coders.ITERABLE.urn),
-          component_coder_ids=[accumulator_coder_id])
-      accumulator_iter_coder_id = context.add_or_get_coder_id(
-          accumulator_iter_coder)
-
-      key_accumulator_iter_coder = beam_runner_api_pb2.Coder(
-          spec=beam_runner_api_pb2.FunctionSpec(
-              urn=common_urns.coders.KV.urn),
-          component_coder_ids=[key_coder_id, accumulator_iter_coder_id])
-      key_accumulator_iter_coder_id = context.add_or_get_coder_id(
-          key_accumulator_iter_coder)
-
-      precombined_pcoll_id = unique_name(
-          context.components.pcollections, 'pcollection')
-      context.components.pcollections[precombined_pcoll_id].CopyFrom(
-          beam_runner_api_pb2.PCollection(
-              unique_name=transform.unique_name + '/Precombine.out',
-              coder_id=key_accumulator_coder_id,
-              windowing_strategy_id=input_pcoll.windowing_strategy_id,
-              is_bounded=input_pcoll.is_bounded))
-
-      grouped_pcoll_id = unique_name(
-          context.components.pcollections, 'pcollection')
-      context.components.pcollections[grouped_pcoll_id].CopyFrom(
-          beam_runner_api_pb2.PCollection(
-              unique_name=transform.unique_name + '/Group.out',
-              coder_id=key_accumulator_iter_coder_id,
-              windowing_strategy_id=output_pcoll.windowing_strategy_id,
-              is_bounded=output_pcoll.is_bounded))
-
-      merged_pcoll_id = unique_name(
-          context.components.pcollections, 'pcollection')
-      context.components.pcollections[merged_pcoll_id].CopyFrom(
-          beam_runner_api_pb2.PCollection(
-              unique_name=transform.unique_name + '/Merge.out',
-              coder_id=key_accumulator_coder_id,
-              windowing_strategy_id=output_pcoll.windowing_strategy_id,
-              is_bounded=output_pcoll.is_bounded))
-
-      def make_stage(base_stage, transform):
-        return Stage(
-            transform.unique_name,
-            [transform],
-            downstream_side_inputs=base_stage.downstream_side_inputs,
-            must_follow=base_stage.must_follow,
-            parent=base_stage,
-            environment=base_stage.environment)
-
-      yield make_stage(
-          stage,
-          beam_runner_api_pb2.PTransform(
-              unique_name=transform.unique_name + '/Precombine',
-              spec=beam_runner_api_pb2.FunctionSpec(
-                  urn=common_urns.combine_components
-                  .COMBINE_PER_KEY_PRECOMBINE.urn,
-                  payload=transform.spec.payload),
-              inputs=transform.inputs,
-              outputs={'out': precombined_pcoll_id}))
-
-      yield make_stage(
-          stage,
-          beam_runner_api_pb2.PTransform(
-              unique_name=transform.unique_name + '/Group',
-              spec=beam_runner_api_pb2.FunctionSpec(
-                  urn=common_urns.primitives.GROUP_BY_KEY.urn),
-              inputs={'in': precombined_pcoll_id},
-              outputs={'out': grouped_pcoll_id}))
-
-      yield make_stage(
-          stage,
-          beam_runner_api_pb2.PTransform(
-              unique_name=transform.unique_name + '/Merge',
-              spec=beam_runner_api_pb2.FunctionSpec(
-                  urn=common_urns.combine_components
-                  .COMBINE_PER_KEY_MERGE_ACCUMULATORS.urn,
-                  payload=transform.spec.payload),
-              inputs={'in': grouped_pcoll_id},
-              outputs={'out': merged_pcoll_id}))
-
-      yield make_stage(
-          stage,
-          beam_runner_api_pb2.PTransform(
-              unique_name=transform.unique_name + '/ExtractOutputs',
-              spec=beam_runner_api_pb2.FunctionSpec(
-                  urn=common_urns.combine_components
-                  .COMBINE_PER_KEY_EXTRACT_OUTPUTS.urn,
-                  payload=transform.spec.payload),
-              inputs={'in': merged_pcoll_id},
-              outputs=transform.outputs))
-
+      expansion = lifted_stages if can_lift(transform) else unlifted_stages
+      for substage in expansion(stage):
+        yield substage
     else:
       yield stage
 
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 4305810..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()
@@ -195,7 +195,7 @@
     self._state = None
     self._state_queues = []
     self._log_queues = []
-    self.state = beam_job_api_pb2.JobState.STARTING
+    self.state = beam_job_api_pb2.JobState.STOPPED
     self.daemon = True
     self.result = None
 
@@ -220,10 +220,12 @@
     return self._artifact_staging_endpoint
 
   def run(self):
+    self.state = beam_job_api_pb2.JobState.STARTING
     self._run_thread = threading.Thread(target=self._run_job)
     self._run_thread.start()
 
   def _run_job(self):
+    self.state = beam_job_api_pb2.JobState.RUNNING
     with JobLogHandler(self._log_queues):
       try:
         result = fn_api_runner.FnApiRunner(
@@ -239,7 +241,7 @@
         raise
 
   def cancel(self):
-    if self.state not in abstract_job_service.TERMINAL_STATES:
+    if not self.is_terminal_state(self.state):
       self.state = beam_job_api_pb2.JobState.CANCELLING
       # TODO(robertwb): Actually cancel...
       self.state = beam_job_api_pb2.JobState.CANCELLED
@@ -253,7 +255,7 @@
     while True:
       current_state = state_queue.get(block=True)
       yield current_state
-      if current_state in abstract_job_service.TERMINAL_STATES:
+      if self.is_terminal_state(current_state):
         break
 
   def get_message_stream(self):
@@ -264,7 +266,7 @@
 
     current_state = self.state
     yield current_state
-    while current_state not in abstract_job_service.TERMINAL_STATES:
+    while not self.is_terminal_state(current_state):
       msg = log_queue.get(block=True)
       yield msg
       if isinstance(msg, int):
diff --git a/sdks/python/apache_beam/runners/portability/portable_runner.py b/sdks/python/apache_beam/runners/portability/portable_runner.py
index 632f940..c3e4176 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
@@ -58,7 +56,7 @@
 
 TERMINAL_STATES = [
     beam_job_api_pb2.JobState.DONE,
-    beam_job_api_pb2.JobState.STOPPED,
+    beam_job_api_pb2.JobState.DRAINED,
     beam_job_api_pb2.JobState.FAILED,
     beam_job_api_pb2.JobState.CANCELLED,
 ]
@@ -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
@@ -157,7 +139,6 @@
               'use_loopback_process_worker', False)
       portable_options.environment_config, server = (
           worker_pool_main.BeamFnExternalWorkerPoolServicer.start(
-              sdk_worker_main._get_worker_count(options),
               state_cache_size=sdk_worker_main._get_state_cache_size(options),
               use_process=use_loopback_process_worker))
       cleanup_callbacks = [functools.partial(server.stop, 1)]
@@ -280,9 +261,12 @@
     prepare_response = job_service.Prepare(
         prepare_request,
         timeout=portable_options.job_server_timeout)
-    if prepare_response.artifact_staging_endpoint.url:
+    artifact_endpoint = (portable_options.artifact_endpoint
+                         if portable_options.artifact_endpoint
+                         else prepare_response.artifact_staging_endpoint.url)
+    if artifact_endpoint:
       stager = portable_stager.PortableStager(
-          grpc.insecure_channel(prepare_response.artifact_staging_endpoint.url),
+          grpc.insecure_channel(artifact_endpoint),
           prepare_response.staging_session_token)
       retrieval_token, _ = stager.stage_job_resources(
           options,
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/portability/spark_runner.py b/sdks/python/apache_beam/runners/portability/spark_runner.py
new file mode 100644
index 0000000..ca03310
--- /dev/null
+++ b/sdks/python/apache_beam/runners/portability/spark_runner.py
@@ -0,0 +1,84 @@
+#
+# 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.
+#
+
+"""A runner for executing portable pipelines on Spark."""
+
+from __future__ import absolute_import
+from __future__ import print_function
+
+import re
+
+from apache_beam.options import pipeline_options
+from apache_beam.runners.portability import job_server
+from apache_beam.runners.portability import portable_runner
+
+# https://spark.apache.org/docs/latest/submitting-applications.html#master-urls
+LOCAL_MASTER_PATTERN = r'^local(\[.+\])?$'
+
+
+class SparkRunner(portable_runner.PortableRunner):
+  def run_pipeline(self, pipeline, options):
+    spark_options = options.view_as(SparkRunnerOptions)
+    portable_options = options.view_as(pipeline_options.PortableOptions)
+    if (re.match(LOCAL_MASTER_PATTERN, spark_options.spark_master_url)
+        and not portable_options.environment_type
+        and not portable_options.output_executable_path):
+      portable_options.environment_type = 'LOOPBACK'
+    return super(SparkRunner, self).run_pipeline(pipeline, options)
+
+  def default_job_server(self, options):
+    # TODO(BEAM-8139) submit a Spark jar to a cluster
+    return job_server.StopOnExitJobServer(SparkJarJobServer(options))
+
+
+class SparkRunnerOptions(pipeline_options.PipelineOptions):
+  @classmethod
+  def _add_argparse_args(cls, parser):
+    parser.add_argument('--spark_master_url',
+                        default='local[4]',
+                        help='Spark master URL (spark://HOST:PORT). '
+                             'Use "local" (single-threaded) or "local[*]" '
+                             '(multi-threaded) to start a local cluster for '
+                             'the execution.')
+    parser.add_argument('--spark_job_server_jar',
+                        help='Path or URL to a Beam Spark jobserver jar.')
+    parser.add_argument('--artifacts_dir', default=None)
+
+
+class SparkJarJobServer(job_server.JavaJarJobServer):
+  def __init__(self, options):
+    super(SparkJarJobServer, self).__init__()
+    options = options.view_as(SparkRunnerOptions)
+    self._jar = options.spark_job_server_jar
+    self._master_url = options.spark_master_url
+    self._artifacts_dir = options.artifacts_dir
+
+  def path_to_jar(self):
+    if self._jar:
+      return self._jar
+    else:
+      return self.path_to_beam_jar('runners:spark:job-server:shadowJar')
+
+  def java_arguments(self, job_port, artifacts_dir):
+    return [
+        '--spark-master-url', self._master_url,
+        '--artifacts-dir', (self._artifacts_dir
+                            if self._artifacts_dir else artifacts_dir),
+        '--job-port', job_port,
+        '--artifact-port', 0,
+        '--expansion-port', 0
+    ]
diff --git a/sdks/python/apache_beam/runners/runner.py b/sdks/python/apache_beam/runners/runner.py
index 1000480..fe9c492 100644
--- a/sdks/python/apache_beam/runners/runner.py
+++ b/sdks/python/apache_beam/runners/runner.py
@@ -38,6 +38,7 @@
     'apache_beam.runners.interactive.interactive_runner.InteractiveRunner',
     'apache_beam.runners.portability.flink_runner.FlinkRunner',
     'apache_beam.runners.portability.portable_runner.PortableRunner',
+    'apache_beam.runners.portability.spark_runner.SparkRunner',
     'apache_beam.runners.test.TestDirectRunner',
     'apache_beam.runners.test.TestDataflowRunner',
 )
@@ -328,7 +329,7 @@
 
   @classmethod
   def is_terminal(cls, state):
-    return state in [cls.STOPPED, cls.DONE, cls.FAILED, cls.CANCELLED,
+    return state in [cls.DONE, cls.FAILED, cls.CANCELLED,
                      cls.UPDATED, cls.DRAINED]
 
 
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.py b/sdks/python/apache_beam/runners/worker/data_plane.py
index 8324e6b..26e4b60 100644
--- a/sdks/python/apache_beam/runners/worker/data_plane.py
+++ b/sdks/python/apache_beam/runners/worker/data_plane.py
@@ -173,7 +173,7 @@
 
   def __init__(self):
     self._to_send = queue.Queue()
-    self._received = collections.defaultdict(queue.Queue)
+    self._received = collections.defaultdict(lambda: queue.Queue(maxsize=5))
     self._receive_lock = threading.Lock()
     self._reads_finished = threading.Event()
     self._closed = False
@@ -267,7 +267,6 @@
         yield beam_fn_api_pb2.Elements(data=data)
 
   def _read_inputs(self, elements_iterator):
-    # TODO(robertwb): Pushback/throttling to avoid unbounded buffering.
     try:
       for elements in elements_iterator:
         for data in elements.data:
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..488f505 100644
--- a/sdks/python/apache_beam/runners/worker/sdk_worker.py
+++ b/sdks/python/apache_beam/runners/worker/sdk_worker.py
@@ -27,11 +27,8 @@
 import queue
 import sys
 import threading
-import time
 import traceback
 from builtins import object
-from builtins import range
-from concurrent import futures
 
 import grpc
 from future.utils import raise_
@@ -45,6 +42,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.
@@ -54,17 +52,15 @@
 
 class SdkHarness(object):
   REQUEST_METHOD_PREFIX = '_request_'
-  SCHEDULING_DELAY_THRESHOLD_SEC = 5*60  # 5 Minutes
 
   def __init__(
-      self, control_address, worker_count,
+      self, control_address,
       credentials=None,
       worker_id=None,
       # Caching is disabled by default
       state_cache_size=0,
       profiler_factory=None):
     self._alive = True
-    self._worker_count = worker_count
     self._worker_index = 0
     self._worker_id = worker_id
     self._state_cache = StateCache(state_cache_size)
@@ -94,43 +90,14 @@
         fns=self._fns)
     # workers for process/finalize bundle.
     self.workers = queue.Queue()
-    # 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)
-    # finalize and process share one thread pool.
-    self._process_thread_pool = futures.ThreadPoolExecutor(
-        max_workers=self._worker_count)
+    self._worker_thread_pool = UnboundedThreadPoolExecutor()
     self._responses = queue.Queue()
-    self._process_bundle_queue = queue.Queue()
-    self._unscheduled_process_bundle = {}
-    logging.info('Initializing SDKHarness with %s workers.', self._worker_count)
+    logging.info('Initializing SDKHarness with unbounded number of workers.')
 
   def run(self):
     control_stub = beam_fn_api_pb2_grpc.BeamFnControlStub(self._control_channel)
     no_more_work = object()
 
-    # Create process workers
-    for _ in range(self._worker_count):
-      # SdkHarness manage function registration and share self._fns with all
-      # the workers. This is needed because function registration (register)
-      # and execution (process_bundle) are send over different request and we
-      # do not really know which worker is going to process bundle
-      # for a function till we get process_bundle request. Moreover
-      # same function is reused by different process bundle calls and
-      # potentially get executed by different worker. Hence we need a
-      # centralized function list shared among all the workers.
-      self.workers.put(
-          SdkWorker(self._bundle_processor_cache,
-                    state_cache_metrics_fn=
-                    self._state_cache.get_monitoring_infos,
-                    profiler_factory=self._profiler_factory))
-
     def get_responses():
       while True:
         response = self._responses.get()
@@ -139,10 +106,6 @@
         yield response
 
     self._alive = True
-    monitoring_thread = threading.Thread(name='SdkHarness_monitor',
-                                         target=self._monitor_process_bundle)
-    monitoring_thread.daemon = True
-    monitoring_thread.start()
 
     try:
       for work_request in control_stub.Control(get_responses()):
@@ -158,8 +121,7 @@
     logging.info('No more requests from control plane')
     logging.info('SDK Harness waiting for in-flight requests to complete')
     # Wait until existing requests are processed.
-    self._progress_thread_pool.shutdown()
-    self._process_thread_pool.shutdown()
+    self._worker_thread_pool.shutdown()
     # get_responses may be blocked on responses.get(), but we need to return
     # control to its caller.
     self._responses.put(no_more_work)
@@ -187,22 +149,15 @@
   def _request_process_bundle(self, request):
 
     def task():
-      # Take the free worker. Wait till a worker is free.
-      worker = self.workers.get()
-      # Get the first work item in the queue
-      work = self._process_bundle_queue.get()
-      self._unscheduled_process_bundle.pop(work.instruction_id, None)
+      worker = self._get_or_create_worker()
       try:
-        self._execute(lambda: worker.do_instruction(work), work)
+        self._execute(lambda: worker.do_instruction(request), request)
       finally:
         # Put the worker back in the free worker pool
         self.workers.put(worker)
-    # Create a task for each process_bundle request and schedule it
-    self._process_bundle_queue.put(request)
-    self._unscheduled_process_bundle[request.instruction_id] = time.time()
-    self._process_thread_pool.submit(task)
+    self._worker_thread_pool.submit(task)
     logging.debug(
-        "Currently using %s threads." % len(self._process_thread_pool._threads))
+        "Currently using %s threads." % len(self._worker_thread_pool._workers))
 
   def _request_process_bundle_split(self, request):
     self._request_process_bundle_action(request)
@@ -218,17 +173,19 @@
       # only process progress/split request when a bundle is in processing.
       if (instruction_id in
           self._bundle_processor_cache.active_bundle_processors):
-        self._execute(
-            lambda: self.progress_worker.do_instruction(request), request)
+        worker = self._get_or_create_worker()
+        try:
+          self._execute(lambda: worker.do_instruction(request), request)
+        finally:
+          # Put the worker back in the free worker pool
+          self.workers.put(worker)
       else:
         self._execute(lambda: beam_fn_api_pb2.InstructionResponse(
             instruction_id=request.instruction_id, error=(
-                'Process bundle request not yet scheduled for instruction {}' if
-                instruction_id in self._unscheduled_process_bundle else
                 'Unknown process bundle instruction {}').format(
                     instruction_id)), request)
 
-    self._progress_thread_pool.submit(task)
+    self._worker_thread_pool.submit(task)
 
   def _request_finalize_bundle(self, request):
     self._request_execute(request)
@@ -237,37 +194,23 @@
 
     def task():
       # Get one available worker.
-      worker = self.workers.get()
+      worker = self._get_or_create_worker()
       try:
-        self._execute(
-            lambda: worker.do_instruction(request), request)
+        self._execute(lambda: worker.do_instruction(request), request)
       finally:
         # Put the worker back in the free worker pool.
         self.workers.put(worker)
 
-    self._process_thread_pool.submit(task)
+    self._worker_thread_pool.submit(task)
 
-  def _monitor_process_bundle(self):
-    """
-    Monitor the unscheduled bundles and log if a bundle is not scheduled for
-    more than SCHEDULING_DELAY_THRESHOLD_SEC.
-    """
-    while self._alive:
-      time.sleep(SdkHarness.SCHEDULING_DELAY_THRESHOLD_SEC)
-      # Check for bundles to be scheduled.
-      if self._unscheduled_process_bundle:
-        current_time = time.time()
-        for instruction_id in self._unscheduled_process_bundle:
-          request_time = None
-          try:
-            request_time = self._unscheduled_process_bundle[instruction_id]
-          except KeyError:
-            pass
-          if request_time:
-            scheduling_delay = current_time - request_time
-            if scheduling_delay > SdkHarness.SCHEDULING_DELAY_THRESHOLD_SEC:
-              logging.warning('Unable to schedule instruction %s for %s',
-                              instruction_id, scheduling_delay)
+  def _get_or_create_worker(self):
+    try:
+      return self.workers.get_nowait()
+    except queue.Empty:
+      return SdkWorker(self._bundle_processor_cache,
+                       state_cache_metrics_fn=
+                       self._state_cache.get_monitoring_infos,
+                       profiler_factory=self._profiler_factory)
 
 
 class BundleProcessorCache(object):
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..2467965 100644
--- a/sdks/python/apache_beam/runners/worker/sdk_worker_main.py
+++ b/sdks/python/apache_beam/runners/worker/sdk_worker_main.py
@@ -147,7 +147,6 @@
     assert not service_descriptor.oauth2_client_credentials_grant.url
     SdkHarness(
         control_address=service_descriptor.url,
-        worker_count=_get_worker_count(sdk_pipeline_options),
         worker_id=_worker_id,
         state_cache_size=_get_state_cache_size(sdk_pipeline_options),
         profiler_factory=profiler.Profile.factory_from_options(
@@ -177,35 +176,6 @@
     })
 
 
-def _get_worker_count(pipeline_options):
-  """Extract worker count from the pipeline_options.
-
-  This defines how many SdkWorkers will be started in this Python process.
-  And each SdkWorker will have its own thread to process data. Name of the
-  experimental parameter is 'worker_threads'
-  Example Usage in the Command Line:
-    --experimental worker_threads=1
-
-  Note: worker_threads is an experimental flag and might not be available in
-  future releases.
-
-  Returns:
-    an int containing the worker_threads to use. Default is 12
-  """
-  experiments = pipeline_options.view_as(DebugOptions).experiments
-
-  experiments = experiments if experiments else []
-
-  for experiment in experiments:
-    # There should only be 1 match so returning from the loop
-    if re.match(r'worker_threads=', experiment):
-      return int(
-          re.match(r'worker_threads=(?P<worker_threads>.*)',
-                   experiment).group('worker_threads'))
-
-  return 12
-
-
 def _get_state_cache_size(pipeline_options):
   """Defines the upper number of state items to cache.
 
diff --git a/sdks/python/apache_beam/runners/worker/sdk_worker_main_test.py b/sdks/python/apache_beam/runners/worker/sdk_worker_main_test.py
index 9703515..cae65a2 100644
--- a/sdks/python/apache_beam/runners/worker/sdk_worker_main_test.py
+++ b/sdks/python/apache_beam/runners/worker/sdk_worker_main_test.py
@@ -20,7 +20,6 @@
 from __future__ import division
 from __future__ import print_function
 
-import json
 import logging
 import unittest
 
@@ -56,40 +55,24 @@
 
     wrapped_method_for_test()
 
-  def test_work_count_default_value(self):
-    self._check_worker_count('{}', 12)
-
   def test_parse_pipeline_options(self):
     expected_options = PipelineOptions([])
     expected_options.view_as(
-        SdkWorkerMainTest.MockOptions).m_m_option = [
-            'worker_threads=1', 'beam_fn_api'
-        ]
+        SdkWorkerMainTest.MockOptions).m_m_option = ['beam_fn_api']
     expected_options.view_as(
         SdkWorkerMainTest.MockOptions).m_option = '/tmp/requirements.txt'
     self.assertEqual(
-        {'m_m_option': ['worker_threads=1']},
-        sdk_worker_main._parse_pipeline_options(
-            '{"options": {"m_m_option":["worker_threads=1"]}}')
-        .get_all_options(drop_default=True))
-    self.assertEqual(
         expected_options.get_all_options(),
         sdk_worker_main._parse_pipeline_options(
             '{"options": {' +
             '"m_option": "/tmp/requirements.txt", ' +
-            '"m_m_option":["worker_threads=1", "beam_fn_api"]' +
+            '"m_m_option":["beam_fn_api"]' +
             '}}').get_all_options())
     self.assertEqual(
-        {'m_m_option': ['worker_threads=1']},
-        sdk_worker_main._parse_pipeline_options(
-            '{"beam:option:m_m_option:v1":["worker_threads=1"]}')
-        .get_all_options(drop_default=True))
-    self.assertEqual(
         expected_options.get_all_options(),
         sdk_worker_main._parse_pipeline_options(
             '{"beam:option:m_option:v1": "/tmp/requirements.txt", ' +
-            '"beam:option:m_m_option:v1":["worker_threads=1", ' +
-            '"beam_fn_api"]}').get_all_options())
+            '"beam:option:m_m_option:v1":["beam_fn_api"]}').get_all_options())
     self.assertEqual(
         {'beam:option:m_option:v': 'mock_val'},
         sdk_worker_main._parse_pipeline_options(
@@ -106,30 +89,6 @@
             '{"options": {"eam:option:m_option:v":"mock_val"}}')
         .get_all_options(drop_default=True))
 
-  def test_work_count_custom_value(self):
-    self._check_worker_count('{"experiments":["worker_threads=1"]}', 1)
-    self._check_worker_count('{"experiments":["worker_threads=4"]}', 4)
-    self._check_worker_count('{"experiments":["worker_threads=12"]}', 12)
-
-  def test_work_count_wrong_format(self):
-    self._check_worker_count(
-        '{"experiments":["worker_threads="]}', exception=True)
-    self._check_worker_count(
-        '{"experiments":["worker_threads=a"]}', exception=True)
-    self._check_worker_count(
-        '{"experiments":["worker_threads=1a"]}', exception=True)
-
-  def _check_worker_count(self, pipeline_options, expected=0, exception=False):
-    if exception:
-      self.assertRaises(
-          Exception, sdk_worker_main._get_worker_count,
-          PipelineOptions.from_dictionary(json.loads(pipeline_options)))
-    else:
-      self.assertEqual(
-          sdk_worker_main._get_worker_count(
-              PipelineOptions.from_dictionary(json.loads(pipeline_options))),
-          expected)
-
 
 if __name__ == '__main__':
   logging.getLogger().setLevel(logging.INFO)
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..a422851 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):
@@ -78,7 +78,7 @@
      tuple of request_count, number of process_bundles per request and workers
      counts to process the request.
     """
-    for (request_count, process_bundles_per_request, worker_count) in args:
+    for (request_count, process_bundles_per_request) in args:
       requests = []
       process_bundle_descriptors = []
 
@@ -93,15 +93,14 @@
 
       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")
       server.start()
 
       harness = sdk_worker.SdkHarness(
-          "localhost:%s" % test_port, worker_count=worker_count,
-          state_cache_size=100)
+          "localhost:%s" % test_port, state_cache_size=100)
       harness.run()
 
       for worker in harness.workers.queue:
@@ -110,7 +109,7 @@
                           for item in process_bundle_descriptors})
 
   def test_fn_registration(self):
-    self._check_fn_registration_multi_request((1, 4, 1), (4, 4, 1), (4, 4, 2))
+    self._check_fn_registration_multi_request((1, 4), (4, 4))
 
 
 if __name__ == "__main__":
diff --git a/sdks/python/apache_beam/runners/worker/statesampler_fast.pxd b/sdks/python/apache_beam/runners/worker/statesampler_fast.pxd
index 799bd0d..aebf9f6 100644
--- a/sdks/python/apache_beam/runners/worker/statesampler_fast.pxd
+++ b/sdks/python/apache_beam/runners/worker/statesampler_fast.pxd
@@ -43,6 +43,9 @@
 
   cdef int32_t current_state_index
 
+  cpdef ScopedState current_state(self)
+  cdef inline ScopedState current_state_c(self)
+
   cpdef _scoped_state(
       self, counter_name, name_context, output_counter, metrics_container)
 
@@ -56,7 +59,7 @@
   cdef readonly object name_context
   cdef readonly int64_t _nsecs
   cdef int32_t old_state_index
-  cdef readonly MetricsContainer _metrics_container
+  cdef readonly MetricsContainer metrics_container
 
   cpdef __enter__(self)
 
diff --git a/sdks/python/apache_beam/runners/worker/statesampler_fast.pyx b/sdks/python/apache_beam/runners/worker/statesampler_fast.pyx
index 325ec99..8d2346a 100644
--- a/sdks/python/apache_beam/runners/worker/statesampler_fast.pyx
+++ b/sdks/python/apache_beam/runners/worker/statesampler_fast.pyx
@@ -159,8 +159,12 @@
       (<ScopedState>state)._nsecs = 0
     self.started = self.finished = False
 
-  def current_state(self):
-    return self.scoped_states_by_index[self.current_state_index]
+  cpdef ScopedState current_state(self):
+    return self.current_state_c()
+
+  cdef inline ScopedState current_state_c(self):
+    # Faster than cpdef due to self always being a Python subclass.
+    return <ScopedState>self.scoped_states_by_index[self.current_state_index]
 
   cpdef _scoped_state(self, counter_name, name_context, output_counter,
                       metrics_container):
@@ -189,6 +193,11 @@
     pythread.PyThread_release_lock(self.lock)
     return scoped_state
 
+  def update_metric(self, typed_metric_name, value):
+    # Each of these is a cdef lookup.
+    self.current_state_c().metrics_container.get_metric_cell(
+        typed_metric_name).update(value)
+
 
 cdef class ScopedState(object):
   """Context manager class managing transitions for a given sampler state."""
@@ -205,7 +214,7 @@
     self.name_context = step_name_context
     self.state_index = state_index
     self.counter = counter
-    self._metrics_container = metrics_container
+    self.metrics_container = metrics_container
 
   @property
   def nsecs(self):
@@ -232,7 +241,3 @@
     self.sampler.current_state_index = self.old_state_index
     self.sampler.state_transition_count += 1
     pythread.PyThread_release_lock(self.sampler.lock)
-
-  @property
-  def metrics_container(self):
-    return self._metrics_container
diff --git a/sdks/python/apache_beam/runners/worker/statesampler_slow.py b/sdks/python/apache_beam/runners/worker/statesampler_slow.py
index 0091828..fb2592c 100644
--- a/sdks/python/apache_beam/runners/worker/statesampler_slow.py
+++ b/sdks/python/apache_beam/runners/worker/statesampler_slow.py
@@ -50,6 +50,10 @@
     return ScopedState(
         self, counter_name, name_context, output_counter, metrics_container)
 
+  def update_metric(self, typed_metric_name, value):
+    self.current_state().metrics_container.get_metric_cell(
+        typed_metric_name).update(value)
+
   def _enter_state(self, state):
     self.state_transition_count += 1
     self._state_stack.append(state)
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..6b1437c 100644
--- a/sdks/python/apache_beam/runners/worker/worker_pool_main.py
+++ b/sdks/python/apache_beam/runners/worker/worker_pool_main.py
@@ -35,36 +35,34 @@
 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(
     beam_fn_api_pb2_grpc.BeamFnExternalWorkerPoolServicer):
 
-  def __init__(self, worker_threads,
+  def __init__(self,
                use_process=False,
                container_executable=None,
                state_cache_size=0):
-    self._worker_threads = worker_threads
     self._use_process = use_process
     self._container_executable = container_executable
     self._state_cache_size = state_cache_size
     self._worker_processes = {}
 
   @classmethod
-  def start(cls, worker_threads=1, use_process=False, port=0,
+  def start(cls, 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,
-                      use_process=use_process,
+    worker_pool = cls(use_process=use_process,
                       container_executable=container_executable,
                       state_cache_size=state_cache_size)
     beam_fn_api_pb2_grpc.add_BeamFnExternalWorkerPoolServicer_to_server(
@@ -88,13 +86,11 @@
                    'import SdkHarness; '
                    'SdkHarness('
                    '"%s",'
-                   'worker_count=%d,'
                    'worker_id="%s",'
                    'state_cache_size=%d'
                    ')'
                    '.run()' % (
                        start_worker_request.control_endpoint.url,
-                       self._worker_threads,
                        start_worker_request.worker_id,
                        self._state_cache_size)]
         if self._container_executable:
@@ -120,7 +116,6 @@
       else:
         worker = sdk_worker.SdkHarness(
             start_worker_request.control_endpoint.url,
-            worker_count=self._worker_threads,
             worker_id=start_worker_request.worker_id,
             state_cache_size=self._state_cache_size)
         worker_thread = threading.Thread(
@@ -157,11 +152,6 @@
   """Entry point for worker pool service for external environments."""
 
   parser = argparse.ArgumentParser()
-  parser.add_argument('--threads_per_worker',
-                      type=int,
-                      default=argparse.SUPPRESS,
-                      dest='worker_threads',
-                      help='Number of threads per SDK worker.')
   parser.add_argument('--container_executable',
                       type=str,
                       default=None,
diff --git a/sdks/python/apache_beam/testing/benchmarks/nexmark/nexmark_util.py b/sdks/python/apache_beam/testing/benchmarks/nexmark/nexmark_util.py
index 688d602..916faa4 100644
--- a/sdks/python/apache_beam/testing/benchmarks/nexmark/nexmark_util.py
+++ b/sdks/python/apache_beam/testing/benchmarks/nexmark/nexmark_util.py
@@ -41,6 +41,8 @@
 import apache_beam as beam
 from apache_beam.testing.benchmarks.nexmark.models import nexmark_model
 
+_LOGGER = logging.getLogger(__name__)
+
 
 class Command(object):
   def __init__(self, cmd, args):
@@ -53,7 +55,7 @@
                     timeout, self.cmd.__name__)
 
       self.cmd(*self.args)
-      logging.info('%d seconds elapsed. Thread (%s) finished.',
+      _LOGGER.info('%d seconds elapsed. Thread (%s) finished.',
                    timeout, self.cmd.__name__)
 
     thread = threading.Thread(target=thread_target, name='Thread-timeout')
diff --git a/sdks/python/apache_beam/testing/data/trigger_transcripts.yaml b/sdks/python/apache_beam/testing/data/trigger_transcripts.yaml
index 0e01ad9..cac0c74 100644
--- a/sdks/python/apache_beam/testing/data/trigger_transcripts.yaml
+++ b/sdks/python/apache_beam/testing/data/trigger_transcripts.yaml
@@ -139,6 +139,32 @@
            final: false, index: 3, nonspeculative_index: 1}
 
 ---
+name: discarding_early_fixed
+window_fn: FixedWindows(10)
+trigger_fn: AfterWatermark(early=AfterCount(2))
+timestamp_combiner: OUTPUT_AT_EOW
+accumulation_mode: discarding
+transcript:
+- input: [1, 2, 3]
+- expect:
+  - {window: [0, 9], values: [1, 2, 3], timestamp: 9, early: true, index: 0}
+- input: [4]    # no output
+- input: [14]   # no output
+- input: [5]
+- expect:
+  - {window: [0, 9], values: [4, 5], timestamp: 9, early: true, index: 1}
+- input: [18]
+- expect:
+  - {window: [10, 19], values: [14, 18], timestamp: 19, early: true, index: 0}
+- input: [6]
+- watermark: 100
+- expect:
+  - {window: [0, 9], values:[6], timestamp: 9, early: false, late: false,
+     final: true, index: 2, nonspeculative_index: 0}
+  - {window: [10, 19], values:[], timestamp: 19, early: false, late: false,
+     final: true, index: 1, nonspeculative_index: 0}
+
+---
 name: garbage_collection
 broken_on:
   - SwitchingDirectRunner  # claims pipeline stall
diff --git a/sdks/python/apache_beam/testing/load_tests/load_test_metrics_utils.py b/sdks/python/apache_beam/testing/load_tests/load_test_metrics_utils.py
index c5c5259..2f22b02 100644
--- a/sdks/python/apache_beam/testing/load_tests/load_test_metrics_utils.py
+++ b/sdks/python/apache_beam/testing/load_tests/load_test_metrics_utils.py
@@ -72,6 +72,8 @@
     }
 ]
 
+_LOGGER = logging.getLogger(__name__)
+
 
 def parse_step(step_name):
   """Replaces white spaces and removes 'Step:' label
@@ -330,13 +332,13 @@
     if len(results) > 0:
       log = "Load test results for test: %s and timestamp: %s:" \
             % (results[0][ID_LABEL], results[0][SUBMIT_TIMESTAMP_LABEL])
-      logging.info(log)
+      _LOGGER.info(log)
       for result in results:
         log = "Metric: %s Value: %d" \
               % (result[METRICS_TYPE_LABEL], result[VALUE_LABEL])
-        logging.info(log)
+        _LOGGER.info(log)
     else:
-      logging.info("No test results were collected.")
+      _LOGGER.info("No test results were collected.")
 
 
 class BigQueryMetricsPublisher(object):
@@ -351,7 +353,7 @@
       for output in outputs:
         errors = output['errors']
         for err in errors:
-          logging.error(err['message'])
+          _LOGGER.error(err['message'])
           raise ValueError(
               'Unable save rows in BigQuery: {}'.format(err['message']))
 
diff --git a/sdks/python/apache_beam/testing/pipeline_verifiers.py b/sdks/python/apache_beam/testing/pipeline_verifiers.py
index 1178672..cf99541 100644
--- a/sdks/python/apache_beam/testing/pipeline_verifiers.py
+++ b/sdks/python/apache_beam/testing/pipeline_verifiers.py
@@ -48,6 +48,8 @@
 
 MAX_RETRIES = 4
 
+_LOGGER = logging.getLogger(__name__)
+
 
 class PipelineStateMatcher(BaseMatcher):
   """Matcher that verify pipeline job terminated in expected state
@@ -121,7 +123,7 @@
     if not matched_path:
       raise IOError('No such file or directory: %s' % self.file_path)
 
-    logging.info('Find %d files in %s: \n%s',
+    _LOGGER.info('Find %d files in %s: \n%s',
                  len(matched_path), self.file_path, '\n'.join(matched_path))
     for path in matched_path:
       with FileSystems.open(path, 'r') as f:
@@ -132,7 +134,7 @@
   def _matches(self, _):
     if self.sleep_secs:
       # Wait to have output file ready on FS
-      logging.info('Wait %d seconds...', self.sleep_secs)
+      _LOGGER.info('Wait %d seconds...', self.sleep_secs)
       time.sleep(self.sleep_secs)
 
     # Read from given file(s) path
@@ -140,7 +142,7 @@
 
     # Compute checksum
     self.checksum = utils.compute_hash(read_lines)
-    logging.info('Read from given path %s, %d lines, checksum: %s.',
+    _LOGGER.info('Read from given path %s, %d lines, checksum: %s.',
                  self.file_path, len(read_lines), self.checksum)
     return self.checksum == self.expected_checksum
 
diff --git a/sdks/python/apache_beam/testing/pipeline_verifiers_test.py b/sdks/python/apache_beam/testing/pipeline_verifiers_test.py
index 16ffee9..ec17ef6 100644
--- a/sdks/python/apache_beam/testing/pipeline_verifiers_test.py
+++ b/sdks/python/apache_beam/testing/pipeline_verifiers_test.py
@@ -66,12 +66,11 @@
 
   def test_pipeline_state_matcher_fails(self):
     """Test PipelineStateMatcher fails when using default expected state
-    and job actually finished in CANCELLED/DRAINED/FAILED/STOPPED/UNKNOWN
+    and job actually finished in CANCELLED/DRAINED/FAILED/UNKNOWN
     """
     failed_state = [PipelineState.CANCELLED,
                     PipelineState.DRAINED,
                     PipelineState.FAILED,
-                    PipelineState.STOPPED,
                     PipelineState.UNKNOWN]
 
     for state in failed_state:
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/testing/test_stream.py b/sdks/python/apache_beam/testing/test_stream.py
index 610a1a8..9d9284c 100644
--- a/sdks/python/apache_beam/testing/test_stream.py
+++ b/sdks/python/apache_beam/testing/test_stream.py
@@ -31,6 +31,8 @@
 from apache_beam import coders
 from apache_beam import core
 from apache_beam import pvalue
+from apache_beam.portability import common_urns
+from apache_beam.portability.api import beam_runner_api_pb2
 from apache_beam.transforms import PTransform
 from apache_beam.transforms import window
 from apache_beam.transforms.window import TimestampedValue
@@ -66,6 +68,28 @@
     # TODO(BEAM-5949): Needed for Python 2 compatibility.
     return not self == other
 
+  @abstractmethod
+  def to_runner_api(self, element_coder):
+    raise NotImplementedError
+
+  @staticmethod
+  def from_runner_api(proto, element_coder):
+    if proto.HasField('element_event'):
+      return ElementEvent(
+          [TimestampedValue(
+              element_coder.decode(tv.encoded_element),
+              timestamp.Timestamp(micros=1000 * tv.timestamp))
+           for tv in proto.element_event.elements])
+    elif proto.HasField('watermark_event'):
+      return WatermarkEvent(timestamp.Timestamp(
+          micros=1000 * proto.watermark_event.new_watermark))
+    elif proto.HasField('processing_time_event'):
+      return ProcessingTimeEvent(timestamp.Duration(
+          micros=1000 * proto.processing_time_event.advance_duration))
+    else:
+      raise ValueError(
+          'Unknown TestStream Event type: %s' % proto.WhichOneof('event'))
+
 
 class ElementEvent(Event):
   """Element-producing test stream event."""
@@ -82,6 +106,15 @@
   def __lt__(self, other):
     return self.timestamped_values < other.timestamped_values
 
+  def to_runner_api(self, element_coder):
+    return beam_runner_api_pb2.TestStreamPayload.Event(
+        element_event=beam_runner_api_pb2.TestStreamPayload.Event.AddElements(
+            elements=[
+                beam_runner_api_pb2.TestStreamPayload.TimestampedElement(
+                    encoded_element=element_coder.encode(tv.value),
+                    timestamp=tv.timestamp.micros // 1000)
+                for tv in self.timestamped_values]))
+
 
 class WatermarkEvent(Event):
   """Watermark-advancing test stream event."""
@@ -98,6 +131,11 @@
   def __lt__(self, other):
     return self.new_watermark < other.new_watermark
 
+  def to_runner_api(self, unused_element_coder):
+    return beam_runner_api_pb2.TestStreamPayload.Event(
+        watermark_event
+        =beam_runner_api_pb2.TestStreamPayload.Event.AdvanceWatermark(
+            new_watermark=self.new_watermark.micros // 1000))
 
 class ProcessingTimeEvent(Event):
   """Processing time-advancing test stream event."""
@@ -114,6 +152,12 @@
   def __lt__(self, other):
     return self.advance_by < other.advance_by
 
+  def to_runner_api(self, unused_element_coder):
+    return beam_runner_api_pb2.TestStreamPayload.Event(
+        processing_time_event
+        =beam_runner_api_pb2.TestStreamPayload.Event.AdvanceProcessingTime(
+            advance_duration=self.advance_by.micros // 1000))
+
 
 class TestStream(PTransform):
   """Test stream that generates events on an unbounded PCollection of elements.
@@ -123,11 +167,12 @@
   output.
   """
 
-  def __init__(self, coder=coders.FastPrimitivesCoder()):
+  def __init__(self, coder=coders.FastPrimitivesCoder(), events=()):
+    super(TestStream, self).__init__()
     assert coder is not None
     self.coder = coder
     self.current_watermark = timestamp.MIN_TIMESTAMP
-    self.events = []
+    self.events = list(events)
 
   def get_windowing(self, unused_inputs):
     return core.Windowing(window.GlobalWindows())
@@ -206,3 +251,19 @@
     """
     self._add(ProcessingTimeEvent(advance_by))
     return self
+
+  def to_runner_api_parameter(self, context):
+    return (
+        common_urns.primitives.TEST_STREAM.urn,
+        beam_runner_api_pb2.TestStreamPayload(
+            coder_id=context.coders.get_id(self.coder),
+            events=[e.to_runner_api(self.coder) for e in self.events]))
+
+  @PTransform.register_urn(
+      common_urns.primitives.TEST_STREAM.urn,
+      beam_runner_api_pb2.TestStreamPayload)
+  def from_runner_api_parameter(payload, context):
+    coder = context.coders.get_by_id(payload.coder_id)
+    return TestStream(
+        coder=coder,
+        events=[Event.from_runner_api(e, coder) for e in payload.events])
diff --git a/sdks/python/apache_beam/transforms/core.py b/sdks/python/apache_beam/transforms/core.py
index 148caae..6c48b23 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
@@ -99,6 +101,8 @@
 K = typing.TypeVar('K')
 V = typing.TypeVar('V')
 
+_LOGGER = logging.getLogger(__name__)
+
 
 class DoFnContext(object):
   """A context available to all methods of DoFn instance."""
@@ -242,6 +246,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 +258,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 +271,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 +293,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)
@@ -362,16 +379,16 @@
       else:
         env1 = id_to_proto_map[env_id]
         env2 = context.environments[env_id]
-        assert env1.urn == env2.proto.urn, (
+        assert env1.urn == env2.to_runner_api(context).urn, (
             'Expected environments with the same ID to be equal but received '
             'environments with different URNs '
             '%r and %r',
-            env1.urn, env2.proto.urn)
-        assert env1.payload == env2.proto.payload, (
+            env1.urn, env2.to_runner_api(context).urn)
+        assert env1.payload == env2.to_runner_api(context).payload, (
             'Expected environments with the same ID to be equal but received '
             'environments with different payloads '
             '%r and %r',
-            env1.payload, env2.proto.payload)
+            env1.payload, env2.to_runner_api(context).payload)
     return self._proto
 
   def get_restriction_coder(self):
@@ -379,6 +396,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."""
 
@@ -450,7 +504,7 @@
       try:
         callback()
       except Exception as e:
-        logging.warning("Got exception from finalization call: %s", e)
+        _LOGGER.warning("Got exception from finalization call: %s", e)
 
   def has_callbacks(self):
     return len(self._callbacks) > 0
@@ -459,6 +513,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 +542,7 @@
   TimestampParam = _DoFnParam('TimestampParam')
   WindowParam = _DoFnParam('WindowParam')
   PaneInfoParam = _DoFnParam('PaneInfoParam')
-  WatermarkReporterParam = _DoFnParam('WatermarkReporterParam')
+  WatermarkEstimatorParam = _WatermarkEstimatorParam
   BundleFinalizerParam = _BundleFinalizerParam
   KeyParam = _DoFnParam('KeyParam')
 
@@ -489,7 +554,7 @@
   TimerParam = _TimerDoFnParam
 
   DoFnProcessParams = [ElementParam, SideInputParam, TimestampParam,
-                       WindowParam, WatermarkReporterParam, PaneInfoParam,
+                       WindowParam, WatermarkEstimatorParam, PaneInfoParam,
                        BundleFinalizerParam, KeyParam, StateParam, TimerParam]
 
   RestrictionParam = _RestrictionDoFnParam
@@ -522,7 +587,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:
@@ -684,7 +749,7 @@
       type_hints = type_hints.strip_iterable()
     except ValueError as e:
       # TODO(BEAM-8466): Raise exception here if using stricter type checking.
-      logging.warning('%s: %s', self.display_data()['fn'].value, e)
+      _LOGGER.warning('%s: %s', self.display_data()['fn'].value, e)
     return type_hints
 
   def infer_output_type(self, input_type):
@@ -1155,7 +1220,7 @@
         key_coder = coders.registry.get_coder(typehints.Any)
 
       if not key_coder.is_deterministic():
-        logging.warning(
+        _LOGGER.warning(
             'Key coder %s for transform %s with stateful DoFn may not '
             'be deterministic. This may cause incorrect behavior for complex '
             'key types. Consider adding an input type hint for this transform.',
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..6f67266 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)
@@ -290,13 +308,11 @@
 @Environment.register_urn(python_urns.EMBEDDED_PYTHON_GRPC, bytes)
 class EmbeddedPythonGrpcEnvironment(Environment):
 
-  def __init__(self, num_workers=None, state_cache_size=None):
-    self.num_workers = num_workers
+  def __init__(self, state_cache_size=None):
     self.state_cache_size = state_cache_size
 
   def __eq__(self, other):
     return self.__class__ == other.__class__ \
-           and self.num_workers == other.num_workers \
            and self.state_cache_size == other.state_cache_size
 
   def __ne__(self, other):
@@ -304,34 +320,26 @@
     return not self == other
 
   def __hash__(self):
-    return hash((self.__class__, self.num_workers, self.state_cache_size))
+    return hash((self.__class__, self.state_cache_size))
 
   def __repr__(self):
     repr_parts = []
-    if not self.num_workers is None:
-      repr_parts.append('num_workers=%d' % self.num_workers)
     if not self.state_cache_size is None:
       repr_parts.append('state_cache_size=%d' % self.state_cache_size)
     return 'EmbeddedPythonGrpcEnvironment(%s)' % ','.join(repr_parts)
 
   def to_runner_api_parameter(self, context):
-    if self.num_workers is None and self.state_cache_size is None:
+    if self.state_cache_size is None:
       payload = b''
-    elif self.num_workers is not None and self.state_cache_size is not None:
-      payload = b'%d,%d' % (self.num_workers, self.state_cache_size)
     else:
-      # We want to make sure that the environment stays the same through the
-      # roundtrip to runner api, so here we don't want to set default for the
-      # other if only one of num workers or state cache size is set
-      raise ValueError('Must provide worker num and state cache size.')
+      payload = b'%d' % self.state_cache_size
     return python_urns.EMBEDDED_PYTHON_GRPC, payload
 
   @staticmethod
   def from_runner_api_parameter(payload, context):
     if payload:
-      num_workers, state_cache_size = payload.decode('utf-8').split(',')
+      state_cache_size = payload.decode('utf-8')
       return EmbeddedPythonGrpcEnvironment(
-          num_workers=int(num_workers),
           state_cache_size=int(state_cache_size))
     else:
       return EmbeddedPythonGrpcEnvironment()
@@ -339,8 +347,8 @@
   @classmethod
   def from_options(cls, options):
     if options.environment_config:
-      num_workers, state_cache_size = options.environment_config.split(',')
-      return cls(num_workers=num_workers, state_cache_size=state_cache_size)
+      state_cache_size = options.environment_config
+      return cls(state_cache_size=state_cache_size)
     else:
       return cls()
 
diff --git a/sdks/python/apache_beam/transforms/environments_test.py b/sdks/python/apache_beam/transforms/environments_test.py
index 0fd568c..46868e8 100644
--- a/sdks/python/apache_beam/transforms/environments_test.py
+++ b/sdks/python/apache_beam/transforms/environments_test.py
@@ -46,7 +46,7 @@
         ExternalEnvironment('localhost:8080', params={'k1': 'v1'}),
         EmbeddedPythonEnvironment(),
         EmbeddedPythonGrpcEnvironment(),
-        EmbeddedPythonGrpcEnvironment(num_workers=2, state_cache_size=0),
+        EmbeddedPythonGrpcEnvironment(state_cache_size=0),
         SubprocessSDKEnvironment(command_string=u'foö')):
       context = pipeline_context.PipelineContext()
       self.assertEqual(
@@ -55,13 +55,6 @@
               environment.to_runner_api(context), context)
       )
 
-    with self.assertRaises(ValueError) as ctx:
-      EmbeddedPythonGrpcEnvironment(num_workers=2).to_runner_api(
-          pipeline_context.PipelineContext()
-      )
-    self.assertIn('Must provide worker num and state cache size.',
-                  ctx.exception.args)
-
 
 if __name__ == '__main__':
   logging.getLogger().setLevel(logging.INFO)
diff --git a/sdks/python/apache_beam/transforms/trigger.py b/sdks/python/apache_beam/transforms/trigger.py
index 9761da2..e5bc20d 100644
--- a/sdks/python/apache_beam/transforms/trigger.py
+++ b/sdks/python/apache_beam/transforms/trigger.py
@@ -67,6 +67,9 @@
     ]
 
 
+_LOGGER = logging.getLogger(__name__)
+
+
 class AccumulationMode(object):
   """Controls what to do with data when a trigger fires multiple times."""
   DISCARDING = beam_runner_api_pb2.AccumulationMode.DISCARDING
@@ -1190,7 +1193,7 @@
             window, self.NONSPECULATIVE_INDEX)
         state.add_state(window, self.NONSPECULATIVE_INDEX, 1)
         windowed_value.PaneInfoTiming.LATE
-        logging.warning('Watermark moved backwards in time '
+        _LOGGER.warning('Watermark moved backwards in time '
                         'or late data moved window end forward.')
     else:
       nonspeculative_index = state.get_state(window, self.NONSPECULATIVE_INDEX)
@@ -1320,7 +1323,7 @@
         elif time_domain == TimeDomain.WATERMARK:
           time_marker = watermark
         else:
-          logging.error(
+          _LOGGER.error(
               'TimeDomain error: No timers defined for time domain %s.',
               time_domain)
         if timestamp <= time_marker:
diff --git a/sdks/python/apache_beam/transforms/trigger_test.py b/sdks/python/apache_beam/transforms/trigger_test.py
index dbc4bcd..d1e5433 100644
--- a/sdks/python/apache_beam/transforms/trigger_test.py
+++ b/sdks/python/apache_beam/transforms/trigger_test.py
@@ -20,6 +20,7 @@
 from __future__ import absolute_import
 
 import collections
+import json
 import os.path
 import pickle
 import unittest
@@ -31,6 +32,7 @@
 import yaml
 
 import apache_beam as beam
+from apache_beam import coders
 from apache_beam.options.pipeline_options import PipelineOptions
 from apache_beam.options.pipeline_options import StandardOptions
 from apache_beam.runners import pipeline_context
@@ -39,6 +41,7 @@
 from apache_beam.testing.test_stream import TestStream
 from apache_beam.testing.util import assert_that
 from apache_beam.testing.util import equal_to
+from apache_beam.transforms import ptransform
 from apache_beam.transforms import trigger
 from apache_beam.transforms.core import Windowing
 from apache_beam.transforms.trigger import AccumulationMode
@@ -502,7 +505,10 @@
     while hasattr(cls, unique_name):
       counter += 1
       unique_name = 'test_%s_%d' % (name, counter)
-    setattr(cls, unique_name, lambda self: self._run_log_test(spec))
+    test_method = lambda self: self._run_log_test(spec)
+    test_method.__name__ = unique_name
+    test_method.__test__ = True
+    setattr(cls, unique_name, test_method)
 
   # We must prepend an underscore to this name so that the open-source unittest
   # runner does not execute this method directly as a test.
@@ -606,24 +612,73 @@
         window_fn, trigger_fn, accumulation_mode, timestamp_combiner,
         transcript, spec)
 
-  def _windowed_value_info(self, windowed_value):
-    # Currently some runners operate at the millisecond level, and some at the
-    # microsecond level.  Trigger transcript timestamps are expressed as
-    # integral units of the finest granularity, whatever that may be.
-    # In these tests we interpret them as integral seconds and then truncate
-    # the results to integral seconds to allow for portability across
-    # different sub-second resolutions.
-    window, = windowed_value.windows
-    return {
-        'window': [int(window.start), int(window.max_timestamp())],
-        'values': sorted(windowed_value.value),
-        'timestamp': int(windowed_value.timestamp),
-        'index': windowed_value.pane_info.index,
-        'nonspeculative_index': windowed_value.pane_info.nonspeculative_index,
-        'early': windowed_value.pane_info.timing == PaneInfoTiming.EARLY,
-        'late': windowed_value.pane_info.timing == PaneInfoTiming.LATE,
-        'final': windowed_value.pane_info.is_last,
-    }
+
+def _windowed_value_info(windowed_value):
+  # Currently some runners operate at the millisecond level, and some at the
+  # microsecond level.  Trigger transcript timestamps are expressed as
+  # integral units of the finest granularity, whatever that may be.
+  # In these tests we interpret them as integral seconds and then truncate
+  # the results to integral seconds to allow for portability across
+  # different sub-second resolutions.
+  window, = windowed_value.windows
+  return {
+      'window': [int(window.start), int(window.max_timestamp())],
+      'values': sorted(windowed_value.value),
+      'timestamp': int(windowed_value.timestamp),
+      'index': windowed_value.pane_info.index,
+      'nonspeculative_index': windowed_value.pane_info.nonspeculative_index,
+      'early': windowed_value.pane_info.timing == PaneInfoTiming.EARLY,
+      'late': windowed_value.pane_info.timing == PaneInfoTiming.LATE,
+      'final': windowed_value.pane_info.is_last,
+  }
+
+
+def _windowed_value_info_map_fn(
+    k, vs,
+    window=beam.DoFn.WindowParam,
+    t=beam.DoFn.TimestampParam,
+    p=beam.DoFn.PaneInfoParam):
+  return (
+      k,
+      _windowed_value_info(WindowedValue(
+          vs, windows=[window], timestamp=t, pane_info=p)))
+
+
+def _windowed_value_info_check(actual, expected):
+
+  def format(panes):
+    return '\n[%s]\n' % '\n '.join(str(pane) for pane in sorted(
+        panes, key=lambda pane: pane.get('timestamp', None)))
+
+  if len(actual) > len(expected):
+    raise AssertionError(
+        'Unexpected output: expected %s but got %s' % (
+            format(expected), format(actual)))
+  elif len(expected) > len(actual):
+    raise AssertionError(
+        'Unmatched output: expected %s but got %s' % (
+            format(expected), format(actual)))
+  else:
+
+    def diff(actual, expected):
+      for key in sorted(expected.keys(), reverse=True):
+        if key in actual:
+          if actual[key] != expected[key]:
+            return key
+
+    for output in actual:
+      diffs = [diff(output, pane) for pane in expected]
+      if all(diffs):
+        raise AssertionError(
+            'Unmatched output: %s not found in %s (diffs in %s)' % (
+                output, format(expected), diffs))
+
+
+class _ConcatCombineFn(beam.CombineFn):
+  create_accumulator = lambda self: []
+  add_input = lambda self, acc, element: acc.append(element) or acc
+  merge_accumulators = lambda self, accs: sum(accs, [])
+  extract_output = lambda self, acc: acc
 
 
 class TriggerDriverTranscriptTest(TranscriptTest):
@@ -645,7 +700,7 @@
         for timer_window, (name, time_domain, t_timestamp) in to_fire:
           for wvalue in driver.process_timer(
               timer_window, name, time_domain, t_timestamp, state):
-            output.append(self._windowed_value_info(wvalue))
+            output.append(_windowed_value_info(wvalue))
         to_fire = state.get_and_clear_timers(watermark)
 
     for action, params in transcript:
@@ -661,7 +716,7 @@
             WindowedValue(t, t, window_fn.assign(WindowFn.AssignContext(t, t)))
             for t in params]
         output = [
-            self._windowed_value_info(wv)
+            _windowed_value_info(wv)
             for wv in driver.process_elements(state, bundle, watermark)]
         fire_timers()
 
@@ -690,7 +745,7 @@
     self.assertEqual([], output, msg='Unexpected output: %s' % output)
 
 
-class TestStreamTranscriptTest(TranscriptTest):
+class BaseTestStreamTranscriptTest(TranscriptTest):
   """A suite of TestStream-based tests based on trigger transcript entries.
   """
 
@@ -702,14 +757,17 @@
     if runner_name in spec.get('broken_on', ()):
       self.skipTest('Known to be broken on %s' % runner_name)
 
-    test_stream = TestStream()
+    # Elements are encoded as a json strings to allow other languages to
+    # decode elements while executing the test stream.
+    # TODO(BEAM-8600): Eliminate these gymnastics.
+    test_stream = TestStream(coder=coders.StrUtf8Coder()).with_output_types(str)
     for action, params in transcript:
       if action == 'expect':
-        test_stream.add_elements([('expect', params)])
+        test_stream.add_elements([json.dumps(('expect', params))])
       else:
-        test_stream.add_elements([('expect', [])])
+        test_stream.add_elements([json.dumps(('expect', []))])
         if action == 'input':
-          test_stream.add_elements([('input', e) for e in params])
+          test_stream.add_elements([json.dumps(('input', e)) for e in params])
         elif action == 'watermark':
           test_stream.advance_watermark_to(params)
         elif action == 'clock':
@@ -718,7 +776,9 @@
           pass  # Requires inspection of implementation details.
         else:
           raise ValueError('Unexpected action: %s' % action)
-    test_stream.add_elements([('expect', [])])
+    test_stream.add_elements([json.dumps(('expect', []))])
+
+    read_test_stream = test_stream | beam.Map(json.loads)
 
     class Check(beam.DoFn):
       """A StatefulDoFn that verifies outputs are produced as expected.
@@ -731,52 +791,63 @@
 
       The key is ignored, but all items must be on the same key to share state.
       """
+      def __init__(self, allow_out_of_order=True):
+        # Some runners don't support cross-stage TestStream semantics.
+        self.allow_out_of_order = allow_out_of_order
+
       def process(
-          self, element, seen=beam.DoFn.StateParam(
+          self,
+          element,
+          seen=beam.DoFn.StateParam(
               beam.transforms.userstate.BagStateSpec(
                   'seen',
+                  beam.coders.FastPrimitivesCoder())),
+          expected=beam.DoFn.StateParam(
+              beam.transforms.userstate.BagStateSpec(
+                  'expected',
                   beam.coders.FastPrimitivesCoder()))):
         _, (action, data) = element
+
+        if self.allow_out_of_order:
+          if action == 'expect' and not list(seen.read()):
+            if data:
+              expected.add(data)
+            return
+          elif action == 'actual' and list(expected.read()):
+            seen.add(data)
+            all_data = list(seen.read())
+            all_expected = list(expected.read())
+            if len(all_data) == len(all_expected[0]):
+              expected.clear()
+              for expect in all_expected[1:]:
+                expected.add(expect)
+              action, data = 'expect', all_expected[0]
+            else:
+              return
+
         if action == 'actual':
           seen.add(data)
 
         elif action == 'expect':
           actual = list(seen.read())
           seen.clear()
-
-          if len(actual) > len(data):
-            raise AssertionError(
-                'Unexpected output: expected %s but got %s' % (data, actual))
-          elif len(data) > len(actual):
-            raise AssertionError(
-                'Unmatched output: expected %s but got %s' % (data, actual))
-          else:
-
-            def diff(actual, expected):
-              for key in sorted(expected.keys(), reverse=True):
-                if key in actual:
-                  if actual[key] != expected[key]:
-                    return key
-
-            for output in actual:
-              diffs = [diff(output, expected) for expected in data]
-              if all(diffs):
-                raise AssertionError(
-                    'Unmatched output: %s not found in %s (diffs in %s)' % (
-                        output, data, diffs))
+          _windowed_value_info_check(actual, data)
 
         else:
           raise ValueError('Unexpected action: %s' % action)
 
-    with TestPipeline(options=PipelineOptions(streaming=True)) as p:
+    @ptransform.ptransform_fn
+    def CheckAggregation(inputs_and_expected, aggregation):
       # Split the test stream into a branch of to-be-processed elements, and
       # a branch of expected results.
       inputs, expected = (
-          p
-          | test_stream
-          | beam.MapTuple(
-              lambda tag, value: beam.pvalue.TaggedOutput(tag, ('key', value))
-              ).with_outputs('input', 'expect'))
+          inputs_and_expected
+          | beam.FlatMapTuple(
+              lambda tag, value: [
+                  beam.pvalue.TaggedOutput(tag, ('key1', value)),
+                  beam.pvalue.TaggedOutput(tag, ('key2', value)),
+              ]).with_outputs('input', 'expect'))
+
       # Process the inputs with the given windowing to produce actual outputs.
       outputs = (
           inputs
@@ -787,15 +858,8 @@
               trigger=trigger_fn,
               accumulation_mode=accumulation_mode,
               timestamp_combiner=timestamp_combiner)
-          | beam.GroupByKey()
-          | beam.MapTuple(
-              lambda k, vs,
-                     window=beam.DoFn.WindowParam,
-                     t=beam.DoFn.TimestampParam,
-                     p=beam.DoFn.PaneInfoParam: (
-                         k,
-                         self._windowed_value_info(WindowedValue(
-                             vs, windows=[window], timestamp=t, pane_info=p))))
+          | aggregation
+          | beam.MapTuple(_windowed_value_info_map_fn)
           # Place outputs back into the global window to allow flattening
           # and share a single state in Check.
           | 'Global' >> beam.WindowInto(beam.transforms.window.GlobalWindows()))
@@ -805,7 +869,104 @@
       tagged_outputs = (
           outputs | beam.MapTuple(lambda key, value: (key, ('actual', value))))
       # pylint: disable=expression-not-assigned
-      (tagged_expected, tagged_outputs) | beam.Flatten() | beam.ParDo(Check())
+      ([tagged_expected, tagged_outputs]
+       | beam.Flatten()
+       | beam.ParDo(Check(self.allow_out_of_order)))
+
+    with TestPipeline() as p:
+      # TODO(BEAM-8601): Pass this during pipeline construction.
+      p.options.view_as(StandardOptions).streaming = True
+
+      # We can have at most one test stream per pipeline, so we share it.
+      inputs_and_expected = p | read_test_stream
+      _ = inputs_and_expected | CheckAggregation(beam.GroupByKey())
+      _ = inputs_and_expected | CheckAggregation(beam.CombinePerKey(
+          _ConcatCombineFn()))
+
+
+class TestStreamTranscriptTest(BaseTestStreamTranscriptTest):
+  allow_out_of_order = False
+
+
+class WeakTestStreamTranscriptTest(BaseTestStreamTranscriptTest):
+  allow_out_of_order = True
+
+
+class BatchTranscriptTest(TranscriptTest):
+
+  def _execute(
+      self, window_fn, trigger_fn, accumulation_mode, timestamp_combiner,
+      transcript, spec):
+    if timestamp_combiner == TimestampCombiner.OUTPUT_AT_EARLIEST_TRANSFORMED:
+      self.skipTest(
+          'Non-fnapi timestamp combiner: %s' % spec.get('timestamp_combiner'))
+
+    if accumulation_mode != AccumulationMode.ACCUMULATING:
+      self.skipTest('Batch mode only makes sense for accumulating.')
+
+    watermark = MIN_TIMESTAMP
+    for action, params in transcript:
+      if action == 'watermark':
+        watermark = params
+      elif action == 'input':
+        if any(t <= watermark for t in params):
+          self.skipTest('Batch mode never has late data.')
+
+    inputs = sum([vs for action, vs in transcript if action == 'input'], [])
+    final_panes_by_window = {}
+    for action, params in transcript:
+      if action == 'expect':
+        for expected in params:
+          trimmed = {}
+          for field in ('window', 'values', 'timestamp'):
+            if field in expected:
+              trimmed[field] = expected[field]
+          final_panes_by_window[tuple(expected['window'])] = trimmed
+    final_panes = list(final_panes_by_window.values())
+
+    if window_fn.is_merging():
+      merged_away = set()
+      class MergeContext(WindowFn.MergeContext):
+        def merge(_, to_be_merged, merge_result):
+          for window in to_be_merged:
+            if window != merge_result:
+              merged_away.add(window)
+      all_windows = [IntervalWindow(*pane['window']) for pane in final_panes]
+      window_fn.merge(MergeContext(all_windows))
+      final_panes = [
+          pane for pane in final_panes
+          if IntervalWindow(*pane['window']) not in merged_away]
+
+    with TestPipeline() as p:
+      input_pc = (
+          p
+          | beam.Create(inputs)
+          | beam.Map(lambda t: TimestampedValue(('key', t), t))
+          | beam.WindowInto(
+              window_fn,
+              trigger=trigger_fn,
+              accumulation_mode=accumulation_mode,
+              timestamp_combiner=timestamp_combiner))
+
+      grouped = input_pc | 'Grouped' >> (
+          beam.GroupByKey()
+          | beam.MapTuple(_windowed_value_info_map_fn)
+          | beam.MapTuple(lambda _, value: value))
+
+      combined = input_pc | 'Combined' >> (
+          beam.CombinePerKey(_ConcatCombineFn())
+          | beam.MapTuple(_windowed_value_info_map_fn)
+          | beam.MapTuple(lambda _, value: value))
+
+      assert_that(
+          grouped,
+          lambda actual: _windowed_value_info_check(actual, final_panes),
+          label='CheckGrouped')
+
+      assert_that(
+          combined,
+          lambda actual: _windowed_value_info_check(actual, final_panes),
+          label='CheckCombined')
 
 
 TRANSCRIPT_TEST_FILE = os.path.join(
@@ -814,6 +975,8 @@
 if os.path.exists(TRANSCRIPT_TEST_FILE):
   TriggerDriverTranscriptTest._create_tests(TRANSCRIPT_TEST_FILE)
   TestStreamTranscriptTest._create_tests(TRANSCRIPT_TEST_FILE)
+  WeakTestStreamTranscriptTest._create_tests(TRANSCRIPT_TEST_FILE)
+  BatchTranscriptTest._create_tests(TRANSCRIPT_TEST_FILE)
 
 
 if __name__ == '__main__':
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..ec32a1e 100644
--- a/sdks/python/apache_beam/transforms/window_test.py
+++ b/sdks/python/apache_beam/transforms/window_test.py
@@ -22,7 +22,10 @@
 import unittest
 from builtins import range
 
+from nose.plugins.attrib import attr
+
 import apache_beam as beam
+from apache_beam.coders import coders
 from apache_beam.runners import pipeline_context
 from apache_beam.testing.test_pipeline import TestPipeline
 from apache_beam.testing.util import assert_that
@@ -43,6 +46,7 @@
 from apache_beam.transforms.window import GlobalWindow
 from apache_beam.transforms.window import GlobalWindows
 from apache_beam.transforms.window import IntervalWindow
+from apache_beam.transforms.window import NonMergingWindowFn
 from apache_beam.transforms.window import Sessions
 from apache_beam.transforms.window import SlidingWindows
 from apache_beam.transforms.window import TimestampCombiner
@@ -65,6 +69,23 @@
 
 reify_windows = core.ParDo(ReifyWindowsFn())
 
+class TestCustomWindows(NonMergingWindowFn):
+  """A custom non merging window fn which assigns elements into interval windows
+  [0, 3), [3, 5) and [5, element timestamp) based on the element timestamps.
+  """
+
+  def assign(self, context):
+    timestamp = context.timestamp
+    if timestamp < 3:
+      return [IntervalWindow(0, 3)]
+    elif timestamp < 5:
+      return [IntervalWindow(3, 5)]
+    else:
+      return [IntervalWindow(5, timestamp)]
+
+  def get_window_coder(self):
+    return coders.IntervalWindowCoder()
+
 
 class WindowTest(unittest.TestCase):
 
@@ -281,6 +302,49 @@
       assert_that(mean_per_window, equal_to([(0, 2.0), (1, 7.0)]),
                   label='assert:mean')
 
+  @attr('ValidatesRunner')
+  def test_custom_windows(self):
+    with TestPipeline() as p:
+      pcoll = self.timestamped_key_values(p, 'key', 0, 1, 2, 3, 4, 5, 6)
+      # pylint: disable=abstract-class-instantiated
+      result = (pcoll
+                | 'custom window' >> WindowInto(TestCustomWindows())
+                | GroupByKey()
+                | 'sort values' >> MapTuple(lambda k, vs: (k, sorted(vs))))
+      assert_that(result, equal_to([('key', [0, 1, 2]),
+                                    ('key', [3, 4]),
+                                    ('key', [5]),
+                                    ('key', [6])]))
+
+  @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/typehints/typehints.py b/sdks/python/apache_beam/typehints/typehints.py
index 6062e6f..b64e020 100644
--- a/sdks/python/apache_beam/typehints/typehints.py
+++ b/sdks/python/apache_beam/typehints/typehints.py
@@ -96,6 +96,8 @@
 # to templated (upper-case) versions instead.
 DISALLOWED_PRIMITIVE_TYPES = (list, set, tuple, dict)
 
+_LOGGER = logging.getLogger(__name__)
+
 
 class SimpleTypeHintError(TypeError):
   pass
@@ -1086,9 +1088,9 @@
     if isinstance(type_params, tuple) and len(type_params) == 3:
       yield_type, send_type, return_type = type_params
       if send_type is not None:
-        logging.warning('Ignoring send_type hint: %s' % send_type)
+        _LOGGER.warning('Ignoring send_type hint: %s' % send_type)
       if send_type is not None:
-        logging.warning('Ignoring return_type hint: %s' % return_type)
+        _LOGGER.warning('Ignoring return_type hint: %s' % return_type)
     else:
       yield_type = type_params
     return self.IteratorTypeConstraint(yield_type)
diff --git a/sdks/python/apache_beam/utils/profiler.py b/sdks/python/apache_beam/utils/profiler.py
index 0606744..c6f7295 100644
--- a/sdks/python/apache_beam/utils/profiler.py
+++ b/sdks/python/apache_beam/utils/profiler.py
@@ -36,6 +36,8 @@
 
 from apache_beam.io import filesystems
 
+_LOGGER = logging.getLogger(__name__)
+
 
 class Profile(object):
   """cProfile wrapper context for saving and logging profiler results."""
@@ -53,14 +55,14 @@
     self.profile_output = None
 
   def __enter__(self):
-    logging.info('Start profiling: %s', self.profile_id)
+    _LOGGER.info('Start profiling: %s', self.profile_id)
     self.profile = cProfile.Profile()
     self.profile.enable()
     return self
 
   def __exit__(self, *args):
     self.profile.disable()
-    logging.info('Stop profiling: %s', self.profile_id)
+    _LOGGER.info('Stop profiling: %s', self.profile_id)
 
     if self.profile_location:
       dump_location = os.path.join(
@@ -70,7 +72,7 @@
       try:
         os.close(fd)
         self.profile.dump_stats(filename)
-        logging.info('Copying profiler data to: [%s]', dump_location)
+        _LOGGER.info('Copying profiler data to: [%s]', dump_location)
         self.file_copy_fn(filename, dump_location)
       finally:
         os.remove(filename)
@@ -81,7 +83,7 @@
       self.stats = pstats.Stats(
           self.profile, stream=s).sort_stats(Profile.SORTBY)
       self.stats.print_stats()
-      logging.info('Profiler data: [%s]', s.getvalue())
+      _LOGGER.info('Profiler data: [%s]', s.getvalue())
 
   @staticmethod
   def default_file_copy_fn(src, dest):
@@ -176,5 +178,5 @@
       return
     report_start_time = time.time()
     heap_profile = self._hpy().heap()
-    logging.info('*** MemoryReport Heap:\n %s\n MemoryReport took %.1f seconds',
+    _LOGGER.info('*** MemoryReport Heap:\n %s\n MemoryReport took %.1f seconds',
                  heap_profile, time.time() - report_start_time)
diff --git a/sdks/python/apache_beam/utils/retry.py b/sdks/python/apache_beam/utils/retry.py
index 59d8dec..e34e364 100644
--- a/sdks/python/apache_beam/utils/retry.py
+++ b/sdks/python/apache_beam/utils/retry.py
@@ -51,6 +51,9 @@
 # pylint: enable=wrong-import-order, wrong-import-position
 
 
+_LOGGER = logging.getLogger(__name__)
+
+
 class PermanentException(Exception):
   """Base class for exceptions that should not be retried."""
   pass
@@ -153,7 +156,7 @@
 
 
 def with_exponential_backoff(
-    num_retries=7, initial_delay_secs=5.0, logger=logging.warning,
+    num_retries=7, initial_delay_secs=5.0, logger=_LOGGER.warning,
     retry_filter=retry_on_server_errors_filter,
     clock=Clock(), fuzz=True, factor=2, max_delay_secs=60 * 60):
   """Decorator with arguments that control the retry logic.
@@ -163,7 +166,7 @@
     initial_delay_secs: The delay before the first retry, in seconds.
     logger: A callable used to report an exception. Must have the same signature
       as functions in the standard logging module. The default is
-      logging.warning.
+      _LOGGER.warning.
     retry_filter: A callable getting the exception raised and returning True
       if the retry should happen. For instance we do not want to retry on
       404 Http errors most of the time. The default value will return true
diff --git a/sdks/python/apache_beam/utils/subprocess_server.py b/sdks/python/apache_beam/utils/subprocess_server.py
index 65dbcae..fd55f18 100644
--- a/sdks/python/apache_beam/utils/subprocess_server.py
+++ b/sdks/python/apache_beam/utils/subprocess_server.py
@@ -33,6 +33,8 @@
 
 from apache_beam.version import __version__ as beam_version
 
+_LOGGER = logging.getLogger(__name__)
+
 
 class SubprocessServer(object):
   """An abstract base class for running GRPC Servers as an external process.
@@ -78,7 +80,7 @@
         port, = pick_port(None)
         cmd = [arg.replace('{{PORT}}', str(port)) for arg in self._cmd]
       endpoint = 'localhost:%s' % port
-      logging.warning("Starting service with %s", str(cmd).replace("',", "'"))
+      _LOGGER.warning("Starting service with %s", str(cmd).replace("',", "'"))
       try:
         self._process = subprocess.Popen(cmd)
         wait_secs = .1
@@ -86,7 +88,7 @@
         channel_ready = grpc.channel_ready_future(channel)
         while True:
           if self._process.poll() is not None:
-            logging.error("Starting job service with %s", cmd)
+            _LOGGER.error("Starting job service with %s", cmd)
             raise RuntimeError(
                 'Service failed to start up with error %s' %
                 self._process.poll())
@@ -100,7 +102,7 @@
                         endpoint)
         return self._stub_class(channel)
       except:  # pylint: disable=bare-except
-        logging.exception("Error bringing up service")
+        _LOGGER.exception("Error bringing up service")
         self.stop()
         raise
 
@@ -169,7 +171,7 @@
             classifier='SNAPSHOT',
             appendix=appendix))
     if os.path.exists(local_path):
-      logging.info('Using pre-built snapshot at %s', local_path)
+      _LOGGER.info('Using pre-built snapshot at %s', local_path)
       return local_path
     elif '.dev' in beam_version:
       # TODO: Attempt to use nightly snapshots?
@@ -187,7 +189,7 @@
     if os.path.exists(url):
       return url
     else:
-      logging.warning('Downloading job server jar from %s' % url)
+      _LOGGER.warning('Downloading job server jar from %s' % url)
       cached_jar = os.path.join(cls.JAR_CACHE, os.path.basename(url))
       if not os.path.exists(cached_jar):
         if not os.path.exists(cls.JAR_CACHE):
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..c82d0f9
--- /dev/null
+++ b/sdks/python/apache_beam/utils/thread_pool_executor_test.py
@@ -0,0 +1,114 @@
+#
+# 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 itertools
+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)
+
+  def test_map(self):
+    with UnboundedThreadPoolExecutor() as executor:
+      executor.map(self.append_and_sleep, itertools.repeat(0.01, 5))
+
+    with self._lock:
+      self.assertEqual(5, len(self._worker_idents))
+
+
+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 ccf90f6..9f1a9f3 100644
--- a/sdks/python/setup.py
+++ b/sdks/python/setup.py
@@ -214,6 +214,7 @@
     ext_modules=cythonize([
         'apache_beam/**/*.pyx',
         'apache_beam/coders/coder_impl.py',
+        'apache_beam/metrics/cells.py',
         'apache_beam/metrics/execution.py',
         'apache_beam/runners/common.py',
         'apache_beam/runners/worker/logger.py',
@@ -227,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/portable/common.gradle b/sdks/python/test-suites/portable/common.gradle
index f04d28d..3cb4362 100644
--- a/sdks/python/test-suites/portable/common.gradle
+++ b/sdks/python/test-suites/portable/common.gradle
@@ -79,3 +79,22 @@
 task flinkValidatesRunner() {
   dependsOn 'flinkCompatibilityMatrixLoopback'
 }
+
+// TODO(BEAM-8598): Enable on pre-commit.
+task flinkTriggerTranscript() {
+  dependsOn 'setupVirtualenv'
+  dependsOn ':runners:flink:1.9:job-server:shadowJar'
+  doLast {
+    exec {
+      executable 'sh'
+      args '-c', """
+          . ${envdir}/bin/activate \\
+          && cd ${pythonRootDir} \\
+          && pip install -e .[test] \\
+          && python setup.py nosetests \\
+              --tests apache_beam.transforms.trigger_test:WeakTestStreamTranscriptTest \\
+              --test-pipeline-options='--runner=FlinkRunner --environment_type=LOOPBACK --flink_job_server_jar=${project(":runners:flink:1.9:job-server:").shadowJar.archivePath}'
+          """
+    }
+  }
+}
diff --git a/sdks/python/test-suites/portable/py2/build.gradle b/sdks/python/test-suites/portable/py2/build.gradle
index 3c1548d..5d967e4 100644
--- a/sdks/python/test-suites/portable/py2/build.gradle
+++ b/sdks/python/test-suites/portable/py2/build.gradle
@@ -39,6 +39,8 @@
   dependsOn ':runners:flink:1.9:job-server:shadowJar'
   dependsOn portableWordCountFlinkRunnerBatch
   dependsOn portableWordCountFlinkRunnerStreaming
+  dependsOn ':runners:spark:job-server:shadowJar'
+  dependsOn portableWordCountSparkRunnerBatch
 }
 
 // TODO: Move the rest of this file into ../common.gradle.
diff --git a/sdks/python/test-suites/portable/py35/build.gradle b/sdks/python/test-suites/portable/py35/build.gradle
index 1b2cb4f..88b4e2f 100644
--- a/sdks/python/test-suites/portable/py35/build.gradle
+++ b/sdks/python/test-suites/portable/py35/build.gradle
@@ -36,4 +36,6 @@
     dependsOn ':runners:flink:1.9:job-server:shadowJar'
     dependsOn portableWordCountFlinkRunnerBatch
     dependsOn portableWordCountFlinkRunnerStreaming
+    dependsOn ':runners:spark:job-server:shadowJar'
+    dependsOn portableWordCountSparkRunnerBatch
 }
diff --git a/sdks/python/test-suites/portable/py36/build.gradle b/sdks/python/test-suites/portable/py36/build.gradle
index 475e110..496777d 100644
--- a/sdks/python/test-suites/portable/py36/build.gradle
+++ b/sdks/python/test-suites/portable/py36/build.gradle
@@ -36,4 +36,6 @@
     dependsOn ':runners:flink:1.9:job-server:shadowJar'
     dependsOn portableWordCountFlinkRunnerBatch
     dependsOn portableWordCountFlinkRunnerStreaming
+    dependsOn ':runners:spark:job-server:shadowJar'
+    dependsOn portableWordCountSparkRunnerBatch
 }
diff --git a/sdks/python/test-suites/portable/py37/build.gradle b/sdks/python/test-suites/portable/py37/build.gradle
index 912b316..924de81 100644
--- a/sdks/python/test-suites/portable/py37/build.gradle
+++ b/sdks/python/test-suites/portable/py37/build.gradle
@@ -36,4 +36,6 @@
     dependsOn ':runners:flink:1.9:job-server:shadowJar'
     dependsOn portableWordCountFlinkRunnerBatch
     dependsOn portableWordCountFlinkRunnerStreaming
+    dependsOn ':runners:spark:job-server:shadowJar'
+    dependsOn portableWordCountSparkRunnerBatch
 }
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}
diff --git a/website/Gemfile b/website/Gemfile
index 4a08725..1050303 100644
--- a/website/Gemfile
+++ b/website/Gemfile
@@ -20,7 +20,7 @@
 
 source 'https://rubygems.org'
 
-gem 'jekyll', '3.2'
+gem 'jekyll', '3.6.3'
 
 # Jekyll plugins
 group :jekyll_plugins do
diff --git a/website/Gemfile.lock b/website/Gemfile.lock
index e94f132..9db2ebe 100644
--- a/website/Gemfile.lock
+++ b/website/Gemfile.lock
@@ -13,7 +13,7 @@
     concurrent-ruby (1.1.4)
     ethon (0.11.0)
       ffi (>= 1.3.0)
-    ffi (1.9.25)
+    ffi (1.11.1)
     forwardable-extended (2.6.0)
     html-proofer (3.9.3)
       activesupport (>= 4.2, < 6.0)
@@ -26,15 +26,16 @@
       yell (~> 2.0)
     i18n (0.9.5)
       concurrent-ruby (~> 1.0)
-    jekyll (3.2.0)
+    jekyll (3.6.3)
+      addressable (~> 2.4)
       colorator (~> 1.0)
       jekyll-sass-converter (~> 1.0)
       jekyll-watch (~> 1.1)
-      kramdown (~> 1.3)
-      liquid (~> 3.0)
+      kramdown (~> 1.14)
+      liquid (~> 4.0)
       mercenary (~> 0.3.3)
       pathutil (~> 0.9)
-      rouge (~> 1.7)
+      rouge (>= 1.7, < 3)
       safe_yaml (~> 1.0)
     jekyll-redirect-from (0.11.0)
       jekyll (>= 2.0)
@@ -45,29 +46,27 @@
     jekyll_github_sample (0.3.1)
       activesupport (~> 4.0)
       jekyll (~> 3.0)
-    kramdown (1.16.2)
-    liquid (3.0.6)
-    listen (3.1.5)
-      rb-fsevent (~> 0.9, >= 0.9.4)
-      rb-inotify (~> 0.9, >= 0.9.7)
-      ruby_dep (~> 1.2)
+    kramdown (1.17.0)
+    liquid (4.0.3)
+    listen (3.2.0)
+      rb-fsevent (~> 0.10, >= 0.10.3)
+      rb-inotify (~> 0.9, >= 0.9.10)
     mercenary (0.3.6)
     mini_portile2 (2.3.0)
     minitest (5.11.3)
     nokogiri (1.8.5)
       mini_portile2 (~> 2.3.0)
     parallel (1.12.1)
-    pathutil (0.16.1)
+    pathutil (0.16.2)
       forwardable-extended (~> 2.6)
     public_suffix (3.0.3)
     rake (12.3.0)
-    rb-fsevent (0.10.2)
-    rb-inotify (0.9.10)
-      ffi (>= 0.5.0, < 2)
-    rouge (1.11.1)
-    ruby_dep (1.5.0)
-    safe_yaml (1.0.4)
-    sass (3.5.5)
+    rb-fsevent (0.10.3)
+    rb-inotify (0.10.0)
+      ffi (~> 1.0)
+    rouge (2.2.1)
+    safe_yaml (1.0.5)
+    sass (3.7.4)
       sass-listen (~> 4.0.0)
     sass-listen (4.0.0)
       rb-fsevent (~> 0.9, >= 0.9.4)
@@ -85,7 +84,7 @@
 DEPENDENCIES
   activesupport (< 5.0.0.0)
   html-proofer
-  jekyll (= 3.2)
+  jekyll (= 3.6.3)
   jekyll-redirect-from
   jekyll-sass-converter
   jekyll_github_sample