Merge pull request #10283 [BEAM-8882] Allow Dataflow to automatically choose portability.

diff --git a/.test-infra/jenkins/job_PostCommit_Python37.groovy b/.test-infra/jenkins/job_PostCommit_Python37.groovy
index e4f2e17..b52994e 100644
--- a/.test-infra/jenkins/job_PostCommit_Python37.groovy
+++ b/.test-infra/jenkins/job_PostCommit_Python37.groovy
@@ -27,7 +27,7 @@
   previousNames('/beam_PostCommit_Python3_Verify/')
 
   // Set common parameters.
-  commonJobProperties.setTopLevelMainJobProperties(delegate, 'master', 150)
+  commonJobProperties.setTopLevelMainJobProperties(delegate, 'master', 100)
 
   publishers {
     archiveJunit('**/nosetests*.xml')
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 ebfe4b2..0e27b13 100644
--- a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy
+++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy
@@ -297,7 +297,7 @@
 
     // Automatically use the official release version if we are performing a release
     // otherwise append '-SNAPSHOT'
-    project.version = '2.18.0'
+    project.version = '2.19.0'
     if (!isRelease(project)) {
       project.version += '-SNAPSHOT'
     }
@@ -373,7 +373,7 @@
     def google_cloud_spanner_version = "1.6.0"
     def grpc_version = "1.17.1"
     def guava_version = "20.0"
-    def hadoop_version = "2.7.3"
+    def hadoop_version = "2.8.5"
     def hamcrest_version = "2.1"
     def jackson_version = "2.9.10"
     def jaxb_api_version = "2.2.12"
diff --git a/gradle.properties b/gradle.properties
index 3f608c2..d758ae4 100644
--- a/gradle.properties
+++ b/gradle.properties
@@ -23,7 +23,7 @@
 signing.gnupg.executable=gpg
 signing.gnupg.useLegacyGpg=true
 
-version=2.18.0-SNAPSHOT
-python_sdk_version=2.18.0.dev
+version=2.19.0-SNAPSHOT
+python_sdk_version=2.19.0.dev
 
 javaVersion=1.8
diff --git a/model/job-management/src/main/proto/beam_artifact_api.proto b/model/job-management/src/main/proto/beam_artifact_api.proto
index 34eb389..2cfede9 100644
--- a/model/job-management/src/main/proto/beam_artifact_api.proto
+++ b/model/job-management/src/main/proto/beam_artifact_api.proto
@@ -29,6 +29,8 @@
 option java_package = "org.apache.beam.model.jobmanagement.v1";
 option java_outer_classname = "ArtifactApi";
 
+import "beam_runner_api.proto";
+
 // A service to stage artifacts for use in a Job.
 service ArtifactStagingService {
   // Stage an artifact to be available during job execution. The first request must contain the
@@ -142,6 +144,10 @@
 
 // The result of committing a manifest.
 message CommitManifestResponse {
+  enum Constants {
+    // Token indicating that no artifacts were staged and therefore no retrieval attempt is necessary.
+    NO_ARTIFACTS_STAGED_TOKEN = 0 [(org.apache.beam.model.pipeline.v1.beam_constant) = "__no_artifacts_staged__"];
+  }
   // (Required) An opaque token representing the entirety of the staged artifacts.
   // This can be used to retrieve the manifest and artifacts from an associated
   // ArtifactRetrievalService.
diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CoderTranslators.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CoderTranslators.java
index f2cc8fa..2bf5264 100644
--- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CoderTranslators.java
+++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CoderTranslators.java
@@ -28,6 +28,7 @@
 import org.apache.beam.sdk.coders.LengthPrefixCoder;
 import org.apache.beam.sdk.coders.RowCoder;
 import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.schemas.SchemaTranslation;
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
 import org.apache.beam.sdk.util.InstanceBuilder;
 import org.apache.beam.sdk.util.WindowedValue;
diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SplittableParDoNaiveBounded.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SplittableParDoNaiveBounded.java
index 340889d..8a41d7e 100644
--- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SplittableParDoNaiveBounded.java
+++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SplittableParDoNaiveBounded.java
@@ -147,7 +147,7 @@
         ProcessContinuation continuation =
             invoker.invokeProcessElement(new NestedProcessContext<>(fn, c, element, w, tracker));
         if (continuation.shouldResume()) {
-          restriction = tracker.checkpoint();
+          restriction = tracker.trySplit(0).getResidual();
           Uninterruptibles.sleepUninterruptibly(
               continuation.resumeDelay().getMillis(), TimeUnit.MILLISECONDS);
         } else {
diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/CoderTranslationTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/CoderTranslationTest.java
index 451568c..c2c6804 100644
--- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/CoderTranslationTest.java
+++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/CoderTranslationTest.java
@@ -45,10 +45,10 @@
 import org.apache.beam.sdk.coders.SerializableCoder;
 import org.apache.beam.sdk.coders.StringUtf8Coder;
 import org.apache.beam.sdk.coders.VarLongCoder;
-import org.apache.beam.sdk.schemas.LogicalTypes;
 import org.apache.beam.sdk.schemas.Schema;
 import org.apache.beam.sdk.schemas.Schema.Field;
 import org.apache.beam.sdk.schemas.Schema.FieldType;
+import org.apache.beam.sdk.schemas.logicaltypes.FixedBytes;
 import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
 import org.apache.beam.sdk.transforms.windowing.IntervalWindow.IntervalWindowCoder;
 import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder;
@@ -86,7 +86,7 @@
                       Field.of("i16", FieldType.INT16),
                       Field.of("array", FieldType.array(FieldType.STRING)),
                       Field.of("map", FieldType.map(FieldType.STRING, FieldType.INT32)),
-                      Field.of("bar", FieldType.logicalType(LogicalTypes.FixedBytes.of(123))))))
+                      Field.of("bar", FieldType.logicalType(FixedBytes.of(123))))))
           .build();
 
   /**
diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/CommonCoderTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/CommonCoderTest.java
index 52dddcc..2de2fe8 100644
--- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/CommonCoderTest.java
+++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/CommonCoderTest.java
@@ -61,6 +61,7 @@
 import org.apache.beam.sdk.coders.StringUtf8Coder;
 import org.apache.beam.sdk.coders.VarLongCoder;
 import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.schemas.SchemaTranslation;
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
 import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
 import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/SchemaTranslationTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/SchemaTranslationTest.java
index 3807712..9324bab 100644
--- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/SchemaTranslationTest.java
+++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/SchemaTranslationTest.java
@@ -21,10 +21,11 @@
 import static org.junit.Assert.assertThat;
 
 import org.apache.beam.model.pipeline.v1.SchemaApi;
-import org.apache.beam.sdk.schemas.LogicalTypes;
 import org.apache.beam.sdk.schemas.Schema;
 import org.apache.beam.sdk.schemas.Schema.Field;
 import org.apache.beam.sdk.schemas.Schema.FieldType;
+import org.apache.beam.sdk.schemas.SchemaTranslation;
+import org.apache.beam.sdk.schemas.logicaltypes.FixedBytes;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
 import org.junit.Test;
 import org.junit.runner.RunWith;
@@ -75,8 +76,7 @@
           .add(
               Schema.of(
                   Field.of("decimal", FieldType.DECIMAL), Field.of("datetime", FieldType.DATETIME)))
-          .add(
-              Schema.of(Field.of("logical", FieldType.logicalType(LogicalTypes.FixedBytes.of(24)))))
+          .add(Schema.of(Field.of("logical", FieldType.logicalType(FixedBytes.of(24)))))
           .build();
     }
 
diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/SplittableParDoTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/SplittableParDoTest.java
index 959120c..41de06d 100644
--- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/SplittableParDoTest.java
+++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/SplittableParDoTest.java
@@ -29,6 +29,7 @@
 import org.apache.beam.sdk.transforms.ParDo;
 import org.apache.beam.sdk.transforms.splittabledofn.HasDefaultTracker;
 import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker;
+import org.apache.beam.sdk.transforms.splittabledofn.SplitResult;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PCollectionTuple;
 import org.apache.beam.sdk.values.TupleTag;
@@ -68,8 +69,8 @@
     }
 
     @Override
-    public SomeRestriction checkpoint() {
-      return someRestriction;
+    public SplitResult<SomeRestriction> trySplit(double fractionOfRemainder) {
+      return SplitResult.of(null, someRestriction);
     }
 
     @Override
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/OutputAndTimeBoundedSplittableProcessElementInvoker.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/OutputAndTimeBoundedSplittableProcessElementInvoker.java
index 67982e1..9edc558 100644
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/OutputAndTimeBoundedSplittableProcessElementInvoker.java
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/OutputAndTimeBoundedSplittableProcessElementInvoker.java
@@ -38,6 +38,7 @@
 import org.apache.beam.sdk.transforms.DoFnOutputReceivers;
 import org.apache.beam.sdk.transforms.reflect.DoFnInvoker;
 import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker;
+import org.apache.beam.sdk.transforms.splittabledofn.SplitResult;
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
 import org.apache.beam.sdk.transforms.windowing.PaneInfo;
 import org.apache.beam.sdk.util.WindowedValue;
@@ -342,7 +343,10 @@
       // Only one of them "wins" - tracker.checkpoint() must be called only once.
       if (checkpoint == null) {
         residualWatermark = lastReportedWatermark;
-        checkpoint = checkNotNull(tracker.checkpoint());
+        SplitResult<RestrictionT> split = tracker.trySplit(0);
+        if (split != null) {
+          checkpoint = checkNotNull(split.getResidual());
+        }
       }
       return getTakenCheckpoint();
     }
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/SplittableParDoProcessFnTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/SplittableParDoProcessFnTest.java
index 43f9f21..685f6d9 100644
--- a/runners/core-java/src/test/java/org/apache/beam/runners/core/SplittableParDoProcessFnTest.java
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/SplittableParDoProcessFnTest.java
@@ -52,6 +52,7 @@
 import org.apache.beam.sdk.transforms.splittabledofn.HasDefaultTracker;
 import org.apache.beam.sdk.transforms.splittabledofn.OffsetRangeTracker;
 import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker;
+import org.apache.beam.sdk.transforms.splittabledofn.SplitResult;
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
 import org.apache.beam.sdk.transforms.windowing.FixedWindows;
 import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
@@ -105,8 +106,8 @@
     }
 
     @Override
-    public SomeRestriction checkpoint() {
-      return someRestriction;
+    public SplitResult<SomeRestriction> trySplit(double fractionOfRemainder) {
+      return SplitResult.of(null, someRestriction);
     }
 
     @Override
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/metrics/MetricsPusherTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/metrics/MetricsPusherTest.java
index 107059a..aba4d09 100644
--- a/runners/core-java/src/test/java/org/apache/beam/runners/core/metrics/MetricsPusherTest.java
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/metrics/MetricsPusherTest.java
@@ -28,6 +28,7 @@
 import org.apache.beam.sdk.testing.UsesAttemptedMetrics;
 import org.apache.beam.sdk.testing.UsesCounterMetrics;
 import org.apache.beam.sdk.testing.UsesMetricsPusher;
+import org.apache.beam.sdk.testing.UsesSystemMetrics;
 import org.apache.beam.sdk.testing.ValidatesRunner;
 import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.transforms.ParDo;
@@ -73,7 +74,12 @@
     assertThat(TestMetricsSink.getCounterValue(COUNTER_NAME), is(NUM_ELEMENTS));
   }
 
-  @Category({ValidatesRunner.class, UsesAttemptedMetrics.class, UsesCounterMetrics.class})
+  @Category({
+    ValidatesRunner.class,
+    UsesAttemptedMetrics.class,
+    UsesCounterMetrics.class,
+    UsesSystemMetrics.class
+  })
   @Test
   public void pushesSystemMetrics() throws InterruptedException {
     TestMetricsSink.clear();
diff --git a/runners/flink/flink_runner.gradle b/runners/flink/flink_runner.gradle
index 6281b94..c7f8cc0 100644
--- a/runners/flink/flink_runner.gradle
+++ b/runners/flink/flink_runner.gradle
@@ -198,6 +198,7 @@
       excludeCategories 'org.apache.beam.sdk.testing.FlattenWithHeterogeneousCoders'
       excludeCategories 'org.apache.beam.sdk.testing.LargeKeys$Above100MB'
       excludeCategories 'org.apache.beam.sdk.testing.UsesCommittedMetrics'
+      excludeCategories 'org.apache.beam.sdk.testing.UsesSystemMetrics'
       if (config.streaming) {
         excludeCategories 'org.apache.beam.sdk.testing.UsesImpulse'
         excludeCategories 'org.apache.beam.sdk.testing.UsesTestStreamWithMultipleStages'  // BEAM-8598
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java
index 45e0c33..fa45b7f 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java
@@ -78,6 +78,7 @@
 import org.apache.beam.sdk.options.StreamingOptions;
 import org.apache.beam.sdk.runners.AppliedPTransform;
 import org.apache.beam.sdk.runners.TransformHierarchy;
+import org.apache.beam.sdk.testing.TestStream;
 import org.apache.beam.sdk.transforms.Combine;
 import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.transforms.DoFnSchemaInformation;
@@ -95,6 +96,7 @@
 import org.apache.beam.sdk.transforms.windowing.DefaultTrigger;
 import org.apache.beam.sdk.transforms.windowing.Window;
 import org.apache.beam.sdk.util.AppliedCombineFn;
+import org.apache.beam.sdk.util.CoderUtils;
 import org.apache.beam.sdk.util.DoFnInfo;
 import org.apache.beam.sdk.util.WindowedValue;
 import org.apache.beam.sdk.values.KV;
@@ -103,8 +105,10 @@
 import org.apache.beam.sdk.values.PInput;
 import org.apache.beam.sdk.values.POutput;
 import org.apache.beam.sdk.values.PValue;
+import org.apache.beam.sdk.values.TimestampedValue;
 import org.apache.beam.sdk.values.TupleTag;
 import org.apache.beam.sdk.values.WindowingStrategy;
+import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
 import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.TextFormat;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Supplier;
@@ -1071,6 +1075,66 @@
 
     registerTransformTranslator(Read.Bounded.class, new ReadTranslator());
 
+    registerTransformTranslator(
+        TestStream.class,
+        new TransformTranslator<TestStream>() {
+          @Override
+          public void translate(TestStream transform, TranslationContext context) {
+            translateTyped(transform, context);
+          }
+
+          private <T> void translateTyped(TestStream<T> transform, TranslationContext context) {
+            try {
+              StepTranslationContext stepContext = context.addStep(transform, "ParallelRead");
+              stepContext.addInput(PropertyNames.FORMAT, "test_stream");
+              RunnerApi.TestStreamPayload.Builder payloadBuilder =
+                  RunnerApi.TestStreamPayload.newBuilder();
+              for (TestStream.Event event : transform.getEvents()) {
+                if (event instanceof TestStream.ElementEvent) {
+                  RunnerApi.TestStreamPayload.Event.AddElements.Builder addElementsBuilder =
+                      RunnerApi.TestStreamPayload.Event.AddElements.newBuilder();
+                  Iterable<TimestampedValue<T>> elements =
+                      ((TestStream.ElementEvent) event).getElements();
+                  for (TimestampedValue<T> element : elements) {
+                    addElementsBuilder.addElements(
+                        RunnerApi.TestStreamPayload.TimestampedElement.newBuilder()
+                            .setEncodedElement(
+                                ByteString.copyFrom(
+                                    CoderUtils.encodeToByteArray(
+                                        transform.getValueCoder(), element.getValue())))
+                            .setTimestamp(element.getTimestamp().getMillis() * 1000));
+                  }
+                  payloadBuilder.addEventsBuilder().setElementEvent(addElementsBuilder);
+                } else if (event instanceof TestStream.WatermarkEvent) {
+                  payloadBuilder
+                      .addEventsBuilder()
+                      .setWatermarkEvent(
+                          RunnerApi.TestStreamPayload.Event.AdvanceWatermark.newBuilder()
+                              .setNewWatermark(
+                                  ((TestStream.WatermarkEvent) event).getWatermark().getMillis()
+                                      * 1000));
+                } else if (event instanceof TestStream.ProcessingTimeEvent) {
+                  payloadBuilder
+                      .addEventsBuilder()
+                      .setProcessingTimeEvent(
+                          RunnerApi.TestStreamPayload.Event.AdvanceProcessingTime.newBuilder()
+                              .setAdvanceDuration(
+                                  ((TestStream.ProcessingTimeEvent) event)
+                                          .getProcessingTimeAdvance()
+                                          .getMillis()
+                                      * 1000));
+                }
+              }
+              stepContext.addInput(
+                  PropertyNames.SERIALIZED_TEST_STREAM,
+                  byteArrayToJsonString(payloadBuilder.build().toByteArray()));
+              stepContext.addOutput(PropertyNames.OUTPUT, context.getOutput(transform));
+            } catch (Exception e) {
+              throw new RuntimeException(e);
+            }
+          }
+        });
+
     ///////////////////////////////////////////////////////////////////////////
     // Legacy Splittable DoFn translation.
 
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/PropertyNames.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/PropertyNames.java
index e44170d..088b759 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/PropertyNames.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/PropertyNames.java
@@ -53,6 +53,7 @@
   public static final String PUBSUB_TOPIC_OVERRIDE = "pubsub_topic_runtime_override";
   public static final String SCALAR_FIELD_NAME = "value";
   public static final String SERIALIZED_FN = "serialized_fn";
+  public static final String SERIALIZED_TEST_STREAM = "serialized_test_stream";
   public static final String SORT_VALUES = "sort_values";
   public static final String TUPLE_TAGS = "tuple_tags";
   public static final String USE_INDEXED_FORMAT = "use_indexed_format";
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/SchemaCoderCloudObjectTranslator.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/SchemaCoderCloudObjectTranslator.java
index 8ea562b..e166a22 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/SchemaCoderCloudObjectTranslator.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/SchemaCoderCloudObjectTranslator.java
@@ -19,10 +19,10 @@
 
 import java.io.IOException;
 import org.apache.beam.model.pipeline.v1.SchemaApi;
-import org.apache.beam.runners.core.construction.SchemaTranslation;
 import org.apache.beam.runners.core.construction.SdkComponents;
 import org.apache.beam.sdk.schemas.Schema;
 import org.apache.beam.sdk.schemas.SchemaCoder;
+import org.apache.beam.sdk.schemas.SchemaTranslation;
 import org.apache.beam.sdk.transforms.SerializableFunction;
 import org.apache.beam.sdk.util.SerializableUtils;
 import org.apache.beam.sdk.util.StringUtils;
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/CloudObjectsTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/CloudObjectsTest.java
index 09f9fa9..23baf9e 100644
--- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/CloudObjectsTest.java
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/CloudObjectsTest.java
@@ -51,10 +51,10 @@
 import org.apache.beam.sdk.coders.SetCoder;
 import org.apache.beam.sdk.coders.StructuredCoder;
 import org.apache.beam.sdk.coders.VarLongCoder;
-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.schemas.SchemaCoder;
+import org.apache.beam.sdk.schemas.logicaltypes.FixedBytes;
 import org.apache.beam.sdk.transforms.SerializableFunction;
 import org.apache.beam.sdk.transforms.join.CoGbkResult.CoGbkResultCoder;
 import org.apache.beam.sdk.transforms.join.CoGbkResultSchema;
@@ -91,7 +91,7 @@
           .addDoubleField("double")
           .addStringField("string")
           .addArrayField("list_int32", FieldType.INT32)
-          .addLogicalTypeField("fixed_bytes", LogicalTypes.FixedBytes.of(4))
+          .addLogicalTypeField("fixed_bytes", FixedBytes.of(4))
           .build();
 
   /** Tests that all of the Default Coders are tested. */
diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/artifact/AbstractArtifactStagingService.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/artifact/AbstractArtifactStagingService.java
index 86e79a5..0b0fadf 100644
--- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/artifact/AbstractArtifactStagingService.java
+++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/artifact/AbstractArtifactStagingService.java
@@ -23,6 +23,7 @@
 import java.nio.channels.WritableByteChannel;
 import java.nio.charset.Charset;
 import java.nio.charset.StandardCharsets;
+import org.apache.beam.model.jobmanagement.v1.ArtifactApi;
 import org.apache.beam.model.jobmanagement.v1.ArtifactApi.ArtifactMetadata;
 import org.apache.beam.model.jobmanagement.v1.ArtifactApi.CommitManifestRequest;
 import org.apache.beam.model.jobmanagement.v1.ArtifactApi.CommitManifestResponse;
@@ -32,6 +33,7 @@
 import org.apache.beam.model.jobmanagement.v1.ArtifactApi.PutArtifactRequest;
 import org.apache.beam.model.jobmanagement.v1.ArtifactApi.PutArtifactResponse;
 import org.apache.beam.model.jobmanagement.v1.ArtifactStagingServiceGrpc.ArtifactStagingServiceImplBase;
+import org.apache.beam.model.pipeline.v1.RunnerApi;
 import org.apache.beam.runners.fnexecution.FnService;
 import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
 import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.util.JsonFormat;
@@ -50,7 +52,11 @@
 public abstract class AbstractArtifactStagingService extends ArtifactStagingServiceImplBase
     implements FnService {
 
-  public static final String NO_ARTIFACTS_STAGED_TOKEN = "__no_artifacts_staged__";
+  public static final String NO_ARTIFACTS_STAGED_TOKEN =
+      ArtifactApi.CommitManifestResponse.Constants.NO_ARTIFACTS_STAGED_TOKEN
+          .getValueDescriptor()
+          .getOptions()
+          .getExtension(RunnerApi.beamConstant);
 
   private static final Logger LOG = LoggerFactory.getLogger(AbstractArtifactStagingService.class);
 
diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/jobsubmission/InMemoryJobService.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/jobsubmission/InMemoryJobService.java
index 59a2647..2f4df48 100644
--- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/jobsubmission/InMemoryJobService.java
+++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/jobsubmission/InMemoryJobService.java
@@ -200,7 +200,7 @@
               try {
                 cleanupJobFn.accept(stagingSessionToken);
               } catch (Exception e) {
-                LOG.error(
+                LOG.warn(
                     "Failed to remove job staging directory for token {}: {}",
                     stagingSessionToken,
                     e);
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 8b7cac0..6516ed3 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
@@ -75,7 +75,7 @@
 	return proto.EnumName(LogEntry_Severity_Enum_name, int32(x))
 }
 func (LogEntry_Severity_Enum) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_232aec3131d70c18, []int{27, 1, 0}
+	return fileDescriptor_beam_fn_api_a35062cff5082b14, []int{27, 1, 0}
 }
 
 // A descriptor for connecting to a remote port using the Beam Fn Data API.
@@ -98,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_232aec3131d70c18, []int{0}
+	return fileDescriptor_beam_fn_api_a35062cff5082b14, []int{0}
 }
 func (m *RemoteGrpcPort) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_RemoteGrpcPort.Unmarshal(m, b)
@@ -158,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_232aec3131d70c18, []int{1}
+	return fileDescriptor_beam_fn_api_a35062cff5082b14, []int{1}
 }
 func (m *InstructionRequest) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_InstructionRequest.Unmarshal(m, b)
@@ -414,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_232aec3131d70c18, []int{2}
+	return fileDescriptor_beam_fn_api_a35062cff5082b14, []int{2}
 }
 func (m *InstructionResponse) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_InstructionResponse.Unmarshal(m, b)
@@ -662,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_232aec3131d70c18, []int{3}
+	return fileDescriptor_beam_fn_api_a35062cff5082b14, []int{3}
 }
 func (m *RegisterRequest) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_RegisterRequest.Unmarshal(m, b)
@@ -700,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_232aec3131d70c18, []int{4}
+	return fileDescriptor_beam_fn_api_a35062cff5082b14, []int{4}
 }
 func (m *RegisterResponse) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_RegisterResponse.Unmarshal(m, b)
@@ -748,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_232aec3131d70c18, []int{5}
+	return fileDescriptor_beam_fn_api_a35062cff5082b14, []int{5}
 }
 func (m *ProcessBundleDescriptor) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_ProcessBundleDescriptor.Unmarshal(m, b)
@@ -856,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_232aec3131d70c18, []int{6}
+	return fileDescriptor_beam_fn_api_a35062cff5082b14, []int{6}
 }
 func (m *BundleApplication) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_BundleApplication.Unmarshal(m, b)
@@ -941,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_232aec3131d70c18, []int{7}
+	return fileDescriptor_beam_fn_api_a35062cff5082b14, []int{7}
 }
 func (m *DelayedBundleApplication) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_DelayedBundleApplication.Unmarshal(m, b)
@@ -1000,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_232aec3131d70c18, []int{8}
+	return fileDescriptor_beam_fn_api_a35062cff5082b14, []int{8}
 }
 func (m *ProcessBundleRequest) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_ProcessBundleRequest.Unmarshal(m, b)
@@ -1054,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_232aec3131d70c18, []int{8, 0}
+	return fileDescriptor_beam_fn_api_a35062cff5082b14, []int{8, 0}
 }
 func (m *ProcessBundleRequest_CacheToken) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_ProcessBundleRequest_CacheToken.Unmarshal(m, b)
@@ -1203,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_232aec3131d70c18, []int{8, 0, 0}
+	return fileDescriptor_beam_fn_api_a35062cff5082b14, []int{8, 0, 0}
 }
 func (m *ProcessBundleRequest_CacheToken_UserState) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_ProcessBundleRequest_CacheToken_UserState.Unmarshal(m, b)
@@ -1238,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_232aec3131d70c18, []int{8, 0, 1}
+	return fileDescriptor_beam_fn_api_a35062cff5082b14, []int{8, 0, 1}
 }
 func (m *ProcessBundleRequest_CacheToken_SideInput) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_ProcessBundleRequest_CacheToken_SideInput.Unmarshal(m, b)
@@ -1292,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_232aec3131d70c18, []int{9}
+	return fileDescriptor_beam_fn_api_a35062cff5082b14, []int{9}
 }
 func (m *ProcessBundleResponse) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_ProcessBundleResponse.Unmarshal(m, b)
@@ -1356,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_232aec3131d70c18, []int{10}
+	return fileDescriptor_beam_fn_api_a35062cff5082b14, []int{10}
 }
 func (m *ProcessBundleProgressRequest) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_ProcessBundleProgressRequest.Unmarshal(m, b)
@@ -1395,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_232aec3131d70c18, []int{11}
+	return fileDescriptor_beam_fn_api_a35062cff5082b14, []int{11}
 }
 func (m *Metrics) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Metrics.Unmarshal(m, b)
@@ -1448,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_232aec3131d70c18, []int{11, 0}
+	return fileDescriptor_beam_fn_api_a35062cff5082b14, []int{11, 0}
 }
 func (m *Metrics_PTransform) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Metrics_PTransform.Unmarshal(m, b)
@@ -1518,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_232aec3131d70c18, []int{11, 0, 0}
+	return fileDescriptor_beam_fn_api_a35062cff5082b14, []int{11, 0, 0}
 }
 func (m *Metrics_PTransform_Measured) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Metrics_PTransform_Measured.Unmarshal(m, b)
@@ -1572,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_232aec3131d70c18, []int{11, 0, 1}
+	return fileDescriptor_beam_fn_api_a35062cff5082b14, []int{11, 0, 1}
 }
 func (m *Metrics_PTransform_ProcessedElements) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Metrics_PTransform_ProcessedElements.Unmarshal(m, b)
@@ -1626,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_232aec3131d70c18, []int{11, 0, 2}
+	return fileDescriptor_beam_fn_api_a35062cff5082b14, []int{11, 0, 2}
 }
 func (m *Metrics_PTransform_ActiveElements) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Metrics_PTransform_ActiveElements.Unmarshal(m, b)
@@ -1687,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_232aec3131d70c18, []int{11, 1}
+	return fileDescriptor_beam_fn_api_a35062cff5082b14, []int{11, 1}
 }
 func (m *Metrics_User) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Metrics_User.Unmarshal(m, b)
@@ -1868,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_232aec3131d70c18, []int{11, 1, 0}
+	return fileDescriptor_beam_fn_api_a35062cff5082b14, []int{11, 1, 0}
 }
 func (m *Metrics_User_MetricName) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Metrics_User_MetricName.Unmarshal(m, b)
@@ -1914,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_232aec3131d70c18, []int{11, 1, 1}
+	return fileDescriptor_beam_fn_api_a35062cff5082b14, []int{11, 1, 1}
 }
 func (m *Metrics_User_CounterData) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Metrics_User_CounterData.Unmarshal(m, b)
@@ -1956,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_232aec3131d70c18, []int{11, 1, 2}
+	return fileDescriptor_beam_fn_api_a35062cff5082b14, []int{11, 1, 2}
 }
 func (m *Metrics_User_DistributionData) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Metrics_User_DistributionData.Unmarshal(m, b)
@@ -2017,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_232aec3131d70c18, []int{11, 1, 3}
+	return fileDescriptor_beam_fn_api_a35062cff5082b14, []int{11, 1, 3}
 }
 func (m *Metrics_User_GaugeData) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Metrics_User_GaugeData.Unmarshal(m, b)
@@ -2069,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_232aec3131d70c18, []int{12}
+	return fileDescriptor_beam_fn_api_a35062cff5082b14, []int{12}
 }
 func (m *ProcessBundleProgressResponse) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_ProcessBundleProgressResponse.Unmarshal(m, b)
@@ -2130,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_232aec3131d70c18, []int{13}
+	return fileDescriptor_beam_fn_api_a35062cff5082b14, []int{13}
 }
 func (m *ProcessBundleSplitRequest) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_ProcessBundleSplitRequest.Unmarshal(m, b)
@@ -2190,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_232aec3131d70c18, []int{13, 0}
+	return fileDescriptor_beam_fn_api_a35062cff5082b14, []int{13, 0}
 }
 func (m *ProcessBundleSplitRequest_DesiredSplit) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_ProcessBundleSplitRequest_DesiredSplit.Unmarshal(m, b)
@@ -2260,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_232aec3131d70c18, []int{14}
+	return fileDescriptor_beam_fn_api_a35062cff5082b14, []int{14}
 }
 func (m *ProcessBundleSplitResponse) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_ProcessBundleSplitResponse.Unmarshal(m, b)
@@ -2331,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_232aec3131d70c18, []int{14, 0}
+	return fileDescriptor_beam_fn_api_a35062cff5082b14, []int{14, 0}
 }
 func (m *ProcessBundleSplitResponse_ChannelSplit) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_ProcessBundleSplitResponse_ChannelSplit.Unmarshal(m, b)
@@ -2385,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_232aec3131d70c18, []int{15}
+	return fileDescriptor_beam_fn_api_a35062cff5082b14, []int{15}
 }
 func (m *FinalizeBundleRequest) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_FinalizeBundleRequest.Unmarshal(m, b)
@@ -2422,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_232aec3131d70c18, []int{16}
+	return fileDescriptor_beam_fn_api_a35062cff5082b14, []int{16}
 }
 func (m *FinalizeBundleResponse) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_FinalizeBundleResponse.Unmarshal(m, b)
@@ -2456,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_232aec3131d70c18, []int{17}
+	return fileDescriptor_beam_fn_api_a35062cff5082b14, []int{17}
 }
 func (m *Elements) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Elements.Unmarshal(m, b)
@@ -2514,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_232aec3131d70c18, []int{17, 0}
+	return fileDescriptor_beam_fn_api_a35062cff5082b14, []int{17, 0}
 }
 func (m *Elements_Data) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Elements_Data.Unmarshal(m, b)
@@ -2582,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_232aec3131d70c18, []int{18}
+	return fileDescriptor_beam_fn_api_a35062cff5082b14, []int{18}
 }
 func (m *StateRequest) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_StateRequest.Unmarshal(m, b)
@@ -2787,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_232aec3131d70c18, []int{19}
+	return fileDescriptor_beam_fn_api_a35062cff5082b14, []int{19}
 }
 func (m *StateResponse) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_StateResponse.Unmarshal(m, b)
@@ -2979,7 +2979,7 @@
 func (m *StateKey) String() string { return proto.CompactTextString(m) }
 func (*StateKey) ProtoMessage()    {}
 func (*StateKey) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_232aec3131d70c18, []int{20}
+	return fileDescriptor_beam_fn_api_a35062cff5082b14, []int{20}
 }
 func (m *StateKey) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_StateKey.Unmarshal(m, b)
@@ -3216,7 +3216,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_232aec3131d70c18, []int{20, 0}
+	return fileDescriptor_beam_fn_api_a35062cff5082b14, []int{20, 0}
 }
 func (m *StateKey_Runner) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_StateKey_Runner.Unmarshal(m, b)
@@ -3271,7 +3271,7 @@
 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_232aec3131d70c18, []int{20, 1}
+	return fileDescriptor_beam_fn_api_a35062cff5082b14, []int{20, 1}
 }
 func (m *StateKey_IterableSideInput) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_StateKey_IterableSideInput.Unmarshal(m, b)
@@ -3343,7 +3343,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_232aec3131d70c18, []int{20, 2}
+	return fileDescriptor_beam_fn_api_a35062cff5082b14, []int{20, 2}
 }
 func (m *StateKey_MultimapSideInput) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_StateKey_MultimapSideInput.Unmarshal(m, b)
@@ -3419,7 +3419,7 @@
 func (m *StateKey_MultimapKeysSideInput) String() string { return proto.CompactTextString(m) }
 func (*StateKey_MultimapKeysSideInput) ProtoMessage()    {}
 func (*StateKey_MultimapKeysSideInput) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_232aec3131d70c18, []int{20, 3}
+	return fileDescriptor_beam_fn_api_a35062cff5082b14, []int{20, 3}
 }
 func (m *StateKey_MultimapKeysSideInput) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_StateKey_MultimapKeysSideInput.Unmarshal(m, b)
@@ -3479,7 +3479,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_232aec3131d70c18, []int{20, 4}
+	return fileDescriptor_beam_fn_api_a35062cff5082b14, []int{20, 4}
 }
 func (m *StateKey_BagUserState) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_StateKey_BagUserState.Unmarshal(m, b)
@@ -3544,7 +3544,7 @@
 func (m *StateGetRequest) String() string { return proto.CompactTextString(m) }
 func (*StateGetRequest) ProtoMessage()    {}
 func (*StateGetRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_232aec3131d70c18, []int{21}
+	return fileDescriptor_beam_fn_api_a35062cff5082b14, []int{21}
 }
 func (m *StateGetRequest) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_StateGetRequest.Unmarshal(m, b)
@@ -3591,7 +3591,7 @@
 func (m *StateGetResponse) String() string { return proto.CompactTextString(m) }
 func (*StateGetResponse) ProtoMessage()    {}
 func (*StateGetResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_232aec3131d70c18, []int{22}
+	return fileDescriptor_beam_fn_api_a35062cff5082b14, []int{22}
 }
 func (m *StateGetResponse) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_StateGetResponse.Unmarshal(m, b)
@@ -3640,7 +3640,7 @@
 func (m *StateAppendRequest) String() string { return proto.CompactTextString(m) }
 func (*StateAppendRequest) ProtoMessage()    {}
 func (*StateAppendRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_232aec3131d70c18, []int{23}
+	return fileDescriptor_beam_fn_api_a35062cff5082b14, []int{23}
 }
 func (m *StateAppendRequest) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_StateAppendRequest.Unmarshal(m, b)
@@ -3678,7 +3678,7 @@
 func (m *StateAppendResponse) String() string { return proto.CompactTextString(m) }
 func (*StateAppendResponse) ProtoMessage()    {}
 func (*StateAppendResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_232aec3131d70c18, []int{24}
+	return fileDescriptor_beam_fn_api_a35062cff5082b14, []int{24}
 }
 func (m *StateAppendResponse) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_StateAppendResponse.Unmarshal(m, b)
@@ -3709,7 +3709,7 @@
 func (m *StateClearRequest) String() string { return proto.CompactTextString(m) }
 func (*StateClearRequest) ProtoMessage()    {}
 func (*StateClearRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_232aec3131d70c18, []int{25}
+	return fileDescriptor_beam_fn_api_a35062cff5082b14, []int{25}
 }
 func (m *StateClearRequest) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_StateClearRequest.Unmarshal(m, b)
@@ -3740,7 +3740,7 @@
 func (m *StateClearResponse) String() string { return proto.CompactTextString(m) }
 func (*StateClearResponse) ProtoMessage()    {}
 func (*StateClearResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_232aec3131d70c18, []int{26}
+	return fileDescriptor_beam_fn_api_a35062cff5082b14, []int{26}
 }
 func (m *StateClearResponse) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_StateClearResponse.Unmarshal(m, b)
@@ -3796,7 +3796,7 @@
 func (m *LogEntry) String() string { return proto.CompactTextString(m) }
 func (*LogEntry) ProtoMessage()    {}
 func (*LogEntry) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_232aec3131d70c18, []int{27}
+	return fileDescriptor_beam_fn_api_a35062cff5082b14, []int{27}
 }
 func (m *LogEntry) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_LogEntry.Unmarshal(m, b)
@@ -3886,7 +3886,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_232aec3131d70c18, []int{27, 0}
+	return fileDescriptor_beam_fn_api_a35062cff5082b14, []int{27, 0}
 }
 func (m *LogEntry_List) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_LogEntry_List.Unmarshal(m, b)
@@ -3936,7 +3936,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_232aec3131d70c18, []int{27, 1}
+	return fileDescriptor_beam_fn_api_a35062cff5082b14, []int{27, 1}
 }
 func (m *LogEntry_Severity) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_LogEntry_Severity.Unmarshal(m, b)
@@ -3966,7 +3966,7 @@
 func (m *LogControl) String() string { return proto.CompactTextString(m) }
 func (*LogControl) ProtoMessage()    {}
 func (*LogControl) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_232aec3131d70c18, []int{28}
+	return fileDescriptor_beam_fn_api_a35062cff5082b14, []int{28}
 }
 func (m *LogControl) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_LogControl.Unmarshal(m, b)
@@ -4002,7 +4002,7 @@
 func (m *StartWorkerRequest) String() string { return proto.CompactTextString(m) }
 func (*StartWorkerRequest) ProtoMessage()    {}
 func (*StartWorkerRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_232aec3131d70c18, []int{29}
+	return fileDescriptor_beam_fn_api_a35062cff5082b14, []int{29}
 }
 func (m *StartWorkerRequest) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_StartWorkerRequest.Unmarshal(m, b)
@@ -4075,7 +4075,7 @@
 func (m *StartWorkerResponse) String() string { return proto.CompactTextString(m) }
 func (*StartWorkerResponse) ProtoMessage()    {}
 func (*StartWorkerResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_232aec3131d70c18, []int{30}
+	return fileDescriptor_beam_fn_api_a35062cff5082b14, []int{30}
 }
 func (m *StartWorkerResponse) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_StartWorkerResponse.Unmarshal(m, b)
@@ -4113,7 +4113,7 @@
 func (m *StopWorkerRequest) String() string { return proto.CompactTextString(m) }
 func (*StopWorkerRequest) ProtoMessage()    {}
 func (*StopWorkerRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_232aec3131d70c18, []int{31}
+	return fileDescriptor_beam_fn_api_a35062cff5082b14, []int{31}
 }
 func (m *StopWorkerRequest) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_StopWorkerRequest.Unmarshal(m, b)
@@ -4151,7 +4151,7 @@
 func (m *StopWorkerResponse) String() string { return proto.CompactTextString(m) }
 func (*StopWorkerResponse) ProtoMessage()    {}
 func (*StopWorkerResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_fn_api_232aec3131d70c18, []int{32}
+	return fileDescriptor_beam_fn_api_a35062cff5082b14, []int{32}
 }
 func (m *StopWorkerResponse) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_StopWorkerResponse.Unmarshal(m, b)
@@ -4755,9 +4755,9 @@
 	Metadata: "beam_fn_api.proto",
 }
 
-func init() { proto.RegisterFile("beam_fn_api.proto", fileDescriptor_beam_fn_api_232aec3131d70c18) }
+func init() { proto.RegisterFile("beam_fn_api.proto", fileDescriptor_beam_fn_api_a35062cff5082b14) }
 
-var fileDescriptor_beam_fn_api_232aec3131d70c18 = []byte{
+var fileDescriptor_beam_fn_api_a35062cff5082b14 = []byte{
 	// 3241 bytes of a gzipped FileDescriptorProto
 	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xbc, 0x5a, 0xdd, 0x6f, 0x1b, 0xc7,
 	0xb5, 0xf7, 0xf2, 0x43, 0x22, 0x0f, 0x29, 0x89, 0x1c, 0x49, 0x36, 0xbd, 0xd7, 0xb9, 0xd7, 0xe1,
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 f843ecc..609c74d 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
@@ -8,6 +8,7 @@
 import math "math"
 import pipeline_v1 "github.com/apache/beam/sdks/go/pkg/beam/model/pipeline_v1"
 import _struct "github.com/golang/protobuf/ptypes/struct"
+import timestamp "github.com/golang/protobuf/ptypes/timestamp"
 
 import (
 	context "golang.org/x/net/context"
@@ -57,7 +58,7 @@
 	return proto.EnumName(JobMessage_MessageImportance_name, int32(x))
 }
 func (JobMessage_MessageImportance) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_beam_job_api_cf64c696c499a6a1, []int{14, 0}
+	return fileDescriptor_beam_job_api_c1d5a4d112319449, []int{14, 0}
 }
 
 type JobState_Enum int32
@@ -122,7 +123,7 @@
 	return proto.EnumName(JobState_Enum_name, int32(x))
 }
 func (JobState_Enum) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_beam_job_api_cf64c696c499a6a1, []int{16, 0}
+	return fileDescriptor_beam_job_api_c1d5a4d112319449, []int{16, 0}
 }
 
 type PipelineOptionType_Enum int32
@@ -158,7 +159,7 @@
 	return proto.EnumName(PipelineOptionType_Enum_name, int32(x))
 }
 func (PipelineOptionType_Enum) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_beam_job_api_cf64c696c499a6a1, []int{21, 0}
+	return fileDescriptor_beam_job_api_c1d5a4d112319449, []int{21, 0}
 }
 
 // Prepare is a synchronous request that returns a preparationId back
@@ -178,7 +179,7 @@
 func (m *PrepareJobRequest) String() string { return proto.CompactTextString(m) }
 func (*PrepareJobRequest) ProtoMessage()    {}
 func (*PrepareJobRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_job_api_cf64c696c499a6a1, []int{0}
+	return fileDescriptor_beam_job_api_c1d5a4d112319449, []int{0}
 }
 func (m *PrepareJobRequest) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_PrepareJobRequest.Unmarshal(m, b)
@@ -238,7 +239,7 @@
 func (m *PrepareJobResponse) String() string { return proto.CompactTextString(m) }
 func (*PrepareJobResponse) ProtoMessage()    {}
 func (*PrepareJobResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_job_api_cf64c696c499a6a1, []int{1}
+	return fileDescriptor_beam_job_api_c1d5a4d112319449, []int{1}
 }
 func (m *PrepareJobResponse) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_PrepareJobResponse.Unmarshal(m, b)
@@ -299,7 +300,7 @@
 func (m *RunJobRequest) String() string { return proto.CompactTextString(m) }
 func (*RunJobRequest) ProtoMessage()    {}
 func (*RunJobRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_job_api_cf64c696c499a6a1, []int{2}
+	return fileDescriptor_beam_job_api_c1d5a4d112319449, []int{2}
 }
 func (m *RunJobRequest) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_RunJobRequest.Unmarshal(m, b)
@@ -344,7 +345,7 @@
 func (m *RunJobResponse) String() string { return proto.CompactTextString(m) }
 func (*RunJobResponse) ProtoMessage()    {}
 func (*RunJobResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_job_api_cf64c696c499a6a1, []int{3}
+	return fileDescriptor_beam_job_api_c1d5a4d112319449, []int{3}
 }
 func (m *RunJobResponse) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_RunJobResponse.Unmarshal(m, b)
@@ -385,7 +386,7 @@
 func (m *CancelJobRequest) String() string { return proto.CompactTextString(m) }
 func (*CancelJobRequest) ProtoMessage()    {}
 func (*CancelJobRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_job_api_cf64c696c499a6a1, []int{4}
+	return fileDescriptor_beam_job_api_c1d5a4d112319449, []int{4}
 }
 func (m *CancelJobRequest) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_CancelJobRequest.Unmarshal(m, b)
@@ -424,7 +425,7 @@
 func (m *CancelJobResponse) String() string { return proto.CompactTextString(m) }
 func (*CancelJobResponse) ProtoMessage()    {}
 func (*CancelJobResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_job_api_cf64c696c499a6a1, []int{5}
+	return fileDescriptor_beam_job_api_c1d5a4d112319449, []int{5}
 }
 func (m *CancelJobResponse) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_CancelJobResponse.Unmarshal(m, b)
@@ -466,7 +467,7 @@
 func (m *JobInfo) String() string { return proto.CompactTextString(m) }
 func (*JobInfo) ProtoMessage()    {}
 func (*JobInfo) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_job_api_cf64c696c499a6a1, []int{6}
+	return fileDescriptor_beam_job_api_c1d5a4d112319449, []int{6}
 }
 func (m *JobInfo) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_JobInfo.Unmarshal(m, b)
@@ -526,7 +527,7 @@
 func (m *GetJobsRequest) String() string { return proto.CompactTextString(m) }
 func (*GetJobsRequest) ProtoMessage()    {}
 func (*GetJobsRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_job_api_cf64c696c499a6a1, []int{7}
+	return fileDescriptor_beam_job_api_c1d5a4d112319449, []int{7}
 }
 func (m *GetJobsRequest) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_GetJobsRequest.Unmarshal(m, b)
@@ -557,7 +558,7 @@
 func (m *GetJobsResponse) String() string { return proto.CompactTextString(m) }
 func (*GetJobsResponse) ProtoMessage()    {}
 func (*GetJobsResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_job_api_cf64c696c499a6a1, []int{8}
+	return fileDescriptor_beam_job_api_c1d5a4d112319449, []int{8}
 }
 func (m *GetJobsResponse) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_GetJobsResponse.Unmarshal(m, b)
@@ -598,7 +599,7 @@
 func (m *GetJobStateRequest) String() string { return proto.CompactTextString(m) }
 func (*GetJobStateRequest) ProtoMessage()    {}
 func (*GetJobStateRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_job_api_cf64c696c499a6a1, []int{9}
+	return fileDescriptor_beam_job_api_c1d5a4d112319449, []int{9}
 }
 func (m *GetJobStateRequest) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_GetJobStateRequest.Unmarshal(m, b)
@@ -626,35 +627,36 @@
 }
 
 type JobStateEvent struct {
-	State                JobState_Enum `protobuf:"varint,1,opt,name=state,proto3,enum=org.apache.beam.model.job_management.v1.JobState_Enum" json:"state,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}      `json:"-"`
-	XXX_unrecognized     []byte        `json:"-"`
-	XXX_sizecache        int32         `json:"-"`
+	State                JobState_Enum        `protobuf:"varint,1,opt,name=state,proto3,enum=org.apache.beam.model.job_management.v1.JobState_Enum" json:"state,omitempty"`
+	Timestamp            *timestamp.Timestamp `protobuf:"bytes,2,opt,name=timestamp,proto3" json:"timestamp,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}             `json:"-"`
+	XXX_unrecognized     []byte               `json:"-"`
+	XXX_sizecache        int32                `json:"-"`
 }
 
 func (m *JobStateEvent) Reset()         { *m = JobStateEvent{} }
 func (m *JobStateEvent) String() string { return proto.CompactTextString(m) }
 func (*JobStateEvent) ProtoMessage()    {}
 func (*JobStateEvent) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_job_api_cf64c696c499a6a1, []int{10}
+	return fileDescriptor_beam_job_api_c1d5a4d112319449, []int{10}
 }
 func (m *JobStateEvent) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_GetJobStateResponse.Unmarshal(m, b)
+	return xxx_messageInfo_JobStateEvent.Unmarshal(m, b)
 }
 func (m *JobStateEvent) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_GetJobStateResponse.Marshal(b, m, deterministic)
+	return xxx_messageInfo_JobStateEvent.Marshal(b, m, deterministic)
 }
 func (dst *JobStateEvent) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_GetJobStateResponse.Merge(dst, src)
+	xxx_messageInfo_JobStateEvent.Merge(dst, src)
 }
 func (m *JobStateEvent) XXX_Size() int {
-	return xxx_messageInfo_GetJobStateResponse.Size(m)
+	return xxx_messageInfo_JobStateEvent.Size(m)
 }
 func (m *JobStateEvent) XXX_DiscardUnknown() {
-	xxx_messageInfo_GetJobStateResponse.DiscardUnknown(m)
+	xxx_messageInfo_JobStateEvent.DiscardUnknown(m)
 }
 
-var xxx_messageInfo_GetJobStateResponse proto.InternalMessageInfo
+var xxx_messageInfo_JobStateEvent proto.InternalMessageInfo
 
 func (m *JobStateEvent) GetState() JobState_Enum {
 	if m != nil {
@@ -663,6 +665,13 @@
 	return JobState_UNSPECIFIED
 }
 
+func (m *JobStateEvent) GetTimestamp() *timestamp.Timestamp {
+	if m != nil {
+		return m.Timestamp
+	}
+	return nil
+}
+
 // GetPipeline is a synchronus request that returns a pipeline back
 // Throws error GRPC_STATUS_UNAVAILABLE if server is down
 // Throws error NOT_FOUND if the jobId is not found
@@ -677,7 +686,7 @@
 func (m *GetJobPipelineRequest) String() string { return proto.CompactTextString(m) }
 func (*GetJobPipelineRequest) ProtoMessage()    {}
 func (*GetJobPipelineRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_job_api_cf64c696c499a6a1, []int{11}
+	return fileDescriptor_beam_job_api_c1d5a4d112319449, []int{11}
 }
 func (m *GetJobPipelineRequest) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_GetJobPipelineRequest.Unmarshal(m, b)
@@ -715,7 +724,7 @@
 func (m *GetJobPipelineResponse) String() string { return proto.CompactTextString(m) }
 func (*GetJobPipelineResponse) ProtoMessage()    {}
 func (*GetJobPipelineResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_job_api_cf64c696c499a6a1, []int{12}
+	return fileDescriptor_beam_job_api_c1d5a4d112319449, []int{12}
 }
 func (m *GetJobPipelineResponse) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_GetJobPipelineResponse.Unmarshal(m, b)
@@ -757,7 +766,7 @@
 func (m *JobMessagesRequest) String() string { return proto.CompactTextString(m) }
 func (*JobMessagesRequest) ProtoMessage()    {}
 func (*JobMessagesRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_job_api_cf64c696c499a6a1, []int{13}
+	return fileDescriptor_beam_job_api_c1d5a4d112319449, []int{13}
 }
 func (m *JobMessagesRequest) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_JobMessagesRequest.Unmarshal(m, b)
@@ -798,7 +807,7 @@
 func (m *JobMessage) String() string { return proto.CompactTextString(m) }
 func (*JobMessage) ProtoMessage()    {}
 func (*JobMessage) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_job_api_cf64c696c499a6a1, []int{14}
+	return fileDescriptor_beam_job_api_c1d5a4d112319449, []int{14}
 }
 func (m *JobMessage) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_JobMessage.Unmarshal(m, b)
@@ -860,7 +869,7 @@
 func (m *JobMessagesResponse) String() string { return proto.CompactTextString(m) }
 func (*JobMessagesResponse) ProtoMessage()    {}
 func (*JobMessagesResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_job_api_cf64c696c499a6a1, []int{15}
+	return fileDescriptor_beam_job_api_c1d5a4d112319449, []int{15}
 }
 func (m *JobMessagesResponse) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_JobMessagesResponse.Unmarshal(m, b)
@@ -1010,7 +1019,7 @@
 func (m *JobState) String() string { return proto.CompactTextString(m) }
 func (*JobState) ProtoMessage()    {}
 func (*JobState) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_job_api_cf64c696c499a6a1, []int{16}
+	return fileDescriptor_beam_job_api_c1d5a4d112319449, []int{16}
 }
 func (m *JobState) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_JobState.Unmarshal(m, b)
@@ -1041,7 +1050,7 @@
 func (m *GetJobMetricsRequest) String() string { return proto.CompactTextString(m) }
 func (*GetJobMetricsRequest) ProtoMessage()    {}
 func (*GetJobMetricsRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_job_api_cf64c696c499a6a1, []int{17}
+	return fileDescriptor_beam_job_api_c1d5a4d112319449, []int{17}
 }
 func (m *GetJobMetricsRequest) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_GetJobMetricsRequest.Unmarshal(m, b)
@@ -1079,7 +1088,7 @@
 func (m *GetJobMetricsResponse) String() string { return proto.CompactTextString(m) }
 func (*GetJobMetricsResponse) ProtoMessage()    {}
 func (*GetJobMetricsResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_job_api_cf64c696c499a6a1, []int{18}
+	return fileDescriptor_beam_job_api_c1d5a4d112319449, []int{18}
 }
 func (m *GetJobMetricsResponse) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_GetJobMetricsResponse.Unmarshal(m, b)
@@ -1119,7 +1128,7 @@
 func (m *MetricResults) String() string { return proto.CompactTextString(m) }
 func (*MetricResults) ProtoMessage()    {}
 func (*MetricResults) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_job_api_cf64c696c499a6a1, []int{19}
+	return fileDescriptor_beam_job_api_c1d5a4d112319449, []int{19}
 }
 func (m *MetricResults) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_MetricResults.Unmarshal(m, b)
@@ -1167,7 +1176,7 @@
 func (m *DescribePipelineOptionsRequest) String() string { return proto.CompactTextString(m) }
 func (*DescribePipelineOptionsRequest) ProtoMessage()    {}
 func (*DescribePipelineOptionsRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_job_api_cf64c696c499a6a1, []int{20}
+	return fileDescriptor_beam_job_api_c1d5a4d112319449, []int{20}
 }
 func (m *DescribePipelineOptionsRequest) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_DescribePipelineOptionsRequest.Unmarshal(m, b)
@@ -1199,7 +1208,7 @@
 func (m *PipelineOptionType) String() string { return proto.CompactTextString(m) }
 func (*PipelineOptionType) ProtoMessage()    {}
 func (*PipelineOptionType) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_job_api_cf64c696c499a6a1, []int{21}
+	return fileDescriptor_beam_job_api_c1d5a4d112319449, []int{21}
 }
 func (m *PipelineOptionType) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_PipelineOptionType.Unmarshal(m, b)
@@ -1240,7 +1249,7 @@
 func (m *PipelineOptionDescriptor) String() string { return proto.CompactTextString(m) }
 func (*PipelineOptionDescriptor) ProtoMessage()    {}
 func (*PipelineOptionDescriptor) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_job_api_cf64c696c499a6a1, []int{22}
+	return fileDescriptor_beam_job_api_c1d5a4d112319449, []int{22}
 }
 func (m *PipelineOptionDescriptor) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_PipelineOptionDescriptor.Unmarshal(m, b)
@@ -1307,7 +1316,7 @@
 func (m *DescribePipelineOptionsResponse) String() string { return proto.CompactTextString(m) }
 func (*DescribePipelineOptionsResponse) ProtoMessage()    {}
 func (*DescribePipelineOptionsResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_job_api_cf64c696c499a6a1, []int{23}
+	return fileDescriptor_beam_job_api_c1d5a4d112319449, []int{23}
 }
 func (m *DescribePipelineOptionsResponse) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_DescribePipelineOptionsResponse.Unmarshal(m, b)
@@ -1810,97 +1819,99 @@
 	Metadata: "beam_job_api.proto",
 }
 
-func init() { proto.RegisterFile("beam_job_api.proto", fileDescriptor_beam_job_api_cf64c696c499a6a1) }
+func init() { proto.RegisterFile("beam_job_api.proto", fileDescriptor_beam_job_api_c1d5a4d112319449) }
 
-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,
+var fileDescriptor_beam_job_api_c1d5a4d112319449 = []byte{
+	// 1444 bytes of a gzipped FileDescriptorProto
+	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x57, 0x5f, 0x6f, 0x1b, 0xc5,
+	0x16, 0xef, 0x3a, 0x76, 0x6c, 0x1f, 0xd7, 0xce, 0x66, 0xda, 0xdc, 0xa4, 0xd6, 0xbd, 0x6d, 0xee,
+	0x5e, 0x5d, 0x5a, 0x54, 0xb1, 0x6d, 0x52, 0x89, 0x96, 0x16, 0x0a, 0xeb, 0x78, 0xeb, 0xda, 0x24,
+	0xb6, 0x35, 0xde, 0x80, 0x00, 0x09, 0xb3, 0xb6, 0x27, 0x66, 0x8b, 0x77, 0x67, 0xd9, 0x1d, 0x5b,
+	0xad, 0x84, 0x40, 0xe2, 0x85, 0x37, 0x78, 0xe4, 0x3b, 0x20, 0x24, 0xc4, 0x13, 0x0f, 0x7c, 0x06,
+	0x3e, 0x04, 0x12, 0x9f, 0x81, 0x17, 0x5e, 0xd0, 0xcc, 0xce, 0x3a, 0xde, 0xa4, 0x21, 0xb6, 0xa9,
+	0xc4, 0x93, 0x77, 0xce, 0x9f, 0xdf, 0xf9, 0x3b, 0x67, 0x8e, 0x01, 0xf5, 0x88, 0xed, 0x76, 0x9f,
+	0xd0, 0x5e, 0xd7, 0xf6, 0x1d, 0xdd, 0x0f, 0x28, 0xa3, 0xe8, 0x3a, 0x0d, 0x86, 0xba, 0xed, 0xdb,
+	0xfd, 0x8f, 0x89, 0xce, 0xd9, 0xba, 0x4b, 0x07, 0x64, 0xa4, 0x73, 0x21, 0xd7, 0xf6, 0xec, 0x21,
+	0x71, 0x89, 0xc7, 0xf4, 0xc9, 0x4e, 0x79, 0x43, 0x28, 0x07, 0x63, 0xcf, 0x23, 0xc1, 0xb1, 0x7e,
+	0x79, 0x8d, 0x78, 0x03, 0x9f, 0x3a, 0x1e, 0x0b, 0x25, 0xe1, 0xdf, 0x43, 0x4a, 0x87, 0x23, 0x72,
+	0x4b, 0x9c, 0x7a, 0xe3, 0xa3, 0x5b, 0x21, 0x0b, 0xc6, 0x7d, 0x26, 0xb9, 0xd7, 0x4e, 0x72, 0x99,
+	0xe3, 0x92, 0x90, 0xd9, 0xae, 0x2f, 0x05, 0x8a, 0x2e, 0x61, 0x81, 0xd3, 0x97, 0x68, 0xda, 0xcf,
+	0x0a, 0xac, 0xb7, 0x03, 0xe2, 0xdb, 0x01, 0x69, 0xd0, 0x1e, 0x26, 0x9f, 0x8e, 0x49, 0xc8, 0x50,
+	0x0d, 0x72, 0xbe, 0xe3, 0x93, 0x91, 0xe3, 0x91, 0x2d, 0x65, 0x5b, 0xb9, 0x51, 0xd8, 0xbd, 0xa9,
+	0x3f, 0x3f, 0x8e, 0x58, 0x4c, 0x9f, 0xec, 0xe8, 0x6d, 0xf9, 0x8d, 0xa7, 0xca, 0xa8, 0x02, 0x6a,
+	0xfc, 0xdd, 0xa5, 0x3e, 0x73, 0xa8, 0x17, 0x6e, 0xa5, 0x04, 0xe0, 0xa6, 0x1e, 0x79, 0xaa, 0xc7,
+	0x9e, 0xea, 0x1d, 0x11, 0x07, 0x5e, 0x8b, 0x15, 0x5a, 0x91, 0x3c, 0xba, 0x02, 0x39, 0x9e, 0x2d,
+	0xcf, 0x76, 0xc9, 0xd6, 0xca, 0xb6, 0x72, 0x23, 0x8f, 0xb3, 0x4f, 0x68, 0xaf, 0x69, 0xbb, 0x44,
+	0xfb, 0x4d, 0x01, 0x34, 0xeb, 0x7d, 0xe8, 0x53, 0x2f, 0x24, 0xe8, 0xff, 0x50, 0xf2, 0x05, 0xd5,
+	0xe6, 0x08, 0x5d, 0x67, 0x20, 0x82, 0xc8, 0xe3, 0xe2, 0x0c, 0xb5, 0x3e, 0x40, 0x21, 0x5c, 0xb1,
+	0x03, 0xe6, 0x1c, 0xd9, 0x7d, 0xd6, 0x0d, 0x99, 0x3d, 0x74, 0xbc, 0x61, 0x37, 0xce, 0xb6, 0xf4,
+	0xf2, 0xee, 0x1c, 0x61, 0x1b, 0xbe, 0xd3, 0x21, 0xc1, 0xc4, 0xe9, 0x93, 0x2a, 0x09, 0xfb, 0x81,
+	0xe3, 0x33, 0x1a, 0xe0, 0xcd, 0x18, 0xb9, 0x13, 0x01, 0x9b, 0x12, 0x17, 0xed, 0xc2, 0x46, 0x6c,
+	0x2b, 0x24, 0x61, 0xc8, 0xfd, 0x63, 0xf4, 0x13, 0xe2, 0xc9, 0xd0, 0x2e, 0x49, 0x66, 0x27, 0xe2,
+	0x59, 0x9c, 0xa5, 0x75, 0xa1, 0x88, 0xc7, 0xde, 0x4c, 0x7d, 0xe6, 0x0c, 0xf0, 0x3a, 0xac, 0x05,
+	0xbc, 0xda, 0x64, 0x62, 0x8f, 0xa4, 0x95, 0x94, 0x90, 0x2b, 0x4d, 0xc9, 0x91, 0x81, 0xeb, 0x50,
+	0x8a, 0x0d, 0xc8, 0x14, 0x6e, 0xc0, 0x2a, 0x4f, 0xfa, 0x14, 0x39, 0xf3, 0x84, 0xf6, 0xea, 0x03,
+	0xed, 0x65, 0x50, 0xf7, 0x6c, 0xaf, 0x4f, 0x46, 0x33, 0xce, 0x9c, 0x21, 0x6a, 0xc3, 0xfa, 0x8c,
+	0xa8, 0x84, 0xdd, 0x87, 0x4c, 0xc8, 0x6c, 0x16, 0x75, 0x55, 0x69, 0xf7, 0x55, 0x7d, 0xce, 0xdb,
+	0xa1, 0x37, 0x68, 0xaf, 0xc3, 0x15, 0x75, 0xd3, 0x1b, 0xbb, 0x38, 0x02, 0xd1, 0x7e, 0x51, 0x20,
+	0xdb, 0xa0, 0xbd, 0xba, 0x77, 0x44, 0xcf, 0xf0, 0x22, 0xd1, 0x3c, 0xa9, 0x44, 0xf3, 0x3c, 0xb7,
+	0x37, 0x57, 0x16, 0xec, 0xcd, 0x69, 0x3c, 0xe9, 0x17, 0x11, 0x8f, 0x0a, 0xa5, 0x1a, 0x61, 0x0d,
+	0xda, 0x0b, 0x65, 0x6e, 0xb5, 0x0f, 0x61, 0x6d, 0x4a, 0x91, 0x29, 0x7c, 0x3b, 0x8a, 0xc8, 0xf1,
+	0x8e, 0xe8, 0x96, 0xb2, 0xbd, 0x72, 0xa3, 0xb0, 0x7b, 0x7b, 0x11, 0xab, 0x3c, 0x59, 0x22, 0x07,
+	0xfc, 0x43, 0xbb, 0x09, 0x28, 0xc2, 0x17, 0xce, 0x9c, 0x53, 0xd1, 0x6f, 0x15, 0x28, 0xc6, 0xa2,
+	0xe6, 0x84, 0x78, 0xec, 0xc5, 0x96, 0x13, 0xdd, 0x83, 0xfc, 0x74, 0x5a, 0xc9, 0xfb, 0x57, 0x3e,
+	0x55, 0x09, 0x2b, 0x96, 0xc0, 0xc7, 0xc2, 0x9a, 0x0e, 0x1b, 0x51, 0x18, 0xd3, 0x11, 0x74, 0x5e,
+	0x6f, 0xfe, 0xeb, 0xa4, 0xbc, 0xcc, 0xee, 0x8b, 0x9a, 0x7c, 0x3c, 0xb3, 0x0d, 0xda, 0x3b, 0x20,
+	0x61, 0x68, 0x0f, 0x49, 0x78, 0x8e, 0x3f, 0x7f, 0xa4, 0x00, 0x8e, 0xa5, 0xd1, 0x7f, 0x00, 0xdc,
+	0xe8, 0xf3, 0x58, 0x32, 0x2f, 0x29, 0xf5, 0x01, 0x42, 0x90, 0xe6, 0xa1, 0xcb, 0x7e, 0x16, 0xdf,
+	0x88, 0x00, 0x38, 0xae, 0x4f, 0x03, 0xc6, 0xaf, 0x9c, 0x68, 0xe3, 0xd2, 0xae, 0xb9, 0x48, 0x39,
+	0xa4, 0x6d, 0x5d, 0xfe, 0xd6, 0xa7, 0x60, 0x78, 0x06, 0x18, 0xfd, 0x17, 0x2e, 0xc6, 0x9e, 0x31,
+	0xf2, 0x94, 0x89, 0xb6, 0xcf, 0xe3, 0x82, 0xa4, 0x59, 0xe4, 0x29, 0xd3, 0x7e, 0x50, 0x60, 0xfd,
+	0x14, 0x08, 0xd2, 0xe0, 0xea, 0x81, 0xd9, 0xe9, 0x18, 0x35, 0xb3, 0x5b, 0x3f, 0x68, 0xb7, 0xb0,
+	0x65, 0x34, 0xf7, 0xcc, 0xee, 0x61, 0xb3, 0xd3, 0x36, 0xf7, 0xea, 0x8f, 0xea, 0x66, 0x55, 0xbd,
+	0x80, 0x36, 0x60, 0xbd, 0xd1, 0xaa, 0x74, 0x63, 0xb9, 0xaa, 0x59, 0x39, 0xac, 0xa9, 0x0a, 0xda,
+	0x82, 0xcb, 0x49, 0xb2, 0x65, 0xd4, 0xf7, 0xcd, 0xaa, 0x9a, 0x3a, 0xa9, 0x50, 0x31, 0x3a, 0xf5,
+	0x3d, 0x75, 0x05, 0x6d, 0xc2, 0xa5, 0x59, 0xf2, 0xbb, 0x06, 0x6e, 0xd6, 0x9b, 0x35, 0x35, 0x7d,
+	0x52, 0xde, 0xc4, 0xb8, 0x85, 0xd5, 0x0c, 0x7f, 0x45, 0x2e, 0x25, 0x6a, 0x25, 0x7b, 0xe1, 0x23,
+	0x50, 0xe3, 0x60, 0x03, 0x49, 0x93, 0x3d, 0x71, 0x67, 0x89, 0xcc, 0x3e, 0xbe, 0x80, 0xd7, 0x24,
+	0xdc, 0xd4, 0x42, 0x17, 0x4a, 0xa2, 0xf5, 0x8f, 0xf1, 0xa3, 0xb6, 0x5f, 0xfc, 0x22, 0x89, 0xfb,
+	0xf8, 0xf8, 0x02, 0x2e, 0x86, 0xd1, 0x45, 0x8e, 0xe0, 0x2a, 0x00, 0xb9, 0x18, 0x5a, 0xfb, 0x49,
+	0x81, 0x5c, 0x2c, 0xae, 0x7d, 0xa7, 0x40, 0x9a, 0xdf, 0x3d, 0xb4, 0x06, 0x85, 0x64, 0x15, 0x0a,
+	0x90, 0xed, 0x58, 0xad, 0x76, 0xdb, 0xac, 0xaa, 0x0a, 0x3f, 0xe0, 0xc3, 0xa6, 0x48, 0x5f, 0x0a,
+	0xe5, 0x20, 0x5d, 0x6d, 0x35, 0x4d, 0x75, 0x05, 0x01, 0xac, 0x3e, 0x8a, 0x8a, 0x90, 0x46, 0x45,
+	0xc8, 0xef, 0xf1, 0x62, 0xee, 0xf3, 0x63, 0x86, 0x6b, 0x1c, 0xb6, 0xab, 0x86, 0x65, 0x56, 0xd5,
+	0x55, 0x74, 0x11, 0x72, 0x55, 0x6c, 0xd4, 0x85, 0x7e, 0x96, 0xb3, 0xc4, 0xc9, 0xac, 0xaa, 0x39,
+	0xce, 0xea, 0x58, 0x06, 0xb6, 0x38, 0x2b, 0x8f, 0x4a, 0x00, 0x12, 0x84, 0x9f, 0x81, 0x73, 0x05,
+	0x0a, 0x3f, 0x15, 0xb4, 0x57, 0xe0, 0x72, 0x74, 0x5d, 0x0f, 0xa2, 0xdd, 0xe5, 0x9c, 0xdb, 0xe4,
+	0xc4, 0xd3, 0x60, 0x2a, 0x2e, 0xd3, 0xdd, 0x86, 0xac, 0xdc, 0x7e, 0x64, 0x1d, 0xe7, 0xcf, 0x73,
+	0x04, 0x85, 0x49, 0x38, 0x1e, 0xb1, 0x10, 0xc7, 0x30, 0xda, 0x8f, 0x0a, 0x14, 0x13, 0x2c, 0xd4,
+	0x82, 0xbc, 0xcd, 0x18, 0x71, 0x7d, 0x46, 0x06, 0x72, 0x3e, 0xef, 0xcc, 0x31, 0x41, 0x0e, 0xa8,
+	0xe7, 0x30, 0x1a, 0x38, 0xde, 0x50, 0x0c, 0xe8, 0x63, 0x0c, 0x0e, 0xd8, 0xa7, 0xae, 0xeb, 0x30,
+	0x0e, 0x98, 0x5a, 0x1a, 0x70, 0x8a, 0xa1, 0x6d, 0xc3, 0xd5, 0x68, 0x51, 0xe9, 0x91, 0x76, 0xf2,
+	0x39, 0x8b, 0x5f, 0x1d, 0x02, 0x28, 0xc9, 0xb1, 0x9e, 0xf9, 0x44, 0x6b, 0xc9, 0x8e, 0x01, 0x58,
+	0xed, 0x58, 0x98, 0x57, 0x46, 0x34, 0x4b, 0xa5, 0xd5, 0xda, 0x37, 0x8d, 0x66, 0xd4, 0x2c, 0xf5,
+	0xa6, 0x65, 0xd6, 0x4c, 0xac, 0xa6, 0xb8, 0x54, 0xf3, 0xf0, 0xa0, 0x62, 0x62, 0x75, 0x05, 0xe5,
+	0x21, 0x63, 0x60, 0x6c, 0xbc, 0xa7, 0xa6, 0x39, 0xb9, 0x55, 0x69, 0x98, 0x7b, 0x96, 0x9a, 0xd1,
+	0x7e, 0x55, 0x60, 0x2b, 0x69, 0xe7, 0x78, 0x81, 0xe2, 0x43, 0x4e, 0x3c, 0xda, 0x51, 0x65, 0xc5,
+	0x37, 0xb2, 0x20, 0xcd, 0x9e, 0xf9, 0xd1, 0x25, 0x29, 0xed, 0xbe, 0x35, 0x77, 0xf1, 0x4e, 0x07,
+	0x13, 0xbd, 0x3b, 0x02, 0x0d, 0x6d, 0x43, 0x61, 0x20, 0xed, 0x3a, 0x34, 0xde, 0xc3, 0x66, 0x49,
+	0xe8, 0x7f, 0x50, 0x1c, 0x90, 0x23, 0x7b, 0x3c, 0x62, 0xdd, 0x89, 0x3d, 0x1a, 0x13, 0x39, 0xf6,
+	0x2e, 0x4a, 0xe2, 0x3b, 0x9c, 0x86, 0x2e, 0x43, 0x66, 0x18, 0xd0, 0xb1, 0xbf, 0x95, 0x89, 0x7a,
+	0x51, 0x1c, 0xb4, 0xcf, 0xe1, 0xda, 0x99, 0xc9, 0x96, 0x5d, 0xf9, 0x01, 0x64, 0xe3, 0xf5, 0x23,
+	0xea, 0x17, 0x63, 0xc9, 0xc0, 0x66, 0xd6, 0xcf, 0x18, 0x71, 0xf7, 0x77, 0x10, 0x2f, 0x8b, 0x5c,
+	0x50, 0xd1, 0x97, 0x0a, 0x64, 0xe5, 0xc2, 0x8c, 0xee, 0xcf, 0x6f, 0xe6, 0xe4, 0x1f, 0x84, 0xf2,
+	0x83, 0xa5, 0x74, 0x65, 0xc0, 0x13, 0x58, 0xc1, 0x63, 0x0f, 0xcd, 0x7f, 0xf9, 0x12, 0xcb, 0x6f,
+	0xf9, 0xee, 0xc2, 0x7a, 0xd2, 0xee, 0x67, 0x90, 0x95, 0xcb, 0x14, 0x9a, 0x1f, 0x23, 0xb9, 0x90,
+	0x95, 0xef, 0x2d, 0xae, 0x28, 0xad, 0x7f, 0x01, 0xb9, 0x1a, 0x61, 0x62, 0xfa, 0xa2, 0x07, 0x0b,
+	0xa2, 0xcc, 0x6e, 0x67, 0xe5, 0x25, 0x1f, 0x07, 0xf4, 0xb5, 0x02, 0x85, 0x1a, 0x61, 0x71, 0xcf,
+	0xa0, 0x87, 0x0b, 0x3a, 0x71, 0x62, 0xb7, 0x2a, 0xbf, 0xb9, 0xb4, 0xfe, 0x34, 0x23, 0xab, 0xd1,
+	0x3f, 0x04, 0xf4, 0xda, 0xdc, 0x50, 0x27, 0xff, 0x7d, 0x94, 0xef, 0x2f, 0xa3, 0x2a, 0x1d, 0xf8,
+	0x4a, 0x11, 0x0b, 0xb7, 0xc8, 0x51, 0x87, 0x05, 0xc4, 0x76, 0xff, 0x91, 0xca, 0xdc, 0x56, 0xd0,
+	0x37, 0x0a, 0xa8, 0x35, 0xc2, 0xe4, 0xaa, 0xb0, 0xb0, 0x2f, 0xa7, 0x37, 0xcd, 0xf2, 0xeb, 0xcb,
+	0x29, 0x47, 0x99, 0xb9, 0xad, 0xf0, 0x6e, 0x29, 0x26, 0x5e, 0x51, 0xf4, 0xc6, 0x82, 0xa9, 0x49,
+	0x3e, 0xd6, 0xe5, 0x87, 0xcb, 0xaa, 0xcb, 0x62, 0x7d, 0xaf, 0xc0, 0xe6, 0x19, 0xa3, 0x14, 0xd5,
+	0xe6, 0xc6, 0xfe, 0xeb, 0x97, 0xaf, 0xfc, 0xf8, 0xef, 0x03, 0xc9, 0xcd, 0xab, 0x02, 0x2f, 0x9d,
+	0x09, 0x95, 0x40, 0xaa, 0xac, 0x36, 0x68, 0xcf, 0xf0, 0x9d, 0xf7, 0xd5, 0x04, 0xa7, 0x3b, 0xd9,
+	0xe9, 0xad, 0x8a, 0x7f, 0x3d, 0x77, 0xfe, 0x0c, 0x00, 0x00, 0xff, 0xff, 0xb7, 0xb1, 0x58, 0x7c,
+	0x58, 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 20f6996..39184e8 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
@@ -8,6 +8,12 @@
 import math "math"
 import descriptor "github.com/golang/protobuf/protoc-gen-go/descriptor"
 import any "github.com/golang/protobuf/ptypes/any"
+import _ "github.com/golang/protobuf/ptypes/timestamp"
+
+import (
+	context "golang.org/x/net/context"
+	grpc "google.golang.org/grpc"
+)
 
 // Reference imports to suppress errors if they are not otherwise used.
 var _ = proto.Marshal
@@ -51,7 +57,7 @@
 	return proto.EnumName(BeamConstants_Constants_name, int32(x))
 }
 func (BeamConstants_Constants) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{0, 0}
+	return fileDescriptor_beam_runner_api_70c7dbd5f3375954, []int{0, 0}
 }
 
 type StandardPTransforms_Primitives int32
@@ -130,7 +136,7 @@
 	return proto.EnumName(StandardPTransforms_Primitives_name, int32(x))
 }
 func (StandardPTransforms_Primitives) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{4, 0}
+	return fileDescriptor_beam_runner_api_70c7dbd5f3375954, []int{4, 0}
 }
 
 type StandardPTransforms_DeprecatedPrimitives int32
@@ -157,7 +163,7 @@
 	return proto.EnumName(StandardPTransforms_DeprecatedPrimitives_name, int32(x))
 }
 func (StandardPTransforms_DeprecatedPrimitives) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{4, 1}
+	return fileDescriptor_beam_runner_api_70c7dbd5f3375954, []int{4, 1}
 }
 
 type StandardPTransforms_Composites int32
@@ -196,7 +202,7 @@
 	return proto.EnumName(StandardPTransforms_Composites_name, int32(x))
 }
 func (StandardPTransforms_Composites) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{4, 2}
+	return fileDescriptor_beam_runner_api_70c7dbd5f3375954, []int{4, 2}
 }
 
 // Payload for all of these: CombinePayload
@@ -242,7 +248,7 @@
 	return proto.EnumName(StandardPTransforms_CombineComponents_name, int32(x))
 }
 func (StandardPTransforms_CombineComponents) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{4, 3}
+	return fileDescriptor_beam_runner_api_70c7dbd5f3375954, []int{4, 3}
 }
 
 // Payload for all of these: ParDoPayload containing the user's SDF
@@ -302,7 +308,7 @@
 	return proto.EnumName(StandardPTransforms_SplittableParDoComponents_name, int32(x))
 }
 func (StandardPTransforms_SplittableParDoComponents) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{4, 4}
+	return fileDescriptor_beam_runner_api_70c7dbd5f3375954, []int{4, 4}
 }
 
 type StandardSideInputTypes_Enum int32
@@ -333,7 +339,7 @@
 	return proto.EnumName(StandardSideInputTypes_Enum_name, int32(x))
 }
 func (StandardSideInputTypes_Enum) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{5, 0}
+	return fileDescriptor_beam_runner_api_70c7dbd5f3375954, []int{5, 0}
 }
 
 type Parameter_Type_Enum int32
@@ -362,7 +368,7 @@
 	return proto.EnumName(Parameter_Type_Enum_name, int32(x))
 }
 func (Parameter_Type_Enum) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{8, 0, 0}
+	return fileDescriptor_beam_runner_api_70c7dbd5f3375954, []int{8, 0, 0}
 }
 
 type IsBounded_Enum int32
@@ -388,7 +394,7 @@
 	return proto.EnumName(IsBounded_Enum_name, int32(x))
 }
 func (IsBounded_Enum) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{16, 0}
+	return fileDescriptor_beam_runner_api_70c7dbd5f3375954, []int{16, 0}
 }
 
 type StandardCoders_Enum int32
@@ -548,7 +554,7 @@
 	return proto.EnumName(StandardCoders_Enum_name, int32(x))
 }
 func (StandardCoders_Enum) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{23, 0}
+	return fileDescriptor_beam_runner_api_70c7dbd5f3375954, []int{24, 0}
 }
 
 type MergeStatus_Enum int32
@@ -585,7 +591,7 @@
 	return proto.EnumName(MergeStatus_Enum_name, int32(x))
 }
 func (MergeStatus_Enum) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{25, 0}
+	return fileDescriptor_beam_runner_api_70c7dbd5f3375954, []int{26, 0}
 }
 
 type AccumulationMode_Enum int32
@@ -617,7 +623,7 @@
 	return proto.EnumName(AccumulationMode_Enum_name, int32(x))
 }
 func (AccumulationMode_Enum) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{26, 0}
+	return fileDescriptor_beam_runner_api_70c7dbd5f3375954, []int{27, 0}
 }
 
 type ClosingBehavior_Enum int32
@@ -646,7 +652,7 @@
 	return proto.EnumName(ClosingBehavior_Enum_name, int32(x))
 }
 func (ClosingBehavior_Enum) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{27, 0}
+	return fileDescriptor_beam_runner_api_70c7dbd5f3375954, []int{28, 0}
 }
 
 type OnTimeBehavior_Enum int32
@@ -675,7 +681,7 @@
 	return proto.EnumName(OnTimeBehavior_Enum_name, int32(x))
 }
 func (OnTimeBehavior_Enum) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{28, 0}
+	return fileDescriptor_beam_runner_api_70c7dbd5f3375954, []int{29, 0}
 }
 
 type OutputTime_Enum int32
@@ -709,7 +715,7 @@
 	return proto.EnumName(OutputTime_Enum_name, int32(x))
 }
 func (OutputTime_Enum) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{29, 0}
+	return fileDescriptor_beam_runner_api_70c7dbd5f3375954, []int{30, 0}
 }
 
 type TimeDomain_Enum int32
@@ -746,7 +752,7 @@
 	return proto.EnumName(TimeDomain_Enum_name, int32(x))
 }
 func (TimeDomain_Enum) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{30, 0}
+	return fileDescriptor_beam_runner_api_70c7dbd5f3375954, []int{31, 0}
 }
 
 type StandardEnvironments_Environments int32
@@ -772,7 +778,7 @@
 	return proto.EnumName(StandardEnvironments_Environments_name, int32(x))
 }
 func (StandardEnvironments_Environments) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{35, 0}
+	return fileDescriptor_beam_runner_api_70c7dbd5f3375954, []int{36, 0}
 }
 
 type DisplayData_Type_Enum int32
@@ -813,7 +819,7 @@
 	return proto.EnumName(DisplayData_Type_Enum_name, int32(x))
 }
 func (DisplayData_Type_Enum) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{41, 2, 0}
+	return fileDescriptor_beam_runner_api_70c7dbd5f3375954, []int{42, 2, 0}
 }
 
 type BeamConstants struct {
@@ -826,7 +832,7 @@
 func (m *BeamConstants) String() string { return proto.CompactTextString(m) }
 func (*BeamConstants) ProtoMessage()    {}
 func (*BeamConstants) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{0}
+	return fileDescriptor_beam_runner_api_70c7dbd5f3375954, []int{0}
 }
 func (m *BeamConstants) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_BeamConstants.Unmarshal(m, b)
@@ -868,7 +874,7 @@
 func (m *Components) String() string { return proto.CompactTextString(m) }
 func (*Components) ProtoMessage()    {}
 func (*Components) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{1}
+	return fileDescriptor_beam_runner_api_70c7dbd5f3375954, []int{1}
 }
 func (m *Components) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Components.Unmarshal(m, b)
@@ -952,7 +958,7 @@
 func (m *Pipeline) String() string { return proto.CompactTextString(m) }
 func (*Pipeline) ProtoMessage()    {}
 func (*Pipeline) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{2}
+	return fileDescriptor_beam_runner_api_70c7dbd5f3375954, []int{2}
 }
 func (m *Pipeline) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Pipeline.Unmarshal(m, b)
@@ -1066,7 +1072,7 @@
 func (m *PTransform) String() string { return proto.CompactTextString(m) }
 func (*PTransform) ProtoMessage()    {}
 func (*PTransform) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{3}
+	return fileDescriptor_beam_runner_api_70c7dbd5f3375954, []int{3}
 }
 func (m *PTransform) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_PTransform.Unmarshal(m, b)
@@ -1138,7 +1144,7 @@
 func (m *StandardPTransforms) String() string { return proto.CompactTextString(m) }
 func (*StandardPTransforms) ProtoMessage()    {}
 func (*StandardPTransforms) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{4}
+	return fileDescriptor_beam_runner_api_70c7dbd5f3375954, []int{4}
 }
 func (m *StandardPTransforms) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_StandardPTransforms.Unmarshal(m, b)
@@ -1168,7 +1174,7 @@
 func (m *StandardSideInputTypes) String() string { return proto.CompactTextString(m) }
 func (*StandardSideInputTypes) ProtoMessage()    {}
 func (*StandardSideInputTypes) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{5}
+	return fileDescriptor_beam_runner_api_70c7dbd5f3375954, []int{5}
 }
 func (m *StandardSideInputTypes) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_StandardSideInputTypes.Unmarshal(m, b)
@@ -1218,7 +1224,7 @@
 func (m *PCollection) String() string { return proto.CompactTextString(m) }
 func (*PCollection) ProtoMessage()    {}
 func (*PCollection) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{6}
+	return fileDescriptor_beam_runner_api_70c7dbd5f3375954, []int{6}
 }
 func (m *PCollection) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_PCollection.Unmarshal(m, b)
@@ -1303,7 +1309,7 @@
 func (m *ParDoPayload) String() string { return proto.CompactTextString(m) }
 func (*ParDoPayload) ProtoMessage()    {}
 func (*ParDoPayload) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{7}
+	return fileDescriptor_beam_runner_api_70c7dbd5f3375954, []int{7}
 }
 func (m *ParDoPayload) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_ParDoPayload.Unmarshal(m, b)
@@ -1404,7 +1410,7 @@
 func (m *Parameter) String() string { return proto.CompactTextString(m) }
 func (*Parameter) ProtoMessage()    {}
 func (*Parameter) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{8}
+	return fileDescriptor_beam_runner_api_70c7dbd5f3375954, []int{8}
 }
 func (m *Parameter) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Parameter.Unmarshal(m, b)
@@ -1441,7 +1447,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_d0b62631d7ed3b4f, []int{8, 0}
+	return fileDescriptor_beam_runner_api_70c7dbd5f3375954, []int{8, 0}
 }
 func (m *Parameter_Type) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Parameter_Type.Unmarshal(m, b)
@@ -1478,7 +1484,7 @@
 func (m *StateSpec) String() string { return proto.CompactTextString(m) }
 func (*StateSpec) ProtoMessage()    {}
 func (*StateSpec) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{9}
+	return fileDescriptor_beam_runner_api_70c7dbd5f3375954, []int{9}
 }
 func (m *StateSpec) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_StateSpec.Unmarshal(m, b)
@@ -1708,7 +1714,7 @@
 func (m *ReadModifyWriteStateSpec) String() string { return proto.CompactTextString(m) }
 func (*ReadModifyWriteStateSpec) ProtoMessage()    {}
 func (*ReadModifyWriteStateSpec) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{10}
+	return fileDescriptor_beam_runner_api_70c7dbd5f3375954, []int{10}
 }
 func (m *ReadModifyWriteStateSpec) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_ReadModifyWriteStateSpec.Unmarshal(m, b)
@@ -1746,7 +1752,7 @@
 func (m *BagStateSpec) String() string { return proto.CompactTextString(m) }
 func (*BagStateSpec) ProtoMessage()    {}
 func (*BagStateSpec) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{11}
+	return fileDescriptor_beam_runner_api_70c7dbd5f3375954, []int{11}
 }
 func (m *BagStateSpec) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_BagStateSpec.Unmarshal(m, b)
@@ -1785,7 +1791,7 @@
 func (m *CombiningStateSpec) String() string { return proto.CompactTextString(m) }
 func (*CombiningStateSpec) ProtoMessage()    {}
 func (*CombiningStateSpec) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{12}
+	return fileDescriptor_beam_runner_api_70c7dbd5f3375954, []int{12}
 }
 func (m *CombiningStateSpec) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_CombiningStateSpec.Unmarshal(m, b)
@@ -1831,7 +1837,7 @@
 func (m *MapStateSpec) String() string { return proto.CompactTextString(m) }
 func (*MapStateSpec) ProtoMessage()    {}
 func (*MapStateSpec) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{13}
+	return fileDescriptor_beam_runner_api_70c7dbd5f3375954, []int{13}
 }
 func (m *MapStateSpec) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_MapStateSpec.Unmarshal(m, b)
@@ -1876,7 +1882,7 @@
 func (m *SetStateSpec) String() string { return proto.CompactTextString(m) }
 func (*SetStateSpec) ProtoMessage()    {}
 func (*SetStateSpec) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{14}
+	return fileDescriptor_beam_runner_api_70c7dbd5f3375954, []int{14}
 }
 func (m *SetStateSpec) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_SetStateSpec.Unmarshal(m, b)
@@ -1915,7 +1921,7 @@
 func (m *TimerSpec) String() string { return proto.CompactTextString(m) }
 func (*TimerSpec) ProtoMessage()    {}
 func (*TimerSpec) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{15}
+	return fileDescriptor_beam_runner_api_70c7dbd5f3375954, []int{15}
 }
 func (m *TimerSpec) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_TimerSpec.Unmarshal(m, b)
@@ -1959,7 +1965,7 @@
 func (m *IsBounded) String() string { return proto.CompactTextString(m) }
 func (*IsBounded) ProtoMessage()    {}
 func (*IsBounded) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{16}
+	return fileDescriptor_beam_runner_api_70c7dbd5f3375954, []int{16}
 }
 func (m *IsBounded) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_IsBounded.Unmarshal(m, b)
@@ -1994,7 +2000,7 @@
 func (m *ReadPayload) String() string { return proto.CompactTextString(m) }
 func (*ReadPayload) ProtoMessage()    {}
 func (*ReadPayload) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{17}
+	return fileDescriptor_beam_runner_api_70c7dbd5f3375954, []int{17}
 }
 func (m *ReadPayload) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_ReadPayload.Unmarshal(m, b)
@@ -2041,7 +2047,7 @@
 func (m *WindowIntoPayload) String() string { return proto.CompactTextString(m) }
 func (*WindowIntoPayload) ProtoMessage()    {}
 func (*WindowIntoPayload) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{18}
+	return fileDescriptor_beam_runner_api_70c7dbd5f3375954, []int{18}
 }
 func (m *WindowIntoPayload) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_WindowIntoPayload.Unmarshal(m, b)
@@ -2083,7 +2089,7 @@
 func (m *CombinePayload) String() string { return proto.CompactTextString(m) }
 func (*CombinePayload) ProtoMessage()    {}
 func (*CombinePayload) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{19}
+	return fileDescriptor_beam_runner_api_70c7dbd5f3375954, []int{19}
 }
 func (m *CombinePayload) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_CombinePayload.Unmarshal(m, b)
@@ -2120,18 +2126,22 @@
 // The payload for the test-only primitive TestStream
 type TestStreamPayload struct {
 	// (Required) the coder for elements in the TestStream events
-	CoderId              string                     `protobuf:"bytes,1,opt,name=coder_id,json=coderId,proto3" json:"coder_id,omitempty"`
-	Events               []*TestStreamPayload_Event `protobuf:"bytes,2,rep,name=events,proto3" json:"events,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}                   `json:"-"`
-	XXX_unrecognized     []byte                     `json:"-"`
-	XXX_sizecache        int32                      `json:"-"`
+	CoderId string `protobuf:"bytes,1,opt,name=coder_id,json=coderId,proto3" json:"coder_id,omitempty"`
+	// (Optional) If specified, the TestStream will replay these events.
+	Events []*TestStreamPayload_Event `protobuf:"bytes,2,rep,name=events,proto3" json:"events,omitempty"`
+	// (Optional) If specified, points to a TestStreamService to be
+	// used to retrieve events.
+	Endpoint             *ApiServiceDescriptor `protobuf:"bytes,3,opt,name=endpoint,proto3" json:"endpoint,omitempty"`
+	XXX_NoUnkeyedLiteral struct{}              `json:"-"`
+	XXX_unrecognized     []byte                `json:"-"`
+	XXX_sizecache        int32                 `json:"-"`
 }
 
 func (m *TestStreamPayload) Reset()         { *m = TestStreamPayload{} }
 func (m *TestStreamPayload) String() string { return proto.CompactTextString(m) }
 func (*TestStreamPayload) ProtoMessage()    {}
 func (*TestStreamPayload) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{20}
+	return fileDescriptor_beam_runner_api_70c7dbd5f3375954, []int{20}
 }
 func (m *TestStreamPayload) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_TestStreamPayload.Unmarshal(m, b)
@@ -2165,6 +2175,13 @@
 	return nil
 }
 
+func (m *TestStreamPayload) GetEndpoint() *ApiServiceDescriptor {
+	if m != nil {
+		return m.Endpoint
+	}
+	return nil
+}
+
 type TestStreamPayload_Event struct {
 	// Types that are valid to be assigned to Event:
 	//	*TestStreamPayload_Event_WatermarkEvent
@@ -2180,7 +2197,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_d0b62631d7ed3b4f, []int{20, 0}
+	return fileDescriptor_beam_runner_api_70c7dbd5f3375954, []int{20, 0}
 }
 func (m *TestStreamPayload_Event) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_TestStreamPayload_Event.Unmarshal(m, b)
@@ -2339,8 +2356,14 @@
 	return n
 }
 
+// Advances the watermark to the specified timestamp.
 type TestStreamPayload_Event_AdvanceWatermark struct {
-	NewWatermark         int64    `protobuf:"varint,1,opt,name=new_watermark,json=newWatermark,proto3" json:"new_watermark,omitempty"`
+	// (Required) The watermark to advance to.
+	NewWatermark int64 `protobuf:"varint,1,opt,name=new_watermark,json=newWatermark,proto3" json:"new_watermark,omitempty"`
+	// (Optional) The output watermark tag for a PCollection. If unspecified
+	// or with an empty string, this will default to the Main PCollection
+	// Output
+	Tag                  string   `protobuf:"bytes,2,opt,name=tag,proto3" json:"tag,omitempty"`
 	XXX_NoUnkeyedLiteral struct{} `json:"-"`
 	XXX_unrecognized     []byte   `json:"-"`
 	XXX_sizecache        int32    `json:"-"`
@@ -2352,7 +2375,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_d0b62631d7ed3b4f, []int{20, 0, 0}
+	return fileDescriptor_beam_runner_api_70c7dbd5f3375954, []int{20, 0, 0}
 }
 func (m *TestStreamPayload_Event_AdvanceWatermark) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_TestStreamPayload_Event_AdvanceWatermark.Unmarshal(m, b)
@@ -2379,7 +2402,16 @@
 	return 0
 }
 
+func (m *TestStreamPayload_Event_AdvanceWatermark) GetTag() string {
+	if m != nil {
+		return m.Tag
+	}
+	return ""
+}
+
+// Advances the processing time clock by the specified amount.
 type TestStreamPayload_Event_AdvanceProcessingTime struct {
+	// (Required) The duration to advance by.
 	AdvanceDuration      int64    `protobuf:"varint,1,opt,name=advance_duration,json=advanceDuration,proto3" json:"advance_duration,omitempty"`
 	XXX_NoUnkeyedLiteral struct{} `json:"-"`
 	XXX_unrecognized     []byte   `json:"-"`
@@ -2394,7 +2426,7 @@
 }
 func (*TestStreamPayload_Event_AdvanceProcessingTime) ProtoMessage() {}
 func (*TestStreamPayload_Event_AdvanceProcessingTime) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{20, 0, 1}
+	return fileDescriptor_beam_runner_api_70c7dbd5f3375954, []int{20, 0, 1}
 }
 func (m *TestStreamPayload_Event_AdvanceProcessingTime) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_TestStreamPayload_Event_AdvanceProcessingTime.Unmarshal(m, b)
@@ -2421,18 +2453,24 @@
 	return 0
 }
 
+// Adds elements to the stream to be emitted.
 type TestStreamPayload_Event_AddElements struct {
-	Elements             []*TestStreamPayload_TimestampedElement `protobuf:"bytes,1,rep,name=elements,proto3" json:"elements,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}                                `json:"-"`
-	XXX_unrecognized     []byte                                  `json:"-"`
-	XXX_sizecache        int32                                   `json:"-"`
+	// (Required) The elements to add to the TestStream.
+	Elements []*TestStreamPayload_TimestampedElement `protobuf:"bytes,1,rep,name=elements,proto3" json:"elements,omitempty"`
+	// (Optional) The output PCollection tag to add these elements to. If
+	// unspecified or with an empty string, this will default to the Main
+	// PCollection Output.
+	Tag                  string   `protobuf:"bytes,3,opt,name=tag,proto3" json:"tag,omitempty"`
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
 }
 
 func (m *TestStreamPayload_Event_AddElements) Reset()         { *m = TestStreamPayload_Event_AddElements{} }
 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_d0b62631d7ed3b4f, []int{20, 0, 2}
+	return fileDescriptor_beam_runner_api_70c7dbd5f3375954, []int{20, 0, 2}
 }
 func (m *TestStreamPayload_Event_AddElements) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_TestStreamPayload_Event_AddElements.Unmarshal(m, b)
@@ -2459,8 +2497,19 @@
 	return nil
 }
 
+func (m *TestStreamPayload_Event_AddElements) GetTag() string {
+	if m != nil {
+		return m.Tag
+	}
+	return ""
+}
+
+// A single element inside of the TestStream.
 type TestStreamPayload_TimestampedElement struct {
-	EncodedElement       []byte   `protobuf:"bytes,1,opt,name=encoded_element,json=encodedElement,proto3" json:"encoded_element,omitempty"`
+	// (Required) The element encoded. Currently the TestStream only supports
+	// encoding primitives.
+	EncodedElement []byte `protobuf:"bytes,1,opt,name=encoded_element,json=encodedElement,proto3" json:"encoded_element,omitempty"`
+	// (Required) The event timestamp of this element.
 	Timestamp            int64    `protobuf:"varint,2,opt,name=timestamp,proto3" json:"timestamp,omitempty"`
 	XXX_NoUnkeyedLiteral struct{} `json:"-"`
 	XXX_unrecognized     []byte   `json:"-"`
@@ -2471,7 +2520,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_d0b62631d7ed3b4f, []int{20, 1}
+	return fileDescriptor_beam_runner_api_70c7dbd5f3375954, []int{20, 1}
 }
 func (m *TestStreamPayload_TimestampedElement) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_TestStreamPayload_TimestampedElement.Unmarshal(m, b)
@@ -2505,6 +2554,36 @@
 	return 0
 }
 
+type EventsRequest struct {
+	XXX_NoUnkeyedLiteral struct{} `json:"-"`
+	XXX_unrecognized     []byte   `json:"-"`
+	XXX_sizecache        int32    `json:"-"`
+}
+
+func (m *EventsRequest) Reset()         { *m = EventsRequest{} }
+func (m *EventsRequest) String() string { return proto.CompactTextString(m) }
+func (*EventsRequest) ProtoMessage()    {}
+func (*EventsRequest) Descriptor() ([]byte, []int) {
+	return fileDescriptor_beam_runner_api_70c7dbd5f3375954, []int{21}
+}
+func (m *EventsRequest) XXX_Unmarshal(b []byte) error {
+	return xxx_messageInfo_EventsRequest.Unmarshal(m, b)
+}
+func (m *EventsRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
+	return xxx_messageInfo_EventsRequest.Marshal(b, m, deterministic)
+}
+func (dst *EventsRequest) XXX_Merge(src proto.Message) {
+	xxx_messageInfo_EventsRequest.Merge(dst, src)
+}
+func (m *EventsRequest) XXX_Size() int {
+	return xxx_messageInfo_EventsRequest.Size(m)
+}
+func (m *EventsRequest) XXX_DiscardUnknown() {
+	xxx_messageInfo_EventsRequest.DiscardUnknown(m)
+}
+
+var xxx_messageInfo_EventsRequest proto.InternalMessageInfo
+
 // The payload for the special-but-not-primitive WriteFiles transform.
 type WriteFilesPayload struct {
 	// (Required) The SdkFunctionSpec of the FileBasedSink.
@@ -2523,7 +2602,7 @@
 func (m *WriteFilesPayload) String() string { return proto.CompactTextString(m) }
 func (*WriteFilesPayload) ProtoMessage()    {}
 func (*WriteFilesPayload) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{21}
+	return fileDescriptor_beam_runner_api_70c7dbd5f3375954, []int{22}
 }
 func (m *WriteFilesPayload) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_WriteFilesPayload.Unmarshal(m, b)
@@ -2600,7 +2679,7 @@
 func (m *Coder) String() string { return proto.CompactTextString(m) }
 func (*Coder) ProtoMessage()    {}
 func (*Coder) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{22}
+	return fileDescriptor_beam_runner_api_70c7dbd5f3375954, []int{23}
 }
 func (m *Coder) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Coder.Unmarshal(m, b)
@@ -2644,7 +2723,7 @@
 func (m *StandardCoders) String() string { return proto.CompactTextString(m) }
 func (*StandardCoders) ProtoMessage()    {}
 func (*StandardCoders) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{23}
+	return fileDescriptor_beam_runner_api_70c7dbd5f3375954, []int{24}
 }
 func (m *StandardCoders) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_StandardCoders.Unmarshal(m, b)
@@ -2715,7 +2794,7 @@
 func (m *WindowingStrategy) String() string { return proto.CompactTextString(m) }
 func (*WindowingStrategy) ProtoMessage()    {}
 func (*WindowingStrategy) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{24}
+	return fileDescriptor_beam_runner_api_70c7dbd5f3375954, []int{25}
 }
 func (m *WindowingStrategy) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_WindowingStrategy.Unmarshal(m, b)
@@ -2818,7 +2897,7 @@
 func (m *MergeStatus) String() string { return proto.CompactTextString(m) }
 func (*MergeStatus) ProtoMessage()    {}
 func (*MergeStatus) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{25}
+	return fileDescriptor_beam_runner_api_70c7dbd5f3375954, []int{26}
 }
 func (m *MergeStatus) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_MergeStatus.Unmarshal(m, b)
@@ -2851,7 +2930,7 @@
 func (m *AccumulationMode) String() string { return proto.CompactTextString(m) }
 func (*AccumulationMode) ProtoMessage()    {}
 func (*AccumulationMode) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{26}
+	return fileDescriptor_beam_runner_api_70c7dbd5f3375954, []int{27}
 }
 func (m *AccumulationMode) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_AccumulationMode.Unmarshal(m, b)
@@ -2883,7 +2962,7 @@
 func (m *ClosingBehavior) String() string { return proto.CompactTextString(m) }
 func (*ClosingBehavior) ProtoMessage()    {}
 func (*ClosingBehavior) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{27}
+	return fileDescriptor_beam_runner_api_70c7dbd5f3375954, []int{28}
 }
 func (m *ClosingBehavior) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_ClosingBehavior.Unmarshal(m, b)
@@ -2915,7 +2994,7 @@
 func (m *OnTimeBehavior) String() string { return proto.CompactTextString(m) }
 func (*OnTimeBehavior) ProtoMessage()    {}
 func (*OnTimeBehavior) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{28}
+	return fileDescriptor_beam_runner_api_70c7dbd5f3375954, []int{29}
 }
 func (m *OnTimeBehavior) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_OnTimeBehavior.Unmarshal(m, b)
@@ -2947,7 +3026,7 @@
 func (m *OutputTime) String() string { return proto.CompactTextString(m) }
 func (*OutputTime) ProtoMessage()    {}
 func (*OutputTime) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{29}
+	return fileDescriptor_beam_runner_api_70c7dbd5f3375954, []int{30}
 }
 func (m *OutputTime) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_OutputTime.Unmarshal(m, b)
@@ -2978,7 +3057,7 @@
 func (m *TimeDomain) String() string { return proto.CompactTextString(m) }
 func (*TimeDomain) ProtoMessage()    {}
 func (*TimeDomain) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{30}
+	return fileDescriptor_beam_runner_api_70c7dbd5f3375954, []int{31}
 }
 func (m *TimeDomain) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_TimeDomain.Unmarshal(m, b)
@@ -3028,7 +3107,7 @@
 func (m *Trigger) String() string { return proto.CompactTextString(m) }
 func (*Trigger) ProtoMessage()    {}
 func (*Trigger) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{31}
+	return fileDescriptor_beam_runner_api_70c7dbd5f3375954, []int{32}
 }
 func (m *Trigger) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Trigger.Unmarshal(m, b)
@@ -3469,7 +3548,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_d0b62631d7ed3b4f, []int{31, 0}
+	return fileDescriptor_beam_runner_api_70c7dbd5f3375954, []int{32, 0}
 }
 func (m *Trigger_AfterAll) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Trigger_AfterAll.Unmarshal(m, b)
@@ -3508,7 +3587,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_d0b62631d7ed3b4f, []int{31, 1}
+	return fileDescriptor_beam_runner_api_70c7dbd5f3375954, []int{32, 1}
 }
 func (m *Trigger_AfterAny) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Trigger_AfterAny.Unmarshal(m, b)
@@ -3548,7 +3627,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_d0b62631d7ed3b4f, []int{31, 2}
+	return fileDescriptor_beam_runner_api_70c7dbd5f3375954, []int{32, 2}
 }
 func (m *Trigger_AfterEach) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Trigger_AfterEach.Unmarshal(m, b)
@@ -3594,7 +3673,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_d0b62631d7ed3b4f, []int{31, 3}
+	return fileDescriptor_beam_runner_api_70c7dbd5f3375954, []int{32, 3}
 }
 func (m *Trigger_AfterEndOfWindow) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Trigger_AfterEndOfWindow.Unmarshal(m, b)
@@ -3642,7 +3721,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_d0b62631d7ed3b4f, []int{31, 4}
+	return fileDescriptor_beam_runner_api_70c7dbd5f3375954, []int{32, 4}
 }
 func (m *Trigger_AfterProcessingTime) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Trigger_AfterProcessingTime.Unmarshal(m, b)
@@ -3683,7 +3762,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_d0b62631d7ed3b4f, []int{31, 5}
+	return fileDescriptor_beam_runner_api_70c7dbd5f3375954, []int{32, 5}
 }
 func (m *Trigger_AfterSynchronizedProcessingTime) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Trigger_AfterSynchronizedProcessingTime.Unmarshal(m, b)
@@ -3715,7 +3794,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_d0b62631d7ed3b4f, []int{31, 6}
+	return fileDescriptor_beam_runner_api_70c7dbd5f3375954, []int{32, 6}
 }
 func (m *Trigger_Default) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Trigger_Default.Unmarshal(m, b)
@@ -3747,7 +3826,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_d0b62631d7ed3b4f, []int{31, 7}
+	return fileDescriptor_beam_runner_api_70c7dbd5f3375954, []int{32, 7}
 }
 func (m *Trigger_ElementCount) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Trigger_ElementCount.Unmarshal(m, b)
@@ -3786,7 +3865,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_d0b62631d7ed3b4f, []int{31, 8}
+	return fileDescriptor_beam_runner_api_70c7dbd5f3375954, []int{32, 8}
 }
 func (m *Trigger_Never) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Trigger_Never.Unmarshal(m, b)
@@ -3818,7 +3897,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_d0b62631d7ed3b4f, []int{31, 9}
+	return fileDescriptor_beam_runner_api_70c7dbd5f3375954, []int{32, 9}
 }
 func (m *Trigger_Always) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Trigger_Always.Unmarshal(m, b)
@@ -3854,7 +3933,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_d0b62631d7ed3b4f, []int{31, 10}
+	return fileDescriptor_beam_runner_api_70c7dbd5f3375954, []int{32, 10}
 }
 func (m *Trigger_OrFinally) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Trigger_OrFinally.Unmarshal(m, b)
@@ -3902,7 +3981,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_d0b62631d7ed3b4f, []int{31, 11}
+	return fileDescriptor_beam_runner_api_70c7dbd5f3375954, []int{32, 11}
 }
 func (m *Trigger_Repeat) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Trigger_Repeat.Unmarshal(m, b)
@@ -3947,7 +4026,7 @@
 func (m *TimestampTransform) String() string { return proto.CompactTextString(m) }
 func (*TimestampTransform) ProtoMessage()    {}
 func (*TimestampTransform) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{32}
+	return fileDescriptor_beam_runner_api_70c7dbd5f3375954, []int{33}
 }
 func (m *TimestampTransform) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_TimestampTransform.Unmarshal(m, b)
@@ -4088,7 +4167,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_d0b62631d7ed3b4f, []int{32, 0}
+	return fileDescriptor_beam_runner_api_70c7dbd5f3375954, []int{33, 0}
 }
 func (m *TimestampTransform_Delay) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_TimestampTransform_Delay.Unmarshal(m, b)
@@ -4131,7 +4210,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_d0b62631d7ed3b4f, []int{32, 1}
+	return fileDescriptor_beam_runner_api_70c7dbd5f3375954, []int{33, 1}
 }
 func (m *TimestampTransform_AlignTo) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_TimestampTransform_AlignTo.Unmarshal(m, b)
@@ -4199,7 +4278,7 @@
 func (m *SideInput) String() string { return proto.CompactTextString(m) }
 func (*SideInput) ProtoMessage()    {}
 func (*SideInput) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{33}
+	return fileDescriptor_beam_runner_api_70c7dbd5f3375954, []int{34}
 }
 func (m *SideInput) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_SideInput.Unmarshal(m, b)
@@ -4257,7 +4336,7 @@
 func (m *Environment) String() string { return proto.CompactTextString(m) }
 func (*Environment) ProtoMessage()    {}
 func (*Environment) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{34}
+	return fileDescriptor_beam_runner_api_70c7dbd5f3375954, []int{35}
 }
 func (m *Environment) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_Environment.Unmarshal(m, b)
@@ -4301,7 +4380,7 @@
 func (m *StandardEnvironments) String() string { return proto.CompactTextString(m) }
 func (*StandardEnvironments) ProtoMessage()    {}
 func (*StandardEnvironments) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{35}
+	return fileDescriptor_beam_runner_api_70c7dbd5f3375954, []int{36}
 }
 func (m *StandardEnvironments) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_StandardEnvironments.Unmarshal(m, b)
@@ -4333,7 +4412,7 @@
 func (m *DockerPayload) String() string { return proto.CompactTextString(m) }
 func (*DockerPayload) ProtoMessage()    {}
 func (*DockerPayload) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{36}
+	return fileDescriptor_beam_runner_api_70c7dbd5f3375954, []int{37}
 }
 func (m *DockerPayload) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_DockerPayload.Unmarshal(m, b)
@@ -4374,7 +4453,7 @@
 func (m *ProcessPayload) String() string { return proto.CompactTextString(m) }
 func (*ProcessPayload) ProtoMessage()    {}
 func (*ProcessPayload) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{37}
+	return fileDescriptor_beam_runner_api_70c7dbd5f3375954, []int{38}
 }
 func (m *ProcessPayload) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_ProcessPayload.Unmarshal(m, b)
@@ -4434,7 +4513,7 @@
 func (m *ExternalPayload) String() string { return proto.CompactTextString(m) }
 func (*ExternalPayload) ProtoMessage()    {}
 func (*ExternalPayload) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{38}
+	return fileDescriptor_beam_runner_api_70c7dbd5f3375954, []int{39}
 }
 func (m *ExternalPayload) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_ExternalPayload.Unmarshal(m, b)
@@ -4485,7 +4564,7 @@
 func (m *SdkFunctionSpec) String() string { return proto.CompactTextString(m) }
 func (*SdkFunctionSpec) ProtoMessage()    {}
 func (*SdkFunctionSpec) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{39}
+	return fileDescriptor_beam_runner_api_70c7dbd5f3375954, []int{40}
 }
 func (m *SdkFunctionSpec) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_SdkFunctionSpec.Unmarshal(m, b)
@@ -4564,7 +4643,7 @@
 func (m *FunctionSpec) String() string { return proto.CompactTextString(m) }
 func (*FunctionSpec) ProtoMessage()    {}
 func (*FunctionSpec) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{40}
+	return fileDescriptor_beam_runner_api_70c7dbd5f3375954, []int{41}
 }
 func (m *FunctionSpec) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_FunctionSpec.Unmarshal(m, b)
@@ -4611,7 +4690,7 @@
 func (m *DisplayData) String() string { return proto.CompactTextString(m) }
 func (*DisplayData) ProtoMessage()    {}
 func (*DisplayData) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{41}
+	return fileDescriptor_beam_runner_api_70c7dbd5f3375954, []int{42}
 }
 func (m *DisplayData) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_DisplayData.Unmarshal(m, b)
@@ -4655,7 +4734,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_d0b62631d7ed3b4f, []int{41, 0}
+	return fileDescriptor_beam_runner_api_70c7dbd5f3375954, []int{42, 0}
 }
 func (m *DisplayData_Identifier) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_DisplayData_Identifier.Unmarshal(m, b)
@@ -4719,7 +4798,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_d0b62631d7ed3b4f, []int{41, 1}
+	return fileDescriptor_beam_runner_api_70c7dbd5f3375954, []int{42, 1}
 }
 func (m *DisplayData_Item) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_DisplayData_Item.Unmarshal(m, b)
@@ -4791,7 +4870,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_d0b62631d7ed3b4f, []int{41, 2}
+	return fileDescriptor_beam_runner_api_70c7dbd5f3375954, []int{42, 2}
 }
 func (m *DisplayData_Type) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_DisplayData_Type.Unmarshal(m, b)
@@ -4845,7 +4924,7 @@
 func (m *MessageWithComponents) String() string { return proto.CompactTextString(m) }
 func (*MessageWithComponents) ProtoMessage()    {}
 func (*MessageWithComponents) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{42}
+	return fileDescriptor_beam_runner_api_70c7dbd5f3375954, []int{43}
 }
 func (m *MessageWithComponents) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_MessageWithComponents.Unmarshal(m, b)
@@ -5289,7 +5368,7 @@
 func (m *ExecutableStagePayload) String() string { return proto.CompactTextString(m) }
 func (*ExecutableStagePayload) ProtoMessage()    {}
 func (*ExecutableStagePayload) Descriptor() ([]byte, []int) {
-	return fileDescriptor_beam_runner_api_d0b62631d7ed3b4f, []int{43}
+	return fileDescriptor_beam_runner_api_70c7dbd5f3375954, []int{44}
 }
 func (m *ExecutableStagePayload) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_ExecutableStagePayload.Unmarshal(m, b)
@@ -5381,7 +5460,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_d0b62631d7ed3b4f, []int{43, 0}
+	return fileDescriptor_beam_runner_api_70c7dbd5f3375954, []int{44, 0}
 }
 func (m *ExecutableStagePayload_SideInputId) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_ExecutableStagePayload_SideInputId.Unmarshal(m, b)
@@ -5431,7 +5510,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_d0b62631d7ed3b4f, []int{43, 1}
+	return fileDescriptor_beam_runner_api_70c7dbd5f3375954, []int{44, 1}
 }
 func (m *ExecutableStagePayload_UserStateId) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_ExecutableStagePayload_UserStateId.Unmarshal(m, b)
@@ -5481,7 +5560,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_d0b62631d7ed3b4f, []int{43, 2}
+	return fileDescriptor_beam_runner_api_70c7dbd5f3375954, []int{44, 2}
 }
 func (m *ExecutableStagePayload_TimerId) XXX_Unmarshal(b []byte) error {
 	return xxx_messageInfo_ExecutableStagePayload_TimerId.Unmarshal(m, b)
@@ -5571,6 +5650,7 @@
 	proto.RegisterType((*TestStreamPayload_Event_AdvanceProcessingTime)(nil), "org.apache.beam.model.pipeline.v1.TestStreamPayload.Event.AdvanceProcessingTime")
 	proto.RegisterType((*TestStreamPayload_Event_AddElements)(nil), "org.apache.beam.model.pipeline.v1.TestStreamPayload.Event.AddElements")
 	proto.RegisterType((*TestStreamPayload_TimestampedElement)(nil), "org.apache.beam.model.pipeline.v1.TestStreamPayload.TimestampedElement")
+	proto.RegisterType((*EventsRequest)(nil), "org.apache.beam.model.pipeline.v1.EventsRequest")
 	proto.RegisterType((*WriteFilesPayload)(nil), "org.apache.beam.model.pipeline.v1.WriteFilesPayload")
 	proto.RegisterMapType((map[string]*SideInput)(nil), "org.apache.beam.model.pipeline.v1.WriteFilesPayload.SideInputsEntry")
 	proto.RegisterType((*Coder)(nil), "org.apache.beam.model.pipeline.v1.Coder")
@@ -5639,331 +5719,437 @@
 	proto.RegisterExtension(E_BeamConstant)
 }
 
-func init() {
-	proto.RegisterFile("beam_runner_api.proto", fileDescriptor_beam_runner_api_d0b62631d7ed3b4f)
+// Reference imports to suppress errors if they are not otherwise used.
+var _ context.Context
+var _ grpc.ClientConn
+
+// This is a compile-time assertion to ensure that this generated file
+// is compatible with the grpc package it is being compiled against.
+const _ = grpc.SupportPackageIsVersion4
+
+// TestStreamServiceClient is the client API for TestStreamService service.
+//
+// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://godoc.org/google.golang.org/grpc#ClientConn.NewStream.
+type TestStreamServiceClient interface {
+	// A TestStream will request for events using this RPC.
+	Events(ctx context.Context, in *EventsRequest, opts ...grpc.CallOption) (TestStreamService_EventsClient, error)
 }
 
-var fileDescriptor_beam_runner_api_d0b62631d7ed3b4f = []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,
+type testStreamServiceClient struct {
+	cc *grpc.ClientConn
+}
+
+func NewTestStreamServiceClient(cc *grpc.ClientConn) TestStreamServiceClient {
+	return &testStreamServiceClient{cc}
+}
+
+func (c *testStreamServiceClient) Events(ctx context.Context, in *EventsRequest, opts ...grpc.CallOption) (TestStreamService_EventsClient, error) {
+	stream, err := c.cc.NewStream(ctx, &_TestStreamService_serviceDesc.Streams[0], "/org.apache.beam.model.pipeline.v1.TestStreamService/Events", opts...)
+	if err != nil {
+		return nil, err
+	}
+	x := &testStreamServiceEventsClient{stream}
+	if err := x.ClientStream.SendMsg(in); err != nil {
+		return nil, err
+	}
+	if err := x.ClientStream.CloseSend(); err != nil {
+		return nil, err
+	}
+	return x, nil
+}
+
+type TestStreamService_EventsClient interface {
+	Recv() (*TestStreamPayload_Event, error)
+	grpc.ClientStream
+}
+
+type testStreamServiceEventsClient struct {
+	grpc.ClientStream
+}
+
+func (x *testStreamServiceEventsClient) Recv() (*TestStreamPayload_Event, error) {
+	m := new(TestStreamPayload_Event)
+	if err := x.ClientStream.RecvMsg(m); err != nil {
+		return nil, err
+	}
+	return m, nil
+}
+
+// TestStreamServiceServer is the server API for TestStreamService service.
+type TestStreamServiceServer interface {
+	// A TestStream will request for events using this RPC.
+	Events(*EventsRequest, TestStreamService_EventsServer) error
+}
+
+func RegisterTestStreamServiceServer(s *grpc.Server, srv TestStreamServiceServer) {
+	s.RegisterService(&_TestStreamService_serviceDesc, srv)
+}
+
+func _TestStreamService_Events_Handler(srv interface{}, stream grpc.ServerStream) error {
+	m := new(EventsRequest)
+	if err := stream.RecvMsg(m); err != nil {
+		return err
+	}
+	return srv.(TestStreamServiceServer).Events(m, &testStreamServiceEventsServer{stream})
+}
+
+type TestStreamService_EventsServer interface {
+	Send(*TestStreamPayload_Event) error
+	grpc.ServerStream
+}
+
+type testStreamServiceEventsServer struct {
+	grpc.ServerStream
+}
+
+func (x *testStreamServiceEventsServer) Send(m *TestStreamPayload_Event) error {
+	return x.ServerStream.SendMsg(m)
+}
+
+var _TestStreamService_serviceDesc = grpc.ServiceDesc{
+	ServiceName: "org.apache.beam.model.pipeline.v1.TestStreamService",
+	HandlerType: (*TestStreamServiceServer)(nil),
+	Methods:     []grpc.MethodDesc{},
+	Streams: []grpc.StreamDesc{
+		{
+			StreamName:    "Events",
+			Handler:       _TestStreamService_Events_Handler,
+			ServerStreams: true,
+		},
+	},
+	Metadata: "beam_runner_api.proto",
+}
+
+func init() {
+	proto.RegisterFile("beam_runner_api.proto", fileDescriptor_beam_runner_api_70c7dbd5f3375954)
+}
+
+var fileDescriptor_beam_runner_api_70c7dbd5f3375954 = []byte{
+	// 5205 bytes of a gzipped FileDescriptorProto
+	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xbc, 0x7c, 0xdb, 0x6f, 0x23, 0xc9,
+	0x75, 0x37, 0x2f, 0xe2, 0xed, 0x90, 0xa2, 0x5a, 0x25, 0xcd, 0xac, 0xb6, 0xbd, 0xde, 0x99, 0xed,
+	0x5d, 0xef, 0x8e, 0xf7, 0x5b, 0x73, 0x47, 0x9a, 0x99, 0x9d, 0x19, 0xd9, 0x9e, 0x35, 0x29, 0x36,
+	0x47, 0x3d, 0xc3, 0x9b, 0x9b, 0x94, 0x34, 0xb3, 0xb6, 0xb7, 0xdd, 0x62, 0x17, 0xa5, 0x86, 0x9a,
+	0xdd, 0x74, 0x77, 0x53, 0xb3, 0x34, 0x6c, 0x7c, 0x40, 0x10, 0x18, 0x41, 0x02, 0xe4, 0xf2, 0x90,
+	0x00, 0x7e, 0x08, 0x02, 0xd8, 0x40, 0x80, 0x24, 0x0f, 0x09, 0xe2, 0x24, 0x40, 0x5e, 0x9d, 0xe4,
+	0x31, 0x0f, 0x01, 0xf2, 0x94, 0x3f, 0x23, 0x81, 0x1f, 0x92, 0xa7, 0xa0, 0x2e, 0xdd, 0x6c, 0x52,
+	0xd2, 0x2c, 0x29, 0x09, 0x79, 0x63, 0x9f, 0xae, 0xf3, 0x3b, 0x55, 0xa7, 0xaa, 0x4e, 0x9d, 0x73,
+	0xea, 0x34, 0xe1, 0xc6, 0x21, 0xd6, 0x07, 0x9a, 0x3b, 0xb2, 0x6d, 0xec, 0x6a, 0xfa, 0xd0, 0x2c,
+	0x0d, 0x5d, 0xc7, 0x77, 0xd0, 0x3b, 0x8e, 0x7b, 0x54, 0xd2, 0x87, 0x7a, 0xef, 0x18, 0x97, 0x48,
+	0x8b, 0xd2, 0xc0, 0x31, 0xb0, 0x55, 0x1a, 0x9a, 0x43, 0x6c, 0x99, 0x36, 0x2e, 0x9d, 0x6e, 0x8a,
+	0x2b, 0xd8, 0x36, 0x86, 0x8e, 0x69, 0xfb, 0x1e, 0xe3, 0x11, 0xdf, 0x3c, 0x72, 0x9c, 0x23, 0x0b,
+	0x7f, 0x4c, 0x9f, 0x0e, 0x47, 0xfd, 0x8f, 0x75, 0x7b, 0xcc, 0x5f, 0xdd, 0x9e, 0x7d, 0x65, 0x60,
+	0xaf, 0xe7, 0x9a, 0x43, 0xdf, 0x71, 0x79, 0x8b, 0x5b, 0xb3, 0x2d, 0x7c, 0x73, 0x80, 0x3d, 0x5f,
+	0x1f, 0x0c, 0x59, 0x03, 0xe9, 0xd7, 0x71, 0x58, 0xae, 0x60, 0x7d, 0xb0, 0xe3, 0xd8, 0x9e, 0xaf,
+	0xdb, 0xbe, 0x27, 0xfd, 0x4d, 0x1c, 0x72, 0xe1, 0x13, 0xda, 0x84, 0xf5, 0x86, 0xd2, 0xd4, 0xba,
+	0x4a, 0x43, 0xee, 0x74, 0xcb, 0x8d, 0xb6, 0xd6, 0x50, 0xea, 0x75, 0xa5, 0x23, 0xc4, 0xc4, 0x37,
+	0xfe, 0xf2, 0xef, 0xfe, 0xe7, 0xd7, 0xa9, 0xd5, 0x6f, 0x3c, 0xde, 0xda, 0xba, 0x77, 0xef, 0xe1,
+	0xd6, 0xdd, 0x7b, 0x9f, 0x3c, 0x7a, 0x70, 0xff, 0xe1, 0xc3, 0x07, 0xe8, 0x2e, 0xac, 0x37, 0xca,
+	0x2f, 0xce, 0xb2, 0xc4, 0xc5, 0x9b, 0x94, 0x45, 0x38, 0xc3, 0xf1, 0x04, 0xa4, 0xa7, 0xf5, 0x56,
+	0xa5, 0x5c, 0xd7, 0x0e, 0x94, 0x66, 0xb5, 0x75, 0xa0, 0x9d, 0xcb, 0x9f, 0x98, 0xe6, 0xdf, 0x7c,
+	0xfc, 0xe0, 0xee, 0x7d, 0xca, 0x2f, 0xfd, 0x43, 0x16, 0x60, 0xc7, 0x19, 0x0c, 0x1d, 0x1b, 0x93,
+	0x3e, 0xff, 0x00, 0xc0, 0x77, 0x75, 0xdb, 0xeb, 0x3b, 0xee, 0xc0, 0xdb, 0x88, 0xdf, 0x4e, 0xde,
+	0xc9, 0x6f, 0x7d, 0xbb, 0xf4, 0xa5, 0xaa, 0x2f, 0x4d, 0x20, 0x4a, 0xdd, 0x90, 0x5f, 0xb6, 0x7d,
+	0x77, 0xac, 0x46, 0x00, 0x51, 0x0f, 0x0a, 0xc3, 0x9e, 0x63, 0x59, 0xb8, 0xe7, 0x9b, 0x8e, 0xed,
+	0x6d, 0x24, 0xa8, 0x80, 0x4f, 0x17, 0x13, 0xd0, 0x8e, 0x20, 0x30, 0x11, 0x53, 0xa0, 0x68, 0x0c,
+	0xeb, 0xaf, 0x4c, 0xdb, 0x70, 0x5e, 0x99, 0xf6, 0x91, 0xe6, 0xf9, 0xae, 0xee, 0xe3, 0x23, 0x13,
+	0x7b, 0x1b, 0x49, 0x2a, 0xac, 0xb6, 0x98, 0xb0, 0x83, 0x00, 0xa9, 0x13, 0x02, 0x31, 0x99, 0x6b,
+	0xaf, 0xce, 0xbe, 0x41, 0xdf, 0x85, 0x74, 0xcf, 0x31, 0xb0, 0xeb, 0x6d, 0x2c, 0x51, 0x61, 0x8f,
+	0x17, 0x13, 0xb6, 0x43, 0x79, 0x19, 0x3e, 0x07, 0x22, 0x2a, 0xc3, 0xf6, 0xa9, 0xe9, 0x3a, 0xf6,
+	0x80, 0xb4, 0xd9, 0x48, 0x5d, 0x46, 0x65, 0x72, 0x04, 0x81, 0xab, 0x2c, 0x0a, 0x2a, 0x5a, 0xb0,
+	0x32, 0x33, 0x6d, 0x48, 0x80, 0xe4, 0x09, 0x1e, 0x6f, 0xc4, 0x6f, 0xc7, 0xef, 0xe4, 0x54, 0xf2,
+	0x13, 0xed, 0x40, 0xea, 0x54, 0xb7, 0x46, 0x78, 0x23, 0x71, 0x3b, 0x7e, 0x27, 0xbf, 0xf5, 0x8d,
+	0x39, 0xba, 0xd0, 0x0e, 0x51, 0x55, 0xc6, 0xbb, 0x9d, 0x78, 0x14, 0x17, 0x1d, 0x58, 0x3d, 0x33,
+	0x87, 0xe7, 0xc8, 0xab, 0x4e, 0xcb, 0x2b, 0xcd, 0x23, 0x6f, 0x27, 0x84, 0x8d, 0x0a, 0xfc, 0x09,
+	0x6c, 0x5c, 0x34, 0x8f, 0xe7, 0xc8, 0x7d, 0x36, 0x2d, 0xf7, 0xfe, 0x1c, 0x72, 0x67, 0xd1, 0xc7,
+	0x51, 0xe9, 0x3d, 0xc8, 0x47, 0x26, 0xf6, 0x1c, 0x81, 0x4f, 0xa6, 0x05, 0xde, 0x99, 0x6b, 0x6e,
+	0x0d, 0xec, 0xce, 0xe8, 0xf4, 0xcc, 0x24, 0x5f, 0x8f, 0x4e, 0x23, 0xb0, 0x11, 0x81, 0xd2, 0x7f,
+	0xc4, 0x21, 0xdb, 0xe6, 0xcd, 0x50, 0x03, 0xa0, 0x17, 0xae, 0x36, 0x2a, 0x6f, 0xbe, 0xf5, 0x31,
+	0x59, 0xa2, 0x6a, 0x04, 0x00, 0x7d, 0x04, 0xc8, 0x75, 0x1c, 0x5f, 0x0b, 0x2d, 0x87, 0x66, 0x1a,
+	0xcc, 0x58, 0xe4, 0x54, 0x81, 0xbc, 0x09, 0x97, 0x95, 0x62, 0x90, 0x4d, 0x57, 0x30, 0x4c, 0x6f,
+	0x68, 0xe9, 0x63, 0xcd, 0xd0, 0x7d, 0x7d, 0x23, 0x39, 0xf7, 0xd0, 0xaa, 0x8c, 0xad, 0xaa, 0xfb,
+	0xba, 0x9a, 0x37, 0x26, 0x0f, 0xd2, 0xef, 0x2d, 0x01, 0x4c, 0xd6, 0x2e, 0xba, 0x05, 0xf9, 0x91,
+	0x6d, 0xfe, 0x68, 0x84, 0x35, 0x5b, 0x1f, 0xe0, 0x8d, 0x14, 0xd5, 0x27, 0x30, 0x52, 0x53, 0x1f,
+	0x60, 0xb4, 0x03, 0x4b, 0xde, 0x10, 0xf7, 0xf8, 0xc8, 0x3f, 0x9e, 0x43, 0x74, 0x6d, 0x64, 0xd3,
+	0x65, 0xda, 0x19, 0xe2, 0x9e, 0x4a, 0x99, 0xd1, 0x7b, 0xb0, 0xec, 0x8d, 0x0e, 0x23, 0xe6, 0x97,
+	0x0d, 0x78, 0x9a, 0x48, 0x4c, 0x8c, 0x69, 0x0f, 0x47, 0x7e, 0x60, 0xcf, 0x1e, 0x2f, 0xb4, 0x0d,
+	0x4b, 0x0a, 0xe5, 0xe5, 0x26, 0x86, 0x01, 0xa1, 0x2e, 0x64, 0x9c, 0x91, 0x4f, 0x31, 0x99, 0xd9,
+	0xda, 0x5e, 0x0c, 0xb3, 0xc5, 0x98, 0x19, 0x68, 0x00, 0x75, 0x66, 0x5a, 0xd2, 0x57, 0x9e, 0x16,
+	0xf1, 0x31, 0xe4, 0x23, 0xfd, 0x3f, 0x67, 0x79, 0xaf, 0x47, 0x97, 0x77, 0x2e, 0xba, 0x3f, 0xb6,
+	0xa1, 0x10, 0xed, 0xe6, 0x22, 0xbc, 0xd2, 0xdf, 0x2f, 0xc3, 0x5a, 0xc7, 0xd7, 0x6d, 0x43, 0x77,
+	0x8d, 0xc9, 0xb0, 0x3d, 0xe9, 0x2f, 0x92, 0x00, 0x6d, 0xd7, 0x1c, 0x98, 0xbe, 0x79, 0x8a, 0x3d,
+	0xf4, 0x75, 0x48, 0xb7, 0xcb, 0xaa, 0x56, 0x6d, 0x09, 0x31, 0xf1, 0xab, 0xbf, 0x20, 0xc7, 0xed,
+	0x1b, 0x64, 0x80, 0xdb, 0xe1, 0xe4, 0x6d, 0x0f, 0x75, 0xd7, 0x70, 0xb6, 0x4f, 0x37, 0xd1, 0x47,
+	0x90, 0xa9, 0xd5, 0xcb, 0xdd, 0xae, 0xdc, 0x14, 0xe2, 0xe2, 0x2d, 0xda, 0xf6, 0xcd, 0x99, 0xb6,
+	0x7d, 0x4b, 0xf7, 0x7d, 0x6c, 0x93, 0xd6, 0x9f, 0x40, 0xe1, 0xa9, 0xda, 0xda, 0x6b, 0x6b, 0x95,
+	0x97, 0xda, 0x73, 0xf9, 0xa5, 0x90, 0x10, 0xdf, 0xa3, 0x2c, 0x6f, 0xcf, 0xb0, 0x1c, 0xb9, 0xce,
+	0x68, 0xa8, 0x1d, 0x8e, 0xb5, 0x13, 0x3c, 0xe6, 0x52, 0x94, 0x46, 0x7b, 0xaf, 0xde, 0x91, 0x85,
+	0xe4, 0x05, 0x52, 0xcc, 0xc1, 0x70, 0x64, 0x79, 0x98, 0xb4, 0x7e, 0x08, 0xc5, 0x72, 0xa7, 0xa3,
+	0x3c, 0x6d, 0x72, 0x4f, 0xa2, 0x23, 0x2c, 0x89, 0xef, 0x52, 0xa6, 0xaf, 0xce, 0x30, 0xb1, 0x93,
+	0x4f, 0x33, 0x6d, 0x9f, 0x0e, 0xe6, 0x1e, 0xe4, 0xbb, 0x72, 0xa7, 0xab, 0x75, 0xba, 0xaa, 0x5c,
+	0x6e, 0x08, 0x29, 0x51, 0xa2, 0x5c, 0x6f, 0xcd, 0x70, 0xf9, 0xd8, 0xf3, 0x3d, 0xdf, 0x25, 0xc4,
+	0xd3, 0x4d, 0x74, 0x1f, 0xf2, 0x8d, 0x72, 0x3b, 0x14, 0x95, 0xbe, 0x40, 0xd4, 0x40, 0x1f, 0x6a,
+	0x4c, 0x9c, 0x47, 0xb8, 0x1e, 0xc1, 0x72, 0x43, 0x56, 0x9f, 0xca, 0x21, 0x5f, 0x46, 0xfc, 0x1a,
+	0xe5, 0xbb, 0x35, 0xcb, 0x87, 0xdd, 0x23, 0x1c, 0xe1, 0x94, 0x7c, 0x58, 0xaf, 0xe2, 0xa1, 0x8b,
+	0x7b, 0xba, 0x8f, 0x8d, 0xc8, 0xa4, 0xbd, 0x0f, 0x4b, 0xaa, 0x5c, 0xae, 0x0a, 0x31, 0xf1, 0x2d,
+	0x0a, 0x74, 0x73, 0x06, 0xc8, 0xc5, 0xba, 0xc1, 0xfb, 0xbb, 0xa3, 0xca, 0xe5, 0xae, 0xac, 0xed,
+	0x2b, 0xf2, 0x81, 0x10, 0xbf, 0xa0, 0xbf, 0x3d, 0x17, 0xeb, 0x3e, 0xd6, 0x4e, 0x4d, 0xfc, 0x8a,
+	0x48, 0xfd, 0xcf, 0x38, 0xf7, 0xae, 0x3c, 0xd3, 0xc7, 0x1e, 0xfa, 0x16, 0xac, 0xec, 0xb4, 0x1a,
+	0x15, 0xa5, 0x29, 0x6b, 0x6d, 0x59, 0xa5, 0x73, 0x19, 0x13, 0x3f, 0xa0, 0x40, 0xef, 0xcc, 0x02,
+	0x39, 0x83, 0x43, 0xd3, 0xc6, 0xda, 0x10, 0xbb, 0xc1, 0x74, 0x3e, 0x01, 0x21, 0xe0, 0x66, 0x2e,
+	0x5f, 0xfd, 0xa5, 0x10, 0x17, 0xef, 0x50, 0x76, 0xe9, 0x02, 0xf6, 0x23, 0xcb, 0x39, 0xd4, 0x2d,
+	0x8b, 0xf2, 0xdf, 0x85, 0x9c, 0x2a, 0x77, 0x76, 0xf7, 0x6a, 0xb5, 0xba, 0x2c, 0x24, 0xc4, 0x77,
+	0x28, 0xe3, 0x57, 0xce, 0x8c, 0xd7, 0x3b, 0x1e, 0xf5, 0xfb, 0x16, 0xe6, 0x83, 0x3e, 0x50, 0x95,
+	0xae, 0xac, 0xd5, 0x94, 0xba, 0xdc, 0x11, 0x92, 0x17, 0xad, 0x07, 0xd7, 0xf4, 0xb1, 0xd6, 0x37,
+	0x2d, 0x4c, 0x55, 0xfd, 0x9b, 0x04, 0xac, 0xee, 0x30, 0xf9, 0x11, 0xcf, 0x52, 0x05, 0x71, 0x66,
+	0xec, 0x5a, 0x5b, 0x95, 0x39, 0x49, 0x88, 0x89, 0x5b, 0x14, 0xfa, 0xa3, 0xd7, 0xab, 0x41, 0x23,
+	0x33, 0xc8, 0x48, 0xa4, 0x7f, 0x87, 0x20, 0xcd, 0x62, 0xb2, 0xe5, 0x51, 0xde, 0xd9, 0xd9, 0x6b,
+	0xec, 0xd5, 0xcb, 0xdd, 0x96, 0x4a, 0x9c, 0xe7, 0x6d, 0x8a, 0x7d, 0xff, 0x4b, 0xb0, 0xd9, 0x9a,
+	0xd1, 0x7b, 0xbd, 0xd1, 0x60, 0x64, 0xe9, 0xbe, 0xe3, 0xd2, 0x25, 0xf7, 0x7d, 0xb8, 0x35, 0x2b,
+	0x43, 0x7e, 0xd1, 0x55, 0xcb, 0x3b, 0x5d, 0xad, 0xb5, 0xd7, 0x6d, 0xef, 0x75, 0x89, 0x77, 0xfd,
+	0x90, 0x0a, 0xd8, 0xfc, 0x12, 0x01, 0xf8, 0x0b, 0xdf, 0xd5, 0x7b, 0xbe, 0xc6, 0x2d, 0x24, 0x41,
+	0x7f, 0x06, 0x37, 0xc3, 0x39, 0x25, 0x5b, 0x5c, 0xae, 0x6a, 0xfb, 0xe5, 0xfa, 0x1e, 0x55, 0x76,
+	0x89, 0x82, 0xde, 0xb9, 0x68, 0x66, 0xc9, 0x66, 0xc7, 0x86, 0x46, 0xcd, 0x14, 0xd5, 0xfb, 0xef,
+	0x2f, 0xc1, 0x9b, 0x9d, 0xa1, 0x65, 0xfa, 0xbe, 0x7e, 0x68, 0xe1, 0xb6, 0xee, 0x56, 0x9d, 0x88,
+	0xfe, 0xeb, 0x70, 0xa3, 0x5d, 0x56, 0x54, 0xed, 0x40, 0xe9, 0xee, 0x6a, 0xaa, 0xdc, 0xe9, 0xaa,
+	0xca, 0x4e, 0x57, 0x69, 0x35, 0x85, 0x98, 0xb8, 0x49, 0x05, 0xfd, 0xbf, 0x19, 0x41, 0x9e, 0xd1,
+	0xd7, 0x86, 0xba, 0xe9, 0x6a, 0xaf, 0x4c, 0xff, 0x58, 0x73, 0xb1, 0xe7, 0xbb, 0x26, 0x3d, 0xb2,
+	0x48, 0xbf, 0xab, 0xb0, 0xda, 0x69, 0xd7, 0x95, 0xee, 0x14, 0x52, 0x5c, 0xfc, 0x06, 0x45, 0xfa,
+	0xe0, 0x1c, 0x24, 0x8f, 0x74, 0x6c, 0x16, 0xa5, 0x09, 0x37, 0xdb, 0x6a, 0x6b, 0x47, 0xee, 0x74,
+	0x88, 0x5e, 0xe5, 0xaa, 0x26, 0xd7, 0xe5, 0x86, 0xdc, 0xa4, 0x2a, 0x3d, 0x7f, 0x3d, 0xd0, 0x4e,
+	0xb9, 0x4e, 0x0f, 0x7b, 0x1e, 0x51, 0x29, 0x36, 0x34, 0x6c, 0x61, 0xea, 0xf1, 0x10, 0xbc, 0x0a,
+	0x08, 0x01, 0x5e, 0x88, 0x94, 0x14, 0x3f, 0xa2, 0x48, 0xef, 0xbf, 0x06, 0x29, 0x8a, 0xf1, 0x02,
+	0xbe, 0xc2, 0x46, 0x56, 0x6e, 0x56, 0xb5, 0x8e, 0xf2, 0x99, 0x1c, 0x1d, 0x22, 0xb1, 0x89, 0xe7,
+	0xcf, 0xf5, 0x64, 0x8c, 0xba, 0x6d, 0x68, 0x9e, 0xf9, 0x63, 0x1c, 0x1d, 0x2c, 0x45, 0x76, 0xe0,
+	0x83, 0xa0, 0x77, 0x04, 0x77, 0x32, 0x5a, 0x2a, 0x6a, 0x4a, 0x4a, 0x4a, 0xac, 0x50, 0x29, 0xdf,
+	0x7a, 0x4d, 0xa7, 0x89, 0x8c, 0x70, 0xf8, 0x54, 0xea, 0x8c, 0x40, 0xe9, 0xb7, 0xe2, 0x70, 0x33,
+	0x38, 0xb7, 0x3a, 0xa6, 0x81, 0xe9, 0xd9, 0xd9, 0x1d, 0x0f, 0xb1, 0x27, 0x1d, 0xc3, 0x92, 0x6c,
+	0x8f, 0x06, 0xe8, 0x63, 0xc8, 0x2a, 0x5d, 0x59, 0x2d, 0x57, 0xea, 0x64, 0x0f, 0x46, 0x4d, 0x82,
+	0x67, 0x1a, 0x58, 0xa3, 0x0e, 0xc2, 0xb6, 0xe9, 0x63, 0x97, 0x2c, 0x29, 0x32, 0x88, 0x8f, 0x21,
+	0xdb, 0xd8, 0xab, 0x77, 0x95, 0x46, 0xb9, 0x2d, 0xc4, 0x2f, 0x62, 0x18, 0x8c, 0x2c, 0xdf, 0x1c,
+	0xe8, 0x43, 0xd2, 0x89, 0x5f, 0x24, 0x20, 0x1f, 0x71, 0xcb, 0x67, 0x7d, 0xa9, 0xf8, 0x19, 0x5f,
+	0xea, 0x4d, 0xc8, 0xd2, 0xd0, 0x47, 0x33, 0x0d, 0x7e, 0x14, 0x67, 0xe8, 0xb3, 0x62, 0xa0, 0x36,
+	0x80, 0xe9, 0x69, 0x87, 0xce, 0xc8, 0x36, 0xb0, 0x41, 0xfd, 0xbc, 0xe2, 0xd6, 0xe6, 0x1c, 0x0e,
+	0x85, 0xe2, 0x55, 0x18, 0x4f, 0x89, 0x0c, 0x5a, 0xcd, 0x99, 0xc1, 0x33, 0xda, 0x82, 0x1b, 0x67,
+	0x62, 0xc5, 0x31, 0x91, 0xbc, 0x44, 0x25, 0x9f, 0x09, 0xf2, 0xc6, 0x8a, 0x71, 0xc6, 0xb1, 0x49,
+	0x5d, 0xdd, 0xdf, 0xfc, 0x79, 0x06, 0x0a, 0x74, 0xc3, 0xb6, 0xf5, 0xb1, 0xe5, 0xe8, 0x06, 0x7a,
+	0x0a, 0x29, 0xc3, 0xd1, 0xfa, 0x36, 0xf7, 0x28, 0xb7, 0xe6, 0x00, 0xef, 0x18, 0x27, 0xd3, 0x4e,
+	0xa5, 0xe1, 0xd4, 0x6c, 0x54, 0x07, 0x18, 0xea, 0xae, 0x3e, 0xc0, 0x3e, 0x89, 0x4a, 0x59, 0xbc,
+	0xfd, 0xd1, 0x3c, 0xee, 0x5d, 0xc0, 0xa4, 0x46, 0xf8, 0xd1, 0x0f, 0x21, 0x3f, 0x99, 0xe6, 0xc0,
+	0x03, 0xfd, 0x74, 0x3e, 0xb8, 0x70, 0x70, 0xa5, 0x70, 0x2d, 0x06, 0x19, 0x02, 0x2f, 0x24, 0x50,
+	0x09, 0x3e, 0x39, 0x42, 0x89, 0x4b, 0x1c, 0xf8, 0xa3, 0x8b, 0x4b, 0x20, 0x10, 0x44, 0x0b, 0xa1,
+	0x84, 0x90, 0x40, 0x24, 0xf8, 0xe6, 0x00, 0xbb, 0x5c, 0x42, 0xea, 0x72, 0x12, 0xba, 0x04, 0x22,
+	0x2a, 0xc1, 0x0f, 0x09, 0xe8, 0x6d, 0x00, 0x2f, 0xb4, 0xc3, 0xd4, 0xef, 0xcd, 0xaa, 0x11, 0x0a,
+	0xba, 0x0b, 0xeb, 0x91, 0xad, 0xaa, 0x85, 0xab, 0x3d, 0x43, 0xd7, 0x1c, 0x8a, 0xbc, 0xdb, 0xe1,
+	0x0b, 0xff, 0x1e, 0xdc, 0x70, 0xf1, 0x8f, 0x46, 0xc4, 0x83, 0xd2, 0xfa, 0xa6, 0xad, 0x5b, 0xe6,
+	0x8f, 0x75, 0xf2, 0x7e, 0x23, 0x4b, 0xc1, 0xd7, 0x83, 0x97, 0xb5, 0xc8, 0x3b, 0xf1, 0x04, 0x56,
+	0x66, 0x34, 0x7d, 0x8e, 0xd7, 0x5b, 0x99, 0x0e, 0x08, 0xe7, 0x59, 0x1a, 0x21, 0x68, 0xd4, 0xbf,
+	0x26, 0xc2, 0xa6, 0x95, 0x7e, 0x4d, 0xc2, 0x02, 0xd0, 0x19, 0x61, 0x33, 0xfa, 0xbf, 0x1e, 0x61,
+	0x21, 0x68, 0xd4, 0xfb, 0xff, 0x55, 0x1c, 0x72, 0xe1, 0x6e, 0x40, 0xcf, 0x60, 0xc9, 0x1f, 0x0f,
+	0x99, 0xdd, 0x2a, 0x6e, 0x7d, 0xb2, 0xc8, 0x4e, 0x2a, 0x11, 0xd3, 0xcb, 0x2c, 0x10, 0xc5, 0x10,
+	0x3f, 0x83, 0x25, 0x42, 0x92, 0x54, 0x6e, 0x8c, 0x57, 0x20, 0xbf, 0xd7, 0xec, 0xb4, 0xe5, 0x1d,
+	0xa5, 0xa6, 0xc8, 0x55, 0x21, 0x86, 0x00, 0xd2, 0xcc, 0xd1, 0x15, 0xe2, 0x68, 0x1d, 0x84, 0xb6,
+	0xd2, 0x96, 0xeb, 0xc4, 0x55, 0x68, 0xb5, 0xd9, 0x31, 0x91, 0x40, 0x6f, 0xc0, 0x5a, 0xe4, 0xe0,
+	0xd0, 0x88, 0x5f, 0xf2, 0x5c, 0x56, 0x85, 0xa4, 0xf4, 0x6f, 0x49, 0xc8, 0x85, 0xba, 0x43, 0x2e,
+	0xdc, 0x24, 0x8e, 0xac, 0x36, 0x70, 0x0c, 0xb3, 0x3f, 0xd6, 0x98, 0xc3, 0x16, 0x89, 0x58, 0xbf,
+	0x39, 0xc7, 0x38, 0x54, 0xac, 0x1b, 0x0d, 0xca, 0x7f, 0x40, 0xd8, 0x43, 0xf0, 0xdd, 0x98, 0xba,
+	0xe6, 0xce, 0xbc, 0x23, 0x32, 0xeb, 0x90, 0x3d, 0xd4, 0x8f, 0x98, 0x94, 0xc4, 0xdc, 0x71, 0x71,
+	0x45, 0x3f, 0x8a, 0x22, 0x67, 0x0e, 0xf5, 0x23, 0x8a, 0xf6, 0x39, 0x14, 0x99, 0xe7, 0x43, 0x0d,
+	0x35, 0xc1, 0x64, 0x61, 0xfe, 0x83, 0xf9, 0xb2, 0x0c, 0x8c, 0x31, 0x8a, 0xbc, 0x1c, 0xc2, 0x05,
+	0xbd, 0x25, 0xb1, 0x06, 0x45, 0x5e, 0x9a, 0xbb, 0xb7, 0x0d, 0x7d, 0x38, 0xd5, 0xdb, 0x81, 0x3e,
+	0x0c, 0xd0, 0x3c, 0xec, 0x33, 0xb4, 0xd4, 0xdc, 0x68, 0x1d, 0xec, 0x4f, 0xa1, 0x79, 0xd8, 0x27,
+	0x3f, 0x2b, 0x69, 0x96, 0x5d, 0x90, 0x1e, 0xc0, 0xc6, 0x45, 0x93, 0x30, 0x75, 0x6a, 0xc6, 0xa7,
+	0x4e, 0x4d, 0xe9, 0x11, 0x14, 0xa2, 0x5a, 0x45, 0x77, 0x40, 0x08, 0xbc, 0x86, 0x19, 0x96, 0x22,
+	0xa7, 0x73, 0xb3, 0x23, 0xfd, 0x3c, 0x0e, 0xe8, 0xac, 0xf2, 0x88, 0xfd, 0x8a, 0x78, 0xc9, 0xb3,
+	0x20, 0x28, 0xf2, 0x2e, 0xb0, 0x5f, 0xdf, 0xa5, 0xf9, 0x21, 0xea, 0xb7, 0xf6, 0x6d, 0xbe, 0x1a,
+	0x2e, 0x73, 0xa6, 0xe5, 0x38, 0x4a, 0xcd, 0x96, 0xf6, 0xa1, 0x10, 0xd5, 0x3e, 0xba, 0x0d, 0x05,
+	0xe2, 0x63, 0xcf, 0x74, 0x06, 0x4e, 0xf0, 0x38, 0xe8, 0xc4, 0x7b, 0x50, 0xa4, 0xbb, 0x5a, 0x9b,
+	0x71, 0x2f, 0x0a, 0x94, 0xba, 0x33, 0xd1, 0x56, 0x74, 0x1e, 0x16, 0xd0, 0xd6, 0xcf, 0xe2, 0x90,
+	0x0b, 0x2d, 0x08, 0xea, 0xb0, 0x63, 0x46, 0x33, 0x9c, 0x81, 0x6e, 0xda, 0xdc, 0x5e, 0x6c, 0xcd,
+	0x69, 0x84, 0xaa, 0x94, 0x89, 0xd9, 0x0a, 0x7a, 0xb2, 0x30, 0x02, 0x19, 0x02, 0x3b, 0xbb, 0x66,
+	0x87, 0x40, 0xa9, 0x41, 0x47, 0xbe, 0x03, 0xb9, 0xd0, 0xe3, 0x91, 0xee, 0x5d, 0x64, 0x5c, 0x96,
+	0x21, 0xb7, 0xd7, 0xac, 0xb4, 0xf6, 0x9a, 0x55, 0xb9, 0x2a, 0xc4, 0x51, 0x1e, 0x32, 0xc1, 0x43,
+	0x42, 0xfa, 0xab, 0x38, 0xe4, 0xc9, 0x52, 0x0b, 0xdc, 0x91, 0x67, 0x90, 0xf6, 0x9c, 0x91, 0xdb,
+	0xc3, 0x57, 0xf0, 0x47, 0x38, 0xc2, 0x8c, 0x13, 0x97, 0xb8, 0xba, 0x13, 0x27, 0x19, 0xb0, 0xca,
+	0x12, 0xb0, 0x8a, 0xed, 0x87, 0x1e, 0x54, 0x0b, 0x72, 0x3c, 0x4f, 0x71, 0x25, 0x2f, 0x2a, 0xcb,
+	0x40, 0x6a, 0xb6, 0xf4, 0xc7, 0x71, 0x28, 0xf2, 0xb0, 0x36, 0x90, 0x31, 0xbd, 0xac, 0xe3, 0xd7,
+	0xb0, 0xac, 0x2f, 0xdc, 0x5b, 0x89, 0x8b, 0xf6, 0x96, 0xf4, 0xaf, 0x19, 0x58, 0xed, 0x62, 0xcf,
+	0xef, 0xd0, 0xdc, 0x4a, 0xd0, 0xb5, 0x8b, 0xed, 0x01, 0x52, 0x21, 0x8d, 0x4f, 0x69, 0xa2, 0x36,
+	0x31, 0x77, 0xb6, 0xef, 0x8c, 0x80, 0x92, 0x4c, 0x20, 0x54, 0x8e, 0x84, 0x3a, 0x90, 0x0d, 0x2e,
+	0xdf, 0xb8, 0x61, 0x7e, 0x38, 0x07, 0x6a, 0x79, 0x68, 0x76, 0xb0, 0x7b, 0x6a, 0xf6, 0x70, 0x35,
+	0xbc, 0x7d, 0x53, 0x43, 0x20, 0xf1, 0x4f, 0x52, 0x90, 0xa2, 0x62, 0xd0, 0x29, 0xac, 0xbc, 0xd2,
+	0x7d, 0xec, 0x0e, 0x74, 0xf7, 0x44, 0xa3, 0x22, 0xb9, 0xb6, 0x9f, 0x5f, 0xbe, 0xef, 0xa5, 0xb2,
+	0x71, 0xaa, 0xdb, 0x3d, 0x7c, 0x10, 0x00, 0xef, 0xc6, 0xd4, 0x62, 0x28, 0x85, 0xc9, 0xfd, 0x59,
+	0x1c, 0x6e, 0xf0, 0x78, 0x8b, 0x9c, 0x3b, 0x74, 0x43, 0x33, 0xf1, 0xcc, 0x86, 0xb5, 0xaf, 0x2e,
+	0xbe, 0x1d, 0xc2, 0x93, 0x8d, 0x4f, 0x0e, 0xd3, 0xe1, 0x14, 0x85, 0x75, 0x64, 0x00, 0xcb, 0x81,
+	0x15, 0x62, 0xf2, 0x99, 0x92, 0x6b, 0x57, 0x92, 0x6f, 0xc8, 0x3c, 0xee, 0xdd, 0x8d, 0xa9, 0x05,
+	0x0e, 0x4f, 0xdf, 0x89, 0x0a, 0x08, 0xb3, 0xda, 0x41, 0xef, 0xc2, 0xb2, 0x8d, 0x5f, 0x69, 0xa1,
+	0x86, 0xe8, 0x0c, 0x24, 0xd5, 0x82, 0x8d, 0x5f, 0x4d, 0x1a, 0x09, 0x90, 0xf4, 0xf5, 0x23, 0xbe,
+	0x5a, 0xc9, 0x4f, 0xb1, 0x02, 0x37, 0xce, 0x1d, 0x29, 0xfa, 0x3a, 0x08, 0x3a, 0x7b, 0xa1, 0x19,
+	0x23, 0x97, 0xb9, 0xb3, 0x0c, 0x72, 0x85, 0xd3, 0xab, 0x9c, 0x2c, 0xfe, 0x76, 0x1c, 0xf2, 0x91,
+	0xee, 0xa2, 0x1e, 0x64, 0x83, 0x90, 0x9d, 0xdf, 0x51, 0x3e, 0xbd, 0x94, 0x22, 0xba, 0xc1, 0x8d,
+	0x2e, 0x0e, 0xb0, 0xd5, 0x10, 0x38, 0x18, 0x4a, 0x32, 0x1c, 0x4a, 0x25, 0x03, 0x29, 0xaa, 0x7c,
+	0xf1, 0x7b, 0x80, 0xce, 0xb2, 0xa2, 0x0f, 0x60, 0x05, 0xdb, 0x64, 0x93, 0x85, 0x51, 0x39, 0x1d,
+	0x4f, 0x41, 0x2d, 0x72, 0x72, 0xd0, 0xf0, 0x2d, 0xc8, 0x85, 0x77, 0xc9, 0x54, 0x55, 0x49, 0x75,
+	0x42, 0x90, 0x56, 0x60, 0x99, 0x4e, 0x82, 0xa7, 0x32, 0xa7, 0x5e, 0xfa, 0xaf, 0x24, 0xac, 0xd2,
+	0xd3, 0xbe, 0x66, 0x5a, 0xd8, 0x0b, 0x36, 0x78, 0x0d, 0x96, 0x3c, 0xd3, 0x3e, 0xb9, 0x4a, 0x80,
+	0x48, 0xf8, 0xd1, 0xf7, 0x60, 0xa5, 0xef, 0xb8, 0x03, 0xdd, 0xd7, 0xfa, 0xfc, 0xe5, 0x15, 0xce,
+	0xe7, 0x22, 0x83, 0x0a, 0x68, 0x44, 0x25, 0xcc, 0x7e, 0x62, 0x83, 0x39, 0x9d, 0x1e, 0xd5, 0x67,
+	0x56, 0x2d, 0x06, 0x64, 0x3a, 0x30, 0x0f, 0x7d, 0x0b, 0x44, 0x7e, 0xe3, 0x6f, 0x10, 0x57, 0x79,
+	0x60, 0xda, 0xd8, 0xd0, 0xbc, 0x63, 0xdd, 0x35, 0x4c, 0xfb, 0x88, 0x3a, 0x64, 0x59, 0x75, 0x83,
+	0xb5, 0xa8, 0x86, 0x0d, 0x3a, 0xfc, 0x3d, 0xc2, 0xd3, 0x61, 0x29, 0x0b, 0xe9, 0xaa, 0xf3, 0xdc,
+	0xdb, 0xcd, 0xaa, 0xf5, 0x75, 0xb1, 0xe9, 0xff, 0x69, 0x40, 0x25, 0xfd, 0x04, 0x52, 0xd4, 0xc2,
+	0x5f, 0xcf, 0xdd, 0x52, 0x09, 0xd6, 0xc2, 0xfb, 0xb5, 0xf0, 0x50, 0x09, 0x6e, 0x98, 0x56, 0xc3,
+	0x57, 0xfc, 0x4c, 0xf1, 0xa4, 0x3f, 0x4d, 0x41, 0x31, 0x48, 0x1d, 0xb1, 0xcb, 0x4b, 0xe9, 0x77,
+	0x53, 0xdc, 0x93, 0x78, 0x0f, 0x52, 0x95, 0x97, 0x5d, 0xb9, 0x23, 0xc4, 0xc4, 0x37, 0x69, 0xfe,
+	0x67, 0x8d, 0xe6, 0x7f, 0x28, 0xea, 0xf6, 0xe1, 0xd8, 0xa7, 0xd9, 0x48, 0x74, 0x17, 0xf2, 0x24,
+	0x2e, 0x69, 0x3e, 0xd5, 0xf6, 0xba, 0xb5, 0x47, 0x02, 0x4c, 0x5d, 0x40, 0xb0, 0xb6, 0x24, 0xcc,
+	0xb5, 0x8f, 0xb4, 0x91, 0xdf, 0x7f, 0x44, 0x38, 0xde, 0x86, 0xc4, 0xf3, 0x7d, 0x21, 0x2e, 0xde,
+	0xa4, 0x0d, 0x85, 0x48, 0xc3, 0x93, 0x53, 0xf2, 0x5e, 0x82, 0xa5, 0x4a, 0xab, 0x55, 0x17, 0x0a,
+	0xe2, 0x06, 0x6d, 0x81, 0xa2, 0x62, 0x1d, 0xc7, 0x22, 0x6d, 0xde, 0x87, 0xf4, 0x7e, 0x59, 0x55,
+	0x9a, 0x5d, 0x21, 0x21, 0x8a, 0xb4, 0xd5, 0x7a, 0xa4, 0xd5, 0xa9, 0xee, 0x9a, 0xb6, 0xcf, 0xdb,
+	0x55, 0x5b, 0x7b, 0x95, 0xba, 0x2c, 0xe4, 0xcf, 0x69, 0x67, 0x38, 0x23, 0x9e, 0xee, 0xfa, 0x30,
+	0x92, 0x1f, 0x4b, 0x4e, 0x5d, 0x11, 0xb0, 0x96, 0xd1, 0xd4, 0xd8, 0x7b, 0x90, 0xea, 0x2a, 0x0d,
+	0x59, 0x15, 0x96, 0xce, 0xd1, 0x0b, 0x75, 0xd0, 0xd8, 0x15, 0xc6, 0x8a, 0xd2, 0xec, 0xca, 0xea,
+	0x7e, 0x58, 0xb2, 0x21, 0xa4, 0xa6, 0xf2, 0xea, 0x1c, 0xd8, 0xf6, 0xb1, 0x7b, 0xaa, 0x5b, 0xfc,
+	0x0e, 0x83, 0x65, 0xe3, 0x97, 0xeb, 0x72, 0xf3, 0x69, 0x77, 0x57, 0x6b, 0xab, 0x72, 0x4d, 0x79,
+	0x21, 0xa4, 0xa7, 0xf2, 0x6f, 0x8c, 0xcf, 0xc2, 0xf6, 0x91, 0x7f, 0xac, 0x0d, 0x5d, 0xdc, 0x37,
+	0xbf, 0xe0, 0x5c, 0x53, 0x05, 0x22, 0x42, 0xe6, 0x1c, 0x2e, 0x76, 0x4d, 0x10, 0x91, 0xf5, 0x09,
+	0x14, 0x59, 0xf3, 0x20, 0x21, 0x2d, 0x64, 0xa7, 0xae, 0x75, 0x18, 0x5b, 0xb8, 0xb7, 0xd9, 0xb2,
+	0xa5, 0x79, 0xe1, 0x1b, 0x9d, 0x6e, 0xb9, 0x2b, 0x6b, 0x15, 0x12, 0x88, 0x56, 0xb5, 0x50, 0x79,
+	0x39, 0xf1, 0xeb, 0x94, 0xfd, 0xdd, 0xa9, 0xf9, 0xd7, 0x7d, 0xac, 0x1d, 0xea, 0xbd, 0x13, 0x6c,
+	0x68, 0x51, 0x4d, 0xde, 0x86, 0xa4, 0xda, 0x3a, 0x10, 0x96, 0xc5, 0x37, 0x28, 0xcf, 0x6a, 0x84,
+	0xc7, 0xa5, 0xfd, 0x93, 0x7e, 0x27, 0x1d, 0xf8, 0x7c, 0x91, 0xdc, 0xdc, 0xb5, 0xfb, 0x7c, 0x68,
+	0x1f, 0x0a, 0xec, 0x56, 0x80, 0x74, 0x75, 0xe4, 0x71, 0x6f, 0xf5, 0xde, 0x3c, 0x91, 0x21, 0x61,
+	0xeb, 0x50, 0x2e, 0xe6, 0xaf, 0xe6, 0x07, 0x13, 0x0a, 0x7a, 0x3f, 0xb0, 0x8b, 0x13, 0x07, 0x8f,
+	0x9d, 0x33, 0xcb, 0x8c, 0x1c, 0x84, 0x2c, 0x55, 0xc8, 0xf8, 0xae, 0x79, 0x74, 0x84, 0x5d, 0x1e,
+	0x94, 0x7e, 0x38, 0xcf, 0x39, 0xc7, 0x38, 0xd4, 0x80, 0x15, 0x61, 0x58, 0x0d, 0xfd, 0x46, 0xd3,
+	0xb1, 0x35, 0xc2, 0x42, 0xc3, 0xd2, 0xe2, 0xd6, 0xa3, 0x79, 0xbc, 0xb4, 0x08, 0x6f, 0xc3, 0x31,
+	0x78, 0x0a, 0x43, 0xd0, 0x67, 0xc8, 0x24, 0xe2, 0x61, 0x37, 0x1b, 0xd4, 0x4f, 0xa2, 0x79, 0xaf,
+	0xf9, 0x22, 0x1e, 0x76, 0x31, 0x4b, 0x4e, 0x54, 0x1e, 0xf1, 0x38, 0x21, 0x01, 0x1d, 0x82, 0xd0,
+	0xb3, 0x1c, 0xea, 0x7d, 0x1d, 0xe2, 0x63, 0xfd, 0xd4, 0x74, 0x5c, 0x9a, 0x27, 0x2b, 0xce, 0xe5,
+	0x60, 0xee, 0x30, 0xd6, 0x0a, 0xe7, 0x64, 0xf0, 0x2b, 0xbd, 0x69, 0x2a, 0xf5, 0x44, 0x2c, 0x8b,
+	0x2e, 0x64, 0x4b, 0xf7, 0xb1, 0x8d, 0x3d, 0x8f, 0x26, 0xd6, 0x88, 0x27, 0xc2, 0xe8, 0x75, 0x4e,
+	0x46, 0x9f, 0x43, 0xb1, 0x65, 0x93, 0x8e, 0x05, 0xcc, 0x1b, 0xb9, 0xb9, 0x13, 0x41, 0xd3, 0x8c,
+	0xac, 0x2f, 0x33, 0x68, 0x68, 0x13, 0x6e, 0xe8, 0x9e, 0x67, 0x1e, 0xd9, 0x9e, 0xe6, 0x3b, 0x9a,
+	0x63, 0x07, 0x77, 0x98, 0x1b, 0x40, 0x8f, 0x40, 0xc4, 0x5f, 0x76, 0x9d, 0x96, 0x8d, 0xd9, 0xfa,
+	0x97, 0xbe, 0x0f, 0xf9, 0xc8, 0x62, 0x93, 0x1a, 0x17, 0xc5, 0x7b, 0x2b, 0x90, 0x6f, 0xb6, 0x9a,
+	0xf4, 0x82, 0x4c, 0x69, 0x3e, 0x15, 0xe2, 0x94, 0x20, 0xcb, 0xd5, 0x0e, 0xbb, 0x33, 0x13, 0x12,
+	0x08, 0x41, 0xb1, 0x5c, 0x57, 0xe5, 0x72, 0x95, 0x5f, 0xa3, 0x55, 0x85, 0xa4, 0xf4, 0x03, 0x10,
+	0x66, 0xe7, 0x5f, 0x52, 0x2e, 0x12, 0x51, 0x04, 0xa8, 0x2a, 0x9d, 0x9d, 0xb2, 0x5a, 0x65, 0x12,
+	0x04, 0x28, 0x84, 0x37, 0x71, 0x84, 0x92, 0x20, 0x2d, 0x54, 0x99, 0xde, 0x9e, 0x91, 0xe7, 0xa4,
+	0xf4, 0x5d, 0x58, 0x99, 0x99, 0x23, 0xe9, 0xc9, 0x6b, 0x06, 0x20, 0x37, 0x94, 0xae, 0x56, 0xae,
+	0x1f, 0x94, 0x5f, 0x76, 0x58, 0x4a, 0x8c, 0x12, 0x94, 0x9a, 0xd6, 0x6c, 0x35, 0xe5, 0x46, 0xbb,
+	0xfb, 0x52, 0x48, 0x48, 0xed, 0xd9, 0x29, 0x7a, 0x2d, 0x62, 0x4d, 0x51, 0xe5, 0x29, 0x44, 0x4a,
+	0x98, 0x46, 0x3c, 0x04, 0x98, 0x2c, 0x51, 0xa9, 0x7b, 0x11, 0xda, 0x2a, 0x2c, 0xcb, 0xcd, 0xaa,
+	0xd6, 0xaa, 0x69, 0x61, 0xd2, 0x0e, 0x41, 0xb1, 0x5e, 0xa6, 0x97, 0xe3, 0x4a, 0x53, 0x6b, 0x97,
+	0x9b, 0x44, 0xcb, 0xa4, 0xd7, 0x65, 0xb5, 0xae, 0x44, 0xa9, 0x49, 0xc9, 0x02, 0x98, 0x04, 0xfe,
+	0xd2, 0xe7, 0xaf, 0xd1, 0xb0, 0xbc, 0x2f, 0x37, 0xbb, 0xb4, 0xc4, 0x4f, 0x88, 0xa3, 0x35, 0x58,
+	0xe1, 0x77, 0x4a, 0xe4, 0xa4, 0xa5, 0xc4, 0x04, 0xba, 0x0d, 0x6f, 0x75, 0x5e, 0x36, 0x77, 0x76,
+	0xd5, 0x56, 0x93, 0xde, 0x33, 0xcd, 0xb6, 0x48, 0x4a, 0xbf, 0x14, 0x20, 0xc3, 0xcd, 0x04, 0x52,
+	0x21, 0xa7, 0xf7, 0x7d, 0xec, 0x6a, 0xba, 0x65, 0x71, 0xa3, 0x79, 0x6f, 0x7e, 0x2b, 0x53, 0x2a,
+	0x13, 0xde, 0xb2, 0x65, 0xed, 0xc6, 0xd4, 0xac, 0xce, 0x7f, 0x47, 0x30, 0xed, 0x31, 0x77, 0x84,
+	0x16, 0xc7, 0xb4, 0xc7, 0x13, 0x4c, 0x7b, 0x8c, 0xf6, 0x00, 0x18, 0x26, 0xd6, 0x7b, 0xc7, 0x3c,
+	0xfe, 0xb9, 0xbf, 0x28, 0xa8, 0xac, 0xf7, 0x8e, 0x77, 0x63, 0x2a, 0xeb, 0x1d, 0x79, 0x40, 0x16,
+	0xac, 0x71, 0x58, 0xdb, 0xd0, 0x9c, 0x7e, 0xb0, 0xdf, 0x96, 0xe6, 0xce, 0x8b, 0x4e, 0xe3, 0xdb,
+	0x46, 0xab, 0xcf, 0x36, 0xe6, 0x6e, 0x4c, 0x15, 0xf4, 0x19, 0x1a, 0xf2, 0xe1, 0x06, 0x93, 0x36,
+	0x13, 0x55, 0xf2, 0x2c, 0xe1, 0x93, 0x45, 0xe5, 0x9d, 0x8d, 0x1e, 0xf5, 0xb3, 0x64, 0xf4, 0xf3,
+	0x38, 0x48, 0x4c, 0xac, 0x37, 0xb6, 0x7b, 0xc7, 0xae, 0x63, 0xd3, 0xbb, 0xc3, 0xd9, 0x3e, 0xb0,
+	0x0a, 0x9d, 0x67, 0x8b, 0xf6, 0xa1, 0x13, 0xc1, 0x3c, 0xd3, 0x9f, 0x5b, 0xfa, 0xeb, 0x9b, 0xa0,
+	0xe7, 0x90, 0xd6, 0xad, 0x57, 0xfa, 0xd8, 0xdb, 0x28, 0x50, 0xf1, 0x9b, 0x8b, 0x88, 0xa7, 0x8c,
+	0xbb, 0x31, 0x95, 0x43, 0xa0, 0x26, 0x64, 0x0c, 0xdc, 0xd7, 0x47, 0x96, 0x4f, 0x0f, 0x89, 0xf9,
+	0x8e, 0xff, 0x00, 0xad, 0xca, 0x38, 0x77, 0x63, 0x6a, 0x00, 0x82, 0x3e, 0x9f, 0x84, 0xdd, 0x3d,
+	0x67, 0x64, 0xfb, 0xf4, 0x58, 0x98, 0x2f, 0xb7, 0x11, 0xa0, 0xca, 0x41, 0x92, 0x70, 0x64, 0xfb,
+	0x91, 0x38, 0x9b, 0x3e, 0xa3, 0x5d, 0x48, 0xd9, 0xf8, 0x14, 0xb3, 0x53, 0x24, 0xbf, 0x75, 0x77,
+	0x01, 0xdc, 0x26, 0xe1, 0xdb, 0x8d, 0xa9, 0x0c, 0x80, 0xec, 0x0e, 0xc7, 0x65, 0x77, 0x43, 0xd6,
+	0x98, 0x9e, 0x16, 0x8b, 0xed, 0x8e, 0x96, 0x5b, 0x63, 0xbc, 0x64, 0x77, 0x38, 0xc1, 0x03, 0x99,
+	0x1d, 0x17, 0x0f, 0xb1, 0xee, 0x6f, 0xe4, 0x17, 0x9e, 0x1d, 0x95, 0x32, 0x92, 0xd9, 0x61, 0x10,
+	0xe2, 0x0b, 0xc8, 0x06, 0xd6, 0x02, 0xd5, 0x21, 0x4f, 0xeb, 0xda, 0x68, 0xd3, 0x20, 0x8a, 0x5f,
+	0xc4, 0xbb, 0x89, 0xb2, 0x4f, 0x90, 0xed, 0xf1, 0x35, 0x23, 0xbf, 0x84, 0x5c, 0x68, 0x38, 0xae,
+	0x19, 0xfa, 0x6f, 0xe3, 0x20, 0xcc, 0x1a, 0x0d, 0xd4, 0x82, 0x65, 0xac, 0xbb, 0xd6, 0x58, 0xeb,
+	0x9b, 0x24, 0x38, 0x0a, 0x8a, 0x29, 0x17, 0x11, 0x52, 0xa0, 0x00, 0x35, 0xc6, 0x8f, 0x1a, 0x50,
+	0x20, 0x4e, 0x4d, 0x88, 0x97, 0x58, 0x18, 0x2f, 0x4f, 0xf8, 0x39, 0x9c, 0xf8, 0xff, 0x61, 0xed,
+	0x1c, 0xc3, 0x83, 0x8e, 0x61, 0x3d, 0xcc, 0x60, 0x68, 0x67, 0x2a, 0xc8, 0x1f, 0xcc, 0x99, 0xf6,
+	0xa6, 0xec, 0x93, 0x92, 0xe1, 0x35, 0xff, 0x0c, 0xcd, 0x13, 0xdf, 0x81, 0x5b, 0x5f, 0x62, 0x75,
+	0xc4, 0x1c, 0x64, 0xf8, 0x5e, 0x16, 0xef, 0x41, 0x21, 0xba, 0x01, 0xd1, 0xbb, 0xb3, 0x1b, 0x9a,
+	0xa8, 0x37, 0x35, 0xbd, 0x2b, 0xc5, 0x0c, 0xa4, 0xe8, 0xee, 0x12, 0xb3, 0x90, 0x66, 0x26, 0x46,
+	0xfc, 0xa3, 0x38, 0xe4, 0xc2, 0x2d, 0x82, 0x9e, 0xc0, 0x52, 0x98, 0xd4, 0x5f, 0x4c, 0x97, 0x94,
+	0x8f, 0xb8, 0xf5, 0xc1, 0x4e, 0x5d, 0x7c, 0x3a, 0x02, 0x56, 0xb1, 0x0b, 0x69, 0xb6, 0xc5, 0xd0,
+	0x33, 0x80, 0xc9, 0xc2, 0xba, 0x44, 0xaf, 0x22, 0xdc, 0x95, 0x5c, 0x18, 0x72, 0x48, 0xff, 0x94,
+	0x88, 0xe4, 0xb9, 0x26, 0xd5, 0xb0, 0x1d, 0x48, 0x19, 0xd8, 0xd2, 0xc7, 0x0b, 0xdc, 0x1d, 0x9e,
+	0x45, 0x29, 0x55, 0x09, 0x04, 0xb1, 0x5f, 0x14, 0x0b, 0x7d, 0x06, 0x59, 0xdd, 0x32, 0x8f, 0x6c,
+	0xcd, 0x77, 0xb8, 0x4e, 0xbe, 0x7d, 0x39, 0xdc, 0x32, 0x41, 0xe9, 0x3a, 0xc4, 0x8a, 0xeb, 0xec,
+	0xa7, 0xf8, 0x21, 0xa4, 0xa8, 0x34, 0xf4, 0x0e, 0x14, 0xa8, 0x34, 0x6d, 0x60, 0x5a, 0x96, 0xe9,
+	0xf1, 0x74, 0x63, 0x9e, 0xd2, 0x1a, 0x94, 0x24, 0x3e, 0x86, 0x0c, 0x47, 0x40, 0x37, 0x21, 0x3d,
+	0xc4, 0xae, 0xe9, 0xb0, 0xd8, 0x2c, 0xa9, 0xf2, 0x27, 0x42, 0x77, 0xfa, 0x7d, 0x0f, 0xfb, 0xd4,
+	0x49, 0x48, 0xaa, 0xfc, 0xa9, 0x72, 0x03, 0xd6, 0xce, 0xd9, 0x03, 0xd2, 0x1f, 0x26, 0x20, 0x17,
+	0x66, 0x78, 0xd0, 0x3e, 0x14, 0xf5, 0x1e, 0xad, 0xdf, 0x19, 0xea, 0xbe, 0x8f, 0x5d, 0xfb, 0xb2,
+	0x79, 0x9d, 0x65, 0x06, 0xd3, 0x66, 0x28, 0xe8, 0x39, 0x64, 0x4e, 0x4d, 0xfc, 0xea, 0x6a, 0xd7,
+	0x6b, 0x69, 0x02, 0x51, 0xb3, 0xd1, 0xe7, 0xb0, 0xca, 0xc3, 0xd3, 0x81, 0x3e, 0x1c, 0x12, 0xff,
+	0xa0, 0x6f, 0x73, 0x8f, 0xeb, 0x32, 0xb0, 0x3c, 0xd6, 0x6d, 0x30, 0xac, 0x9a, 0x2d, 0x7d, 0x0a,
+	0xf9, 0x48, 0x55, 0x39, 0x12, 0x20, 0x39, 0x72, 0xed, 0x20, 0x69, 0x3c, 0x72, 0x6d, 0xb4, 0x01,
+	0x99, 0x21, 0x4b, 0xc8, 0x51, 0xb1, 0x05, 0x35, 0x78, 0x7c, 0xb6, 0x94, 0x8d, 0x0b, 0x09, 0xe9,
+	0xcf, 0xe2, 0xb0, 0x1e, 0xa4, 0xa7, 0xa2, 0x65, 0xef, 0xd2, 0xcf, 0xe2, 0x50, 0x88, 0x12, 0xd0,
+	0x7b, 0x90, 0xae, 0xb6, 0xe8, 0x9d, 0x78, 0x6c, 0x2a, 0x6d, 0x84, 0xed, 0xd3, 0x6d, 0xc3, 0xe9,
+	0x9d, 0xb0, 0xa4, 0xcc, 0xfb, 0x90, 0xe1, 0x4e, 0xb2, 0x10, 0x9f, 0x4a, 0xde, 0x90, 0x66, 0xdc,
+	0x4d, 0x22, 0xed, 0xee, 0x40, 0x56, 0x7e, 0xd1, 0x95, 0xd5, 0x66, 0xb9, 0x3e, 0x93, 0x60, 0x22,
+	0x0d, 0xf1, 0x17, 0x64, 0x2a, 0x74, 0x6b, 0xfb, 0x74, 0x53, 0x7a, 0x04, 0xcb, 0x55, 0x0a, 0x1f,
+	0xe4, 0x6b, 0x3f, 0x80, 0x95, 0x9e, 0x63, 0xfb, 0xba, 0x69, 0x93, 0x78, 0x7f, 0xa0, 0x1f, 0x05,
+	0xb5, 0x4f, 0xc5, 0x90, 0xac, 0x10, 0xaa, 0xf4, 0xef, 0x71, 0x28, 0x72, 0x83, 0x16, 0xf0, 0x16,
+	0x21, 0xe1, 0x78, 0xbc, 0x79, 0xc2, 0xf1, 0x10, 0x82, 0x25, 0xdd, 0xed, 0x1d, 0x73, 0x8d, 0xd1,
+	0xdf, 0x44, 0x65, 0x3d, 0x67, 0x30, 0xd0, 0xed, 0x20, 0x95, 0x10, 0x3c, 0xa2, 0x3a, 0x24, 0xb1,
+	0x7d, 0xba, 0x48, 0x69, 0xf7, 0x94, 0xf4, 0x92, 0x6c, 0x9f, 0xb2, 0x5c, 0x28, 0x81, 0x11, 0x3f,
+	0x81, 0x6c, 0x40, 0x58, 0xa8, 0x88, 0xfa, 0xbf, 0xe3, 0xb0, 0x22, 0x73, 0x05, 0x05, 0xe3, 0x8a,
+	0xde, 0x1a, 0xc5, 0xaf, 0xe9, 0xd6, 0x08, 0xed, 0x43, 0x9a, 0x56, 0x2c, 0x05, 0xd7, 0x5b, 0xf3,
+	0xf8, 0xd4, 0x33, 0x1d, 0x63, 0x35, 0x1b, 0x41, 0x95, 0x3c, 0x43, 0x13, 0x1f, 0x43, 0x3e, 0x42,
+	0x5e, 0x68, 0xec, 0x3f, 0x85, 0x95, 0x99, 0x3d, 0x71, 0x3d, 0x59, 0xdd, 0xaf, 0x41, 0x31, 0xf2,
+	0x19, 0xcf, 0xe4, 0x9a, 0x70, 0x39, 0x42, 0x55, 0x0c, 0x69, 0x1b, 0x0a, 0x53, 0xb2, 0xf9, 0x7e,
+	0x8b, 0xcf, 0xb1, 0xdf, 0xa4, 0xdf, 0x2c, 0x41, 0x3e, 0x52, 0xb6, 0x86, 0x14, 0x48, 0x99, 0x3e,
+	0x0e, 0x4f, 0xf6, 0x7b, 0x8b, 0x55, 0xbd, 0x95, 0x14, 0x1f, 0x0f, 0x54, 0x86, 0x20, 0xf6, 0x01,
+	0x14, 0x03, 0xdb, 0xbe, 0xd9, 0x37, 0xb1, 0x4b, 0x6c, 0x73, 0xf4, 0x73, 0x0f, 0xde, 0xbb, 0xbc,
+	0x3f, 0xf9, 0xd2, 0x83, 0x1c, 0xde, 0x93, 0x26, 0x13, 0x8b, 0x31, 0xe1, 0xdb, 0x73, 0xed, 0x60,
+	0x5e, 0x92, 0xe1, 0xbc, 0x88, 0xbf, 0x4a, 0xc0, 0x12, 0x91, 0x8b, 0x14, 0x48, 0x70, 0xe0, 0xf9,
+	0x3e, 0x9b, 0x98, 0xea, 0x78, 0xd8, 0x53, 0x35, 0x61, 0x92, 0x3d, 0xc5, 0xca, 0x80, 0x12, 0x73,
+	0x67, 0xd1, 0xa2, 0x60, 0x33, 0x85, 0x40, 0xe8, 0xc3, 0x60, 0xe5, 0x30, 0x1b, 0xbb, 0x5e, 0x62,
+	0x9f, 0x1e, 0x96, 0x82, 0x4f, 0x0f, 0x4b, 0x65, 0x3b, 0xf8, 0xa2, 0x08, 0x3d, 0x80, 0xbc, 0x77,
+	0xec, 0xb8, 0x3e, 0xcb, 0xb9, 0xf2, 0x38, 0xf5, 0x7c, 0x0e, 0xa0, 0x0d, 0xf7, 0x29, 0xdb, 0x3a,
+	0xa4, 0x2c, 0xfd, 0x10, 0x5b, 0xfc, 0xe3, 0x15, 0xf6, 0x80, 0xde, 0x84, 0xac, 0x65, 0xda, 0x27,
+	0xda, 0xc8, 0xb5, 0x68, 0xf4, 0x97, 0x53, 0x33, 0xe4, 0x79, 0xcf, 0xb5, 0xc4, 0x9f, 0xf2, 0xe2,
+	0xa4, 0xd1, 0x6b, 0x8a, 0x93, 0x58, 0x82, 0x9f, 0x15, 0x0f, 0x28, 0xcd, 0xae, 0xfc, 0x54, 0x56,
+	0x85, 0x04, 0xca, 0x41, 0xaa, 0x56, 0x6f, 0x95, 0xbb, 0x42, 0x92, 0x15, 0x15, 0xb4, 0xea, 0x72,
+	0xb9, 0x29, 0x2c, 0xa1, 0x65, 0xc8, 0x85, 0x1f, 0x26, 0x0a, 0x29, 0x54, 0x80, 0x6c, 0x75, 0x4f,
+	0x2d, 0xd3, 0xca, 0xe1, 0x34, 0x2a, 0x02, 0x3c, 0x2b, 0xef, 0x97, 0xb5, 0x9d, 0x7a, 0xb9, 0xd3,
+	0x11, 0x32, 0xd2, 0x3f, 0x66, 0xe1, 0x46, 0x03, 0x7b, 0x9e, 0x7e, 0x84, 0x0f, 0x4c, 0xff, 0x38,
+	0x52, 0xc8, 0x7c, 0xcd, 0xdf, 0x1a, 0x7d, 0x07, 0x52, 0x34, 0x07, 0xbb, 0xe8, 0xc7, 0x57, 0xc4,
+	0x75, 0xa1, 0x8c, 0xe8, 0xfb, 0xc4, 0xb2, 0xf3, 0x4a, 0xef, 0xc8, 0x26, 0x9a, 0x2f, 0x58, 0x9a,
+	0xae, 0x28, 0xd8, 0x8d, 0xa9, 0xbc, 0xcc, 0x29, 0xac, 0x31, 0xf8, 0x21, 0xac, 0x7a, 0xc6, 0x49,
+	0x78, 0x39, 0x17, 0xad, 0x50, 0xba, 0xc4, 0x59, 0xbc, 0x1b, 0x53, 0x57, 0xbc, 0x19, 0x53, 0x74,
+	0x00, 0xc5, 0xa1, 0xee, 0x6a, 0x86, 0x13, 0x76, 0x3f, 0x3d, 0xb7, 0x51, 0x8a, 0xd6, 0x44, 0x92,
+	0xe8, 0x76, 0x18, 0x2d, 0x62, 0x6d, 0x01, 0x0c, 0xc3, 0xbd, 0xc9, 0x03, 0xf2, 0xc5, 0xbe, 0x1a,
+	0xdc, 0x8d, 0xa9, 0x11, 0x08, 0xa4, 0x42, 0x3e, 0xf2, 0xa5, 0x27, 0x0f, 0xc6, 0x17, 0xfc, 0x2e,
+	0x70, 0x37, 0xa6, 0x46, 0x41, 0x50, 0x07, 0x0a, 0xb4, 0x34, 0x2e, 0x18, 0x7b, 0x6e, 0x6e, 0xd0,
+	0x48, 0x81, 0x0c, 0x01, 0x75, 0x23, 0xf5, 0x32, 0x0d, 0x80, 0xc9, 0x85, 0x24, 0x0f, 0x9d, 0x17,
+	0xba, 0x09, 0x24, 0x51, 0x78, 0x78, 0xf3, 0x88, 0xfa, 0xb0, 0x16, 0xf9, 0xe6, 0x26, 0xec, 0x6a,
+	0x61, 0xc1, 0xef, 0x13, 0x23, 0xe5, 0x31, 0xbb, 0x31, 0x95, 0xbb, 0x78, 0xd1, 0x9a, 0x19, 0x0c,
+	0xe8, 0x6c, 0x35, 0xf4, 0xc6, 0xf2, 0xe5, 0x3f, 0x83, 0x9c, 0x88, 0x89, 0x5e, 0xd3, 0xec, 0xc3,
+	0xf2, 0xf4, 0x72, 0x2e, 0x5e, 0xea, 0x10, 0x24, 0xeb, 0xad, 0x1f, 0x79, 0xae, 0xa4, 0x61, 0xc9,
+	0x75, 0x1c, 0x5f, 0xfa, 0x65, 0x1a, 0x6e, 0xca, 0x5f, 0xe0, 0xde, 0x88, 0x96, 0xdb, 0x76, 0x7c,
+	0xfd, 0x28, 0xdc, 0x4d, 0x6d, 0xc8, 0x47, 0xce, 0x46, 0x6e, 0x3d, 0x16, 0xfd, 0x0a, 0x32, 0x0a,
+	0x41, 0x0c, 0x2b, 0x9b, 0x65, 0x7e, 0xea, 0x9b, 0x7c, 0xc6, 0xce, 0x29, 0x94, 0x96, 0xe7, 0xf2,
+	0x44, 0xce, 0xeb, 0xf7, 0x64, 0x61, 0x28, 0xc6, 0x54, 0xb9, 0xf4, 0xdb, 0x53, 0xdf, 0x6b, 0x2f,
+	0xd1, 0xeb, 0xdc, 0xe8, 0x07, 0xd7, 0x1b, 0x93, 0x4f, 0xfb, 0x52, 0xf4, 0x65, 0xf8, 0x79, 0xde,
+	0xb4, 0x19, 0x4d, 0x5f, 0xd5, 0x8c, 0xf6, 0x21, 0x3f, 0xf2, 0xb0, 0x4b, 0x2f, 0xca, 0xb0, 0xb7,
+	0x91, 0xb9, 0xea, 0x80, 0xf7, 0x3c, 0xec, 0xd2, 0x22, 0x3c, 0x32, 0xe0, 0x51, 0xf0, 0xe0, 0xa1,
+	0x97, 0x90, 0xa6, 0x57, 0xa9, 0xde, 0x46, 0x96, 0x8a, 0x28, 0x5f, 0x5e, 0x04, 0xad, 0xd5, 0x53,
+	0x0c, 0x95, 0x03, 0x8a, 0x2d, 0xc8, 0x47, 0xd4, 0x3c, 0x8f, 0x43, 0xf2, 0x55, 0x00, 0xcb, 0xe9,
+	0xe9, 0x16, 0xfb, 0x94, 0x81, 0x2d, 0x80, 0x1c, 0xa5, 0x34, 0xf5, 0x01, 0x26, 0x80, 0x91, 0x61,
+	0x5c, 0x03, 0xe0, 0x73, 0xc8, 0xf0, 0x4e, 0x5f, 0x1d, 0x6c, 0xeb, 0x0f, 0xe2, 0xd1, 0xba, 0x31,
+	0xee, 0x51, 0xa3, 0x1f, 0x43, 0x9a, 0x55, 0x9f, 0xa0, 0x79, 0x92, 0x91, 0x53, 0x85, 0x2a, 0xe2,
+	0x15, 0x0a, 0xc9, 0xa4, 0xd8, 0xdd, 0xf8, 0xf6, 0xa7, 0x90, 0xa5, 0xff, 0xfd, 0x40, 0x3c, 0xd2,
+	0x77, 0xce, 0x78, 0x34, 0xc4, 0x0b, 0xa1, 0xbe, 0x4c, 0x6b, 0xc8, 0xfe, 0x3c, 0xe0, 0x9f, 0xff,
+	0xfc, 0xaf, 0x5f, 0x30, 0x9f, 0x85, 0x70, 0xed, 0xb9, 0xf6, 0xb6, 0x02, 0xcb, 0x14, 0xa0, 0xc7,
+	0xff, 0x83, 0x61, 0x1e, 0x94, 0x7f, 0x09, 0x50, 0x0a, 0x87, 0x91, 0xff, 0x72, 0xa8, 0x7c, 0x13,
+	0xbe, 0xfc, 0x0f, 0x27, 0x2a, 0x39, 0x95, 0x56, 0xa4, 0x94, 0x87, 0xe6, 0x67, 0xf9, 0x80, 0xae,
+	0x9d, 0x6e, 0x1e, 0xa6, 0xa9, 0xb8, 0x7b, 0xff, 0x1b, 0x00, 0x00, 0xff, 0xff, 0xa7, 0x38, 0x87,
+	0x1a, 0xcb, 0x42, 0x00, 0x00,
 }
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/FieldValueTypeInformation.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/FieldValueTypeInformation.java
index dc27606..a6ecc45 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/FieldValueTypeInformation.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/FieldValueTypeInformation.java
@@ -17,17 +17,11 @@
  */
 package org.apache.beam.sdk.schemas;
 
-import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument;
-
 import com.google.auto.value.AutoValue;
 import java.io.Serializable;
 import java.lang.reflect.Field;
 import java.lang.reflect.Method;
-import java.lang.reflect.ParameterizedType;
-import java.lang.reflect.Type;
 import java.util.Arrays;
-import java.util.Collection;
-import java.util.Map;
 import javax.annotation.Nullable;
 import org.apache.beam.sdk.schemas.utils.ReflectUtils;
 import org.apache.beam.sdk.values.TypeDescriptor;
@@ -129,9 +123,13 @@
   }
 
   public static FieldValueTypeInformation forSetter(Method method) {
+    return forSetter(method, "set");
+  }
+
+  public static FieldValueTypeInformation forSetter(Method method, String setterPrefix) {
     String name;
-    if (method.getName().startsWith("set")) {
-      name = ReflectUtils.stripPrefix(method.getName(), "set");
+    if (method.getName().startsWith(setterPrefix)) {
+      name = ReflectUtils.stripPrefix(method.getName(), setterPrefix);
     } else {
       throw new RuntimeException("Setter has wrong prefix " + method.getName());
     }
@@ -162,25 +160,9 @@
   }
 
   @Nullable
-  private static FieldValueTypeInformation getIterableComponentType(TypeDescriptor valueType) {
+  static FieldValueTypeInformation getIterableComponentType(TypeDescriptor valueType) {
     // TODO: Figure out nullable elements.
-    TypeDescriptor componentType = null;
-    if (valueType.isArray()) {
-      Type component = valueType.getComponentType().getType();
-      if (!component.equals(byte.class)) {
-        componentType = TypeDescriptor.of(component);
-      }
-    } else if (valueType.isSubtypeOf(TypeDescriptor.of(Iterable.class))) {
-      TypeDescriptor<Iterable<?>> collection = valueType.getSupertype(Iterable.class);
-      if (collection.getType() instanceof ParameterizedType) {
-        ParameterizedType ptype = (ParameterizedType) collection.getType();
-        java.lang.reflect.Type[] params = ptype.getActualTypeArguments();
-        checkArgument(params.length == 1);
-        componentType = TypeDescriptor.of(params[0]);
-      } else {
-        throw new RuntimeException("Collection parameter is not parameterized!");
-      }
-    }
+    TypeDescriptor componentType = ReflectUtils.getIterableComponentType(valueType);
     if (componentType == null) {
       return null;
     }
@@ -223,17 +205,7 @@
   @SuppressWarnings("unchecked")
   @Nullable
   private static FieldValueTypeInformation getMapType(TypeDescriptor valueType, int index) {
-    TypeDescriptor mapType = null;
-    if (valueType.isSubtypeOf(TypeDescriptor.of(Map.class))) {
-      TypeDescriptor<Collection<?>> map = valueType.getSupertype(Map.class);
-      if (map.getType() instanceof ParameterizedType) {
-        ParameterizedType ptype = (ParameterizedType) map.getType();
-        java.lang.reflect.Type[] params = ptype.getActualTypeArguments();
-        mapType = TypeDescriptor.of(params[index]);
-      } else {
-        throw new RuntimeException("Map type is not parameterized! " + map);
-      }
-    }
+    TypeDescriptor mapType = ReflectUtils.getMapType(valueType, index);
     if (mapType == null) {
       return null;
     }
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/FromRowUsingCreator.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/FromRowUsingCreator.java
index b1b8ee8..61c0d05 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/FromRowUsingCreator.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/FromRowUsingCreator.java
@@ -21,7 +21,7 @@
 import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkState;
 
 import java.lang.reflect.Type;
-import java.util.Iterator;
+import java.util.Collection;
 import java.util.List;
 import java.util.Map;
 import java.util.Objects;
@@ -31,6 +31,9 @@
 import org.apache.beam.sdk.transforms.SerializableFunction;
 import org.apache.beam.sdk.values.Row;
 import org.apache.beam.sdk.values.RowWithGetters;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Function;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Collections2;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Lists;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Maps;
 
@@ -107,7 +110,8 @@
       return (ValueT) fromRow((Row) value, (Class) fieldType, typeFactory);
     } else if (TypeName.ARRAY.equals(type.getTypeName())) {
       return (ValueT)
-          fromListValue(type.getCollectionElementType(), (List) value, elementType, typeFactory);
+          fromCollectionValue(
+              type.getCollectionElementType(), (Collection) value, elementType, typeFactory);
     } else if (TypeName.ITERABLE.equals(type.getTypeName())) {
       return (ValueT)
           fromIterableValue(
@@ -127,25 +131,35 @@
     }
   }
 
+  private static <SourceT, DestT> Collection<DestT> transformCollection(
+      Collection<SourceT> collection, Function<SourceT, DestT> function) {
+    if (collection instanceof List) {
+      // For performance reasons if the input is a list, make sure that we produce a list. Otherwise
+      // Row unwrapping
+      // is forced to physically copy the collection into a new List object.
+      return Lists.transform((List) collection, function);
+    } else {
+      return Collections2.transform(collection, function);
+    }
+  }
+
   @SuppressWarnings("unchecked")
-  private <ElementT> List fromListValue(
+  private <ElementT> Collection fromCollectionValue(
       FieldType elementType,
-      List<ElementT> rowList,
+      Collection<ElementT> rowCollection,
       FieldValueTypeInformation elementTypeInformation,
       Factory<List<FieldValueTypeInformation>> typeFactory) {
-    List list = Lists.newArrayList();
-    for (ElementT element : rowList) {
-      list.add(
-          fromValue(
-              elementType,
-              element,
-              elementTypeInformation.getType().getType(),
-              elementTypeInformation.getElementType(),
-              elementTypeInformation.getMapKeyType(),
-              elementTypeInformation.getMapValueType(),
-              typeFactory));
-    }
-    return list;
+    return transformCollection(
+        rowCollection,
+        element ->
+            fromValue(
+                elementType,
+                element,
+                elementTypeInformation.getType().getType(),
+                elementTypeInformation.getElementType(),
+                elementTypeInformation.getMapKeyType(),
+                elementTypeInformation.getMapValueType(),
+                typeFactory));
   }
 
   @SuppressWarnings("unchecked")
@@ -154,32 +168,17 @@
       Iterable<ElementT> rowIterable,
       FieldValueTypeInformation elementTypeInformation,
       Factory<List<FieldValueTypeInformation>> typeFactory) {
-    return new Iterable<ElementT>() {
-      @Override
-      public Iterator<ElementT> iterator() {
-        return new Iterator<ElementT>() {
-          Iterator<ElementT> innerIter = rowIterable.iterator();
-
-          @Override
-          public boolean hasNext() {
-            return innerIter.hasNext();
-          }
-
-          @Override
-          public ElementT next() {
-            ElementT element = innerIter.next();
-            return fromValue(
+    return Iterables.transform(
+        rowIterable,
+        element ->
+            fromValue(
                 elementType,
                 element,
                 elementTypeInformation.getType().getType(),
                 elementTypeInformation.getElementType(),
                 elementTypeInformation.getMapKeyType(),
                 elementTypeInformation.getMapValueType(),
-                typeFactory);
-          }
-        };
-      }
-    };
+                typeFactory));
   }
 
   @SuppressWarnings("unchecked")
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/LogicalTypes.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/LogicalTypes.java
deleted file mode 100644
index 8aead16..0000000
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/LogicalTypes.java
+++ /dev/null
@@ -1,114 +0,0 @@
-/*
- * 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.schemas;
-
-import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument;
-
-import java.util.Arrays;
-import org.apache.beam.sdk.schemas.Schema.FieldType;
-import org.apache.beam.sdk.schemas.Schema.LogicalType;
-
-/** A collection of common {@link Schema.LogicalType} classes. */
-public class LogicalTypes {
-  /** A base class for LogicalTypes that use the same Java type as the underlying base type. */
-  public abstract static class PassThroughLogicalType<T> implements LogicalType<T, T> {
-    private final String identifier;
-    private final String argument;
-    private final FieldType fieldType;
-
-    protected PassThroughLogicalType(String identifier, String argument, FieldType fieldType) {
-      this.identifier = identifier;
-      this.argument = argument;
-      this.fieldType = fieldType;
-    }
-
-    @Override
-    public String getIdentifier() {
-      return identifier;
-    }
-
-    @Override
-    public String getArgument() {
-      return argument;
-    }
-
-    @Override
-    public FieldType getBaseType() {
-      return fieldType;
-    }
-
-    @Override
-    public T toBaseType(T input) {
-      return input;
-    }
-
-    @Override
-    public T toInputType(T base) {
-      return base;
-    }
-  }
-
-  /** A LogicalType representing a fixed-size byte array. */
-  public static class FixedBytes implements LogicalType<byte[], byte[]> {
-    public static final String IDENTIFIER = "FixedBytes";
-    private final int byteArraySize;
-
-    private FixedBytes(int byteArraySize) {
-      this.byteArraySize = byteArraySize;
-    }
-
-    public static FixedBytes of(int byteArraySize) {
-      return new FixedBytes(byteArraySize);
-    }
-
-    public int getLength() {
-      return byteArraySize;
-    }
-
-    @Override
-    public String getIdentifier() {
-      return IDENTIFIER;
-    }
-
-    @Override
-    public String getArgument() {
-      return Integer.toString(byteArraySize);
-    }
-
-    @Override
-    public FieldType getBaseType() {
-      return FieldType.BYTES;
-    }
-
-    @Override
-    public byte[] toBaseType(byte[] input) {
-      checkArgument(input.length == byteArraySize);
-      return input;
-    }
-
-    @Override
-    public byte[] toInputType(byte[] base) {
-      checkArgument(base.length <= byteArraySize);
-      if (base.length == byteArraySize) {
-        return base;
-      } else {
-        return Arrays.copyOf(base, byteArraySize);
-      }
-    }
-  }
-}
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/Schema.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/Schema.java
index affaae6..c998037 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/Schema.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/Schema.java
@@ -75,6 +75,11 @@
     public int hashCode() {
       return Arrays.hashCode(array);
     }
+
+    @Override
+    public String toString() {
+      return Arrays.toString(array);
+    }
   }
   // A mapping between field names an indices.
   private final BiMap<String, Integer> fieldIndices = HashBiMap.create();
@@ -497,15 +502,18 @@
     /** The unique identifier for this type. */
     String getIdentifier();
 
+    /** A schema type representing how to interpret the argument. */
+    FieldType getArgumentType();
+
     /** An optional argument to configure the type. */
-    default String getArgument() {
-      return "";
+    @SuppressWarnings("TypeParameterUnusedInFormals")
+    default <T> T getArgument() {
+      return null;
     }
 
     /** The base {@link FieldType} used to store values of this type. */
     FieldType getBaseType();
 
-    /** Convert the input Java type to one appropriate for the base {@link FieldType}. */
     BaseT toBaseType(InputT input);
 
     /** Convert the Java type used by the base {@link FieldType} to the input type. */
@@ -665,11 +673,7 @@
     /** Creates a logical type based on a primitive field type. */
     public static final <InputT, BaseT> FieldType logicalType(
         LogicalType<InputT, BaseT> logicalType) {
-      return FieldType.forTypeName(TypeName.LOGICAL_TYPE)
-          .setLogicalType(logicalType)
-          .build()
-          .withMetadata(LOGICAL_TYPE_IDENTIFIER, logicalType.getIdentifier())
-          .withMetadata(LOGICAL_TYPE_ARGUMENT, logicalType.getArgument());
+      return FieldType.forTypeName(TypeName.LOGICAL_TYPE).setLogicalType(logicalType).build();
     }
 
     /** Set the metadata map for the type, overriding any existing metadata.. */
@@ -720,10 +724,26 @@
       if (!(o instanceof FieldType)) {
         return false;
       }
-      // Logical type not included here, since the logical type identifier is included in the
-      // metadata. The LogicalType class is cached in this object just for convenience.
-      // TODO: this is wrong, since LogicalTypes have metadata associated.
+
       FieldType other = (FieldType) o;
+      if (getTypeName().isLogicalType()) {
+        if (!other.getTypeName().isLogicalType()) {
+          return false;
+        }
+        if (!Objects.equals(
+            getLogicalType().getIdentifier(), other.getLogicalType().getIdentifier())) {
+          return false;
+        }
+        if (!getLogicalType().getArgumentType().equals(other.getLogicalType().getArgumentType())) {
+          return false;
+        }
+        if (!Row.Equals.deepEquals(
+            getLogicalType().getArgument(),
+            other.getLogicalType().getArgument(),
+            getLogicalType().getArgumentType())) {
+          return false;
+        }
+      }
       return Objects.equals(getTypeName(), other.getTypeName())
           && Objects.equals(getNullable(), other.getNullable())
           && Objects.equals(getCollectionElementType(), other.getCollectionElementType())
@@ -738,6 +758,24 @@
       if (!Objects.equals(getTypeName(), other.getTypeName())) {
         return false;
       }
+      if (getTypeName().isLogicalType()) {
+        if (!other.getTypeName().isLogicalType()) {
+          return false;
+        }
+        if (!Objects.equals(
+            getLogicalType().getIdentifier(), other.getLogicalType().getIdentifier())) {
+          return false;
+        }
+        if (!getLogicalType().getArgumentType().equals(other.getLogicalType().getArgumentType())) {
+          return false;
+        }
+        if (!Row.Equals.deepEquals(
+            getLogicalType().getArgument(),
+            other.getLogicalType().getArgument(),
+            getLogicalType().getArgumentType())) {
+          return false;
+        }
+      }
       if (!Objects.equals(getMetadata(), other.getMetadata())) {
         return false;
       }
diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SchemaTranslation.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/SchemaTranslation.java
similarity index 98%
rename from runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SchemaTranslation.java
rename to sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/SchemaTranslation.java
index 78fbbd5..cea324a 100644
--- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SchemaTranslation.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/SchemaTranslation.java
@@ -15,12 +15,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.beam.runners.core.construction;
+package org.apache.beam.sdk.schemas;
 
 import java.util.Map;
 import java.util.UUID;
 import org.apache.beam.model.pipeline.v1.SchemaApi;
-import org.apache.beam.sdk.schemas.Schema;
 import org.apache.beam.sdk.schemas.Schema.Field;
 import org.apache.beam.sdk.schemas.Schema.FieldType;
 import org.apache.beam.sdk.schemas.Schema.LogicalType;
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/logicaltypes/EnumerationType.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/logicaltypes/EnumerationType.java
new file mode 100644
index 0000000..172992b
--- /dev/null
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/logicaltypes/EnumerationType.java
@@ -0,0 +1,165 @@
+/*
+ * 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.schemas.logicaltypes;
+
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Objects;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+import org.apache.beam.sdk.schemas.Schema.FieldType;
+import org.apache.beam.sdk.schemas.Schema.LogicalType;
+import org.apache.beam.sdk.schemas.logicaltypes.EnumerationType.Value;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.BiMap;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.HashBiMap;
+
+/** This {@link LogicalType} represent an enumeration over a fixed set of values. */
+public class EnumerationType implements LogicalType<Value, Integer> {
+  public static final String IDENTIFIER = "Enum";
+  final BiMap<String, Integer> enumValues = HashBiMap.create();
+  final List<String> values;
+
+  private EnumerationType(Map<String, Integer> enumValues) {
+    this.enumValues.putAll(enumValues);
+    values =
+        enumValues.entrySet().stream()
+            .sorted(Comparator.comparingInt(e -> e.getValue()))
+            .map(Entry::getKey)
+            .collect(Collectors.toList());
+  }
+
+  /** Create an enumeration type over a set of String->Integer values. */
+  public static EnumerationType create(Map<String, Integer> enumValues) {
+    return new EnumerationType(enumValues);
+  }
+
+  /**
+   * Create an enumeration type from a fixed set of String values; integer values will be
+   * automatically chosen.
+   */
+  public static EnumerationType create(List<String> enumValues) {
+    return new EnumerationType(
+        IntStream.range(0, enumValues.size())
+            .boxed()
+            .collect(Collectors.toMap(i -> enumValues.get(i), i -> i)));
+  }
+
+  /**
+   * Create an enumeration type from a fixed set of String values; integer values will be
+   * automatically chosen.
+   */
+  public static EnumerationType create(String... enumValues) {
+    return create(Arrays.asList(enumValues));
+  }
+  /** Return an {@link Value} corresponding to one of the enumeration strings. */
+  public Value valueOf(String stringValue) {
+    return new Value(stringValue, enumValues.get(stringValue));
+  }
+
+  /** Return an {@link Value} corresponding to one of the enumeration integer values. */
+  public Value valueOf(int value) {
+    return new Value(enumValues.inverse().get(value), value);
+  }
+
+  @Override
+  public String getIdentifier() {
+    return IDENTIFIER;
+  }
+
+  @Override
+  public FieldType getArgumentType() {
+    return FieldType.map(FieldType.STRING, FieldType.INT32);
+  }
+
+  @Override
+  public Map<String, Integer> getArgument() {
+    return enumValues;
+  }
+
+  @Override
+  public FieldType getBaseType() {
+    return FieldType.INT32;
+  }
+
+  @Override
+  public Integer toBaseType(Value input) {
+    return input.getValue();
+  }
+
+  @Override
+  public Value toInputType(Integer base) {
+    return valueOf(base);
+  }
+
+  public Map<String, Integer> getValuesMap() {
+    return enumValues;
+  }
+
+  public List<String> getValues() {
+    return values;
+  }
+
+  @Override
+  public String toString() {
+    return "Enumeration: " + enumValues;
+  }
+
+  /**
+   * This class represents a single enum value. It can be referenced as a String or as an integer.
+   */
+  public static class Value {
+    private final String stringValue;
+    private final int value;
+
+    public Value(String stringValue, int value) {
+      this.stringValue = stringValue;
+      this.value = value;
+    }
+
+    /** Return the integer enum value. */
+    public int getValue() {
+      return value;
+    }
+
+    /** Return the String enum value. */
+    @Override
+    public String toString() {
+      return stringValue;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+      Value enumValue = (Value) o;
+      return value == enumValue.value && Objects.equals(stringValue, enumValue.stringValue);
+    }
+
+    @Override
+    public int hashCode() {
+      return Objects.hash(stringValue, value);
+    }
+  }
+}
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/logicaltypes/FixedBytes.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/logicaltypes/FixedBytes.java
new file mode 100644
index 0000000..ba8221b
--- /dev/null
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/logicaltypes/FixedBytes.java
@@ -0,0 +1,83 @@
+/*
+ * 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.schemas.logicaltypes;
+
+import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument;
+
+import java.util.Arrays;
+import org.apache.beam.sdk.schemas.Schema.FieldType;
+import org.apache.beam.sdk.schemas.Schema.LogicalType;
+
+/** A LogicalType representing a fixed-size byte array. */
+public class FixedBytes implements LogicalType<byte[], byte[]> {
+  public static final String IDENTIFIER = "FixedBytes";
+  private final int byteArraySize;
+
+  private FixedBytes(int byteArraySize) {
+    this.byteArraySize = byteArraySize;
+  }
+
+  public static FixedBytes of(int byteArraySize) {
+    return new FixedBytes(byteArraySize);
+  }
+
+  public int getLength() {
+    return byteArraySize;
+  }
+
+  @Override
+  public String getIdentifier() {
+    return IDENTIFIER;
+  }
+
+  @Override
+  public FieldType getArgumentType() {
+    return FieldType.INT32;
+  }
+
+  @Override
+  public Integer getArgument() {
+    return byteArraySize;
+  }
+
+  @Override
+  public FieldType getBaseType() {
+    return FieldType.BYTES;
+  }
+
+  @Override
+  public byte[] toBaseType(byte[] input) {
+    checkArgument(input.length == byteArraySize);
+    return input;
+  }
+
+  @Override
+  public byte[] toInputType(byte[] base) {
+    checkArgument(base.length <= byteArraySize);
+    if (base.length == byteArraySize) {
+      return base;
+    } else {
+      return Arrays.copyOf(base, byteArraySize);
+    }
+  }
+
+  @Override
+  public String toString() {
+    return "FixedBytes: " + byteArraySize;
+  }
+}
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/logicaltypes/OneOfType.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/logicaltypes/OneOfType.java
new file mode 100644
index 0000000..214eeb5
--- /dev/null
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/logicaltypes/OneOfType.java
@@ -0,0 +1,172 @@
+/*
+ * 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.schemas.logicaltypes;
+
+import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument;
+import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkNotNull;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.stream.Collectors;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.schemas.Schema.Field;
+import org.apache.beam.sdk.schemas.Schema.FieldType;
+import org.apache.beam.sdk.schemas.Schema.LogicalType;
+import org.apache.beam.sdk.schemas.SchemaTranslation;
+import org.apache.beam.sdk.values.Row;
+
+/**
+ * A logical type representing a union of fields. This logical type is initialized with a set of
+ * field and represents a union of those fields. This logical type is backed by a Row object
+ * containing one nullable field matching each input field, and one additional {@link
+ * EnumerationType} logical type field that indicates which field is set.
+ */
+public class OneOfType implements LogicalType<OneOfType.Value, Row> {
+  public static final String IDENTIFIER = "OneOf";
+
+  private final Schema oneOfSchema;
+  private final EnumerationType enumerationType;
+  private final byte[] schemaProtoRepresentation;
+
+  private OneOfType(List<Field> fields) {
+    List<Field> nullableFields =
+        fields.stream()
+            .map(f -> Field.nullable(f.getName(), f.getType()))
+            .collect(Collectors.toList());
+    List<String> enumValues =
+        nullableFields.stream().map(Field::getName).collect(Collectors.toList());
+    oneOfSchema = Schema.builder().addFields(nullableFields).build();
+    enumerationType = EnumerationType.create(enumValues);
+    schemaProtoRepresentation = SchemaTranslation.schemaToProto(oneOfSchema).toByteArray();
+  }
+
+  /** Create an {@link OneOfType} logical type. */
+  public static OneOfType create(Field... fields) {
+    return create(Arrays.asList(fields));
+  }
+
+  /** Create an {@link OneOfType} logical type. */
+  public static OneOfType create(List<Field> fields) {
+    return new OneOfType(fields);
+  }
+
+  /** Returns the schema of the underlying {@link Row} that is used to represent the union. */
+  public Schema getOneOfSchema() {
+    return oneOfSchema;
+  }
+
+  /** Returns the {@link EnumerationType} that is used to represent the case type. */
+  public EnumerationType getCaseEnumType() {
+    return enumerationType;
+  }
+
+  @Override
+  public String getIdentifier() {
+    return IDENTIFIER;
+  }
+
+  @Override
+  public FieldType getArgumentType() {
+    return FieldType.BYTES;
+  }
+
+  @Override
+  public byte[] getArgument() {
+    return schemaProtoRepresentation;
+  }
+
+  @Override
+  public FieldType getBaseType() {
+    return FieldType.row(oneOfSchema);
+  }
+
+  /** Create a {@link Value} specifying which field to set and the value to set. */
+  public <T> Value createValue(String caseType, T value) {
+    return createValue(getCaseEnumType().valueOf(caseType), value);
+  }
+
+  /** Create a {@link Value} specifying which field to set and the value to set. */
+  public <T> Value createValue(EnumerationType.Value caseType, T value) {
+    return new Value(caseType, oneOfSchema.getField(caseType.toString()).getType(), value);
+  }
+
+  @Override
+  public Row toBaseType(Value input) {
+    EnumerationType.Value caseType = input.getCaseType();
+    int setFieldIndex = oneOfSchema.indexOf(caseType.toString());
+    Row.Builder builder = Row.withSchema(oneOfSchema);
+    for (int i = 0; i < oneOfSchema.getFieldCount(); ++i) {
+      Object value = (i == setFieldIndex) ? input.getValue() : null;
+      builder = builder.addValue(value);
+    }
+    return builder.build();
+  }
+
+  @Override
+  public Value toInputType(Row base) {
+    EnumerationType.Value caseType = null;
+    Object oneOfValue = null;
+    for (int i = 0; i < base.getFieldCount(); ++i) {
+      Object value = base.getValue(i);
+      if (value != null) {
+        checkArgument(caseType == null, "More than one field set in union " + this);
+        caseType = enumerationType.valueOf(oneOfSchema.getField(i).getName());
+        oneOfValue = value;
+      }
+    }
+    checkNotNull(oneOfValue, "No value set in union" + this);
+    return createValue(caseType, oneOfValue);
+  }
+
+  @Override
+  public String toString() {
+    return "OneOf: " + oneOfSchema;
+  }
+
+  /**
+   * Represents a single OneOf value. Each object contains an {@link EnumerationType.Value}
+   * specifying which field is set along with the value of that field.
+   */
+  public static class Value {
+    private final EnumerationType.Value caseType;
+    private final FieldType fieldType;
+    private final Object value;
+
+    public Value(EnumerationType.Value caseType, FieldType fieldType, Object value) {
+      this.caseType = caseType;
+      this.fieldType = fieldType;
+      this.value = value;
+    }
+
+    /** Returns the enumeration that specified which OneOf field is set. */
+    public EnumerationType.Value getCaseType() {
+      return caseType;
+    }
+
+    /** Returns the current value of the OneOf. */
+    @SuppressWarnings("TypeParameterUnusedInFormals")
+    public <T> T getValue() {
+      return (T) value;
+    }
+
+    /** Return the type of this union field. */
+    public FieldType getFieldType() {
+      return fieldType;
+    }
+  }
+}
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/logicaltypes/PassThroughLogicalType.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/logicaltypes/PassThroughLogicalType.java
new file mode 100644
index 0000000..db03a52
--- /dev/null
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/logicaltypes/PassThroughLogicalType.java
@@ -0,0 +1,68 @@
+/*
+ * 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.schemas.logicaltypes;
+
+import org.apache.beam.sdk.schemas.Schema.FieldType;
+import org.apache.beam.sdk.schemas.Schema.LogicalType;
+
+/** A base class for LogicalTypes that use the same Java type as the underlying base type. */
+public abstract class PassThroughLogicalType<T> implements LogicalType<T, T> {
+  private final String identifier;
+  private final FieldType argumentType;
+  private final Object argument;
+  private final FieldType fieldType;
+
+  protected PassThroughLogicalType(
+      String identifier, FieldType argumentType, Object argument, FieldType fieldType) {
+    this.identifier = identifier;
+    this.argumentType = argumentType;
+    this.argument = argument;
+    this.fieldType = fieldType;
+  }
+
+  @Override
+  public String getIdentifier() {
+    return identifier;
+  }
+
+  @Override
+  public FieldType getArgumentType() {
+    return argumentType;
+  }
+
+  @Override
+  @SuppressWarnings("TypeParameterUnusedInFormals")
+  public <ArgumentT> ArgumentT getArgument() {
+    return (ArgumentT) argument;
+  }
+
+  @Override
+  public FieldType getBaseType() {
+    return fieldType;
+  }
+
+  @Override
+  public T toBaseType(T input) {
+    return input;
+  }
+
+  @Override
+  public T toInputType(T base) {
+    return base;
+  }
+}
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/logicaltypes/package-info.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/logicaltypes/package-info.java
new file mode 100644
index 0000000..2a6b018
--- /dev/null
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/logicaltypes/package-info.java
@@ -0,0 +1,27 @@
+/*
+ * 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 set of common LogicalTypes for use with schemas.
+ *
+ * <p>For further details, see the documentation for each class in this package.
+ */
+@DefaultAnnotation(NonNull.class)
+package org.apache.beam.sdk.schemas.logicaltypes;
+
+import edu.umd.cs.findbugs.annotations.DefaultAnnotation;
+import edu.umd.cs.findbugs.annotations.NonNull;
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/AvroUtils.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/AvroUtils.java
index ecd5906..80c337e 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/AvroUtils.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/AvroUtils.java
@@ -50,13 +50,14 @@
 import org.apache.beam.sdk.schemas.AvroRecordSchema;
 import org.apache.beam.sdk.schemas.FieldValueGetter;
 import org.apache.beam.sdk.schemas.FieldValueTypeInformation;
-import org.apache.beam.sdk.schemas.LogicalTypes.FixedBytes;
 import org.apache.beam.sdk.schemas.Schema;
 import org.apache.beam.sdk.schemas.Schema.Field;
 import org.apache.beam.sdk.schemas.Schema.FieldType;
 import org.apache.beam.sdk.schemas.Schema.TypeName;
 import org.apache.beam.sdk.schemas.SchemaCoder;
 import org.apache.beam.sdk.schemas.SchemaUserTypeCreator;
+import org.apache.beam.sdk.schemas.logicaltypes.EnumerationType;
+import org.apache.beam.sdk.schemas.logicaltypes.FixedBytes;
 import org.apache.beam.sdk.schemas.utils.ByteBuddyUtils.ConvertType;
 import org.apache.beam.sdk.schemas.utils.ByteBuddyUtils.ConvertValueForGetter;
 import org.apache.beam.sdk.schemas.utils.ByteBuddyUtils.ConvertValueForSetter;
@@ -612,7 +613,7 @@
           break;
 
         case ENUM:
-          fieldType = Schema.FieldType.STRING;
+          fieldType = FieldType.logicalType(EnumerationType.create(type.type.getEnumSymbols()));
           break;
 
         case ARRAY:
@@ -661,10 +662,10 @@
           break;
 
         case UNION:
-          throw new RuntimeException("Can't convert 'union' to FieldType");
+          throw new IllegalArgumentException("Union types not yet supported");
 
         case NULL:
-          throw new RuntimeException("Can't convert 'null' to FieldType");
+          throw new IllegalArgumentException("Can't convert 'null' to FieldType");
 
         default:
           throw new AssertionError("Unexpected AVRO Schema.Type: " + avroSchema.getType());
@@ -722,12 +723,20 @@
         break;
 
       case LOGICAL_TYPE:
-        FixedBytesField fixedBytesField = FixedBytesField.fromBeamFieldType(fieldType);
-        if (fixedBytesField != null) {
-          baseType = fixedBytesField.toAvroType("fixed", namespace + "." + fieldName);
-        } else {
-          throw new RuntimeException(
-              "Unhandled logical type " + fieldType.getLogicalType().getIdentifier());
+        switch (fieldType.getLogicalType().getIdentifier()) {
+          case FixedBytes.IDENTIFIER:
+            FixedBytesField fixedBytesField =
+                checkNotNull(FixedBytesField.fromBeamFieldType(fieldType));
+            baseType = fixedBytesField.toAvroType("fixed", namespace + "." + fieldName);
+            break;
+          case EnumerationType.IDENTIFIER:
+            EnumerationType enumerationType = fieldType.getLogicalType(EnumerationType.class);
+            baseType =
+                org.apache.avro.Schema.createEnum(fieldName, "", "", enumerationType.getValues());
+            break;
+          default:
+            throw new RuntimeException(
+                "Unhandled logical type " + fieldType.getLogicalType().getIdentifier());
         }
         break;
 
@@ -810,16 +819,24 @@
         return ByteBuffer.wrap((byte[]) value);
 
       case LOGICAL_TYPE:
-        FixedBytesField fixedBytesField = FixedBytesField.fromBeamFieldType(fieldType);
-        if (fixedBytesField != null) {
-          byte[] byteArray = (byte[]) value;
-          if (byteArray.length != fixedBytesField.getSize()) {
-            throw new IllegalArgumentException("Incorrectly sized byte array.");
-          }
-          return GenericData.get().createFixed(null, (byte[]) value, typeWithNullability.type);
+        switch (fieldType.getLogicalType().getIdentifier()) {
+          case FixedBytes.IDENTIFIER:
+            FixedBytesField fixedBytesField =
+                checkNotNull(FixedBytesField.fromBeamFieldType(fieldType));
+            byte[] byteArray = (byte[]) value;
+            if (byteArray.length != fixedBytesField.getSize()) {
+              throw new IllegalArgumentException("Incorrectly sized byte array.");
+            }
+            return GenericData.get().createFixed(null, (byte[]) value, typeWithNullability.type);
+          case EnumerationType.IDENTIFIER:
+            EnumerationType enumerationType = fieldType.getLogicalType(EnumerationType.class);
+            return GenericData.get()
+                .createEnum(
+                    enumerationType.valueOf((int) value).toString(), typeWithNullability.type);
+          default:
+            throw new RuntimeException(
+                "Unhandled logical type " + fieldType.getLogicalType().getIdentifier());
         }
-        throw new RuntimeException(
-            "Unknown logical type " + fieldType.getLogicalType().getIdentifier());
 
       case ARRAY:
       case ITERABLE:
@@ -942,8 +959,7 @@
             (Map<CharSequence, Object>) value, type.type.getValueType(), fieldType);
 
       case UNION:
-        throw new IllegalArgumentException(
-            "Can't convert 'union', only nullable fields are supported");
+        throw new IllegalArgumentException("Union types not yet supported");
 
       case NULL:
         throw new IllegalArgumentException("Can't convert 'null' to non-nullable field");
@@ -1018,8 +1034,10 @@
   }
 
   private static Object convertEnumStrict(Object value, Schema.FieldType fieldType) {
-    checkTypeName(fieldType.getTypeName(), Schema.TypeName.STRING, "enum");
-    return value.toString();
+    checkTypeName(fieldType.getTypeName(), TypeName.LOGICAL_TYPE, "enum");
+    checkArgument(fieldType.getLogicalType().getIdentifier().equals(EnumerationType.IDENTIFIER));
+    EnumerationType enumerationType = fieldType.getLogicalType(EnumerationType.class);
+    return enumerationType.valueOf(value.toString());
   }
 
   private static Object convertArrayStrict(
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/ByteBuddyUtils.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/ByteBuddyUtils.java
index 9cc2fb4..791dafb 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/ByteBuddyUtils.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/ByteBuddyUtils.java
@@ -31,6 +31,8 @@
 import java.util.Collections;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
+import java.util.SortedMap;
 import org.apache.beam.sdk.schemas.FieldValueGetter;
 import org.apache.beam.sdk.schemas.FieldValueSetter;
 import org.apache.beam.sdk.schemas.FieldValueTypeInformation;
@@ -44,6 +46,7 @@
 import org.apache.beam.vendor.bytebuddy.v1_9_3.net.bytebuddy.description.type.TypeDescription;
 import org.apache.beam.vendor.bytebuddy.v1_9_3.net.bytebuddy.description.type.TypeDescription.ForLoadedType;
 import org.apache.beam.vendor.bytebuddy.v1_9_3.net.bytebuddy.dynamic.DynamicType;
+import org.apache.beam.vendor.bytebuddy.v1_9_3.net.bytebuddy.dynamic.loading.ClassLoadingStrategy;
 import org.apache.beam.vendor.bytebuddy.v1_9_3.net.bytebuddy.dynamic.scaffold.InstrumentedType;
 import org.apache.beam.vendor.bytebuddy.v1_9_3.net.bytebuddy.implementation.Implementation;
 import org.apache.beam.vendor.bytebuddy.v1_9_3.net.bytebuddy.implementation.bytecode.ByteCodeAppender;
@@ -64,8 +67,12 @@
 import org.apache.beam.vendor.bytebuddy.v1_9_3.net.bytebuddy.implementation.bytecode.member.MethodVariableAccess;
 import org.apache.beam.vendor.bytebuddy.v1_9_3.net.bytebuddy.matcher.ElementMatchers;
 import org.apache.beam.vendor.bytebuddy.v1_9_3.net.bytebuddy.utility.RandomString;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Function;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Collections2;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Lists;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Maps;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.primitives.Primitives;
 import org.apache.commons.lang3.ArrayUtils;
 import org.apache.commons.lang3.ClassUtils;
 import org.joda.time.DateTimeZone;
@@ -82,12 +89,16 @@
   private static final ForLoadedType CHAR_SEQUENCE_TYPE = new ForLoadedType(CharSequence.class);
   private static final ForLoadedType INSTANT_TYPE = new ForLoadedType(Instant.class);
   private static final ForLoadedType DATE_TIME_ZONE_TYPE = new ForLoadedType(DateTimeZone.class);
-  private static final ForLoadedType LIST_TYPE = new ForLoadedType(List.class);
+  private static final ForLoadedType COLLECTION_TYPE = new ForLoadedType(Collection.class);
   private static final ForLoadedType READABLE_INSTANT_TYPE =
       new ForLoadedType(ReadableInstant.class);
   private static final ForLoadedType READABLE_PARTIAL_TYPE =
       new ForLoadedType(ReadablePartial.class);
   private static final ForLoadedType OBJECT_TYPE = new ForLoadedType(Object.class);
+  private static final ForLoadedType INTEGER_TYPE = new ForLoadedType(Integer.class);
+  private static final ForLoadedType ENUM_TYPE = new ForLoadedType(Enum.class);
+  private static final ForLoadedType BYTE_BUDDY_UTILS_TYPE =
+      new ForLoadedType(ByteBuddyUtils.class);
 
   /**
    * A naming strategy for ByteBuddy classes.
@@ -96,7 +107,7 @@
    * This way, if the class fields or methods are package private, our generated class can still
    * access them.
    */
-  static class InjectPackageStrategy extends NamingStrategy.AbstractBase {
+  public static class InjectPackageStrategy extends NamingStrategy.AbstractBase {
     /** A resolver for the base name for naming the unnamed type. */
     private static final BaseNameResolver baseNameResolver =
         BaseNameResolver.ForUnnamedType.INSTANCE;
@@ -121,6 +132,30 @@
     }
   };
 
+  // Create a new FieldValueGetter subclass.
+  @SuppressWarnings("unchecked")
+  static DynamicType.Builder<FieldValueGetter> subclassGetterInterface(
+      ByteBuddy byteBuddy, Type objectType, Type fieldType) {
+    TypeDescription.Generic getterGenericType =
+        TypeDescription.Generic.Builder.parameterizedType(
+                FieldValueGetter.class, objectType, fieldType)
+            .build();
+    return (DynamicType.Builder<FieldValueGetter>)
+        byteBuddy.with(new InjectPackageStrategy((Class) objectType)).subclass(getterGenericType);
+  }
+
+  // Create a new FieldValueSetter subclass.
+  @SuppressWarnings("unchecked")
+  static DynamicType.Builder<FieldValueSetter> subclassSetterInterface(
+      ByteBuddy byteBuddy, Type objectType, Type fieldType) {
+    TypeDescription.Generic setterGenericType =
+        TypeDescription.Generic.Builder.parameterizedType(
+                FieldValueSetter.class, objectType, fieldType)
+            .build();
+    return (DynamicType.Builder<FieldValueSetter>)
+        byteBuddy.with(new InjectPackageStrategy((Class) objectType)).subclass(setterGenericType);
+  }
+
   public interface TypeConversionsFactory {
     TypeConversion<Type> createTypeConversion(boolean returnRawTypes);
 
@@ -146,30 +181,6 @@
     }
   }
 
-  // Create a new FieldValueGetter subclass.
-  @SuppressWarnings("unchecked")
-  static DynamicType.Builder<FieldValueGetter> subclassGetterInterface(
-      ByteBuddy byteBuddy, Type objectType, Type fieldType) {
-    TypeDescription.Generic getterGenericType =
-        TypeDescription.Generic.Builder.parameterizedType(
-                FieldValueGetter.class, objectType, fieldType)
-            .build();
-    return (DynamicType.Builder<FieldValueGetter>)
-        byteBuddy.with(new InjectPackageStrategy((Class) objectType)).subclass(getterGenericType);
-  }
-
-  // Create a new FieldValueSetter subclass.
-  @SuppressWarnings("unchecked")
-  static DynamicType.Builder<FieldValueSetter> subclassSetterInterface(
-      ByteBuddy byteBuddy, Type objectType, Type fieldType) {
-    TypeDescription.Generic setterGenericType =
-        TypeDescription.Generic.Builder.parameterizedType(
-                FieldValueSetter.class, objectType, fieldType)
-            .build();
-    return (DynamicType.Builder<FieldValueSetter>)
-        byteBuddy.with(new InjectPackageStrategy((Class) objectType)).subclass(setterGenericType);
-  }
-
   // Base class used below to convert types.
   @SuppressWarnings("unchecked")
   public abstract static class TypeConversion<T> {
@@ -193,7 +204,9 @@
       } else if (typeDescriptor.getRawType().isEnum()) {
         return convertEnum(typeDescriptor);
       } else if (typeDescriptor.isSubtypeOf(TypeDescriptor.of(Iterable.class))) {
-        if (typeDescriptor.isSubtypeOf(TypeDescriptor.of(Collection.class))) {
+        if (typeDescriptor.isSubtypeOf(TypeDescriptor.of(List.class))) {
+          return convertList(typeDescriptor);
+        } else if (typeDescriptor.isSubtypeOf(TypeDescriptor.of(Collection.class))) {
           return convertCollection(typeDescriptor);
         } else {
           return convertIterable(typeDescriptor);
@@ -209,6 +222,8 @@
 
     protected abstract T convertCollection(TypeDescriptor<?> type);
 
+    protected abstract T convertList(TypeDescriptor<?> type);
+
     protected abstract T convertMap(TypeDescriptor<?> type);
 
     protected abstract T convertDateTime(TypeDescriptor<?> type);
@@ -251,18 +266,26 @@
 
     @Override
     protected Type convertArray(TypeDescriptor<?> type) {
-      TypeDescriptor ret = createListType(type);
+      TypeDescriptor ret = createCollectionType(type.getComponentType());
       return returnRawTypes ? ret.getRawType() : ret.getType();
     }
 
     @Override
     protected Type convertCollection(TypeDescriptor<?> type) {
-      return Collection.class;
+      TypeDescriptor ret = createCollectionType(ReflectUtils.getIterableComponentType(type));
+      return returnRawTypes ? ret.getRawType() : ret.getType();
+    }
+
+    @Override
+    protected Type convertList(TypeDescriptor<?> type) {
+      TypeDescriptor ret = createCollectionType(ReflectUtils.getIterableComponentType(type));
+      return returnRawTypes ? ret.getRawType() : ret.getType();
     }
 
     @Override
     protected Type convertIterable(TypeDescriptor<?> type) {
-      return Iterable.class;
+      TypeDescriptor ret = createIterableType(ReflectUtils.getIterableComponentType(type));
+      return returnRawTypes ? ret.getRawType() : ret.getType();
     }
 
     @Override
@@ -292,7 +315,9 @@
 
     @Override
     protected Type convertEnum(TypeDescriptor<?> type) {
-      return String.class;
+      // We represent enums in the Row as Integers. The EnumerationType handles the mapping to the
+      // actual enum type.
+      return Integer.class;
     }
 
     @Override
@@ -301,11 +326,190 @@
     }
 
     @SuppressWarnings("unchecked")
-    private <ElementT> TypeDescriptor<List<ElementT>> createListType(TypeDescriptor<?> type) {
-      TypeDescriptor componentType =
-          TypeDescriptor.of(ClassUtils.primitiveToWrapper(type.getComponentType().getRawType()));
-      return new TypeDescriptor<List<ElementT>>() {}.where(
-          new TypeParameter<ElementT>() {}, componentType);
+    private <ElementT> TypeDescriptor<Collection<ElementT>> createCollectionType(
+        TypeDescriptor<?> componentType) {
+      TypeDescriptor wrappedComponentType =
+          TypeDescriptor.of(ClassUtils.primitiveToWrapper(componentType.getRawType()));
+      return new TypeDescriptor<Collection<ElementT>>() {}.where(
+          new TypeParameter<ElementT>() {}, wrappedComponentType);
+    }
+
+    @SuppressWarnings("unchecked")
+    private <ElementT> TypeDescriptor<Iterable<ElementT>> createIterableType(
+        TypeDescriptor<?> componentType) {
+      TypeDescriptor wrappedComponentType =
+          TypeDescriptor.of(ClassUtils.primitiveToWrapper(componentType.getRawType()));
+      return new TypeDescriptor<Iterable<ElementT>>() {}.where(
+          new TypeParameter<ElementT>() {}, wrappedComponentType);
+    }
+  }
+
+  private static final ByteBuddy BYTE_BUDDY = new ByteBuddy();
+
+  // When processing a container (e.g. List<T>) we need to recursively process the element type.
+  // This function
+  // generates a subclass of Function that can be used to recursively transform each element of the
+  // container.
+  static Class createCollectionTransformFunction(
+      Type fromType, Type toType, Function<StackManipulation, StackManipulation> convertElement) {
+    // Generate a TypeDescription for the class we want to generate.
+    TypeDescription.Generic functionGenericType =
+        TypeDescription.Generic.Builder.parameterizedType(
+                Function.class, Primitives.wrap((Class) fromType), Primitives.wrap((Class) toType))
+            .build();
+
+    DynamicType.Builder<Function> builder =
+        (DynamicType.Builder<Function>)
+            BYTE_BUDDY
+                .subclass(functionGenericType)
+                .method(ElementMatchers.named("apply"))
+                .intercept(
+                    new Implementation() {
+                      @Override
+                      public ByteCodeAppender appender(Target target) {
+                        return (methodVisitor, implementationContext, instrumentedMethod) -> {
+                          // this + method parameters.
+                          int numLocals = 1 + instrumentedMethod.getParameters().size();
+
+                          StackManipulation readValue = MethodVariableAccess.REFERENCE.loadFrom(1);
+                          StackManipulation stackManipulation =
+                              new StackManipulation.Compound(
+                                  convertElement.apply(readValue), MethodReturn.REFERENCE);
+
+                          StackManipulation.Size size =
+                              stackManipulation.apply(methodVisitor, implementationContext);
+                          return new Size(size.getMaximalSize(), numLocals);
+                        };
+                      }
+
+                      @Override
+                      public InstrumentedType prepare(InstrumentedType instrumentedType) {
+                        return instrumentedType;
+                      }
+                    });
+
+    return builder
+        .make()
+        .load(ByteBuddyUtils.class.getClassLoader(), ClassLoadingStrategy.Default.INJECTION)
+        .getLoaded();
+  }
+
+  // A function to transform a container, special casing List and Collection types. This is used in
+  // byte-buddy
+  // generated code.
+  public static <FromT, DestT> Iterable<DestT> transformContainer(
+      Iterable<FromT> iterable, Function<FromT, DestT> function) {
+    if (iterable instanceof List) {
+      return Lists.transform((List<FromT>) iterable, function);
+    } else if (iterable instanceof Collection) {
+      return Collections2.transform((Collection<FromT>) iterable, function);
+    } else {
+      return Iterables.transform(iterable, function);
+    }
+  }
+
+  static StackManipulation createTransformingContainer(
+      ForLoadedType functionType, StackManipulation readValue) {
+    StackManipulation stackManipulation =
+        new Compound(
+            readValue,
+            TypeCreation.of(functionType),
+            Duplication.SINGLE,
+            MethodInvocation.invoke(
+                functionType
+                    .getDeclaredMethods()
+                    .filter(ElementMatchers.isConstructor().and(ElementMatchers.takesArguments(0)))
+                    .getOnly()),
+            MethodInvocation.invoke(
+                BYTE_BUDDY_UTILS_TYPE
+                    .getDeclaredMethods()
+                    .filter(ElementMatchers.named("transformContainer"))
+                    .getOnly()));
+    return stackManipulation;
+  }
+
+  public static <K1, V1, K2, V2> TransformingMap<K1, V1, K2, V2> getTransformingMap(
+      Map<K1, V1> sourceMap, Function<K1, K2> keyFunction, Function<V1, V2> valueFunction) {
+    return new TransformingMap<>(sourceMap, keyFunction, valueFunction);
+  }
+
+  public static class TransformingMap<K1, V1, K2, V2> implements Map<K2, V2> {
+    private final Map<K2, V2> delegateMap;
+
+    public TransformingMap(
+        Map<K1, V1> sourceMap, Function<K1, K2> keyFunction, Function<V1, V2> valueFunction) {
+      if (sourceMap instanceof SortedMap) {
+        delegateMap =
+            (Map<K2, V2>)
+                Maps.newTreeMap(); // We don't support copying the comparator. Makes no sense if key
+        // is changing.
+      } else {
+        delegateMap = Maps.newHashMap();
+      }
+      for (Map.Entry<K1, V1> entry : sourceMap.entrySet()) {
+        delegateMap.put(keyFunction.apply(entry.getKey()), valueFunction.apply(entry.getValue()));
+      }
+    }
+
+    @Override
+    public int size() {
+      return delegateMap.size();
+    }
+
+    @Override
+    public boolean isEmpty() {
+      return delegateMap.isEmpty();
+    }
+
+    @Override
+    public boolean containsKey(Object key) {
+      return delegateMap.containsKey(key);
+    }
+
+    @Override
+    public boolean containsValue(Object value) {
+      return delegateMap.containsValue(value);
+    }
+
+    @Override
+    public V2 get(Object key) {
+      return delegateMap.get(key);
+    }
+
+    @Override
+    public V2 put(K2 key, V2 value) {
+      return delegateMap.put(key, value);
+    }
+
+    @Override
+    public V2 remove(Object key) {
+      return delegateMap.remove(key);
+    }
+
+    @Override
+    public void putAll(Map<? extends K2, ? extends V2> m) {
+      delegateMap.putAll(m);
+    }
+
+    @Override
+    public void clear() {
+      delegateMap.clear();
+      ;
+    }
+
+    @Override
+    public Set<K2> keySet() {
+      return delegateMap.keySet();
+    }
+
+    @Override
+    public Collection<V2> values() {
+      return delegateMap.values();
+    }
+
+    @Override
+    public Set<Entry<K2, V2>> entrySet() {
+      return delegateMap.entrySet();
     }
   }
 
@@ -334,46 +538,153 @@
       // return isComponentTypePrimitive ? Arrays.asList(ArrayUtils.toObject(value))
       //     : Arrays.asList(value);
 
-      ForLoadedType loadedType = new ForLoadedType(type.getRawType());
-      StackManipulation stackManipulation = readValue;
+      TypeDescriptor<?> componentType = type.getComponentType();
+      ForLoadedType loadedArrayType = new ForLoadedType(type.getRawType());
+      StackManipulation readArrayValue = readValue;
       // Row always expects to get an Iterable back for array types. Wrap this array into a
       // List using Arrays.asList before returning.
-      if (loadedType.getComponentType().isPrimitive()) {
+      if (loadedArrayType.getComponentType().isPrimitive()) {
         // Arrays.asList doesn't take primitive arrays, so convert first using ArrayUtils.toObject.
-        stackManipulation =
+        readArrayValue =
             new Compound(
-                stackManipulation,
+                readArrayValue,
                 MethodInvocation.invoke(
                     ARRAY_UTILS_TYPE
                         .getDeclaredMethods()
                         .filter(
                             ElementMatchers.isStatic()
                                 .and(ElementMatchers.named("toObject"))
-                                .and(ElementMatchers.takesArguments(loadedType)))
+                                .and(ElementMatchers.takesArguments(loadedArrayType)))
                         .getOnly()));
+
+        componentType = TypeDescriptor.of(Primitives.wrap(componentType.getRawType()));
       }
-      return new Compound(
-          stackManipulation,
-          MethodInvocation.invoke(
-              ARRAYS_TYPE
-                  .getDeclaredMethods()
-                  .filter(ElementMatchers.isStatic().and(ElementMatchers.named("asList")))
-                  .getOnly()));
+      // Now convert to a List object.
+      StackManipulation readListValue =
+          new Compound(
+              readArrayValue,
+              MethodInvocation.invoke(
+                  ARRAYS_TYPE
+                      .getDeclaredMethods()
+                      .filter(ElementMatchers.isStatic().and(ElementMatchers.named("asList")))
+                      .getOnly()));
+
+      // Generate a SerializableFunction to convert the element-type objects.
+      final TypeDescriptor finalComponentType = ReflectUtils.boxIfPrimitive(componentType);
+      if (!finalComponentType.hasUnresolvedParameters()) {
+        Type convertedComponentType =
+            getFactory().createTypeConversion(true).convert(componentType);
+        ForLoadedType functionType =
+            new ForLoadedType(
+                createCollectionTransformFunction(
+                    componentType.getRawType(),
+                    convertedComponentType,
+                    (s) -> getFactory().createGetterConversions(s).convert(finalComponentType)));
+        return createTransformingContainer(functionType, readListValue);
+      } else {
+        return readListValue;
+      }
     }
 
     @Override
     protected StackManipulation convertIterable(TypeDescriptor<?> type) {
-      return readValue;
+      TypeDescriptor componentType = ReflectUtils.getIterableComponentType(type);
+      Type convertedComponentType = getFactory().createTypeConversion(true).convert(componentType);
+      final TypeDescriptor finalComponentType = ReflectUtils.boxIfPrimitive(componentType);
+      if (!finalComponentType.hasUnresolvedParameters()) {
+        ForLoadedType functionType =
+            new ForLoadedType(
+                createCollectionTransformFunction(
+                    componentType.getRawType(),
+                    convertedComponentType,
+                    (s) -> getFactory().createGetterConversions(s).convert(finalComponentType)));
+        return createTransformingContainer(functionType, readValue);
+      } else {
+        return readValue;
+      }
     }
 
     @Override
     protected StackManipulation convertCollection(TypeDescriptor<?> type) {
-      return readValue;
+      TypeDescriptor componentType = ReflectUtils.getIterableComponentType(type);
+      Type convertedComponentType = getFactory().createTypeConversion(true).convert(componentType);
+      final TypeDescriptor finalComponentType = ReflectUtils.boxIfPrimitive(componentType);
+      if (!finalComponentType.hasUnresolvedParameters()) {
+        ForLoadedType functionType =
+            new ForLoadedType(
+                createCollectionTransformFunction(
+                    componentType.getRawType(),
+                    convertedComponentType,
+                    (s) -> getFactory().createGetterConversions(s).convert(finalComponentType)));
+        return createTransformingContainer(functionType, readValue);
+      } else {
+        return readValue;
+      }
+    }
+
+    @Override
+    protected StackManipulation convertList(TypeDescriptor<?> type) {
+      TypeDescriptor componentType = ReflectUtils.getIterableComponentType(type);
+      Type convertedComponentType = getFactory().createTypeConversion(true).convert(componentType);
+      final TypeDescriptor finalComponentType = ReflectUtils.boxIfPrimitive(componentType);
+      if (!finalComponentType.hasUnresolvedParameters()) {
+        ForLoadedType functionType =
+            new ForLoadedType(
+                createCollectionTransformFunction(
+                    componentType.getRawType(),
+                    convertedComponentType,
+                    (s) -> getFactory().createGetterConversions(s).convert(finalComponentType)));
+        return createTransformingContainer(functionType, readValue);
+      } else {
+        return readValue;
+      }
     }
 
     @Override
     protected StackManipulation convertMap(TypeDescriptor<?> type) {
-      return readValue;
+      final TypeDescriptor keyType = ReflectUtils.getMapType(type, 0);
+      final TypeDescriptor valueType = ReflectUtils.getMapType(type, 1);
+
+      Type convertedKeyType = getFactory().createTypeConversion(true).convert(keyType);
+      Type convertedValueType = getFactory().createTypeConversion(true).convert(valueType);
+
+      if (!keyType.hasUnresolvedParameters() && !valueType.hasUnresolvedParameters()) {
+        ForLoadedType keyFunctionType =
+            new ForLoadedType(
+                createCollectionTransformFunction(
+                    keyType.getRawType(),
+                    convertedKeyType,
+                    (s) -> getFactory().createGetterConversions(s).convert(keyType)));
+        ForLoadedType valueFunctionType =
+            new ForLoadedType(
+                createCollectionTransformFunction(
+                    valueType.getRawType(),
+                    convertedValueType,
+                    (s) -> getFactory().createGetterConversions(s).convert(valueType)));
+        return new Compound(
+            readValue,
+            TypeCreation.of(keyFunctionType),
+            Duplication.SINGLE,
+            MethodInvocation.invoke(
+                keyFunctionType
+                    .getDeclaredMethods()
+                    .filter(ElementMatchers.isConstructor().and(ElementMatchers.takesArguments(0)))
+                    .getOnly()),
+            TypeCreation.of(valueFunctionType),
+            Duplication.SINGLE,
+            MethodInvocation.invoke(
+                valueFunctionType
+                    .getDeclaredMethods()
+                    .filter(ElementMatchers.isConstructor().and(ElementMatchers.takesArguments(0)))
+                    .getOnly()),
+            MethodInvocation.invoke(
+                BYTE_BUDDY_UTILS_TYPE
+                    .getDeclaredMethods()
+                    .filter(ElementMatchers.named("getTransformingMap"))
+                    .getOnly()));
+      } else {
+        return readValue;
+      }
     }
 
     @Override
@@ -502,10 +813,14 @@
       return new Compound(
           readValue,
           MethodInvocation.invoke(
-              OBJECT_TYPE
+              ENUM_TYPE
                   .getDeclaredMethods()
-                  .filter(ElementMatchers.named("toString").and(ElementMatchers.takesArguments(0)))
-                  .getOnly()));
+                  .filter(ElementMatchers.named("ordinal").and(ElementMatchers.takesArguments(0)))
+                  .getOnly()),
+          Assigner.DEFAULT.assign(
+              INTEGER_TYPE.asUnboxed().asGenericType(),
+              INTEGER_TYPE.asGenericType(),
+              Typing.STATIC));
     }
 
     @Override
@@ -521,7 +836,7 @@
    * there. This class generates code to convert between these types.
    */
   public static class ConvertValueForSetter extends TypeConversion<StackManipulation> {
-    StackManipulation readValue;
+    protected StackManipulation readValue;
 
     protected ConvertValueForSetter(StackManipulation readValue) {
       this.readValue = readValue;
@@ -545,18 +860,31 @@
               .build()
               .asErasure();
 
+      Type rowElementType =
+          getFactory().createTypeConversion(false).convert(type.getComponentType());
+      final TypeDescriptor arrayElementType = ReflectUtils.boxIfPrimitive(type.getComponentType());
+      if (!arrayElementType.hasUnresolvedParameters()) {
+        ForLoadedType conversionFunction =
+            new ForLoadedType(
+                createCollectionTransformFunction(
+                    TypeDescriptor.of(rowElementType).getRawType(),
+                    Primitives.wrap(arrayElementType.getRawType()),
+                    (s) -> getFactory().createSetterConversions(s).convert(arrayElementType)));
+        readValue = createTransformingContainer(conversionFunction, readValue);
+      }
+
       // Extract an array from the collection.
       StackManipulation stackManipulation =
           new Compound(
               readValue,
-              TypeCasting.to(LIST_TYPE),
+              TypeCasting.to(COLLECTION_TYPE),
               // Call Collection.toArray(T[[]) to extract the array. Push new T[0] on the stack
               // before
               // calling toArray.
               ArrayFactory.forType(loadedType.getComponentType().asBoxed().asGenericType())
                   .withValues(Collections.emptyList()),
               MethodInvocation.invoke(
-                  LIST_TYPE
+                  COLLECTION_TYPE
                       .getDeclaredMethods()
                       .filter(
                           ElementMatchers.named("toArray").and(ElementMatchers.takesArguments(1)))
@@ -583,16 +911,128 @@
 
     @Override
     protected StackManipulation convertIterable(TypeDescriptor<?> type) {
-      return readValue;
+      Type rowElementType =
+          getFactory()
+              .createTypeConversion(false)
+              .convert(ReflectUtils.getIterableComponentType(type));
+      final TypeDescriptor iterableElementType = ReflectUtils.getIterableComponentType(type);
+      if (!iterableElementType.hasUnresolvedParameters()) {
+        ForLoadedType conversionFunction =
+            new ForLoadedType(
+                createCollectionTransformFunction(
+                    TypeDescriptor.of(rowElementType).getRawType(),
+                    iterableElementType.getRawType(),
+                    (s) -> getFactory().createSetterConversions(s).convert(iterableElementType)));
+        StackManipulation transformedContainer =
+            createTransformingContainer(conversionFunction, readValue);
+        return transformedContainer;
+      } else {
+        return readValue;
+      }
     }
 
     @Override
     protected StackManipulation convertCollection(TypeDescriptor<?> type) {
-      return readValue;
+      Type rowElementType =
+          getFactory()
+              .createTypeConversion(false)
+              .convert(ReflectUtils.getIterableComponentType(type));
+      final TypeDescriptor collectionElementType = ReflectUtils.getIterableComponentType(type);
+
+      if (!collectionElementType.hasUnresolvedParameters()) {
+        ForLoadedType conversionFunction =
+            new ForLoadedType(
+                createCollectionTransformFunction(
+                    TypeDescriptor.of(rowElementType).getRawType(),
+                    collectionElementType.getRawType(),
+                    (s) -> getFactory().createSetterConversions(s).convert(collectionElementType)));
+        StackManipulation transformedContainer =
+            createTransformingContainer(conversionFunction, readValue);
+        return transformedContainer;
+      } else {
+        return readValue;
+      }
+    }
+
+    @Override
+    protected StackManipulation convertList(TypeDescriptor<?> type) {
+      Type rowElementType =
+          getFactory()
+              .createTypeConversion(false)
+              .convert(ReflectUtils.getIterableComponentType(type));
+      final TypeDescriptor collectionElementType = ReflectUtils.getIterableComponentType(type);
+
+      if (!collectionElementType.hasUnresolvedParameters()) {
+        ForLoadedType conversionFunction =
+            new ForLoadedType(
+                createCollectionTransformFunction(
+                    TypeDescriptor.of(rowElementType).getRawType(),
+                    collectionElementType.getRawType(),
+                    (s) -> getFactory().createSetterConversions(s).convert(collectionElementType)));
+        readValue = createTransformingContainer(conversionFunction, readValue);
+      }
+      // TODO: Don't copy if already a list!
+      StackManipulation transformedList =
+          new Compound(
+              readValue,
+              MethodInvocation.invoke(
+                  new ForLoadedType(Lists.class)
+                      .getDeclaredMethods()
+                      .filter(
+                          ElementMatchers.named("newArrayList")
+                              .and(ElementMatchers.takesArguments(Iterable.class)))
+                      .getOnly()));
+      return transformedList;
     }
 
     @Override
     protected StackManipulation convertMap(TypeDescriptor<?> type) {
+      Type rowKeyType =
+          getFactory().createTypeConversion(false).convert(ReflectUtils.getMapType(type, 0));
+      final TypeDescriptor keyElementType = ReflectUtils.getMapType(type, 0);
+      Type rowValueType =
+          getFactory().createTypeConversion(false).convert(ReflectUtils.getMapType(type, 1));
+      final TypeDescriptor valueElementType = ReflectUtils.getMapType(type, 1);
+
+      if (!keyElementType.hasUnresolvedParameters()
+          && !valueElementType.hasUnresolvedParameters()) {
+        ForLoadedType keyConversionFunction =
+            new ForLoadedType(
+                createCollectionTransformFunction(
+                    TypeDescriptor.of(rowKeyType).getRawType(),
+                    keyElementType.getRawType(),
+                    (s) -> getFactory().createSetterConversions(s).convert(keyElementType)));
+        ForLoadedType valueConversionFunction =
+            new ForLoadedType(
+                createCollectionTransformFunction(
+                    TypeDescriptor.of(rowValueType).getRawType(),
+                    valueElementType.getRawType(),
+                    (s) -> getFactory().createSetterConversions(s).convert(valueElementType)));
+        readValue =
+            new Compound(
+                readValue,
+                TypeCreation.of(keyConversionFunction),
+                Duplication.SINGLE,
+                MethodInvocation.invoke(
+                    keyConversionFunction
+                        .getDeclaredMethods()
+                        .filter(
+                            ElementMatchers.isConstructor().and(ElementMatchers.takesArguments(0)))
+                        .getOnly()),
+                TypeCreation.of(valueConversionFunction),
+                Duplication.SINGLE,
+                MethodInvocation.invoke(
+                    valueConversionFunction
+                        .getDeclaredMethods()
+                        .filter(
+                            ElementMatchers.isConstructor().and(ElementMatchers.takesArguments(0)))
+                        .getOnly()),
+                MethodInvocation.invoke(
+                    BYTE_BUDDY_UTILS_TYPE
+                        .getDeclaredMethods()
+                        .filter(ElementMatchers.named("getTransformingMap"))
+                        .getOnly()));
+      }
       return readValue;
     }
 
@@ -723,17 +1163,25 @@
     protected StackManipulation convertEnum(TypeDescriptor<?> type) {
       ForLoadedType loadedType = new ForLoadedType(type.getRawType());
 
+      // Convert the stored ordinal back to the Java enum constant.
       return new Compound(
-          readValue,
+          // Call EnumType::values() to get an array of all enum constants.
           MethodInvocation.invoke(
               loadedType
                   .getDeclaredMethods()
                   .filter(
-                      ElementMatchers.named("valueOf")
-                          .and(
-                              ElementMatchers.isStatic()
-                                  .and(ElementMatchers.takesArguments(String.class))))
-                  .getOnly()));
+                      ElementMatchers.named("values")
+                          .and(ElementMatchers.isStatic().and(ElementMatchers.takesArguments(0))))
+                  .getOnly()),
+          // Read the integer enum value.
+          readValue,
+          // Unbox Integer -> int before accessing the array.
+          Assigner.DEFAULT.assign(
+              INTEGER_TYPE.asBoxed().asGenericType(),
+              INTEGER_TYPE.asUnboxed().asGenericType(),
+              Typing.STATIC),
+          // Access the array to return the Java enum type.
+          ArrayAccess.REFERENCE.load());
     }
 
     @Override
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/ReflectUtils.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/ReflectUtils.java
index b9f1ae5..d56f0bd 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/ReflectUtils.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/ReflectUtils.java
@@ -23,8 +23,11 @@
 import java.lang.reflect.Field;
 import java.lang.reflect.Method;
 import java.lang.reflect.Modifier;
+import java.lang.reflect.ParameterizedType;
+import java.lang.reflect.Type;
 import java.security.InvalidParameterException;
 import java.util.Arrays;
+import java.util.Collection;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
@@ -33,16 +36,19 @@
 import javax.annotation.Nullable;
 import org.apache.beam.sdk.schemas.Schema;
 import org.apache.beam.sdk.schemas.annotations.SchemaCreate;
+import org.apache.beam.sdk.values.TypeDescriptor;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Lists;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Maps;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.primitives.Primitives;
 
 /** A set of reflection helper methods. */
 public class ReflectUtils {
-  static class ClassWithSchema {
+  /** Represents a class and a schema. */
+  public static class ClassWithSchema {
     private final Class clazz;
     private final Schema schema;
 
-    ClassWithSchema(Class clazz, Schema schema) {
+    public ClassWithSchema(Class clazz, Schema schema) {
       this.clazz = clazz;
       this.schema = schema;
     }
@@ -78,6 +84,9 @@
         clazz,
         c -> {
           return Arrays.stream(c.getDeclaredMethods())
+              .filter(
+                  m -> !m.isBridge()) // Covariant overloads insert bridge functions, which we must
+              // ignore.
               .filter(m -> !Modifier.isPrivate(m.getModifiers()))
               .filter(m -> !Modifier.isProtected(m.getModifiers()))
               .filter(m -> !Modifier.isStatic(m.getModifiers()))
@@ -183,4 +192,49 @@
   public static String stripSetterPrefix(String method) {
     return stripPrefix(method, "set");
   }
+
+  /** For an array T[] or a subclass of Iterable<T>, return a TypeDescriptor describing T. */
+  @Nullable
+  public static TypeDescriptor getIterableComponentType(TypeDescriptor valueType) {
+    TypeDescriptor componentType = null;
+    if (valueType.isArray()) {
+      Type component = valueType.getComponentType().getType();
+      if (!component.equals(byte.class)) {
+        // Byte arrays are special cased since we have a schema type corresponding to them.
+        componentType = TypeDescriptor.of(component);
+      }
+    } else if (valueType.isSubtypeOf(TypeDescriptor.of(Iterable.class))) {
+      TypeDescriptor<Iterable<?>> collection = valueType.getSupertype(Iterable.class);
+      if (collection.getType() instanceof ParameterizedType) {
+        ParameterizedType ptype = (ParameterizedType) collection.getType();
+        java.lang.reflect.Type[] params = ptype.getActualTypeArguments();
+        checkArgument(params.length == 1);
+        componentType = TypeDescriptor.of(params[0]);
+      } else {
+        throw new RuntimeException("Collection parameter is not parameterized!");
+      }
+    }
+    return componentType;
+  }
+
+  public static TypeDescriptor getMapType(TypeDescriptor valueType, int index) {
+    TypeDescriptor mapType = null;
+    if (valueType.isSubtypeOf(TypeDescriptor.of(Map.class))) {
+      TypeDescriptor<Collection<?>> map = valueType.getSupertype(Map.class);
+      if (map.getType() instanceof ParameterizedType) {
+        ParameterizedType ptype = (ParameterizedType) map.getType();
+        java.lang.reflect.Type[] params = ptype.getActualTypeArguments();
+        mapType = TypeDescriptor.of(params[index]);
+      } else {
+        throw new RuntimeException("Map type is not parameterized! " + map);
+      }
+    }
+    return mapType;
+  }
+
+  public static TypeDescriptor boxIfPrimitive(TypeDescriptor typeDescriptor) {
+    return typeDescriptor.getRawType().isPrimitive()
+        ? TypeDescriptor.of(Primitives.wrap(typeDescriptor.getRawType()))
+        : typeDescriptor;
+  }
 }
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/StaticSchemaInference.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/StaticSchemaInference.java
index 5f49894..0d4fc9a 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/StaticSchemaInference.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/StaticSchemaInference.java
@@ -22,6 +22,7 @@
 import java.lang.reflect.ParameterizedType;
 import java.math.BigDecimal;
 import java.nio.ByteBuffer;
+import java.util.Arrays;
 import java.util.Collection;
 import java.util.List;
 import java.util.Map;
@@ -30,6 +31,7 @@
 import org.apache.beam.sdk.schemas.FieldValueTypeInformation;
 import org.apache.beam.sdk.schemas.Schema;
 import org.apache.beam.sdk.schemas.Schema.FieldType;
+import org.apache.beam.sdk.schemas.logicaltypes.EnumerationType;
 import org.apache.beam.sdk.values.TypeDescriptor;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
 import org.joda.time.ReadableInstant;
@@ -99,6 +101,13 @@
       return primitiveType;
     }
 
+    if (type.getRawType().isEnum()) {
+      Map<String, Integer> enumValues =
+          Arrays.stream(type.getRawType().getEnumConstants())
+              .map(Enum.class::cast)
+              .collect(Collectors.toMap(Enum::toString, Enum::ordinal));
+      return FieldType.logicalType(EnumerationType.create(enumValues));
+    }
     if (type.isArray()) {
       // If the type is T[] where T is byte, this is a BYTES type.
       TypeDescriptor component = type.getComponentType();
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/UsesSystemMetrics.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/UsesSystemMetrics.java
new file mode 100644
index 0000000..ea71215
--- /dev/null
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/UsesSystemMetrics.java
@@ -0,0 +1,24 @@
+/*
+ * 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;
+
+/**
+ * Category tag for tests that use System metrics. These metrics are available only in portable
+ * runners.
+ */
+public interface UsesSystemMetrics {}
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java
index 03b6263..27e8517 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java
@@ -587,7 +587,6 @@
    * <ul>
    *   <li>It <i>must</i> define a {@link GetInitialRestriction} method.
    *   <li>It <i>may</i> define a {@link GetSize} method.
-   *   <li>It <i>may</i> define a {@link GetPartitition} method.
    *   <li>It <i>may</i> define a {@link SplitRestriction} method.
    *   <li>It <i>may</i> define a {@link NewTracker} method returning a subtype of {@code
    *       RestrictionTracker<R>} where {@code R} is the restriction type returned by {@link
@@ -741,10 +740,6 @@
    *
    * <p>Returns a double representing the size of the element and restriction.
    *
-   * <p>A representation for the amount of known work represented as a size. Size representations
-   * should preferably represent a linear space and be comparable within the same partition (see
-   * {@link GetPartition} for details on partition identifiers}).
-   *
    * <p>Splittable {@link DoFn}s should only provide this method if the default implementation
    * within the {@link RestrictionTracker} is an inaccurate representation of known work.
    *
@@ -771,38 +766,6 @@
   public @interface GetSize {}
 
   /**
-   * Annotation for the method that returns the corresponding partition identifier for an element
-   * and restriction pair.
-   *
-   * <p>Signature: {@code byte[] getPartitition(InputT element, RestrictionT restriction);}
-   *
-   * <p>Returns an immutable representation of the partition identifier as a byte[].
-   *
-   * <p>By default, the partition identifier is represented as the encoded element and restriction
-   * pair and should only be provided if the splittable {@link DoFn} can only provide a size over a
-   * shared resource such as a message queue that potentially multiple element and restriction pairs
-   * are doing work on. The partition identifier is used by runners for various size calculations.
-   * Sizes reported with the same partition identifier represent a point in time reporting of the
-   * size for that partition. For example, a runner can compute a global size by summing all
-   * reported sizes over all unique partition identifiers while it can compute the size of a
-   * specific partition based upon the last reported value.
-   *
-   * <p>For example splittable {@link DoFn}s which consume elements from:
-   *
-   * <ul>
-   *   <li>a globally shared resource such as a Pubsub queue should set this to "".
-   *   <li>a shared partitioned resource should use the partition identifier.
-   *   <li>a uniquely partitioned resource such as a file and offset range should not override this
-   *       since the default element and restriction pair should suffice.
-   * </ul>
-   */
-  @Documented
-  @Retention(RetentionPolicy.RUNTIME)
-  @Target(ElementType.METHOD)
-  @Experimental(Kind.SPLITTABLE_DO_FN)
-  public @interface GetPartition {}
-
-  /**
    * Annotation for the method that returns the coder to use for the restriction of a <a
    * href="https://s.apache.org/splittable-do-fn">splittable</a> {@link DoFn}.
    *
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Watch.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Watch.java
index 8d5ed4b..6d34eba 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Watch.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Watch.java
@@ -57,6 +57,7 @@
 import org.apache.beam.sdk.transforms.Watch.Growth.PollResult;
 import org.apache.beam.sdk.transforms.splittabledofn.OffsetRangeTracker;
 import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker;
+import org.apache.beam.sdk.transforms.splittabledofn.SplitResult;
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
 import org.apache.beam.sdk.util.VarInt;
 import org.apache.beam.sdk.values.KV;
@@ -1028,7 +1029,10 @@
     }
 
     @Override
-    public GrowthState checkpoint() {
+    public SplitResult<GrowthState> trySplit(double fractionOfRemainder) {
+      // TODO(BEAM-8873): Add support for splitting off a fixed amount of work for this restriction
+      // instead of only supporting checkpointing.
+
       // residual should contain exactly the work *not* claimed in the current ProcessElement call -
       // unclaimed pending outputs or future polling output
       GrowthState residual;
@@ -1061,7 +1065,7 @@
       }
 
       shouldStop = true;
-      return residual;
+      return SplitResult.of(state, residual);
     }
 
     private HashCode hash128(OutputT value) {
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/splittabledofn/ByteKeyRangeTracker.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/splittabledofn/ByteKeyRangeTracker.java
index f8a577d..0c8a5aa3 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/splittabledofn/ByteKeyRangeTracker.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/splittabledofn/ByteKeyRangeTracker.java
@@ -60,20 +60,23 @@
   }
 
   @Override
-  public ByteKeyRange checkpoint() {
+  public SplitResult<ByteKeyRange> trySplit(double fractionOfRemainder) {
+    // TODO(BEAM-8871): Add support for splitting off a fixed amount of work for this restriction
+    // instead of only supporting checkpointing.
+
     // If we haven't done any work, we should return the original range we were processing
     // as the checkpoint.
     if (lastAttemptedKey == null) {
       ByteKeyRange rval = range;
       // We update our current range to an interval that contains no elements.
       range = NO_KEYS;
-      return rval;
+      return SplitResult.of(range, rval);
     }
 
     // Return an empty range if the current range is done.
     if (lastAttemptedKey.isEmpty()
         || !(range.getEndKey().isEmpty() || range.getEndKey().compareTo(lastAttemptedKey) > 0)) {
-      return NO_KEYS;
+      return SplitResult.of(range, NO_KEYS);
     }
 
     // Otherwise we compute the "remainder" of the range from the last key.
@@ -82,7 +85,7 @@
     ByteKey nextKey = next(lastAttemptedKey);
     ByteKeyRange res = ByteKeyRange.of(nextKey, range.getEndKey());
     this.range = ByteKeyRange.of(range.getStartKey(), nextKey);
-    return res;
+    return SplitResult.of(range, res);
   }
 
   /**
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/splittabledofn/OffsetRangeTracker.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/splittabledofn/OffsetRangeTracker.java
index 69641a7..490f7fc 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/splittabledofn/OffsetRangeTracker.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/splittabledofn/OffsetRangeTracker.java
@@ -45,12 +45,15 @@
   }
 
   @Override
-  public OffsetRange checkpoint() {
+  public SplitResult<OffsetRange> trySplit(double fractionOfRemainder) {
+    // TODO(BEAM-8872): Add support for splitting off a fixed amount of work for this restriction
+    // instead of only supporting checkpointing.
+
     checkState(
         lastClaimedOffset != null, "Can't checkpoint before any offset was successfully claimed");
     OffsetRange res = new OffsetRange(lastClaimedOffset + 1, range.getTo());
     this.range = new OffsetRange(range.getFrom(), lastClaimedOffset + 1);
-    return res;
+    return SplitResult.of(range, res);
   }
 
   /**
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/splittabledofn/RestrictionTracker.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/splittabledofn/RestrictionTracker.java
index 1ed6a97..c36b942 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/splittabledofn/RestrictionTracker.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/splittabledofn/RestrictionTracker.java
@@ -47,17 +47,37 @@
   public abstract RestrictionT currentRestriction();
 
   /**
-   * Signals that the current {@link DoFn.ProcessElement} call should terminate as soon as possible:
-   * after this method returns, the tracker MUST refuse all future claim calls, and {@link
-   * #checkDone} MUST succeed.
+   * Splits current restriction based on {@code fractionOfRemainder}.
    *
-   * <p>Modifies {@link #currentRestriction}. Returns a restriction representing the rest of the
-   * work: the old value of {@link #currentRestriction} is equivalent to the new value and the
-   * return value of this method combined.
+   * <p>If splitting the current restriction is possible, the current restriction is split into a
+   * primary and residual restriction pair. This invocation updates the {@link
+   * #currentRestriction()} to be the primary restriction effectively having the current {@link
+   * DoFn.ProcessElement} execution responsible for performing the work that the primary restriction
+   * represents. The residual restriction will be executed in a separate {@link DoFn.ProcessElement}
+   * invocation (likely in a different process). The work performed by executing the primary and
+   * residual restrictions as separate {@link DoFn.ProcessElement} invocations MUST be equivalent to
+   * the work performed as if this split never occurred.
    *
-   * <p>Must be called at most once on a given object.
+   * <p>The {@code fractionOfRemainder} should be used in a best effort manner to choose a primary
+   * and residual restriction based upon the fraction of the remaining work that the current {@link
+   * DoFn.ProcessElement} invocation is responsible for. For example, if a {@link
+   * DoFn.ProcessElement} was reading a file with a restriction representing the offset range {@code
+   * [100, 200)} and has processed up to offset 130 with a {@code fractionOfRemainder} of {@code
+   * 0.7}, the primary and residual restrictions returned would be {@code [100, 179), [179, 200)}
+   * (note: {@code currentOffset + fractionOfRemainder * remainingWork = 130 + 0.7 * 70 = 179}).
+   *
+   * <p>{@code fractionOfRemainder = 0} means a checkpoint is required.
+   *
+   * <p>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.
+   *
+   * <p>The API is required to be implemented for a streaming pipeline.
+   *
+   * @param fractionOfRemainder A hint as to the fraction of work the primary restriction should
+   *     represent based upon the current known remaining amount of work.
+   * @return a {@link SplitResult} if a split was possible, otherwise returns {@code null}.
    */
-  public abstract RestrictionT checkpoint();
+  public abstract SplitResult<RestrictionT> trySplit(double fractionOfRemainder);
 
   /**
    * Called by the runner after {@link DoFn.ProcessElement} returns.
@@ -66,6 +86,4 @@
    * work remaining in the restriction.
    */
   public abstract void checkDone() throws IllegalStateException;
-
-  // TODO: Add the more general splitRemainderAfterFraction() and other methods.
 }
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/splittabledofn/Sizes.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/splittabledofn/Sizes.java
index 3bbcb7e..495bf5d 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/splittabledofn/Sizes.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/splittabledofn/Sizes.java
@@ -17,8 +17,6 @@
  */
 package org.apache.beam.sdk.transforms.splittabledofn;
 
-import org.apache.beam.sdk.transforms.DoFn;
-
 /** Definitions and convenience methods for reporting sizes for SplittableDoFns. */
 public final class Sizes {
   /**
@@ -29,8 +27,7 @@
   public interface HasSize {
     /**
      * A representation for the amount of known work represented as a size. Size {@code double}
-     * representations should preferably represent a linear space and be comparable within the same
-     * partition.
+     * representations should preferably represent a linear space.
      *
      * <p>It is up to each restriction tracker to convert between their natural representation of
      * outstanding work and this representation. For example:
@@ -47,10 +44,6 @@
      *       of encoded bytes for the keys and values is known for the key range, the number of
      *       remaining bytes can be used.
      * </ul>
-     *
-     * <p>{@link DoFn}s should provide a method annotated with {@link DoFn.GetPartition} to report a
-     * partition identifier if the element and restriction represent the size for a shared resource
-     * such as a message queue topic. See {@link DoFn.GetPartition} for additional details.
      */
     double getSize();
   }
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/splittabledofn/SplitResult.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/splittabledofn/SplitResult.java
new file mode 100644
index 0000000..0844ebf
--- /dev/null
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/splittabledofn/SplitResult.java
@@ -0,0 +1,39 @@
+/*
+ * 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.transforms.splittabledofn;
+
+import com.google.auto.value.AutoValue;
+import javax.annotation.Nullable;
+
+/** A representation of a split result. */
+@AutoValue
+public abstract class SplitResult<RestrictionT> {
+  /** Returns a {@link SplitResult} for the specified primary and residual restrictions. */
+  public static <RestrictionT> SplitResult<RestrictionT> of(
+      RestrictionT primary, RestrictionT residual) {
+    return new AutoValue_SplitResult(primary, residual);
+  }
+
+  /** Returns the primary restriction. */
+  @Nullable
+  public abstract RestrictionT getPrimary();
+
+  /** Returns the residual restriction. */
+  @Nullable
+  public abstract RestrictionT getResidual();
+}
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/Row.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/Row.java
index afeb6f6..be28467 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/Row.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/Row.java
@@ -25,6 +25,7 @@
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.Iterator;
 import java.util.List;
@@ -187,7 +188,7 @@
    * match.
    */
   @Nullable
-  public <T> List<T> getArray(String fieldName) {
+  public <T> Collection<T> getArray(String fieldName) {
     return getArray(getSchema().indexOf(fieldName));
   }
 
@@ -209,6 +210,16 @@
   }
 
   /**
+   * Returns the Logical Type input type for this field. {@link IllegalStateException} is thrown if
+   * schema doesn't match.
+   */
+  @Nullable
+  @SuppressWarnings("TypeParameterUnusedInFormals")
+  public <T> T getLogicalTypeValue(String fieldName) {
+    return getLogicalTypeValue(getSchema().indexOf(fieldName));
+  }
+
+  /**
    * Get a {@link TypeName#ROW} value by field name, {@link IllegalStateException} is thrown if
    * schema doesn't match.
    */
@@ -322,7 +333,7 @@
    * match.
    */
   @Nullable
-  public <T> List<T> getArray(int idx) {
+  public <T> Collection<T> getArray(int idx) {
     return getValue(idx);
   }
 
@@ -345,6 +356,17 @@
   }
 
   /**
+   * Returns the Logical Type input type for this field. {@link IllegalStateException} is thrown if
+   * schema doesn't match.
+   */
+  @Nullable
+  @SuppressWarnings("TypeParameterUnusedInFormals")
+  public <T> T getLogicalTypeValue(int idx) {
+    LogicalType logicalType = checkNotNull(getSchema().getField(idx).getType().getLogicalType());
+    return (T) logicalType.toInputType(getValue(idx));
+  }
+
+  /**
    * Get a {@link Row} value by field index, {@link IllegalStateException} is thrown if schema
    * doesn't match.
    */
@@ -391,8 +413,8 @@
     return h;
   }
 
-  static class Equals {
-    static boolean deepEquals(Object a, Object b, Schema.FieldType fieldType) {
+  public static class Equals {
+    public static boolean deepEquals(Object a, Object b, Schema.FieldType fieldType) {
       if (a == null || b == null) {
         return a == b;
       } else if (fieldType.getTypeName() == TypeName.LOGICAL_TYPE) {
@@ -400,8 +422,8 @@
       } else if (fieldType.getTypeName() == Schema.TypeName.BYTES) {
         return Arrays.equals((byte[]) a, (byte[]) b);
       } else if (fieldType.getTypeName() == TypeName.ARRAY) {
-        return deepEqualsForList(
-            (List<Object>) a, (List<Object>) b, fieldType.getCollectionElementType());
+        return deepEqualsForCollection(
+            (Collection<Object>) a, (Collection<Object>) b, fieldType.getCollectionElementType());
       } else if (fieldType.getTypeName() == TypeName.ITERABLE) {
         return deepEqualsForIterable(
             (Iterable<Object>) a, (Iterable<Object>) b, fieldType.getCollectionElementType());
@@ -472,7 +494,8 @@
       return h;
     }
 
-    static boolean deepEqualsForList(List<Object> a, List<Object> b, Schema.FieldType elementType) {
+    static boolean deepEqualsForCollection(
+        Collection<Object> a, Collection<Object> b, Schema.FieldType elementType) {
       if (a == b) {
         return true;
       }
@@ -563,7 +586,7 @@
       return addValues(Arrays.asList(values));
     }
 
-    public <T> Builder addArray(List<T> values) {
+    public <T> Builder addArray(Collection<T> values) {
       this.values.add(values);
       return this;
     }
@@ -641,16 +664,16 @@
     private List<Object> verifyArray(
         Object value, FieldType collectionElementType, String fieldName) {
       boolean collectionElementTypeNullable = collectionElementType.getNullable();
-      if (!(value instanceof List)) {
+      if (!(value instanceof Collection)) {
         throw new IllegalArgumentException(
             String.format(
-                "For field name %s and array type expected List class. Instead "
+                "For field name %s and array type expected Collection class. Instead "
                     + "class type was %s.",
                 fieldName, value.getClass()));
       }
-      List<Object> valueList = (List<Object>) value;
-      List<Object> verifiedList = Lists.newArrayListWithCapacity(valueList.size());
-      for (Object listValue : valueList) {
+      Collection<Object> valueCollection = (Collection<Object>) value;
+      List<Object> verifiedList = Lists.newArrayListWithCapacity(valueCollection.size());
+      for (Object listValue : valueCollection) {
         if (listValue == null) {
           if (!collectionElementTypeNullable) {
             throw new IllegalArgumentException(
@@ -675,8 +698,8 @@
                     + "class type was %s.",
                 fieldName, value.getClass()));
       }
-      Iterable<Object> valueList = (Iterable<Object>) value;
-      for (Object listValue : valueList) {
+      Iterable<Object> valueIterable = (Iterable<Object>) value;
+      for (Object listValue : valueIterable) {
         if (listValue == null) {
           if (!collectionElementTypeNullable) {
             throw new IllegalArgumentException(
@@ -687,7 +710,7 @@
           verify(listValue, collectionElementType, fieldName);
         }
       }
-      return valueList;
+      return valueIterable;
     }
 
     private Map<Object, Object> verifyMap(
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/RowWithGetters.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/RowWithGetters.java
index 0d78731..ebf59b9 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/RowWithGetters.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/RowWithGetters.java
@@ -17,7 +17,7 @@
  */
 package org.apache.beam.sdk.values;
 
-import java.util.Iterator;
+import java.util.Collection;
 import java.util.List;
 import java.util.Map;
 import java.util.Objects;
@@ -29,6 +29,8 @@
 import org.apache.beam.sdk.schemas.Schema.Field;
 import org.apache.beam.sdk.schemas.Schema.FieldType;
 import org.apache.beam.sdk.schemas.Schema.TypeName;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Collections2;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Lists;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Maps;
 
@@ -44,7 +46,7 @@
   private final Object getterTarget;
   private final List<FieldValueGetter> getters;
 
-  private final Map<Integer, List> cachedLists = Maps.newHashMap();
+  private final Map<Integer, Collection> cachedCollections = Maps.newHashMap();
   private final Map<Integer, Iterable> cachedIterables = Maps.newHashMap();
   private final Map<Integer, Map> cachedMaps = Maps.newHashMap();
 
@@ -69,36 +71,22 @@
     return fieldValue != null ? getValue(type, fieldValue, fieldIdx) : null;
   }
 
-  private List getListValue(FieldType elementType, Object fieldValue) {
-    Iterable iterable = (Iterable) fieldValue;
-    List<Object> list = Lists.newArrayList();
-    for (Object o : iterable) {
-      list.add(getValue(elementType, o, null));
+  private Collection getCollectionValue(FieldType elementType, Object fieldValue) {
+    Collection collection = (Collection) fieldValue;
+    if (collection instanceof List) {
+      // For performance reasons if the input is a list, make sure that we produce a list. Otherwise
+      // Row forwarding
+      // is forced to physically copy the collection into a new List object.
+      return Lists.transform((List) collection, v -> getValue(elementType, v, null));
+    } else {
+      return Collections2.transform(collection, v -> getValue(elementType, v, null));
     }
-    return list;
   }
 
   private Iterable getIterableValue(FieldType elementType, Object fieldValue) {
     Iterable iterable = (Iterable) fieldValue;
     // Wrap the iterable to avoid having to materialize the entire collection.
-    return new Iterable() {
-      @Override
-      public Iterator iterator() {
-        return new Iterator() {
-          Iterator iterator = iterable.iterator();
-
-          @Override
-          public boolean hasNext() {
-            return iterator.hasNext();
-          }
-
-          @Override
-          public Object next() {
-            return getValue(elementType, iterator.next(), null);
-          }
-        };
-      }
-    };
+    return Iterables.transform(iterable, v -> getValue(elementType, v, null));
   }
 
   private Map<?, ?> getMapValue(FieldType keyType, FieldType valueType, Map<?, ?> fieldValue) {
@@ -117,9 +105,9 @@
     } else if (type.getTypeName().equals(TypeName.ARRAY)) {
       return cacheKey != null
           ? (T)
-              cachedLists.computeIfAbsent(
-                  cacheKey, i -> getListValue(type.getCollectionElementType(), fieldValue))
-          : (T) getListValue(type.getCollectionElementType(), fieldValue);
+              cachedCollections.computeIfAbsent(
+                  cacheKey, i -> getCollectionValue(type.getCollectionElementType(), fieldValue))
+          : (T) getCollectionValue(type.getCollectionElementType(), fieldValue);
     } else if (type.getTypeName().equals(TypeName.ITERABLE)) {
       return cacheKey != null
           ? (T)
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/AvroSchemaTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/AvroSchemaTest.java
index ccbd3bb..31f5a31 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/AvroSchemaTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/AvroSchemaTest.java
@@ -31,9 +31,10 @@
 import org.apache.avro.reflect.AvroName;
 import org.apache.avro.reflect.AvroSchema;
 import org.apache.avro.util.Utf8;
-import org.apache.beam.sdk.schemas.LogicalTypes.FixedBytes;
 import org.apache.beam.sdk.schemas.Schema.Field;
 import org.apache.beam.sdk.schemas.Schema.FieldType;
+import org.apache.beam.sdk.schemas.logicaltypes.EnumerationType;
+import org.apache.beam.sdk.schemas.logicaltypes.FixedBytes;
 import org.apache.beam.sdk.schemas.transforms.Group;
 import org.apache.beam.sdk.schemas.utils.AvroUtils;
 import org.apache.beam.sdk.testing.PAssert;
@@ -213,6 +214,44 @@
     }
 
     public AvroPojo() {}
+
+    @Override
+    public String toString() {
+      return "AvroPojo{"
+          + "boolNonNullable="
+          + boolNonNullable
+          + ", anInt="
+          + anInt
+          + ", aLong="
+          + aLong
+          + ", aFloat="
+          + aFloat
+          + ", aDouble="
+          + aDouble
+          + ", string='"
+          + string
+          + '\''
+          + ", bytes="
+          + bytes
+          + ", fixed="
+          + Arrays.toString(fixed)
+          + ", date="
+          + date
+          + ", timestampMillis="
+          + timestampMillis
+          + ", testEnum="
+          + testEnum
+          + ", row="
+          + row
+          + ", array="
+          + array
+          + ", map="
+          + map
+          + ", extraField='"
+          + extraField
+          + '\''
+          + '}';
+    }
   }
 
   private static final Schema SUBSCHEMA =
@@ -222,6 +261,8 @@
           .build();
   private static final FieldType SUB_TYPE = FieldType.row(SUBSCHEMA).withNullable(true);
 
+  private static final EnumerationType TEST_ENUM_TYPE = EnumerationType.create("abc", "cde");
+
   private static final Schema SCHEMA =
       Schema.builder()
           .addField("bool_non_nullable", FieldType.BOOLEAN)
@@ -234,7 +275,7 @@
           .addField("fixed", FieldType.logicalType(FixedBytes.of(4)))
           .addField("date", FieldType.DATETIME)
           .addField("timestampMillis", FieldType.DATETIME)
-          .addField("testEnum", FieldType.STRING)
+          .addField("testEnum", FieldType.logicalType(TEST_ENUM_TYPE))
           .addNullableField("row", SUB_TYPE)
           .addNullableField("array", FieldType.array(SUB_TYPE))
           .addNullableField("map", FieldType.map(FieldType.STRING, SUB_TYPE))
@@ -252,7 +293,7 @@
           .addField("fixed", FieldType.logicalType(FixedBytes.of(4)))
           .addField("date", FieldType.DATETIME)
           .addField("timestampMillis", FieldType.DATETIME)
-          .addField("testEnum", FieldType.STRING)
+          .addField("testEnum", FieldType.logicalType(TEST_ENUM_TYPE))
           .addNullableField("row", SUB_TYPE)
           .addNullableField("array", FieldType.array(SUB_TYPE.withNullable(false)))
           .addNullableField("map", FieldType.map(FieldType.STRING, SUB_TYPE.withNullable(false)))
@@ -324,7 +365,7 @@
               BYTE_ARRAY,
               DATE.toDateTimeAtStartOfDay(DateTimeZone.UTC),
               DATE_TIME,
-              "abc",
+              TEST_ENUM_TYPE.valueOf("abc"),
               NESTED_ROW,
               ImmutableList.of(NESTED_ROW, NESTED_ROW),
               ImmutableMap.of("k1", NESTED_ROW, "k2", NESTED_ROW))
@@ -399,7 +440,7 @@
               BYTE_ARRAY,
               DATE.toDateTimeAtStartOfDay(DateTimeZone.UTC),
               DATE_TIME,
-              "abc",
+              TEST_ENUM_TYPE.valueOf("abc"),
               NESTED_ROW,
               ImmutableList.of(NESTED_ROW, NESTED_ROW),
               ImmutableMap.of("k1", NESTED_ROW, "k2", NESTED_ROW))
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/JavaBeanSchemaTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/JavaBeanSchemaTest.java
index 6ceac8b..feb51db 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/JavaBeanSchemaTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/JavaBeanSchemaTest.java
@@ -17,6 +17,7 @@
  */
 package org.apache.beam.sdk.schemas;
 
+import static org.apache.beam.sdk.schemas.utils.TestJavaBeans.ARRAY_OF_BYTE_ARRAY_BEAM_SCHEMA;
 import static org.apache.beam.sdk.schemas.utils.TestJavaBeans.ITERABLE_BEAM_SCHEMA;
 import static org.apache.beam.sdk.schemas.utils.TestJavaBeans.NESTED_ARRAYS_BEAM_SCHEMA;
 import static org.apache.beam.sdk.schemas.utils.TestJavaBeans.NESTED_ARRAY_BEAN_SCHEMA;
@@ -30,11 +31,13 @@
 import static org.junit.Assert.assertTrue;
 
 import java.math.BigDecimal;
+import java.nio.ByteBuffer;
 import java.nio.charset.Charset;
 import java.util.Arrays;
 import java.util.List;
 import java.util.Map;
 import org.apache.beam.sdk.schemas.utils.SchemaTestUtils;
+import org.apache.beam.sdk.schemas.utils.TestJavaBeans.ArrayOfByteArray;
 import org.apache.beam.sdk.schemas.utils.TestJavaBeans.IterableBean;
 import org.apache.beam.sdk.schemas.utils.TestJavaBeans.MismatchingNullableBean;
 import org.apache.beam.sdk.schemas.utils.TestJavaBeans.NestedArrayBean;
@@ -272,7 +275,7 @@
 
     NestedArrayBean bean = new NestedArrayBean(simple1, simple2, simple3);
     Row row = registry.getToRowFunction(NestedArrayBean.class).apply(bean);
-    List<Row> rows = row.getArray("beans");
+    List<Row> rows = (List) row.getArray("beans");
     assertSame(simple1, registry.getFromRowFunction(SimpleBean.class).apply(rows.get(0)));
     assertSame(simple2, registry.getFromRowFunction(SimpleBean.class).apply(rows.get(1)));
     assertSame(simple3, registry.getFromRowFunction(SimpleBean.class).apply(rows.get(2)));
@@ -422,4 +425,38 @@
     list.add("three");
     assertEquals(list, Lists.newArrayList(converted.getStrings()));
   }
+
+  @Test
+  public void testToRowArrayOfBytes() throws NoSuchSchemaException {
+    SchemaRegistry registry = SchemaRegistry.createDefault();
+    Schema schema = registry.getSchema(ArrayOfByteArray.class);
+    SchemaTestUtils.assertSchemaEquivalent(ARRAY_OF_BYTE_ARRAY_BEAM_SCHEMA, schema);
+
+    ArrayOfByteArray arrayOfByteArray =
+        new ArrayOfByteArray(
+            ImmutableList.of(ByteBuffer.wrap(BYTE_ARRAY), ByteBuffer.wrap(BYTE_ARRAY)));
+    Row expectedRow =
+        Row.withSchema(ARRAY_OF_BYTE_ARRAY_BEAM_SCHEMA)
+            .addArray(ImmutableList.of(BYTE_ARRAY, BYTE_ARRAY))
+            .build();
+    Row converted = registry.getToRowFunction(ArrayOfByteArray.class).apply(arrayOfByteArray);
+    assertEquals(expectedRow, converted);
+  }
+
+  @Test
+  public void testFromRowArrayOfBytes() throws NoSuchSchemaException {
+    SchemaRegistry registry = SchemaRegistry.createDefault();
+    Schema schema = registry.getSchema(ArrayOfByteArray.class);
+    SchemaTestUtils.assertSchemaEquivalent(ARRAY_OF_BYTE_ARRAY_BEAM_SCHEMA, schema);
+
+    ArrayOfByteArray expectedArrayOfByteArray =
+        new ArrayOfByteArray(
+            ImmutableList.of(ByteBuffer.wrap(BYTE_ARRAY), ByteBuffer.wrap(BYTE_ARRAY)));
+    Row row =
+        Row.withSchema(ARRAY_OF_BYTE_ARRAY_BEAM_SCHEMA)
+            .addArray(ImmutableList.of(BYTE_ARRAY, BYTE_ARRAY))
+            .build();
+    ArrayOfByteArray converted = registry.getFromRowFunction(ArrayOfByteArray.class).apply(row);
+    assertEquals(expectedArrayOfByteArray, converted);
+  }
 }
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/JavaFieldSchemaTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/JavaFieldSchemaTest.java
index 51e3ddc..4134a57 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/JavaFieldSchemaTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/JavaFieldSchemaTest.java
@@ -23,6 +23,7 @@
 import static org.apache.beam.sdk.schemas.utils.TestPOJOs.NESTED_NULLABLE_SCHEMA;
 import static org.apache.beam.sdk.schemas.utils.TestPOJOs.NESTED_POJO_SCHEMA;
 import static org.apache.beam.sdk.schemas.utils.TestPOJOs.NULLABLES_SCHEMA;
+import static org.apache.beam.sdk.schemas.utils.TestPOJOs.POJO_WITH_ENUM_SCHEMA;
 import static org.apache.beam.sdk.schemas.utils.TestPOJOs.POJO_WITH_ITERABLE;
 import static org.apache.beam.sdk.schemas.utils.TestPOJOs.POJO_WITH_NESTED_ARRAY_SCHEMA;
 import static org.apache.beam.sdk.schemas.utils.TestPOJOs.PRIMITIVE_ARRAY_POJO_SCHEMA;
@@ -39,6 +40,7 @@
 import java.util.Arrays;
 import java.util.List;
 import java.util.Map;
+import org.apache.beam.sdk.schemas.logicaltypes.EnumerationType;
 import org.apache.beam.sdk.schemas.utils.SchemaTestUtils;
 import org.apache.beam.sdk.schemas.utils.TestPOJOs.AnnotatedSimplePojo;
 import org.apache.beam.sdk.schemas.utils.TestPOJOs.NestedArrayPOJO;
@@ -47,11 +49,14 @@
 import org.apache.beam.sdk.schemas.utils.TestPOJOs.NestedPOJO;
 import org.apache.beam.sdk.schemas.utils.TestPOJOs.POJOWithNestedNullable;
 import org.apache.beam.sdk.schemas.utils.TestPOJOs.POJOWithNullables;
+import org.apache.beam.sdk.schemas.utils.TestPOJOs.PojoWithEnum;
+import org.apache.beam.sdk.schemas.utils.TestPOJOs.PojoWithEnum.Color;
 import org.apache.beam.sdk.schemas.utils.TestPOJOs.PojoWithIterable;
 import org.apache.beam.sdk.schemas.utils.TestPOJOs.PojoWithNestedArray;
 import org.apache.beam.sdk.schemas.utils.TestPOJOs.PrimitiveArrayPOJO;
 import org.apache.beam.sdk.schemas.utils.TestPOJOs.SimplePOJO;
 import org.apache.beam.sdk.schemas.utils.TestPOJOs.StaticCreationSimplePojo;
+import org.apache.beam.sdk.transforms.SerializableFunction;
 import org.apache.beam.sdk.util.SerializableUtils;
 import org.apache.beam.sdk.values.Row;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
@@ -300,7 +305,7 @@
 
     NestedArrayPOJO pojo = new NestedArrayPOJO(simple1, simple2, simple3);
     Row row = registry.getToRowFunction(NestedArrayPOJO.class).apply(pojo);
-    List<Row> rows = row.getArray("pojos");
+    List<Row> rows = (List) row.getArray("pojos");
     assertSame(simple1, registry.getFromRowFunction(SimplePOJO.class).apply(rows.get(0)));
     assertSame(simple2, registry.getFromRowFunction(SimplePOJO.class).apply(rows.get(1)));
     assertSame(simple3, registry.getFromRowFunction(SimplePOJO.class).apply(rows.get(2)));
@@ -527,4 +532,47 @@
     list.add("three");
     assertEquals(list, Lists.newArrayList(converted.strings));
   }
+
+  @Test
+  public void testEnumFieldToRow() throws NoSuchSchemaException {
+    SchemaRegistry registry = SchemaRegistry.createDefault();
+    Schema schema = registry.getSchema(PojoWithEnum.class);
+    SchemaTestUtils.assertSchemaEquivalent(POJO_WITH_ENUM_SCHEMA, schema);
+    EnumerationType enumerationType =
+        POJO_WITH_ENUM_SCHEMA.getField(0).getType().getLogicalType(EnumerationType.class);
+
+    Row redRow =
+        Row.withSchema(POJO_WITH_ENUM_SCHEMA).addValue(enumerationType.valueOf("RED")).build();
+    Row greenRow =
+        Row.withSchema(POJO_WITH_ENUM_SCHEMA).addValue(enumerationType.valueOf("GREEN")).build();
+    Row blueRow =
+        Row.withSchema(POJO_WITH_ENUM_SCHEMA).addValue(enumerationType.valueOf("BLUE")).build();
+
+    SerializableFunction<PojoWithEnum, Row> toRow = registry.getToRowFunction(PojoWithEnum.class);
+    assertEquals(redRow, toRow.apply(new PojoWithEnum(Color.RED)));
+    assertEquals(greenRow, toRow.apply(new PojoWithEnum(Color.GREEN)));
+    assertEquals(blueRow, toRow.apply(new PojoWithEnum(Color.BLUE)));
+  }
+
+  @Test
+  public void testEnumFieldFromRow() throws NoSuchSchemaException {
+    SchemaRegistry registry = SchemaRegistry.createDefault();
+    Schema schema = registry.getSchema(PojoWithEnum.class);
+    SchemaTestUtils.assertSchemaEquivalent(POJO_WITH_ENUM_SCHEMA, schema);
+    EnumerationType enumerationType =
+        POJO_WITH_ENUM_SCHEMA.getField(0).getType().getLogicalType(EnumerationType.class);
+
+    Row redRow =
+        Row.withSchema(POJO_WITH_ENUM_SCHEMA).addValue(enumerationType.valueOf("RED")).build();
+    Row greenRow =
+        Row.withSchema(POJO_WITH_ENUM_SCHEMA).addValue(enumerationType.valueOf("GREEN")).build();
+    Row blueRow =
+        Row.withSchema(POJO_WITH_ENUM_SCHEMA).addValue(enumerationType.valueOf("BLUE")).build();
+
+    SerializableFunction<Row, PojoWithEnum> fromRow =
+        registry.getFromRowFunction(PojoWithEnum.class);
+    assertEquals(new PojoWithEnum(Color.RED), fromRow.apply(redRow));
+    assertEquals(new PojoWithEnum(Color.GREEN), fromRow.apply(greenRow));
+    assertEquals(new PojoWithEnum(Color.BLUE), fromRow.apply(blueRow));
+  }
 }
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/SchemaTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/SchemaTest.java
index 1cbb84b..1369f59 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/SchemaTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/SchemaTest.java
@@ -24,9 +24,9 @@
 import static org.junit.Assert.assertTrue;
 
 import java.util.stream.Stream;
-import org.apache.beam.sdk.schemas.LogicalTypes.PassThroughLogicalType;
 import org.apache.beam.sdk.schemas.Schema.Field;
 import org.apache.beam.sdk.schemas.Schema.FieldType;
+import org.apache.beam.sdk.schemas.logicaltypes.PassThroughLogicalType;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.ExpectedException;
@@ -296,7 +296,7 @@
 
   static class TestType extends PassThroughLogicalType<Long> {
     TestType(String id, String arg) {
-      super(id, arg, FieldType.INT64);
+      super(id, FieldType.STRING, arg, FieldType.INT64);
     }
   }
 
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/logicaltypes/LogicalTypesTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/logicaltypes/LogicalTypesTest.java
new file mode 100644
index 0000000..657af04
--- /dev/null
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/logicaltypes/LogicalTypesTest.java
@@ -0,0 +1,74 @@
+/*
+ * 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.schemas.logicaltypes;
+
+import static org.junit.Assert.assertEquals;
+
+import java.util.Map;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.schemas.Schema.Field;
+import org.apache.beam.sdk.schemas.Schema.FieldType;
+import org.apache.beam.sdk.schemas.logicaltypes.OneOfType.Value;
+import org.apache.beam.sdk.values.Row;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
+import org.junit.Test;
+
+/** Unit tests for logical types. */
+public class LogicalTypesTest {
+  @Test
+  public void testEnumeration() {
+    Map<String, Integer> enumMap = ImmutableMap.of("FIRST", 1, "SECOND", 2);
+    EnumerationType enumeration = EnumerationType.create(enumMap);
+    assertEquals(enumeration.valueOf(1), enumeration.valueOf("FIRST"));
+    assertEquals(enumeration.valueOf(2), enumeration.valueOf("SECOND"));
+    assertEquals("FIRST", enumeration.valueOf(1).toString());
+    assertEquals(1, enumeration.valueOf("FIRST").getValue());
+    assertEquals("SECOND", enumeration.valueOf(2).toString());
+    assertEquals(2, enumeration.valueOf("SECOND").getValue());
+
+    Schema schema =
+        Schema.builder().addLogicalTypeField("enum", EnumerationType.create(enumMap)).build();
+    Row row1 = Row.withSchema(schema).addValue(enumeration.valueOf(1)).build();
+    Row row2 = Row.withSchema(schema).addValue(enumeration.valueOf("FIRST")).build();
+    assertEquals(row1, row2);
+    assertEquals(1, row1.<EnumerationType.Value>getLogicalTypeValue(0).getValue());
+
+    Row row3 = Row.withSchema(schema).addValue(enumeration.valueOf(2)).build();
+    Row row4 = Row.withSchema(schema).addValue(enumeration.valueOf("SECOND")).build();
+    assertEquals(row3, row4);
+    assertEquals(2, row3.<EnumerationType.Value>getLogicalTypeValue(0).getValue());
+  }
+
+  @Test
+  public void testOneOf() {
+    OneOfType oneOf =
+        OneOfType.create(Field.of("string", FieldType.STRING), Field.of("int32", FieldType.INT32));
+    Schema schema = Schema.builder().addLogicalTypeField("union", oneOf).build();
+
+    Row stringOneOf =
+        Row.withSchema(schema).addValue(oneOf.createValue("string", "stringValue")).build();
+    Value union = stringOneOf.getLogicalTypeValue(0);
+    assertEquals("string", union.getCaseType().toString());
+    assertEquals("stringValue", union.getValue());
+
+    Row intOneOf = Row.withSchema(schema).addValue(oneOf.createValue("int32", 42)).build();
+    union = intOneOf.getLogicalTypeValue(0);
+    assertEquals("int32", union.getCaseType().toString());
+    assertEquals(42, (int) union.getValue());
+  }
+}
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/transforms/CoGroupTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/transforms/CoGroupTest.java
index 73aa085..81517af 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/transforms/CoGroupTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/transforms/CoGroupTest.java
@@ -23,6 +23,7 @@
 import static org.hamcrest.collection.IsIterableContainingInAnyOrder.containsInAnyOrder;
 import static org.junit.Assert.assertThat;
 
+import java.util.Collection;
 import java.util.List;
 import org.apache.beam.sdk.TestUtils.KvMatcher;
 import org.apache.beam.sdk.schemas.Schema;
@@ -691,7 +692,7 @@
       Schema valueSchema = value.getSchema();
       for (int i = 0; i < valueSchema.getFieldCount(); ++i) {
         assertEquals(TypeName.ARRAY, valueSchema.getField(i).getType().getTypeName());
-        fieldMatchers.add(new ArrayFieldMatchesAnyOrder(i, value.getArray(i)));
+        fieldMatchers.add(new ArrayFieldMatchesAnyOrder(i, (List) value.getArray(i)));
       }
       matchers.add(
           KvMatcher.isKv(equalTo(row.getKey()), allOf(fieldMatchers.toArray(new Matcher[0]))));
@@ -715,7 +716,7 @@
         return false;
       }
       Row row = (Row) item;
-      List<Row> actual = row.getArray(fieldIndex);
+      Collection<Row> actual = row.getArray(fieldIndex);
       return containsInAnyOrder(expected).matches(actual);
     }
 
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/transforms/SelectTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/transforms/SelectTest.java
index 15d1379..6deab6d 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/transforms/SelectTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/transforms/SelectTest.java
@@ -424,12 +424,17 @@
         return false;
       }
       PartialRowMultipleArray that = (PartialRowMultipleArray) o;
-      return Objects.equals(field1, that.field1);
+      return Objects.equals(field1, that.field1) && Objects.equals(field3, that.field3);
     }
 
     @Override
     public int hashCode() {
-      return Objects.hash(field1);
+      return Objects.hash(field1, field3);
+    }
+
+    @Override
+    public String toString() {
+      return "PartialRowMultipleArray{" + "field1=" + field1 + ", field3=" + field3 + '}';
     }
   }
 
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/SchemaTestUtils.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/SchemaTestUtils.java
index f1d24b3..15fc797 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/SchemaTestUtils.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/SchemaTestUtils.java
@@ -26,6 +26,6 @@
   // Assert that two schemas are equivalent, ignoring field order. This tests that both schemas
   // (recursively) contain the same fields with the same names, but possibly different orders.
   public static void assertSchemaEquivalent(Schema expected, Schema actual) {
-    assertTrue(actual.equivalent(expected));
+    assertTrue("Expected: " + expected + "  Got: " + actual, actual.equivalent(expected));
   }
 }
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/TestJavaBeans.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/TestJavaBeans.java
index f137477..32cf264 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/TestJavaBeans.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/TestJavaBeans.java
@@ -972,4 +972,45 @@
   /** The schema for {@link NestedArrayBean}. * */
   public static final Schema ITERABLE_BEAM_SCHEMA =
       Schema.builder().addIterableField("strings", FieldType.STRING).build();
+
+  /** A bean containing an Array of ByteArray. * */
+  @DefaultSchema(JavaBeanSchema.class)
+  public static class ArrayOfByteArray {
+    private List<ByteBuffer> byteBuffers;
+
+    public ArrayOfByteArray(List<ByteBuffer> byteBuffers) {
+      this.byteBuffers = byteBuffers;
+    }
+
+    public ArrayOfByteArray() {}
+
+    public List<ByteBuffer> getByteBuffers() {
+      return byteBuffers;
+    }
+
+    public void setByteBuffers(List<ByteBuffer> byteBuffers) {
+      this.byteBuffers = byteBuffers;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+      ArrayOfByteArray that = (ArrayOfByteArray) o;
+      return Objects.equals(byteBuffers, that.byteBuffers);
+    }
+
+    @Override
+    public int hashCode() {
+      return Objects.hash(byteBuffers);
+    }
+  }
+
+  /** The schema for {@link NestedArrayBean}. * */
+  public static final Schema ARRAY_OF_BYTE_ARRAY_BEAM_SCHEMA =
+      Schema.builder().addArrayField("byteBuffers", FieldType.BYTES).build();
 }
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/TestPOJOs.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/TestPOJOs.java
index 4605e54..0952d17 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/TestPOJOs.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/TestPOJOs.java
@@ -31,6 +31,7 @@
 import org.apache.beam.sdk.schemas.annotations.SchemaCreate;
 import org.apache.beam.sdk.schemas.annotations.SchemaFieldName;
 import org.apache.beam.sdk.schemas.annotations.SchemaIgnore;
+import org.apache.beam.sdk.schemas.logicaltypes.EnumerationType;
 import org.joda.time.DateTime;
 import org.joda.time.Instant;
 
@@ -860,4 +861,44 @@
   /** The schema for {@link PojoWithNestedArray}. */
   public static final Schema POJO_WITH_ITERABLE =
       Schema.builder().addIterableField("strings", FieldType.STRING).build();
+
+  /** A Pojo containing an enum type. */
+  @DefaultSchema(JavaFieldSchema.class)
+  public static class PojoWithEnum {
+    public enum Color {
+      RED,
+      GREEN,
+      BLUE
+    };
+
+    public final Color color;
+
+    @SchemaCreate
+    public PojoWithEnum(Color color) {
+      this.color = color;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+      PojoWithEnum that = (PojoWithEnum) o;
+      return color == that.color;
+    }
+
+    @Override
+    public int hashCode() {
+      return Objects.hash(color);
+    }
+  }
+
+  /** The schema for {@link PojoWithEnum}. */
+  public static final Schema POJO_WITH_ENUM_SCHEMA =
+      Schema.builder()
+          .addLogicalTypeField("color", EnumerationType.create("RED", "GREEN", "BLUE"))
+          .build();
 }
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/WatchTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/WatchTest.java
index 615eaf6..7242ad4 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/WatchTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/WatchTest.java
@@ -554,7 +554,8 @@
 
     assertTrue(tracker.tryClaim(KV.of(claim, 1 /* termination state */)));
 
-    PollingGrowthState<Integer> residual = (PollingGrowthState<Integer>) tracker.checkpoint();
+    PollingGrowthState<Integer> residual =
+        (PollingGrowthState<Integer>) tracker.trySplit(0).getResidual();
     NonPollingGrowthState<String> primary =
         (NonPollingGrowthState<String>) tracker.currentRestriction();
     tracker.checkDone();
@@ -574,7 +575,8 @@
   public void testPollingGrowthTrackerCheckpointEmpty() {
     GrowthTracker<String, Integer> tracker = newPollingGrowthTracker();
 
-    PollingGrowthState<Integer> residual = (PollingGrowthState<Integer>) tracker.checkpoint();
+    PollingGrowthState<Integer> residual =
+        (PollingGrowthState<Integer>) tracker.trySplit(0).getResidual();
     GrowthState primary = tracker.currentRestriction();
     tracker.checkDone();
 
@@ -603,7 +605,8 @@
 
     assertTrue(tracker.tryClaim(KV.of(claim, 1 /* termination state */)));
 
-    PollingGrowthState<Integer> residual = (PollingGrowthState<Integer>) tracker.checkpoint();
+    PollingGrowthState<Integer> residual =
+        (PollingGrowthState<Integer>) tracker.trySplit(0).getResidual();
 
     assertFalse(newTracker(residual).tryClaim(KV.of(claim, 2)));
   }
@@ -623,7 +626,7 @@
     GrowthTracker<String, Integer> tracker = newTracker(NonPollingGrowthState.of(claim));
 
     assertTrue(tracker.tryClaim(KV.of(claim, 1 /* termination state */)));
-    GrowthState residual = tracker.checkpoint();
+    GrowthState residual = tracker.trySplit(0).getResidual();
     NonPollingGrowthState<String> primary =
         (NonPollingGrowthState<String>) tracker.currentRestriction();
     tracker.checkDone();
@@ -649,7 +652,8 @@
 
     GrowthTracker<String, Integer> tracker = newTracker(NonPollingGrowthState.of(claim));
 
-    NonPollingGrowthState<String> residual = (NonPollingGrowthState<String>) tracker.checkpoint();
+    NonPollingGrowthState<String> residual =
+        (NonPollingGrowthState<String>) tracker.trySplit(0).getResidual();
     GrowthState primary = tracker.currentRestriction();
     tracker.checkDone();
 
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokersTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokersTest.java
index 0d2be5a..44e943e 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokersTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokersTest.java
@@ -55,6 +55,7 @@
 import org.apache.beam.sdk.transforms.reflect.testhelper.DoFnInvokersTestHelper;
 import org.apache.beam.sdk.transforms.splittabledofn.HasDefaultTracker;
 import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker;
+import org.apache.beam.sdk.transforms.splittabledofn.SplitResult;
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
 import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
 import org.apache.beam.sdk.util.UserCodeException;
@@ -506,7 +507,7 @@
     }
 
     @Override
-    public RestrictionWithDefaultTracker checkpoint() {
+    public SplitResult<RestrictionWithDefaultTracker> trySplit(double fractionOfRemainder) {
       throw new UnsupportedOperationException();
     }
 
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/splittabledofn/ByteKeyRangeTrackerTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/splittabledofn/ByteKeyRangeTrackerTest.java
index da7cd53..57ec887 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/splittabledofn/ByteKeyRangeTrackerTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/splittabledofn/ByteKeyRangeTrackerTest.java
@@ -57,7 +57,7 @@
     ByteKeyRangeTracker tracker =
         ByteKeyRangeTracker.of(ByteKeyRange.of(ByteKey.of(0x10), ByteKey.of(0xc0)));
 
-    ByteKeyRange checkpoint = tracker.checkpoint();
+    ByteKeyRange checkpoint = tracker.trySplit(0).getResidual();
     // We expect to get the original range back and that the current restriction
     // is effectively made empty.
     assertEquals(ByteKeyRange.of(ByteKey.of(0x10), ByteKey.of(0xc0)), checkpoint);
@@ -68,7 +68,7 @@
   public void testCheckpointUnstartedForAllKeysRange() throws Exception {
     ByteKeyRangeTracker tracker = ByteKeyRangeTracker.of(ByteKeyRange.ALL_KEYS);
 
-    ByteKeyRange checkpoint = tracker.checkpoint();
+    ByteKeyRange checkpoint = tracker.trySplit(0).getResidual();
     // We expect to get the original range back and that the current restriction
     // is effectively made empty.
     assertEquals(ByteKeyRange.ALL_KEYS, checkpoint);
@@ -80,7 +80,7 @@
     ByteKeyRangeTracker tracker =
         ByteKeyRangeTracker.of(ByteKeyRange.of(ByteKey.of(0x10), ByteKey.of(0xc0)));
     assertFalse(tracker.tryClaim(ByteKey.of(0xd0)));
-    ByteKeyRange checkpoint = tracker.checkpoint();
+    ByteKeyRange checkpoint = tracker.trySplit(0).getResidual();
     assertEquals(ByteKeyRange.of(ByteKey.of(0x10), ByteKey.of(0xc0)), tracker.currentRestriction());
     assertEquals(ByteKeyRangeTracker.NO_KEYS, checkpoint);
   }
@@ -90,7 +90,7 @@
     ByteKeyRangeTracker tracker =
         ByteKeyRangeTracker.of(ByteKeyRange.of(ByteKey.of(0x10), ByteKey.of(0xc0)));
     assertTrue(tracker.tryClaim(ByteKey.of(0x10)));
-    ByteKeyRange checkpoint = tracker.checkpoint();
+    ByteKeyRange checkpoint = tracker.trySplit(0).getResidual();
     assertEquals(
         ByteKeyRange.of(ByteKey.of(0x10), ByteKey.of(0x10, 0x00)), tracker.currentRestriction());
     assertEquals(ByteKeyRange.of(ByteKey.of(0x10, 0x00), ByteKey.of(0xc0)), checkpoint);
@@ -102,7 +102,7 @@
         ByteKeyRangeTracker.of(ByteKeyRange.of(ByteKey.of(0x10), ByteKey.of(0xc0)));
     assertTrue(tracker.tryClaim(ByteKey.of(0x50)));
     assertTrue(tracker.tryClaim(ByteKey.of(0x90)));
-    ByteKeyRange checkpoint = tracker.checkpoint();
+    ByteKeyRange checkpoint = tracker.trySplit(0).getResidual();
     assertEquals(
         ByteKeyRange.of(ByteKey.of(0x10), ByteKey.of(0x90, 0x00)), tracker.currentRestriction());
     assertEquals(ByteKeyRange.of(ByteKey.of(0x90, 0x00), ByteKey.of(0xc0)), checkpoint);
@@ -115,7 +115,7 @@
     assertTrue(tracker.tryClaim(ByteKey.of(0x50)));
     assertTrue(tracker.tryClaim(ByteKey.of(0x90)));
     assertFalse(tracker.tryClaim(ByteKey.of(0xc0)));
-    ByteKeyRange checkpoint = tracker.checkpoint();
+    ByteKeyRange checkpoint = tracker.trySplit(0).getResidual();
     assertEquals(ByteKeyRange.of(ByteKey.of(0x10), ByteKey.of(0xc0)), tracker.currentRestriction());
     assertEquals(ByteKeyRangeTracker.NO_KEYS, checkpoint);
   }
@@ -126,7 +126,7 @@
     assertTrue(tracker.tryClaim(ByteKey.of(0x50)));
     assertTrue(tracker.tryClaim(ByteKey.of(0x90)));
     assertFalse(tracker.tryClaim(ByteKey.EMPTY));
-    ByteKeyRange checkpoint = tracker.checkpoint();
+    ByteKeyRange checkpoint = tracker.trySplit(0).getResidual();
     assertEquals(ByteKeyRange.ALL_KEYS, tracker.currentRestriction());
     assertEquals(ByteKeyRangeTracker.NO_KEYS, checkpoint);
   }
@@ -139,7 +139,7 @@
     assertTrue(tracker.tryClaim(ByteKey.of(0x90)));
     assertTrue(tracker.tryClaim(ByteKey.of(0xa0)));
     assertFalse(tracker.tryClaim(ByteKey.of(0xd0)));
-    ByteKeyRange checkpoint = tracker.checkpoint();
+    ByteKeyRange checkpoint = tracker.trySplit(0).getResidual();
     assertEquals(ByteKeyRange.of(ByteKey.of(0x10), ByteKey.of(0xc0)), tracker.currentRestriction());
     assertEquals(ByteKeyRangeTracker.NO_KEYS, checkpoint);
   }
@@ -152,7 +152,7 @@
     assertTrue(tracker.tryClaim(ByteKey.of(0x90)));
     assertTrue(tracker.tryClaim(ByteKey.of(0xa0)));
     assertFalse(tracker.tryClaim(ByteKey.EMPTY));
-    ByteKeyRange checkpoint = tracker.checkpoint();
+    ByteKeyRange checkpoint = tracker.trySplit(0).getResidual();
     assertEquals(ByteKeyRange.of(ByteKey.of(0x10), ByteKey.of(0xc0)), tracker.currentRestriction());
     assertEquals(ByteKeyRangeTracker.NO_KEYS, checkpoint);
   }
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/splittabledofn/OffsetRangeTrackerTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/splittabledofn/OffsetRangeTrackerTest.java
index f209247..e4d79da 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/splittabledofn/OffsetRangeTrackerTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/splittabledofn/OffsetRangeTrackerTest.java
@@ -48,7 +48,7 @@
   public void testCheckpointUnstarted() throws Exception {
     OffsetRangeTracker tracker = new OffsetRangeTracker(new OffsetRange(100, 200));
     expected.expect(IllegalStateException.class);
-    tracker.checkpoint();
+    tracker.trySplit(0).getResidual();
   }
 
   @Test
@@ -56,14 +56,14 @@
     OffsetRangeTracker tracker = new OffsetRangeTracker(new OffsetRange(100, 200));
     assertFalse(tracker.tryClaim(250L));
     expected.expect(IllegalStateException.class);
-    OffsetRange checkpoint = tracker.checkpoint();
+    OffsetRange checkpoint = tracker.trySplit(0).getResidual();
   }
 
   @Test
   public void testCheckpointJustStarted() throws Exception {
     OffsetRangeTracker tracker = new OffsetRangeTracker(new OffsetRange(100, 200));
     assertTrue(tracker.tryClaim(100L));
-    OffsetRange checkpoint = tracker.checkpoint();
+    OffsetRange checkpoint = tracker.trySplit(0).getResidual();
     assertEquals(new OffsetRange(100, 101), tracker.currentRestriction());
     assertEquals(new OffsetRange(101, 200), checkpoint);
   }
@@ -73,7 +73,7 @@
     OffsetRangeTracker tracker = new OffsetRangeTracker(new OffsetRange(100, 200));
     assertTrue(tracker.tryClaim(105L));
     assertTrue(tracker.tryClaim(110L));
-    OffsetRange checkpoint = tracker.checkpoint();
+    OffsetRange checkpoint = tracker.trySplit(0).getResidual();
     assertEquals(new OffsetRange(100, 111), tracker.currentRestriction());
     assertEquals(new OffsetRange(111, 200), checkpoint);
   }
@@ -84,7 +84,7 @@
     assertTrue(tracker.tryClaim(105L));
     assertTrue(tracker.tryClaim(110L));
     assertTrue(tracker.tryClaim(199L));
-    OffsetRange checkpoint = tracker.checkpoint();
+    OffsetRange checkpoint = tracker.trySplit(0).getResidual();
     assertEquals(new OffsetRange(100, 200), tracker.currentRestriction());
     assertEquals(new OffsetRange(200, 200), checkpoint);
   }
@@ -96,7 +96,7 @@
     assertTrue(tracker.tryClaim(110L));
     assertTrue(tracker.tryClaim(160L));
     assertFalse(tracker.tryClaim(240L));
-    OffsetRange checkpoint = tracker.checkpoint();
+    OffsetRange checkpoint = tracker.trySplit(0).getResidual();
     assertEquals(new OffsetRange(100, 161), tracker.currentRestriction());
     assertEquals(new OffsetRange(161, 200), checkpoint);
   }
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/splittabledofn/SplitResultTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/splittabledofn/SplitResultTest.java
new file mode 100644
index 0000000..47c54bc
--- /dev/null
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/splittabledofn/SplitResultTest.java
@@ -0,0 +1,34 @@
+/*
+ * 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.transforms.splittabledofn;
+
+import static org.junit.Assert.assertEquals;
+
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+@RunWith(JUnit4.class)
+public class SplitResultTest {
+  @Test
+  public void testPrimaryAndResidualAreSet() {
+    SplitResult<String> result = SplitResult.of("primary", "residual");
+    assertEquals("primary", result.getPrimary());
+    assertEquals("residual", result.getResidual());
+  }
+}
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 85b9aae..df77a36 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
@@ -30,9 +30,9 @@
 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.schemas.logicaltypes.PassThroughLogicalType;
 import org.apache.beam.sdk.util.RowJson.RowJsonDeserializer.UnsupportedRowJsonException;
 import org.apache.beam.sdk.values.Row;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
@@ -126,8 +126,8 @@
           Schema.builder()
               .addLogicalTypeField(
                   "f_passThroughString",
-                  new LogicalTypes.PassThroughLogicalType<String>(
-                      "SqlCharType", "", FieldType.STRING) {})
+                  new PassThroughLogicalType<String>(
+                      "SqlCharType", FieldType.STRING, "", FieldType.STRING) {})
               .build();
 
       String rowString = "{\n" + "\"f_passThroughString\" : \"hello\"\n" + "}";
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamUnnestRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamUnnestRel.java
index 1263b3d..9f27a4a 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamUnnestRel.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamUnnestRel.java
@@ -17,7 +17,7 @@
  */
 package org.apache.beam.sdk.extensions.sql.impl.rel;
 
-import java.util.List;
+import java.util.Collection;
 import javax.annotation.Nullable;
 import org.apache.beam.sdk.extensions.sql.impl.planner.BeamCostModel;
 import org.apache.beam.sdk.extensions.sql.impl.planner.NodeStats;
@@ -129,7 +129,7 @@
     @ProcessElement
     public void process(@Element Row row, OutputReceiver<Row> out) {
 
-      @Nullable List<Object> rawValues = row.getArray(unnestIndex);
+      @Nullable Collection<Object> rawValues = row.getArray(unnestIndex);
 
       if (rawValues == null) {
         return;
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/utils/CalciteUtils.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/utils/CalciteUtils.java
index c6a0670..90c49aa 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/utils/CalciteUtils.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/utils/CalciteUtils.java
@@ -21,10 +21,10 @@
 import java.util.Date;
 import java.util.Map;
 import java.util.stream.IntStream;
-import org.apache.beam.sdk.schemas.LogicalTypes.PassThroughLogicalType;
 import org.apache.beam.sdk.schemas.Schema;
 import org.apache.beam.sdk.schemas.Schema.FieldType;
 import org.apache.beam.sdk.schemas.Schema.TypeName;
+import org.apache.beam.sdk.schemas.logicaltypes.PassThroughLogicalType;
 import org.apache.beam.vendor.calcite.v1_20_0.com.google.common.collect.BiMap;
 import org.apache.beam.vendor.calcite.v1_20_0.com.google.common.collect.ImmutableBiMap;
 import org.apache.beam.vendor.calcite.v1_20_0.com.google.common.collect.ImmutableMap;
@@ -48,7 +48,7 @@
     public static final String IDENTIFIER = "SqlDateType";
 
     public DateType() {
-      super(IDENTIFIER, "", FieldType.DATETIME);
+      super(IDENTIFIER, FieldType.STRING, "", FieldType.DATETIME);
     }
   }
 
@@ -57,7 +57,7 @@
     public static final String IDENTIFIER = "SqlTimeType";
 
     public TimeType() {
-      super(IDENTIFIER, "", FieldType.DATETIME);
+      super(IDENTIFIER, FieldType.STRING, "", FieldType.DATETIME);
     }
   }
 
@@ -66,7 +66,7 @@
     public static final String IDENTIFIER = "SqlTimeWithLocalTzType";
 
     public TimeWithLocalTzType() {
-      super(IDENTIFIER, "", FieldType.DATETIME);
+      super(IDENTIFIER, FieldType.STRING, "", FieldType.DATETIME);
     }
   }
 
@@ -75,7 +75,7 @@
     public static final String IDENTIFIER = "SqlTimestampWithLocalTzType";
 
     public TimestampWithLocalTzType() {
-      super(IDENTIFIER, "", FieldType.DATETIME);
+      super(IDENTIFIER, FieldType.STRING, "", FieldType.DATETIME);
     }
   }
 
@@ -84,7 +84,7 @@
     public static final String IDENTIFIER = "SqlCharType";
 
     public CharType() {
-      super(IDENTIFIER, "", FieldType.STRING);
+      super(IDENTIFIER, FieldType.STRING, "", FieldType.STRING);
     }
   }
 
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 7b8ce03..9b06a12 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
@@ -20,15 +20,22 @@
 import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument;
 
 import java.io.Serializable;
+import java.util.List;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 import org.apache.beam.sdk.annotations.Experimental;
 import org.apache.beam.sdk.extensions.sql.impl.BeamTableStatistics;
+import org.apache.beam.sdk.extensions.sql.meta.BeamSqlTableFilter;
+import org.apache.beam.sdk.extensions.sql.meta.DefaultTableFilter;
+import org.apache.beam.sdk.extensions.sql.meta.ProjectSupport;
 import org.apache.beam.sdk.extensions.sql.meta.SchemaBaseBeamTable;
 import org.apache.beam.sdk.extensions.sql.meta.Table;
+import org.apache.beam.sdk.io.mongodb.FindQuery;
 import org.apache.beam.sdk.io.mongodb.MongoDbIO;
 import org.apache.beam.sdk.options.PipelineOptions;
+import org.apache.beam.sdk.schemas.FieldAccessDescriptor;
 import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.schemas.utils.SelectHelpers;
 import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.transforms.JsonToRow;
 import org.apache.beam.sdk.transforms.MapElements;
@@ -85,6 +92,29 @@
   }
 
   @Override
+  public PCollection<Row> buildIOReader(
+      PBegin begin, BeamSqlTableFilter filters, List<String> fieldNames) {
+    MongoDbIO.Read readInstance =
+        MongoDbIO.read().withUri(dbUri).withDatabase(dbName).withCollection(dbCollection);
+
+    final FieldAccessDescriptor resolved =
+        FieldAccessDescriptor.withFieldNames(fieldNames)
+            .withOrderByFieldInsertionOrder()
+            .resolve(getSchema());
+    final Schema newSchema = SelectHelpers.getOutputSchema(getSchema(), resolved);
+
+    if (!(filters instanceof DefaultTableFilter)) {
+      throw new AssertionError("Predicate push-down is unsupported, yet received a predicate.");
+    }
+
+    if (!fieldNames.isEmpty()) {
+      readInstance = readInstance.withQueryFn(FindQuery.create().withProjection(fieldNames));
+    }
+
+    return readInstance.expand(begin).apply(DocumentToRow.withSchema(newSchema));
+  }
+
+  @Override
   public POutput buildIOWriter(PCollection<Row> input) {
     return input
         .apply(new RowToDocument())
@@ -92,6 +122,11 @@
   }
 
   @Override
+  public ProjectSupport supportsProjects() {
+    return ProjectSupport.WITH_FIELD_REORDERING;
+  }
+
+  @Override
   public IsBounded isBounded() {
     return IsBounded.BOUNDED;
   }
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamComplexTypeTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamComplexTypeTest.java
index 24d23c9..45e07ff 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamComplexTypeTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamComplexTypeTest.java
@@ -380,6 +380,16 @@
     }
 
     @Override
+    public FieldType getArgumentType() {
+      return FieldType.STRING;
+    }
+
+    @Override
+    public String getArgument() {
+      return "";
+    }
+
+    @Override
     public Schema.FieldType getBaseType() {
       return Schema.FieldType.DATETIME;
     }
@@ -402,6 +412,16 @@
     }
 
     @Override
+    public FieldType getArgumentType() {
+      return FieldType.STRING;
+    }
+
+    @Override
+    public String getArgument() {
+      return "";
+    }
+
+    @Override
     public Schema.FieldType getBaseType() {
       return Schema.FieldType.DATETIME;
     }
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 aa11690..0d4296a 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
@@ -25,9 +25,16 @@
 import static org.apache.beam.sdk.schemas.Schema.FieldType.INT32;
 import static org.apache.beam.sdk.schemas.Schema.FieldType.INT64;
 import static org.apache.beam.sdk.schemas.Schema.FieldType.STRING;
-import static org.junit.Assert.assertEquals;
+import static org.apache.beam.sdk.testing.SerializableMatchers.containsInAnyOrder;
+import static org.apache.beam.sdk.testing.SerializableMatchers.equalTo;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.core.IsInstanceOf.instanceOf;
 
+import com.mongodb.BasicDBObject;
 import com.mongodb.MongoClient;
+import com.mongodb.client.MongoCollection;
+import com.mongodb.client.MongoDatabase;
+import com.mongodb.client.model.Filters;
 import de.flapdoodle.embed.mongo.MongodExecutable;
 import de.flapdoodle.embed.mongo.MongodProcess;
 import de.flapdoodle.embed.mongo.MongodStarter;
@@ -40,6 +47,7 @@
 import de.flapdoodle.embed.process.runtime.Network;
 import java.util.Arrays;
 import org.apache.beam.sdk.extensions.sql.impl.BeamSqlEnv;
+import org.apache.beam.sdk.extensions.sql.impl.rel.BeamPushDownIOSourceRel;
 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.common.NetworkTestHelper;
@@ -49,7 +57,10 @@
 import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.Row;
+import org.bson.Document;
+import org.junit.After;
 import org.junit.AfterClass;
+import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.ClassRule;
 import org.junit.Rule;
@@ -69,7 +80,6 @@
   private static final Logger LOG = LoggerFactory.getLogger(MongoDbReadWriteIT.class);
   private static final Schema SOURCE_SCHEMA =
       Schema.builder()
-          .addNullableField("_id", STRING)
           .addNullableField("c_bigint", INT64)
           .addNullableField("c_tinyint", BYTE)
           .addNullableField("c_smallint", INT16)
@@ -83,7 +93,6 @@
   private static final String hostname = "localhost";
   private static final String database = "beam";
   private static final String collection = "collection";
-  private static int port;
 
   @ClassRule public static final TemporaryFolder MONGODB_LOCATION = new TemporaryFolder();
 
@@ -92,12 +101,15 @@
   private static MongodProcess mongodProcess;
   private static MongoClient client;
 
+  private static BeamSqlEnv sqlEnv;
+  private static String mongoSqlUrl;
+
   @Rule public final TestPipeline writePipeline = TestPipeline.create();
   @Rule public final TestPipeline readPipeline = TestPipeline.create();
 
   @BeforeClass
   public static void setUp() throws Exception {
-    port = NetworkTestHelper.getAvailableLocalPort();
+    int port = NetworkTestHelper.getAvailableLocalPort();
     LOG.info("Starting MongoDB embedded instance on {}", port);
     IMongodConfig mongodConfig =
         new MongodConfigBuilder()
@@ -117,6 +129,8 @@
     mongodExecutable = mongodStarter.prepare(mongodConfig);
     mongodProcess = mongodExecutable.start();
     client = new MongoClient(hostname, port);
+
+    mongoSqlUrl = String.format("mongodb://%s:%d/%s/%s", hostname, port, database, collection);
   }
 
   @AfterClass
@@ -127,15 +141,23 @@
     mongodExecutable.stop();
   }
 
+  @Before
+  public void init() {
+    sqlEnv = BeamSqlEnv.inMemory(new MongoDbTableProvider());
+    MongoDatabase db = client.getDatabase(database);
+    Document r = db.runCommand(new BasicDBObject().append("profile", 2));
+  }
+
+  @After
+  public void cleanUp() {
+    client.getDatabase(database).drop();
+  }
+
   @Test
   public void testWriteAndRead() {
-    final String mongoSqlUrl =
-        String.format("mongodb://%s:%d/%s/%s", hostname, port, database, collection);
-
     Row testRow =
         row(
             SOURCE_SCHEMA,
-            "object_id",
             9223372036854775807L,
             (byte) 127,
             (short) 32767,
@@ -148,7 +170,6 @@
 
     String createTableStatement =
         "CREATE EXTERNAL TABLE TEST( \n"
-            + "   _id VARCHAR, \n "
             + "   c_bigint BIGINT, \n "
             + "   c_tinyint TINYINT, \n"
             + "   c_smallint SMALLINT, \n"
@@ -163,12 +184,10 @@
             + "LOCATION '"
             + mongoSqlUrl
             + "'";
-    BeamSqlEnv sqlEnv = BeamSqlEnv.inMemory(new MongoDbTableProvider());
     sqlEnv.executeDdl(createTableStatement);
 
     String insertStatement =
         "INSERT INTO TEST VALUES ("
-            + "'object_id', "
             + "9223372036854775807, "
             + "127, "
             + "32767, "
@@ -187,13 +206,95 @@
     PCollection<Row> output =
         BeamSqlRelUtils.toPCollection(readPipeline, sqlEnv.parseQuery("select * from TEST"));
 
-    assertEquals(output.getSchema(), SOURCE_SCHEMA);
+    assertThat(output.getSchema(), equalTo(SOURCE_SCHEMA));
 
     PAssert.that(output).containsInAnyOrder(testRow);
 
     readPipeline.run().waitUntilFinish();
   }
 
+  @Test
+  public void testProjectPushDown() {
+    final Schema expectedSchema =
+        Schema.builder()
+            .addNullableField("c_varchar", STRING)
+            .addNullableField("c_boolean", BOOLEAN)
+            .addNullableField("c_integer", INT32)
+            .build();
+    Row testRow = row(expectedSchema, "varchar", true, 2147483647);
+
+    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_varchar VARCHAR, \n "
+            + "   c_arr ARRAY<VARCHAR> \n"
+            + ") \n"
+            + "TYPE 'mongodb' \n"
+            + "LOCATION '"
+            + mongoSqlUrl
+            + "'";
+    sqlEnv.executeDdl(createTableStatement);
+
+    String insertStatement =
+        "INSERT INTO TEST VALUES ("
+            + "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();
+
+    BeamRelNode node = sqlEnv.parseQuery("select c_varchar, c_boolean, c_integer from TEST");
+    // Calc should be dropped, since MongoDb supports project push-down and field reordering.
+    assertThat(node, instanceOf(BeamPushDownIOSourceRel.class));
+    // Only selected fields are projected.
+    assertThat(
+        node.getRowType().getFieldNames(),
+        containsInAnyOrder("c_varchar", "c_boolean", "c_integer"));
+    PCollection<Row> output = BeamSqlRelUtils.toPCollection(readPipeline, node);
+
+    assertThat(output.getSchema(), equalTo(expectedSchema));
+    PAssert.that(output).containsInAnyOrder(testRow);
+
+    readPipeline.run().waitUntilFinish();
+
+    MongoDatabase db = client.getDatabase(database);
+    MongoCollection coll = db.getCollection("system.profile");
+    // Find the last executed query.
+    Object query =
+        coll.find()
+            .filter(Filters.eq("op", "query"))
+            .sort(new BasicDBObject().append("ts", -1))
+            .iterator()
+            .next();
+
+    // Retrieve a projection parameters.
+    assertThat(query, instanceOf(Document.class));
+    Object command = ((Document) query).get("command");
+    assertThat(command, instanceOf(Document.class));
+    Object projection = ((Document) command).get("projection");
+    assertThat(projection, instanceOf(Document.class));
+
+    // Validate projected fields.
+    assertThat(
+        ((Document) projection).keySet(),
+        containsInAnyOrder("c_varchar", "c_boolean", "c_integer"));
+  }
+
   private Row row(Schema schema, Object... values) {
     return Row.withSchema(schema).addValues(values).build();
   }
diff --git a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/splittabledofn/RestrictionTrackers.java b/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/splittabledofn/RestrictionTrackers.java
index 88428af..82935d8 100644
--- a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/splittabledofn/RestrictionTrackers.java
+++ b/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/splittabledofn/RestrictionTrackers.java
@@ -20,6 +20,7 @@
 import javax.annotation.concurrent.ThreadSafe;
 import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker;
 import org.apache.beam.sdk.transforms.splittabledofn.Sizes;
+import org.apache.beam.sdk.transforms.splittabledofn.SplitResult;
 
 /** Support utilities for interacting with {@link RestrictionTracker RestrictionTrackers}. */
 public class RestrictionTrackers {
@@ -67,8 +68,8 @@
     }
 
     @Override
-    public synchronized RestrictionT checkpoint() {
-      return delegate.checkpoint();
+    public synchronized SplitResult<RestrictionT> trySplit(double fractionOfRemainder) {
+      return delegate.trySplit(fractionOfRemainder);
     }
 
     @Override
diff --git a/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/splittabledofn/RestrictionTrackersTest.java b/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/splittabledofn/RestrictionTrackersTest.java
index 8f2b5de..e595151 100644
--- a/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/splittabledofn/RestrictionTrackersTest.java
+++ b/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/splittabledofn/RestrictionTrackersTest.java
@@ -27,6 +27,7 @@
 import org.apache.beam.sdk.fn.splittabledofn.RestrictionTrackers.ClaimObserver;
 import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker;
 import org.apache.beam.sdk.transforms.splittabledofn.Sizes;
+import org.apache.beam.sdk.transforms.splittabledofn.SplitResult;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
@@ -50,7 +51,7 @@
           }
 
           @Override
-          public Object checkpoint() {
+          public SplitResult<Object> trySplit(double fractionOfRemainder) {
             throw new UnsupportedOperationException();
           }
 
@@ -103,7 +104,7 @@
     }
 
     @Override
-    public Object checkpoint() {
+    public SplitResult<Object> trySplit(double fractionOfRemainder) {
       return null;
     }
 
diff --git a/sdks/java/io/clickhouse/src/main/java/org/apache/beam/sdk/io/clickhouse/ClickHouseIO.java b/sdks/java/io/clickhouse/src/main/java/org/apache/beam/sdk/io/clickhouse/ClickHouseIO.java
index 5d7d79f..05afcb3 100644
--- a/sdks/java/io/clickhouse/src/main/java/org/apache/beam/sdk/io/clickhouse/ClickHouseIO.java
+++ b/sdks/java/io/clickhouse/src/main/java/org/apache/beam/sdk/io/clickhouse/ClickHouseIO.java
@@ -33,8 +33,8 @@
 import org.apache.beam.sdk.metrics.Distribution;
 import org.apache.beam.sdk.metrics.Metrics;
 import org.apache.beam.sdk.schemas.FieldAccessDescriptor;
-import org.apache.beam.sdk.schemas.LogicalTypes;
 import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.schemas.logicaltypes.FixedBytes;
 import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.transforms.PTransform;
 import org.apache.beam.sdk.transforms.ParDo;
@@ -93,7 +93,7 @@
  * <tr><th>ClickHouse</th> <th>Beam</th></tr>
  * <tr><td>{@link TableSchema.TypeName#FLOAT32}</td> <td>{@link Schema.TypeName#FLOAT}</td></tr>
  * <tr><td>{@link TableSchema.TypeName#FLOAT64}</td> <td>{@link Schema.TypeName#DOUBLE}</td></tr>
- * <tr><td>{@link TableSchema.TypeName#FIXEDSTRING}</td> <td>{@link LogicalTypes.FixedBytes}</td></tr>
+ * <tr><td>{@link TableSchema.TypeName#FIXEDSTRING}</td> <td>{@link FixedBytes}</td></tr>
  * <tr><td>{@link TableSchema.TypeName#INT8}</td> <td>{@link Schema.TypeName#BYTE}</td></tr>
  * <tr><td>{@link TableSchema.TypeName#INT16}</td> <td>{@link Schema.TypeName#INT16}</td></tr>
  * <tr><td>{@link TableSchema.TypeName#INT32}</td> <td>{@link Schema.TypeName#INT32}</td></tr>
diff --git a/sdks/java/io/clickhouse/src/main/java/org/apache/beam/sdk/io/clickhouse/TableSchema.java b/sdks/java/io/clickhouse/src/main/java/org/apache/beam/sdk/io/clickhouse/TableSchema.java
index 0600993..82d5b66 100644
--- a/sdks/java/io/clickhouse/src/main/java/org/apache/beam/sdk/io/clickhouse/TableSchema.java
+++ b/sdks/java/io/clickhouse/src/main/java/org/apache/beam/sdk/io/clickhouse/TableSchema.java
@@ -26,8 +26,8 @@
 import java.util.Optional;
 import javax.annotation.Nullable;
 import org.apache.beam.sdk.annotations.Experimental;
-import org.apache.beam.sdk.schemas.LogicalTypes;
 import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.schemas.logicaltypes.FixedBytes;
 
 /** A descriptor for ClickHouse table schema. */
 @Experimental(Experimental.Kind.SOURCE_SINK)
@@ -76,7 +76,7 @@
 
       case FIXEDSTRING:
         int size = columnType.fixedStringSize(); // non-null for fixed strings
-        return Schema.FieldType.logicalType(LogicalTypes.FixedBytes.of(size));
+        return Schema.FieldType.logicalType(FixedBytes.of(size));
 
       case FLOAT32:
         return Schema.FieldType.FLOAT;
diff --git a/sdks/java/io/clickhouse/src/test/java/org/apache/beam/sdk/io/clickhouse/ClickHouseIOTest.java b/sdks/java/io/clickhouse/src/test/java/org/apache/beam/sdk/io/clickhouse/ClickHouseIOTest.java
index 5230481..a7a8004 100644
--- a/sdks/java/io/clickhouse/src/test/java/org/apache/beam/sdk/io/clickhouse/ClickHouseIOTest.java
+++ b/sdks/java/io/clickhouse/src/test/java/org/apache/beam/sdk/io/clickhouse/ClickHouseIOTest.java
@@ -25,10 +25,10 @@
 import java.util.Objects;
 import org.apache.beam.sdk.io.clickhouse.TableSchema.ColumnType;
 import org.apache.beam.sdk.schemas.JavaFieldSchema;
-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.schemas.annotations.DefaultSchema;
+import org.apache.beam.sdk.schemas.logicaltypes.FixedBytes;
 import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.transforms.Create;
 import org.apache.beam.sdk.values.Row;
@@ -158,7 +158,7 @@
             Schema.Field.of("f14", FieldType.STRING),
             Schema.Field.of("f15", FieldType.STRING),
             Schema.Field.of("f16", FieldType.BYTES),
-            Schema.Field.of("f17", FieldType.logicalType(LogicalTypes.FixedBytes.of(3))));
+            Schema.Field.of("f17", FieldType.logicalType(FixedBytes.of(3))));
     Row row1 =
         Row.withSchema(schema)
             .addValue(new DateTime(2030, 10, 1, 0, 0, 0, DateTimeZone.UTC))
diff --git a/sdks/java/io/common/src/test/java/org/apache/beam/sdk/io/common/DatabaseTestHelper.java b/sdks/java/io/common/src/test/java/org/apache/beam/sdk/io/common/DatabaseTestHelper.java
index c324c4d..56b7230 100644
--- a/sdks/java/io/common/src/test/java/org/apache/beam/sdk/io/common/DatabaseTestHelper.java
+++ b/sdks/java/io/common/src/test/java/org/apache/beam/sdk/io/common/DatabaseTestHelper.java
@@ -18,10 +18,12 @@
 package org.apache.beam.sdk.io.common;
 
 import java.sql.Connection;
+import java.sql.ResultSet;
 import java.sql.SQLException;
 import java.sql.Statement;
 import java.text.SimpleDateFormat;
 import java.util.Date;
+import java.util.Optional;
 import javax.sql.DataSource;
 import org.postgresql.ds.PGSimpleDataSource;
 
@@ -79,6 +81,21 @@
         options.getPostgresDatabaseName());
   }
 
+  public static Optional<Long> getPostgresTableSize(DataSource dataSource, String tableName) {
+    try (Connection connection = dataSource.getConnection()) {
+      try (Statement statement = connection.createStatement()) {
+        ResultSet resultSet =
+            statement.executeQuery(String.format("select pg_relation_size('%s')", tableName));
+        if (resultSet.next()) {
+          return Optional.of(resultSet.getLong(1));
+        }
+      }
+    } catch (SQLException e) {
+      return Optional.empty();
+    }
+    return Optional.empty();
+  }
+
   public static void createTableWithStatement(DataSource dataSource, String stmt)
       throws SQLException {
     try (Connection connection = dataSource.getConnection()) {
diff --git a/sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java b/sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java
index 13073a1..59f6057 100644
--- a/sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java
+++ b/sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java
@@ -73,6 +73,7 @@
 import org.apache.http.client.CredentialsProvider;
 import org.apache.http.client.config.RequestConfig;
 import org.apache.http.conn.ssl.TrustSelfSignedStrategy;
+import org.apache.http.conn.ssl.TrustStrategy;
 import org.apache.http.entity.BufferedHttpEntity;
 import org.apache.http.entity.ContentType;
 import org.apache.http.impl.client.BasicCredentialsProvider;
@@ -253,6 +254,8 @@
     @Nullable
     public abstract Integer getConnectTimeout();
 
+    public abstract boolean isTrustSelfSignedCerts();
+
     abstract Builder builder();
 
     @AutoValue.Builder
@@ -275,6 +278,8 @@
 
       abstract Builder setConnectTimeout(Integer connectTimeout);
 
+      abstract Builder setTrustSelfSignedCerts(boolean trustSelfSignedCerts);
+
       abstract ConnectionConfiguration build();
     }
 
@@ -295,6 +300,7 @@
           .setAddresses(Arrays.asList(addresses))
           .setIndex(index)
           .setType(type)
+          .setTrustSelfSignedCerts(false)
           .build();
     }
 
@@ -352,6 +358,18 @@
     }
 
     /**
+     * If Elasticsearch uses SSL/TLS then configure whether to trust self signed certs or not. The
+     * default is false.
+     *
+     * @param trustSelfSignedCerts Whether to trust self signed certs
+     * @return a {@link ConnectionConfiguration} describes a connection configuration to
+     *     Elasticsearch.
+     */
+    public ConnectionConfiguration withTrustSelfSignedCerts(boolean trustSelfSignedCerts) {
+      return builder().setTrustSelfSignedCerts(trustSelfSignedCerts).build();
+    }
+
+    /**
      * If set, overwrites the default max retry timeout (30000ms) in the Elastic {@link RestClient}
      * and the default socket timeout (30000ms) in the {@link RequestConfig} of the Elastic {@link
      * RestClient}.
@@ -386,6 +404,7 @@
       builder.addIfNotNull(DisplayData.item("keystore.path", getKeystorePath()));
       builder.addIfNotNull(DisplayData.item("socketAndRetryTimeout", getSocketAndRetryTimeout()));
       builder.addIfNotNull(DisplayData.item("connectTimeout", getConnectTimeout()));
+      builder.addIfNotNull(DisplayData.item("trustSelfSignedCerts", isTrustSelfSignedCerts()));
     }
 
     @VisibleForTesting
@@ -413,10 +432,10 @@
             String keystorePassword = getKeystorePassword();
             keyStore.load(is, (keystorePassword == null) ? null : keystorePassword.toCharArray());
           }
+          final TrustStrategy trustStrategy =
+              isTrustSelfSignedCerts() ? new TrustSelfSignedStrategy() : null;
           final SSLContext sslContext =
-              SSLContexts.custom()
-                  .loadTrustMaterial(keyStore, new TrustSelfSignedStrategy())
-                  .build();
+              SSLContexts.custom().loadTrustMaterial(keyStore, trustStrategy).build();
           final SSLIOSessionStrategy sessionStrategy = new SSLIOSessionStrategy(sslContext);
           restClientBuilder.setHttpClientConfigCallback(
               httpClientBuilder ->
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/AvroWriteRequest.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/AvroWriteRequest.java
index bea79c6..24f7890 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/AvroWriteRequest.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/AvroWriteRequest.java
@@ -23,7 +23,7 @@
   private final T element;
   private final Schema schema;
 
-  AvroWriteRequest(T element, Schema schema) {
+  public AvroWriteRequest(T element, Schema schema) {
     this.element = element;
     this.schema = schema;
   }
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtils.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtils.java
index c5f1279..e1a171a 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtils.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtils.java
@@ -36,11 +36,11 @@
 import java.util.stream.IntStream;
 import org.apache.avro.generic.GenericData;
 import org.apache.avro.generic.GenericRecord;
-import org.apache.beam.sdk.schemas.LogicalTypes;
 import org.apache.beam.sdk.schemas.Schema;
 import org.apache.beam.sdk.schemas.Schema.Field;
 import org.apache.beam.sdk.schemas.Schema.FieldType;
 import org.apache.beam.sdk.schemas.Schema.TypeName;
+import org.apache.beam.sdk.schemas.logicaltypes.PassThroughLogicalType;
 import org.apache.beam.sdk.transforms.SerializableFunction;
 import org.apache.beam.sdk.transforms.SerializableFunctions;
 import org.apache.beam.sdk.values.Row;
@@ -237,16 +237,16 @@
         return FieldType.DATETIME;
       case "TIME":
         return FieldType.logicalType(
-            new LogicalTypes.PassThroughLogicalType<Instant>(
-                "SqlTimeType", "", FieldType.DATETIME) {});
+            new PassThroughLogicalType<Instant>(
+                "SqlTimeType", FieldType.STRING, "", FieldType.DATETIME) {});
       case "DATE":
         return FieldType.logicalType(
-            new LogicalTypes.PassThroughLogicalType<Instant>(
-                "SqlDateType", "", FieldType.DATETIME) {});
+            new PassThroughLogicalType<Instant>(
+                "SqlDateType", FieldType.STRING, "", FieldType.DATETIME) {});
       case "DATETIME":
         return FieldType.logicalType(
-            new LogicalTypes.PassThroughLogicalType<Instant>(
-                "SqlTimestampWithLocalTzType", "", FieldType.DATETIME) {});
+            new PassThroughLogicalType<Instant>(
+                "SqlTimestampWithLocalTzType", FieldType.STRING, "", FieldType.DATETIME) {});
       case "STRUCT":
       case "RECORD":
         Schema rowSchema = fromTableFieldSchema(nestedFields);
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtilsTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtilsTest.java
index a215635..f4588b0 100644
--- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtilsTest.java
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtilsTest.java
@@ -38,6 +38,7 @@
 import org.apache.avro.generic.GenericData;
 import org.apache.beam.sdk.io.gcp.bigquery.BigQueryUtils.ConversionOptions.TruncateTimestamps;
 import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.schemas.Schema.FieldType;
 import org.apache.beam.sdk.schemas.utils.AvroUtils;
 import org.apache.beam.sdk.values.Row;
 import org.joda.time.DateTime;
@@ -480,6 +481,11 @@
     }
 
     @Override
+    public FieldType getArgumentType() {
+      return FieldType.STRING;
+    }
+
+    @Override
     public Schema.FieldType getBaseType() {
       return Schema.FieldType.DATETIME;
     }
diff --git a/sdks/java/io/hadoop-file-system/src/main/java/org/apache/beam/sdk/io/hdfs/HadoopFileSystemRegistrar.java b/sdks/java/io/hadoop-file-system/src/main/java/org/apache/beam/sdk/io/hdfs/HadoopFileSystemRegistrar.java
index 9d16a41..8368aff 100644
--- a/sdks/java/io/hadoop-file-system/src/main/java/org/apache/beam/sdk/io/hdfs/HadoopFileSystemRegistrar.java
+++ b/sdks/java/io/hadoop-file-system/src/main/java/org/apache/beam/sdk/io/hdfs/HadoopFileSystemRegistrar.java
@@ -36,7 +36,6 @@
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
 
 /** {@link AutoService} registrar for the {@link HadoopFileSystem}. */
 @AutoService(FileSystemRegistrar.class)
@@ -45,6 +44,11 @@
 
   private static final List<String> HA_SCHEMES = Arrays.asList("hdfs", "webhdfs");
 
+  // Using hard-coded value to avoid incompatibility between HDFS client
+  // (org.apache.hadoop:hadoop-dfs-client) version 2.7's DFSConfigKeys and version 2.8's
+  // HdfsClientConfigKeys.
+  private static final String CONFIG_KEY_DFS_NAMESERVICES = "dfs.nameservices";
+
   @Override
   public Iterable<FileSystem> fromOptions(@Nonnull PipelineOptions options) {
     final List<Configuration> configurations =
@@ -75,7 +79,7 @@
       builder.add(new HadoopFileSystem(scheme, configuration));
       registeredSchemes.add(scheme);
     }
-    final String nameServices = configuration.get(DFSConfigKeys.DFS_NAMESERVICES);
+    final String nameServices = configuration.get(CONFIG_KEY_DFS_NAMESERVICES);
     if (nameServices != null && !nameServices.isEmpty()) {
       // we can register schemes that are support by HA cluster
       for (String scheme : HA_SCHEMES) {
diff --git a/sdks/java/io/hadoop-format/build.gradle b/sdks/java/io/hadoop-format/build.gradle
index d575d40..1e0cf94 100644
--- a/sdks/java/io/hadoop-format/build.gradle
+++ b/sdks/java/io/hadoop-format/build.gradle
@@ -73,6 +73,8 @@
     exclude group: "org.apache.spark", module: "spark-sql_2.10"
     exclude group: "org.apache.storm", module: "storm-core"
   }
+  // elasticsearch-hadoop 5.0.0 uses commons-httpclient's URIException
+  testCompile "commons-httpclient:commons-httpclient:3.1"
   testCompile library.java.cassandra_driver_core
   testCompile library.java.cassandra_driver_mapping
   testCompile "org.apache.cassandra:cassandra-all:3.11.3"
diff --git a/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcUtil.java b/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcUtil.java
index 023dbf5..d2c76c3 100644
--- a/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcUtil.java
+++ b/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcUtil.java
@@ -186,9 +186,9 @@
   private static void validateLogicalTypeLength(Schema.Field field, Integer length) {
     try {
       if (field.getType().getTypeName().isLogicalType()
-          && !field.getType().getLogicalType().getArgument().isEmpty()) {
-        int maxLimit = Integer.parseInt(field.getType().getLogicalType().getArgument());
-        if (field.getType().getTypeName().isLogicalType() && length >= maxLimit) {
+          && field.getType().getLogicalType().getArgument() != null) {
+        int maxLimit = (Integer) field.getType().getLogicalType().getArgument();
+        if (length >= maxLimit) {
           throw new RuntimeException(
               String.format(
                   "Length of Schema.Field[%s] data exceeds database column capacity",
diff --git a/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/LogicalTypes.java b/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/LogicalTypes.java
index e8b67e6..01eee3f 100644
--- a/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/LogicalTypes.java
+++ b/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/LogicalTypes.java
@@ -26,34 +26,40 @@
 import java.util.Objects;
 import org.apache.beam.repackaged.core.org.apache.commons.lang3.StringUtils;
 import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.schemas.Schema.FieldType;
+import org.apache.beam.sdk.schemas.logicaltypes.PassThroughLogicalType;
+import org.apache.beam.sdk.values.Row;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
 
 /** Beam {@link org.apache.beam.sdk.schemas.Schema.LogicalType} implementations of JDBC types. */
 public class LogicalTypes {
   public static final Schema.FieldType JDBC_BIT_TYPE =
       Schema.FieldType.logicalType(
-          new org.apache.beam.sdk.schemas.LogicalTypes.PassThroughLogicalType<Boolean>(
-              JDBCType.BIT.getName(), "", Schema.FieldType.BOOLEAN) {});
+          new PassThroughLogicalType<Boolean>(
+              JDBCType.BIT.getName(), FieldType.STRING, "", Schema.FieldType.BOOLEAN) {});
 
   public static final Schema.FieldType JDBC_DATE_TYPE =
       Schema.FieldType.logicalType(
-          new org.apache.beam.sdk.schemas.LogicalTypes.PassThroughLogicalType<Instant>(
-              JDBCType.DATE.getName(), "", Schema.FieldType.DATETIME) {});
+          new PassThroughLogicalType<Instant>(
+              JDBCType.DATE.getName(), FieldType.STRING, "", Schema.FieldType.DATETIME) {});
 
   public static final Schema.FieldType JDBC_FLOAT_TYPE =
       Schema.FieldType.logicalType(
-          new org.apache.beam.sdk.schemas.LogicalTypes.PassThroughLogicalType<Double>(
-              JDBCType.FLOAT.getName(), "", Schema.FieldType.DOUBLE) {});
+          new PassThroughLogicalType<Double>(
+              JDBCType.FLOAT.getName(), FieldType.STRING, "", Schema.FieldType.DOUBLE) {});
 
   public static final Schema.FieldType JDBC_TIME_TYPE =
       Schema.FieldType.logicalType(
-          new org.apache.beam.sdk.schemas.LogicalTypes.PassThroughLogicalType<Instant>(
-              JDBCType.TIME.getName(), "", Schema.FieldType.DATETIME) {});
+          new PassThroughLogicalType<Instant>(
+              JDBCType.TIME.getName(), FieldType.STRING, "", Schema.FieldType.DATETIME) {});
 
   public static final Schema.FieldType JDBC_TIMESTAMP_WITH_TIMEZONE_TYPE =
       Schema.FieldType.logicalType(
-          new org.apache.beam.sdk.schemas.LogicalTypes.PassThroughLogicalType<Instant>(
-              JDBCType.TIMESTAMP_WITH_TIMEZONE.getName(), "", Schema.FieldType.DATETIME) {});
+          new PassThroughLogicalType<Instant>(
+              JDBCType.TIMESTAMP_WITH_TIMEZONE.getName(),
+              FieldType.STRING,
+              "",
+              Schema.FieldType.DATETIME) {});
 
   @VisibleForTesting
   static Schema.FieldType fixedLengthString(JDBCType jdbcType, int length) {
@@ -84,11 +90,17 @@
   /** Base class for JDBC logical types. */
   public abstract static class JdbcLogicalType<T> implements Schema.LogicalType<T, T> {
     protected final String identifier;
+    protected final Schema.FieldType argumentType;
     protected final Schema.FieldType baseType;
-    protected final String argument;
+    protected final Object argument;
 
-    protected JdbcLogicalType(String identifier, Schema.FieldType baseType, String argument) {
+    protected JdbcLogicalType(
+        String identifier,
+        Schema.FieldType argumentType,
+        Schema.FieldType baseType,
+        Object argument) {
       this.identifier = identifier;
+      this.argumentType = argumentType;
       this.baseType = baseType;
       this.argument = argument;
     }
@@ -99,8 +111,14 @@
     }
 
     @Override
-    public String getArgument() {
-      return argument;
+    public FieldType getArgumentType() {
+      return argumentType;
+    }
+
+    @Override
+    @SuppressWarnings("TypeParameterUnusedInFormals")
+    public <ArgumentT> ArgumentT getArgument() {
+      return (ArgumentT) argument;
     }
 
     @Override
@@ -142,7 +160,7 @@
     }
 
     private FixedLengthString(String identifier, int length) {
-      super(identifier, Schema.FieldType.STRING, String.valueOf(length));
+      super(identifier, FieldType.INT32, Schema.FieldType.STRING, length);
       this.length = length;
     }
 
@@ -162,7 +180,7 @@
     }
 
     private FixedLengthBytes(String identifier, int length) {
-      super(identifier, Schema.FieldType.BYTES, String.valueOf(length));
+      super(identifier, FieldType.INT32, Schema.FieldType.BYTES, length);
       this.length = length;
     }
 
@@ -186,7 +204,7 @@
     }
 
     private VariableLengthString(String identifier, int maxLength) {
-      super(identifier, Schema.FieldType.STRING, String.valueOf(maxLength));
+      super(identifier, FieldType.INT32, Schema.FieldType.STRING, maxLength);
       this.maxLength = maxLength;
     }
 
@@ -206,7 +224,7 @@
     }
 
     private VariableLengthBytes(String identifier, int maxLength) {
-      super(identifier, Schema.FieldType.BYTES, String.valueOf(maxLength));
+      super(identifier, FieldType.INT32, Schema.FieldType.BYTES, maxLength);
       this.maxLength = maxLength;
     }
 
@@ -223,11 +241,17 @@
     private final int scale;
 
     public static FixedPrecisionNumeric of(String identifier, int precision, int scale) {
-      return new FixedPrecisionNumeric(identifier, precision, scale);
+      Schema schema = Schema.builder().addInt32Field("precision").addInt32Field("scale").build();
+      return new FixedPrecisionNumeric(schema, identifier, precision, scale);
     }
 
-    private FixedPrecisionNumeric(String identifier, int precision, int scale) {
-      super(identifier, Schema.FieldType.DECIMAL, precision + ":" + scale);
+    private FixedPrecisionNumeric(
+        Schema argumentSchema, String identifier, int precision, int scale) {
+      super(
+          identifier,
+          FieldType.row(argumentSchema),
+          Schema.FieldType.DECIMAL,
+          Row.withSchema(argumentSchema).addValues(precision, scale).build());
       this.precision = precision;
       this.scale = scale;
     }
diff --git a/sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/JdbcIOIT.java b/sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/JdbcIOIT.java
index 311cfe0..81b9799 100644
--- a/sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/JdbcIOIT.java
+++ b/sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/JdbcIOIT.java
@@ -24,6 +24,7 @@
 import java.sql.SQLException;
 import java.util.HashSet;
 import java.util.List;
+import java.util.Optional;
 import java.util.Set;
 import java.util.UUID;
 import java.util.function.Function;
@@ -83,6 +84,7 @@
   private static String tableName;
   private static String bigQueryDataset;
   private static String bigQueryTable;
+  private static Long tableSize;
   @Rule public TestPipeline pipelineWrite = TestPipeline.create();
   @Rule public TestPipeline pipelineRead = TestPipeline.create();
 
@@ -97,6 +99,7 @@
     dataSource = DatabaseTestHelper.getPostgresDataSource(options);
     tableName = DatabaseTestHelper.getTestTableName("IT");
     executeWithRetry(JdbcIOIT::createTable);
+    tableSize = DatabaseTestHelper.getPostgresTableSize(dataSource, tableName).orElse(0L);
   }
 
   private static void createTable() throws SQLException {
@@ -141,6 +144,9 @@
   private Set<Function<MetricsReader, NamedTestResult>> getWriteMetricSuppliers(
       String uuid, String timestamp) {
     Set<Function<MetricsReader, NamedTestResult>> suppliers = new HashSet<>();
+    Optional<Long> postgresTableSize =
+        DatabaseTestHelper.getPostgresTableSize(dataSource, tableName);
+
     suppliers.add(
         reader -> {
           long writeStart = reader.getStartTimeMetric("write_time");
@@ -148,6 +154,13 @@
           return NamedTestResult.create(
               uuid, timestamp, "write_time", (writeEnd - writeStart) / 1e3);
         });
+
+    postgresTableSize.ifPresent(
+        tableFinalSize ->
+            suppliers.add(
+                ignore ->
+                    NamedTestResult.create(
+                        uuid, timestamp, "total_size", tableFinalSize - tableSize)));
     return suppliers;
   }
 
diff --git a/sdks/python/apache_beam/examples/snippets/transforms/aggregation/sample.py b/sdks/python/apache_beam/examples/snippets/transforms/aggregation/sample.py
new file mode 100644
index 0000000..d5abc37
--- /dev/null
+++ b/sdks/python/apache_beam/examples/snippets/transforms/aggregation/sample.py
@@ -0,0 +1,69 @@
+# coding=utf-8
+#
+# 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
+from __future__ import print_function
+
+
+def sample_fixed_size_globally(test=None):
+  # [START sample_fixed_size_globally]
+  import apache_beam as beam
+
+  with beam.Pipeline() as pipeline:
+    sample = (
+        pipeline
+        | 'Create produce' >> beam.Create([
+            '🍓 Strawberry',
+            '🥕 Carrot',
+            '🍆 Eggplant',
+            '🍅 Tomato',
+            '🥔 Potato',
+        ])
+        | 'Sample N elements' >> beam.combiners.Sample.FixedSizeGlobally(3)
+        | beam.Map(print)
+    )
+    # [END sample_fixed_size_globally]
+    if test:
+      test(sample)
+
+
+def sample_fixed_size_per_key(test=None):
+  # [START sample_fixed_size_per_key]
+  import apache_beam as beam
+
+  with beam.Pipeline() as pipeline:
+    samples_per_key = (
+        pipeline
+        | 'Create produce' >> beam.Create([
+            ('spring', '🍓'),
+            ('spring', '🥕'),
+            ('spring', '🍆'),
+            ('spring', '🍅'),
+            ('summer', '🥕'),
+            ('summer', '🍅'),
+            ('summer', '🌽'),
+            ('fall', '🥕'),
+            ('fall', '🍅'),
+            ('winter', '🍆'),
+        ])
+        | 'Samples per key' >> beam.combiners.Sample.FixedSizePerKey(3)
+        | beam.Map(print)
+    )
+    # [END sample_fixed_size_per_key]
+    if test:
+      test(samples_per_key)
diff --git a/sdks/python/apache_beam/examples/snippets/transforms/aggregation/sample_test.py b/sdks/python/apache_beam/examples/snippets/transforms/aggregation/sample_test.py
new file mode 100644
index 0000000..22cd656
--- /dev/null
+++ b/sdks/python/apache_beam/examples/snippets/transforms/aggregation/sample_test.py
@@ -0,0 +1,63 @@
+# coding=utf-8
+#
+# 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
+from __future__ import print_function
+
+import unittest
+
+import mock
+
+from apache_beam.examples.snippets.util import assert_matches_stdout
+from apache_beam.testing.test_pipeline import TestPipeline
+
+from . import sample
+
+
+def check_sample(actual):
+  expected = '''[START sample]
+['🥕 Carrot', '🍆 Eggplant', '🍅 Tomato']
+[END sample]'''.splitlines()[1:-1]
+  # The sampled elements are non-deterministic, so check the sample size.
+  assert_matches_stdout(actual, expected, lambda elements: len(elements))
+
+
+def check_samples_per_key(actual):
+  expected = '''[START samples_per_key]
+('spring', ['🍓', '🥕', '🍆'])
+('summer', ['🥕', '🍅', '🌽'])
+('fall', ['🥕', '🍅'])
+('winter', ['🍆'])
+[END samples_per_key]'''.splitlines()[1:-1]
+  # The sampled elements are non-deterministic, so check the sample size.
+  assert_matches_stdout(actual, expected, lambda pair: (pair[0], len(pair[1])))
+
+
+@mock.patch('apache_beam.Pipeline', TestPipeline)
+@mock.patch(
+    'apache_beam.examples.snippets.transforms.aggregation.sample.print', str)
+class SampleTest(unittest.TestCase):
+  def test_sample_fixed_size_globally(self):
+    sample.sample_fixed_size_globally(check_sample)
+
+  def test_sample_fixed_size_per_key(self):
+    sample.sample_fixed_size_per_key(check_samples_per_key)
+
+
+if __name__ == '__main__':
+  unittest.main()
diff --git a/sdks/python/apache_beam/internal/pickler.py b/sdks/python/apache_beam/internal/pickler.py
index 3bb038a..8083091 100644
--- a/sdks/python/apache_beam/internal/pickler.py
+++ b/sdks/python/apache_beam/internal/pickler.py
@@ -54,7 +54,7 @@
   # if executed concurrently, see: BEAM-8651, http://bugs.python.org/issue38884.
   pickle_lock_unless_py2 = threading.RLock()
 else:
-  # Avoid slow reentrant locks on Py2.
+  # Avoid slow reentrant locks on Py2. See: https://bugs.python.org/issue3001.
   pickle_lock_unless_py2 = _NoOpContextManager()
 # Dill 0.28.0 renamed dill.dill to dill._dill:
 # https://github.com/uqfoundation/dill/commit/f0972ecc7a41d0b8acada6042d557068cac69baa
diff --git a/sdks/python/apache_beam/io/gcp/bigquery_test.py b/sdks/python/apache_beam/io/gcp/bigquery_test.py
index 505a683..ac62774 100644
--- a/sdks/python/apache_beam/io/gcp/bigquery_test.py
+++ b/sdks/python/apache_beam/io/gcp/bigquery_test.py
@@ -43,6 +43,7 @@
 from apache_beam.io.gcp.bigquery import _StreamToBigQuery
 from apache_beam.io.gcp.bigquery_file_loads_test import _ELEMENTS
 from apache_beam.io.gcp.bigquery_tools import JSON_COMPLIANCE_ERROR
+from apache_beam.io.gcp.bigquery_tools import RetryStrategy
 from apache_beam.io.gcp.internal.clients import bigquery
 from apache_beam.io.gcp.pubsub import ReadFromPubSub
 from apache_beam.io.gcp.tests import utils
@@ -647,7 +648,6 @@
                method='FILE_LOADS'))
 
   @attr('IT')
-  @unittest.skip('BEAM-8842: Disabled due to reliance on old retry behavior.')
   def test_multiple_destinations_transform(self):
     streaming = self.test_pipeline.options.view_as(StandardOptions).streaming
     if streaming and isinstance(self.test_pipeline.runner, TestDataflowRunner):
@@ -735,6 +735,7 @@
                table_side_inputs=(table_record_pcv,),
                schema=lambda dest, table_map: table_map.get(dest, None),
                schema_side_inputs=(schema_table_pcv,),
+               insert_retry_strategy=RetryStrategy.RETRY_ON_TRANSIENT_ERROR,
                method='STREAMING_INSERTS'))
 
       assert_that(r[beam.io.gcp.bigquery.BigQueryWriteFn.FAILED_ROWS],
diff --git a/sdks/python/apache_beam/io/mongodbio.py b/sdks/python/apache_beam/io/mongodbio.py
index cb1f5b1..fba5b43 100644
--- a/sdks/python/apache_beam/io/mongodbio.py
+++ b/sdks/python/apache_beam/io/mongodbio.py
@@ -174,7 +174,7 @@
     for split_key_id in split_keys:
       if bundle_start >= stop_position:
         break
-      bundle_end = min(stop_position, split_key_id)
+      bundle_end = min(stop_position, split_key_id['_id'])
       yield iobase.SourceBundle(weight=desired_bundle_size_in_mb,
                                 source=self,
                                 start_position=bundle_start,
diff --git a/sdks/python/apache_beam/io/mongodbio_test.py b/sdks/python/apache_beam/io/mongodbio_test.py
index 3f07ec1..72ae0ad 100644
--- a/sdks/python/apache_beam/io/mongodbio_test.py
+++ b/sdks/python/apache_beam/io/mongodbio_test.py
@@ -123,10 +123,8 @@
     # Return ids of elements in the range with chunk size skip and exclude
     # head element. For simplicity of tests every document is considered 1Mb
     # by default.
-    return {
-        'splitKeys':
-        [x['_id'] for x in self.docs[start_index:end_index:maxChunkSize]][1:]
-    }
+    return {'splitKeys': [{'_id': x['_id']} for x in
+                          self.docs[start_index:end_index:maxChunkSize]][1:]}
 
 
 class _MockMongoClient(object):
diff --git a/sdks/python/apache_beam/runners/common.py b/sdks/python/apache_beam/runners/common.py
index 8632cfd..c045231 100644
--- a/sdks/python/apache_beam/runners/common.py
+++ b/sdks/python/apache_beam/runners/common.py
@@ -50,13 +50,14 @@
 class NameContext(object):
   """Holds the name information for a step."""
 
-  def __init__(self, step_name):
+  def __init__(self, step_name, transform_id=None):
     """Creates a new step NameContext.
 
     Args:
       step_name: The name of the step.
     """
     self.step_name = step_name
+    self.transform_id = transform_id
 
   def __eq__(self, other):
     return self.step_name == other.step_name
diff --git a/sdks/python/apache_beam/runners/direct/transform_evaluator.py b/sdks/python/apache_beam/runners/direct/transform_evaluator.py
index 08def43..c893d8b 100644
--- a/sdks/python/apache_beam/runners/direct/transform_evaluator.py
+++ b/sdks/python/apache_beam/runners/direct/transform_evaluator.py
@@ -823,15 +823,17 @@
     k = self.key_coder.decode(encoded_k)
     state = self._step_context.get_keyed_state(encoded_k)
 
+    watermarks = self._evaluation_context._watermark_manager.get_watermarks(
+        self._applied_ptransform)
     for timer_firing in timer_firings:
       for wvalue in self.driver.process_timer(
           timer_firing.window, timer_firing.name, timer_firing.time_domain,
-          timer_firing.timestamp, state):
+          timer_firing.timestamp, state, watermarks.input_watermark):
         self.gabw_items.append(wvalue.with_value((k, wvalue.value)))
-    watermark = self._evaluation_context._watermark_manager.get_watermarks(
-        self._applied_ptransform).output_watermark
     if vs:
-      for wvalue in self.driver.process_elements(state, vs, watermark):
+      for wvalue in self.driver.process_elements(state, vs,
+                                                 watermarks.output_watermark,
+                                                 watermarks.input_watermark):
         self.gabw_items.append(wvalue.with_value((k, wvalue.value)))
 
     self.keyed_holds[encoded_k] = state.get_earliest_hold()
diff --git a/sdks/python/apache_beam/runners/direct/util.py b/sdks/python/apache_beam/runners/direct/util.py
index 407ea39..57650ac 100644
--- a/sdks/python/apache_beam/runners/direct/util.py
+++ b/sdks/python/apache_beam/runners/direct/util.py
@@ -64,9 +64,8 @@
     self.timestamp = timestamp
 
   def __repr__(self):
-    return 'TimerFiring(%r, %r, %s, %s)' % (self.encoded_key,
-                                            self.name, self.time_domain,
-                                            self.timestamp)
+    return 'TimerFiring({!r}, {!r}, {}, {})'.format(
+        self.encoded_key, self.name, self.time_domain, self.timestamp)
 
 
 class KeyedWorkItem(object):
@@ -75,3 +74,7 @@
     self.encoded_key = encoded_key
     self.timer_firings = timer_firings or []
     self.elements = elements or []
+
+  def __repr__(self):
+    return 'KeyedWorkItem({!r}, {}, {})'.format(
+        self.encoded_key, self.timer_firings, self.elements)
diff --git a/sdks/python/apache_beam/runners/interactive/pipeline_instrument.py b/sdks/python/apache_beam/runners/interactive/pipeline_instrument.py
index af9b5c5..cd06b42 100644
--- a/sdks/python/apache_beam/runners/interactive/pipeline_instrument.py
+++ b/sdks/python/apache_beam/runners/interactive/pipeline_instrument.py
@@ -26,6 +26,7 @@
 import apache_beam as beam
 from apache_beam.io.gcp.pubsub import ReadFromPubSub
 from apache_beam.pipeline import PipelineVisitor
+from apache_beam.portability.api import beam_runner_api_pb2
 from apache_beam.runners.interactive import cache_manager as cache
 from apache_beam.runners.interactive import interactive_environment as ie
 
@@ -73,13 +74,20 @@
         pipeline.to_runner_api(use_fake_coders=True),
         pipeline.runner,
         options)
+
+    self._background_caching_pipeline = beam.pipeline.Pipeline.from_runner_api(
+        pipeline.to_runner_api(use_fake_coders=True),
+        pipeline.runner,
+        options)
+
     # Snapshot of original pipeline information.
     (self._original_pipeline_proto,
      self._original_context) = self._pipeline_snap.to_runner_api(
          return_context=True, use_fake_coders=True)
 
     # All compute-once-against-original-pipeline fields.
-    self._has_unbounded_source = has_unbounded_source(self._pipeline_snap)
+    self._unbounded_sources = unbounded_sources(
+        self._background_caching_pipeline)
     # TODO(BEAM-7760): once cache scope changed, this is not needed to manage
     # relationships across pipelines, runners, and jobs.
     self._pcolls_to_pcoll_id = pcolls_to_pcoll_id(self._pipeline_snap,
@@ -103,11 +111,131 @@
     """Always returns a new instance of portable instrumented proto."""
     return self._pipeline.to_runner_api(use_fake_coders=True)
 
+  def _required_components(self, pipeline_proto, required_transforms_ids):
+    """Returns the components and subcomponents of the given transforms.
+
+    This method returns all the components (transforms, PCollections, coders,
+    and windowing stratgies) related to the given transforms and to all of their
+    subtransforms. This method accomplishes this recursively.
+    """
+    transforms = pipeline_proto.components.transforms
+    pcollections = pipeline_proto.components.pcollections
+    coders = pipeline_proto.components.coders
+    windowing_strategies = pipeline_proto.components.windowing_strategies
+
+    # Cache the transforms that will be copied into the new pipeline proto.
+    required_transforms = {k: transforms[k] for k in required_transforms_ids}
+
+    # Cache all the output PCollections of the transforms.
+    pcollection_ids = [pc for t in required_transforms.values()
+                       for pc in t.outputs.values()]
+    required_pcollections = {pc_id: pcollections[pc_id]
+                             for pc_id in pcollection_ids}
+
+    # Cache all the PCollection coders.
+    coder_ids = [pc.coder_id for pc in required_pcollections.values()]
+    required_coders = {c_id: coders[c_id] for c_id in coder_ids}
+
+    # Cache all the windowing strategy ids.
+    windowing_strategies_ids = [pc.windowing_strategy_id
+                                for pc in required_pcollections.values()]
+    required_windowing_strategies = {ws_id: windowing_strategies[ws_id]
+                                     for ws_id in windowing_strategies_ids}
+
+    subtransforms = {}
+    subpcollections = {}
+    subcoders = {}
+    subwindowing_strategies = {}
+
+    # Recursively go through all the subtransforms and add their components.
+    for transform_id, transform in required_transforms.items():
+      if transform_id in pipeline_proto.root_transform_ids:
+        continue
+      (t, pc, c, ws) = self._required_components(pipeline_proto,
+                                                 transform.subtransforms)
+      subtransforms.update(t)
+      subpcollections.update(pc)
+      subcoders.update(c)
+      subwindowing_strategies.update(ws)
+
+    # Now we got all the components and their subcomponents, so return the
+    # complete collection.
+    required_transforms.update(subtransforms)
+    required_pcollections.update(subpcollections)
+    required_coders.update(subcoders)
+    required_windowing_strategies.update(subwindowing_strategies)
+
+    return (required_transforms, required_pcollections, required_coders,
+            required_windowing_strategies)
+
+  def background_caching_pipeline_proto(self):
+    """Returns the background caching pipeline.
+
+    This method creates a background caching pipeline by: adding writes to cache
+    from each unbounded source (done in the instrument method), and cutting out
+    all components (transform, PCollections, coders, windowing strategies) that
+    are not the unbounded sources or writes to cache (or subtransforms thereof).
+    """
+    # Create the pipeline_proto to read all the components from. It will later
+    # create a new pipeline proto from the cut out components.
+    pipeline_proto = self._background_caching_pipeline.to_runner_api(
+        return_context=False, use_fake_coders=True)
+
+    # Get all the sources we want to cache.
+    sources = unbounded_sources(self._background_caching_pipeline)
+
+    # Get all the root transforms. The caching transforms will be subtransforms
+    # of one of these roots.
+    roots = [root for root in pipeline_proto.root_transform_ids]
+
+    # Get the transform IDs of the caching transforms. These caching operations
+    # are added the the _background_caching_pipeline in the instrument() method.
+    # It's added there so that multiple calls to this method won't add multiple
+    # caching operations (idempotent).
+    transforms = pipeline_proto.components.transforms
+    caching_transform_ids = [t_id for root in roots
+                             for t_id in transforms[root].subtransforms
+                             if WRITE_CACHE in t_id]
+
+    # Get the IDs of the unbounded sources.
+    required_transform_labels = [src.full_label for src in sources]
+    unbounded_source_ids = [k for k, v in transforms.items()
+                            if v.unique_name in required_transform_labels]
+
+    # The required transforms are the tranforms that we want to cut out of
+    # the pipeline_proto and insert into a new pipeline to return.
+    required_transform_ids = (roots + caching_transform_ids +
+                              unbounded_source_ids)
+    (t, p, c, w) = self._required_components(pipeline_proto,
+                                             required_transform_ids)
+
+    def set_proto_map(proto_map, new_value):
+      proto_map.clear()
+      for key, value in new_value.items():
+        proto_map[key].CopyFrom(value)
+
+    # Copy the transforms into the new pipeline.
+    pipeline_to_execute = beam_runner_api_pb2.Pipeline()
+    pipeline_to_execute.root_transform_ids[:] = roots
+    set_proto_map(pipeline_to_execute.components.transforms, t)
+    set_proto_map(pipeline_to_execute.components.pcollections, p)
+    set_proto_map(pipeline_to_execute.components.coders, c)
+    set_proto_map(pipeline_to_execute.components.windowing_strategies, w)
+
+    # Cut out all subtransforms in the root that aren't the required transforms.
+    for root_id in roots:
+      root = pipeline_to_execute.components.transforms[root_id]
+      root.subtransforms[:] = [
+          transform_id for transform_id in root.subtransforms
+          if transform_id in pipeline_to_execute.components.transforms]
+
+    return pipeline_to_execute
+
   @property
-  def has_unbounded_source(self):
+  def has_unbounded_sources(self):
     """Returns whether the pipeline has any `REPLACEABLE_UNBOUNDED_SOURCES`.
     """
-    return self._has_unbounded_source
+    return len(self._unbounded_sources) > 0
 
   @property
   def cacheables(self):
@@ -165,12 +293,19 @@
     self._pipeline.visit(v)
     # Create ReadCache transforms.
     for cacheable_input in cacheable_inputs:
-      self._read_cache(cacheable_input)
+      self._read_cache(self._pipeline, cacheable_input)
     # Replace/wire inputs w/ cached PCollections from ReadCache transforms.
-    self._replace_with_cached_inputs()
+    self._replace_with_cached_inputs(self._pipeline)
     # Write cache for all cacheables.
     for _, cacheable in self.cacheables.items():
-      self._write_cache(cacheable['pcoll'])
+      self._write_cache(self._pipeline, cacheable['pcoll'])
+
+    # Instrument the background caching pipeline if we can.
+    if self.has_unbounded_sources:
+      for source in self._unbounded_sources:
+        self._write_cache(self._background_caching_pipeline,
+                          source.outputs[None])
+
     # TODO(BEAM-7760): prune sub graphs that doesn't need to be executed.
 
   def preprocess(self):
@@ -207,7 +342,7 @@
     v = PreprocessVisitor(self)
     self._pipeline.visit(v)
 
-  def _write_cache(self, pcoll):
+  def _write_cache(self, pipeline, pcoll):
     """Caches a cacheable PCollection.
 
     For the given PCollection, by appending sub transform part that materialize
@@ -218,29 +353,29 @@
     the pipeline being instrumented and the keyed cache is absent.
 
     Modifies:
-      self._pipeline
+      pipeline
     """
     # Makes sure the pcoll belongs to the pipeline being instrumented.
-    if pcoll.pipeline is not self._pipeline:
+    if pcoll.pipeline is not pipeline:
       return
     # The keyed cache is always valid within this instrumentation.
     key = self.cache_key(pcoll)
     # Only need to write when the cache with expected key doesn't exist.
     if not self._cache_manager.exists('full', key):
-      _ = pcoll | '{}{}'.format(WRITE_CACHE, key) >> cache.WriteCache(
-          self._cache_manager, key)
+      label = '{}{}'.format(WRITE_CACHE, key)
+      _ = pcoll | label >> cache.WriteCache(self._cache_manager, key)
 
-  def _read_cache(self, pcoll):
+  def _read_cache(self, pipeline, pcoll):
     """Reads a cached pvalue.
 
     A noop will cause the pipeline to execute the transform as
     it is and cache nothing from this transform for next run.
 
     Modifies:
-      self._pipeline
+      pipeline
     """
     # Makes sure the pcoll belongs to the pipeline being instrumented.
-    if pcoll.pipeline is not self._pipeline:
+    if pcoll.pipeline is not pipeline:
       return
     # The keyed cache is always valid within this instrumentation.
     key = self.cache_key(pcoll)
@@ -250,13 +385,13 @@
         # Mutates the pipeline with cache read transform attached
         # to root of the pipeline.
         pcoll_from_cache = (
-            self._pipeline
+            pipeline
             | '{}{}'.format(READ_CACHE, key) >> cache.ReadCache(
                 self._cache_manager, key))
         self._cached_pcoll_read[key] = pcoll_from_cache
     # else: NOOP when cache doesn't exist, just compute the original graph.
 
-  def _replace_with_cached_inputs(self):
+  def _replace_with_cached_inputs(self, pipeline):
     """Replace PCollection inputs in the pipeline with cache if possible.
 
     For any input PCollection, find out whether there is valid cache. If so,
@@ -287,7 +422,7 @@
           transform_node.inputs = tuple(input_list)
 
     v = ReadCacheWireVisitor(self)
-    self._pipeline.visit(v)
+    pipeline.visit(v)
 
   def _cacheable_inputs(self, transform):
     inputs = set()
@@ -374,9 +509,10 @@
         cacheable['version'] = str(id(val))
         cacheable['pcoll'] = val
         cacheable['producer_version'] = str(id(val.producer))
-        cacheables[cacheable_key(val, pcolls_to_pcoll_id)] = cacheable
         pcoll_version_map[cacheable['pcoll_id']] = cacheable['version']
+        cacheables[cacheable_key(val, pcolls_to_pcoll_id)] = cacheable
         cacheable_var_by_pcoll_id[cacheable['pcoll_id']] = key
+
   return pcoll_version_map, cacheables, cacheable_var_by_pcoll_id
 
 
@@ -390,8 +526,13 @@
   return '_'.join((pcoll_version, pcoll_id))
 
 
-def has_unbounded_source(pipeline):
+def has_unbounded_sources(pipeline):
   """Checks if a given pipeline has replaceable unbounded sources."""
+  return len(unbounded_sources(pipeline)) > 0
+
+
+def unbounded_sources(pipeline):
+  """Returns a pipeline's replaceable unbounded sources."""
 
   class CheckUnboundednessVisitor(PipelineVisitor):
     """Visitor checks if there are any unbounded read sources in the Pipeline.
@@ -401,18 +542,18 @@
     """
 
     def __init__(self):
-      self.has_unbounded_source = False
+      self.unbounded_sources = []
 
     def enter_composite_transform(self, transform_node):
       self.visit_transform(transform_node)
 
     def visit_transform(self, transform_node):
-      self.has_unbounded_source |= isinstance(transform_node.transform,
-                                              REPLACEABLE_UNBOUNDED_SOURCES)
+      if isinstance(transform_node.transform, REPLACEABLE_UNBOUNDED_SOURCES):
+        self.unbounded_sources.append(transform_node)
 
   v = CheckUnboundednessVisitor()
   pipeline.visit(v)
-  return v.has_unbounded_source
+  return v.unbounded_sources
 
 
 def pcolls_to_pcoll_id(pipeline, original_context):
diff --git a/sdks/python/apache_beam/runners/interactive/pipeline_instrument_test.py b/sdks/python/apache_beam/runners/interactive/pipeline_instrument_test.py
index f65b133..c45b8e3 100644
--- a/sdks/python/apache_beam/runners/interactive/pipeline_instrument_test.py
+++ b/sdks/python/apache_beam/runners/interactive/pipeline_instrument_test.py
@@ -44,10 +44,8 @@
   def setUp(self):
     ie.new_env(cache_manager=cache.FileBasedCacheManager())
 
-  def assertPipelineEqual(self, actual_pipeline, expected_pipeline):
-    actual_pipeline_proto = actual_pipeline.to_runner_api(use_fake_coders=True)
-    expected_pipeline_proto = expected_pipeline.to_runner_api(
-        use_fake_coders=True)
+  def assertPipelineProtoEqual(self, actual_pipeline_proto,
+                               expected_pipeline_proto):
     components1 = actual_pipeline_proto.components
     components2 = expected_pipeline_proto.components
     self.assertEqual(len(components1.transforms), len(components2.transforms))
@@ -64,6 +62,13 @@
                               expected_pipeline_proto,
                               expected_pipeline_proto.root_transform_ids[0])
 
+  def assertPipelineEqual(self, actual_pipeline, expected_pipeline):
+    actual_pipeline_proto = actual_pipeline.to_runner_api(use_fake_coders=True)
+    expected_pipeline_proto = expected_pipeline.to_runner_api(
+        use_fake_coders=True)
+    self.assertPipelineProtoEqual(actual_pipeline_proto,
+                                  expected_pipeline_proto)
+
   def assertTransformEqual(self, actual_pipeline_proto, actual_transform_id,
                            expected_pipeline_proto, expected_transform_id):
     transform_proto1 = actual_pipeline_proto.components.transforms[
@@ -178,14 +183,48 @@
     p = beam.Pipeline(interactive_runner.InteractiveRunner())
     _ = p | 'ReadUnboundedSource' >> beam.io.ReadFromPubSub(
         subscription='projects/fake-project/subscriptions/fake_sub')
-    self.assertTrue(instr.has_unbounded_source(p))
+    self.assertTrue(instr.has_unbounded_sources(p))
 
   def test_not_has_unbounded_source(self):
     p = beam.Pipeline(interactive_runner.InteractiveRunner())
     with tempfile.NamedTemporaryFile(delete=False) as f:
       f.write(b'test')
     _ = p | 'ReadBoundedSource' >> beam.io.ReadFromText(f.name)
-    self.assertFalse(instr.has_unbounded_source(p))
+    self.assertFalse(instr.has_unbounded_sources(p))
+
+  def test_background_caching_pipeline_proto(self):
+    p = beam.Pipeline(interactive_runner.InteractiveRunner())
+
+    # Test that the two ReadFromPubSub are correctly cut out.
+    a = p | 'ReadUnboundedSourceA' >> beam.io.ReadFromPubSub(
+        subscription='projects/fake-project/subscriptions/fake_sub')
+    b = p | 'ReadUnboundedSourceB' >> beam.io.ReadFromPubSub(
+        subscription='projects/fake-project/subscriptions/fake_sub')
+
+    # Add some extra PTransform afterwards to make sure that only the unbounded
+    # sources remain.
+    c = (a, b) | beam.CoGroupByKey()
+    _ = c | beam.Map(lambda x: x)
+
+    ib.watch(locals())
+    instrumenter = instr.pin(p)
+    actual_pipeline = instrumenter.background_caching_pipeline_proto()
+
+    # Now recreate the expected pipeline, which should only have the unbounded
+    # sources.
+    p = beam.Pipeline(interactive_runner.InteractiveRunner())
+    a = p | 'ReadUnboundedSourceA' >> beam.io.ReadFromPubSub(
+        subscription='projects/fake-project/subscriptions/fake_sub')
+    _ = a | 'a' >> cache.WriteCache(ie.current_env().cache_manager(), '')
+
+    b = p | 'ReadUnboundedSourceB' >> beam.io.ReadFromPubSub(
+        subscription='projects/fake-project/subscriptions/fake_sub')
+    _ = b | 'b' >> cache.WriteCache(ie.current_env().cache_manager(), '')
+
+    expected_pipeline = p.to_runner_api(return_context=False,
+                                        use_fake_coders=True)
+
+    self.assertPipelineProtoEqual(actual_pipeline, expected_pipeline)
 
   def _example_pipeline(self, watch=True):
     p = beam.Pipeline(interactive_runner.InteractiveRunner())
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 b97f683..2bbafbb 100644
--- a/sdks/python/apache_beam/runners/portability/local_job_service.py
+++ b/sdks/python/apache_beam/runners/portability/local_job_service.py
@@ -98,16 +98,36 @@
         provision_info,
         self._artifact_staging_endpoint)
 
+  def get_bind_address(self):
+    """Return the address used to open the port on the gRPC server.
+
+    This is often, but not always the same as the service address.  For
+    example, to make the service accessible to external machines, override this
+    to return '[::]' and override `get_service_address()` to return a publicly
+    accessible host name.
+    """
+    return self.get_service_address()
+
+  def get_service_address(self):
+    """Return the host name at which this server will be accessible.
+
+    In particular, this is provided to the client upon connection as the
+    artifact staging endpoint.
+    """
+    return 'localhost'
+
   def start_grpc_server(self, port=0):
     self._server = grpc.server(UnboundedThreadPoolExecutor())
-    port = self._server.add_insecure_port('localhost:%d' % port)
+    port = self._server.add_insecure_port(
+        '%s:%d' % (self.get_bind_address(), port))
     beam_job_api_pb2_grpc.add_JobServiceServicer_to_server(self, self._server)
     beam_artifact_api_pb2_grpc.add_ArtifactStagingServiceServicer_to_server(
         self._artifact_service, self._server)
+    hostname = self.get_service_address()
     self._artifact_staging_endpoint = endpoints_pb2.ApiServiceDescriptor(
-        url='localhost:%d' % port)
+        url='%s:%d' % (hostname, port))
     self._server.start()
-    _LOGGER.info('Grpc server started on port %s', port)
+    _LOGGER.info('Grpc server started at %s on port %d' % (hostname, port))
     return port
 
   def stop(self, timeout=1):
diff --git a/sdks/python/apache_beam/runners/worker/bundle_processor.py b/sdks/python/apache_beam/runners/worker/bundle_processor.py
index c62f194..fd2528d 100644
--- a/sdks/python/apache_beam/runners/worker/bundle_processor.py
+++ b/sdks/python/apache_beam/runners/worker/bundle_processor.py
@@ -978,7 +978,7 @@
         transform_id)
     output_coder = factory.get_only_output_coder(transform_proto)
   return DataInputOperation(
-      transform_proto.unique_name,
+      common.NameContext(transform_proto.unique_name, transform_id),
       transform_proto.unique_name,
       consumers,
       factory.counter_factory,
@@ -1000,7 +1000,7 @@
         transform_id)
     output_coder = factory.get_only_input_coder(transform_proto)
   return DataOutputOperation(
-      transform_proto.unique_name,
+      common.NameContext(transform_proto.unique_name, transform_id),
       transform_proto.unique_name,
       consumers,
       factory.counter_factory,
@@ -1019,7 +1019,7 @@
       [factory.get_only_output_coder(transform_proto)])
   return factory.augment_oldstyle_op(
       operations.ReadOperation(
-          transform_proto.unique_name,
+          common.NameContext(transform_proto.unique_name, transform_id),
           spec,
           factory.counter_factory,
           factory.state_sampler),
@@ -1036,7 +1036,7 @@
       [WindowedValueCoder(source.default_output_coder())])
   return factory.augment_oldstyle_op(
       operations.ReadOperation(
-          transform_proto.unique_name,
+          common.NameContext(transform_proto.unique_name, transform_id),
           spec,
           factory.counter_factory,
           factory.state_sampler),
@@ -1048,7 +1048,7 @@
     python_urns.IMPULSE_READ_TRANSFORM, beam_runner_api_pb2.ReadPayload)
 def create(factory, transform_id, transform_proto, parameter, consumers):
   return operations.ImpulseReadOperation(
-      transform_proto.unique_name,
+      common.NameContext(transform_proto.unique_name, transform_id),
       factory.counter_factory,
       factory.state_sampler,
       consumers,
@@ -1227,7 +1227,7 @@
 
   result = factory.augment_oldstyle_op(
       operation_cls(
-          transform_proto.unique_name,
+          common.NameContext(transform_proto.unique_name, transform_id),
           spec,
           factory.counter_factory,
           factory.state_sampler,
@@ -1276,7 +1276,7 @@
 def create(factory, transform_id, transform_proto, unused_parameter, consumers):
   return factory.augment_oldstyle_op(
       operations.FlattenOperation(
-          transform_proto.unique_name,
+          common.NameContext(transform_proto.unique_name, transform_id),
           operation_specs.WorkerFlatten(
               None, [factory.get_only_output_coder(transform_proto)]),
           factory.counter_factory,
@@ -1294,7 +1294,7 @@
        [], {}))
   return factory.augment_oldstyle_op(
       operations.PGBKCVOperation(
-          transform_proto.unique_name,
+          common.NameContext(transform_proto.unique_name, transform_id),
           operation_specs.WorkerPartialGroupByKey(
               serialized_combine_fn,
               None,
@@ -1310,7 +1310,7 @@
     beam_runner_api_pb2.CombinePayload)
 def create(factory, transform_id, transform_proto, payload, consumers):
   return _create_combine_phase_operation(
-      factory, transform_proto, payload, consumers, 'merge')
+      factory, transform_id, transform_proto, payload, consumers, 'merge')
 
 
 @BeamTransformFactory.register_urn(
@@ -1318,7 +1318,7 @@
     beam_runner_api_pb2.CombinePayload)
 def create(factory, transform_id, transform_proto, payload, consumers):
   return _create_combine_phase_operation(
-      factory, transform_proto, payload, consumers, 'extract')
+      factory, transform_id, transform_proto, payload, consumers, 'extract')
 
 
 @BeamTransformFactory.register_urn(
@@ -1326,17 +1326,17 @@
     beam_runner_api_pb2.CombinePayload)
 def create(factory, transform_id, transform_proto, payload, consumers):
   return _create_combine_phase_operation(
-      factory, transform_proto, payload, consumers, 'all')
+      factory, transform_id, transform_proto, payload, consumers, 'all')
 
 
 def _create_combine_phase_operation(
-    factory, transform_proto, payload, consumers, phase):
+    factory, transform_id, transform_proto, payload, consumers, phase):
   serialized_combine_fn = pickler.dumps(
       (beam.CombineFn.from_runner_api(payload.combine_fn, factory.context),
        [], {}))
   return factory.augment_oldstyle_op(
       operations.CombineOperation(
-          transform_proto.unique_name,
+          common.NameContext(transform_proto.unique_name, transform_id),
           operation_specs.WorkerCombineFn(
               serialized_combine_fn,
               phase,
@@ -1352,7 +1352,7 @@
 def create(factory, transform_id, transform_proto, unused_parameter, consumers):
   return factory.augment_oldstyle_op(
       operations.FlattenOperation(
-          transform_proto.unique_name,
+          common.NameContext(transform_proto.unique_name, transform_id),
           operation_specs.WorkerFlatten(
               None,
               [factory.get_only_output_coder(transform_proto)]),
diff --git a/sdks/python/apache_beam/runners/worker/log_handler.py b/sdks/python/apache_beam/runners/worker/log_handler.py
index 08dac3a..12f162b 100644
--- a/sdks/python/apache_beam/runners/worker/log_handler.py
+++ b/sdks/python/apache_beam/runners/worker/log_handler.py
@@ -25,12 +25,13 @@
 import sys
 import threading
 import time
-from builtins import range
+import traceback
 
 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 statesampler
 from apache_beam.runners.worker.channel_factory import GRPCChannelFactory
 from apache_beam.runners.worker.worker_id_interceptor import WorkerIdInterceptor
 
@@ -54,7 +55,8 @@
       logging.ERROR: beam_fn_api_pb2.LogEntry.Severity.ERROR,
       logging.WARNING: beam_fn_api_pb2.LogEntry.Severity.WARN,
       logging.INFO: beam_fn_api_pb2.LogEntry.Severity.INFO,
-      logging.DEBUG: beam_fn_api_pb2.LogEntry.Severity.DEBUG
+      logging.DEBUG: beam_fn_api_pb2.LogEntry.Severity.DEBUG,
+      -float('inf'): beam_fn_api_pb2.LogEntry.Severity.DEBUG,
   }
 
   def __init__(self, log_service_descriptor):
@@ -81,16 +83,37 @@
         self._log_channel)
     return self._logging_stub.Logging(self._write_log_entries())
 
+  def map_log_level(self, level):
+    try:
+      return self.LOG_LEVEL_MAP[level]
+    except KeyError:
+      return max(
+          beam_level for python_level, beam_level in self.LOG_LEVEL_MAP.items()
+          if python_level <= level)
+
   def emit(self, record):
     log_entry = beam_fn_api_pb2.LogEntry()
-    log_entry.severity = self.LOG_LEVEL_MAP[record.levelno]
+    log_entry.severity = self.map_log_level(record.levelno)
     log_entry.message = self.format(record)
     log_entry.thread = record.threadName
-    log_entry.log_location = record.module + '.' + record.funcName
+    log_entry.log_location = '%s:%s' % (
+        record.pathname or record.module, record.lineno or record.funcName)
     (fraction, seconds) = math.modf(record.created)
     nanoseconds = 1e9 * fraction
     log_entry.timestamp.seconds = int(seconds)
     log_entry.timestamp.nanos = int(nanoseconds)
+    if record.exc_info:
+      log_entry.trace = ''.join(traceback.format_exception(*record.exc_info))
+    instruction_id = statesampler.get_current_instruction_id()
+    if instruction_id:
+      log_entry.instruction_id = instruction_id
+    tracker = statesampler.get_current_tracker()
+    if tracker:
+      current_state = tracker.current_state()
+      if (current_state
+          and current_state.name_context
+          and current_state.name_context.transform_id):
+        log_entry.transform_id = current_state.name_context.transform_id
 
     try:
       self._log_entry_queue.put(log_entry, block=False)
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 a651409..c79ccf9 100644
--- a/sdks/python/apache_beam/runners/worker/log_handler_test.py
+++ b/sdks/python/apache_beam/runners/worker/log_handler_test.py
@@ -18,6 +18,7 @@
 from __future__ import absolute_import
 
 import logging
+import re
 import unittest
 from builtins import range
 
@@ -26,7 +27,9 @@
 from apache_beam.portability.api import beam_fn_api_pb2
 from apache_beam.portability.api import beam_fn_api_pb2_grpc
 from apache_beam.portability.api import endpoints_pb2
+from apache_beam.runners.common import NameContext
 from apache_beam.runners.worker import log_handler
+from apache_beam.runners.worker import statesampler
 from apache_beam.utils.thread_pool_executor import UnboundedThreadPoolExecutor
 
 _LOGGER = logging.getLogger(__name__)
@@ -83,14 +86,58 @@
                          log_entry.severity)
         self.assertEqual('%s: %s' % (msg, num_received_log_entries),
                          log_entry.message)
-        self.assertEqual(u'log_handler_test._verify_fn_log_handler',
-                         log_entry.log_location)
+        self.assertTrue(
+            re.match(r'.*/log_handler_test.py:\d+', log_entry.log_location),
+            log_entry.log_location)
         self.assertGreater(log_entry.timestamp.seconds, 0)
         self.assertGreaterEqual(log_entry.timestamp.nanos, 0)
         num_received_log_entries += 1
 
     self.assertEqual(num_received_log_entries, num_log_entries)
 
+  def assertContains(self, haystack, needle):
+    self.assertTrue(
+        needle in haystack, 'Expected %r to contain %r.' % (haystack, needle))
+
+  def test_exc_info(self):
+    try:
+      raise ValueError('some message')
+    except ValueError:
+      _LOGGER.error('some error', exc_info=True)
+
+    self.fn_log_handler.close()
+
+    log_entry = self.test_logging_service.log_records_received[0].log_entries[0]
+    self.assertContains(log_entry.message, 'some error')
+    self.assertContains(log_entry.trace, 'some message')
+    self.assertContains(log_entry.trace, 'log_handler_test.py')
+
+  def test_context(self):
+    try:
+      with statesampler.instruction_id('A'):
+        tracker = statesampler.for_test()
+        with tracker.scoped_state(NameContext('name', 'tid'), 'stage'):
+          _LOGGER.info('message a')
+      with statesampler.instruction_id('B'):
+        _LOGGER.info('message b')
+      _LOGGER.info('message c')
+
+      self.fn_log_handler.close()
+      a, b, c = sum(
+          [list(logs.log_entries)
+           for logs in self.test_logging_service.log_records_received], [])
+
+      self.assertEqual(a.instruction_id, 'A')
+      self.assertEqual(b.instruction_id, 'B')
+      self.assertEqual(c.instruction_id, '')
+
+      self.assertEqual(a.transform_id, 'tid')
+      self.assertEqual(b.transform_id, '')
+      self.assertEqual(c.transform_id, '')
+
+    finally:
+      statesampler.set_current_tracker(None)
+
 
 # Test cases.
 data = {
diff --git a/sdks/python/apache_beam/runners/worker/sdk_worker.py b/sdks/python/apache_beam/runners/worker/sdk_worker.py
index e0534ff..00a0ac2 100644
--- a/sdks/python/apache_beam/runners/worker/sdk_worker.py
+++ b/sdks/python/apache_beam/runners/worker/sdk_worker.py
@@ -39,6 +39,7 @@
 from apache_beam.portability.api import beam_fn_api_pb2_grpc
 from apache_beam.runners.worker import bundle_processor
 from apache_beam.runners.worker import data_plane
+from apache_beam.runners.worker import statesampler
 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
@@ -132,17 +133,18 @@
     _LOGGER.info('Done consuming work.')
 
   def _execute(self, task, request):
-    try:
-      response = task()
-    except Exception:  # pylint: disable=broad-except
-      traceback_string = traceback.format_exc()
-      print(traceback_string, file=sys.stderr)
-      _LOGGER.error(
-          'Error processing instruction %s. Original traceback is\n%s\n',
-          request.instruction_id, traceback_string)
-      response = beam_fn_api_pb2.InstructionResponse(
-          instruction_id=request.instruction_id, error=traceback_string)
-    self._responses.put(response)
+    with statesampler.instruction_id(request.instruction_id):
+      try:
+        response = task()
+      except Exception:  # pylint: disable=broad-except
+        traceback_string = traceback.format_exc()
+        print(traceback_string, file=sys.stderr)
+        _LOGGER.error(
+            'Error processing instruction %s. Original traceback is\n%s\n',
+            request.instruction_id, traceback_string)
+        response = beam_fn_api_pb2.InstructionResponse(
+            instruction_id=request.instruction_id, error=traceback_string)
+      self._responses.put(response)
 
   def _request_register(self, request):
     # registration request is handled synchronously
diff --git a/sdks/python/apache_beam/runners/worker/statesampler.py b/sdks/python/apache_beam/runners/worker/statesampler.py
index 707ee1f..e57815e 100644
--- a/sdks/python/apache_beam/runners/worker/statesampler.py
+++ b/sdks/python/apache_beam/runners/worker/statesampler.py
@@ -19,6 +19,7 @@
 
 from __future__ import absolute_import
 
+import contextlib
 import threading
 from collections import namedtuple
 
@@ -49,6 +50,25 @@
     return None
 
 
+_INSTRUCTION_IDS = threading.local()
+
+
+def get_current_instruction_id():
+  try:
+    return _INSTRUCTION_IDS.instruction_id
+  except AttributeError:
+    return None
+
+
+@contextlib.contextmanager
+def instruction_id(id):
+  try:
+    _INSTRUCTION_IDS.instruction_id = id
+    yield
+  finally:
+    _INSTRUCTION_IDS.instruction_id = None
+
+
 def for_test():
   set_current_tracker(StateSampler('test', CounterFactory()))
   return get_current_tracker()
diff --git a/sdks/python/apache_beam/testing/data/trigger_transcripts.yaml b/sdks/python/apache_beam/testing/data/trigger_transcripts.yaml
index fdda05c..cac0c74 100644
--- a/sdks/python/apache_beam/testing/data/trigger_transcripts.yaml
+++ b/sdks/python/apache_beam/testing/data/trigger_transcripts.yaml
@@ -55,24 +55,6 @@
       - {window: [20, 29], values: [25], timestamp: 25, late: false}
 
 ---
-name: timestamp_combiner_earliest_separate_bundles
-window_fn: FixedWindows(10)
-trigger_fn: Default
-timestamp_combiner: OUTPUT_AT_EARLIEST
-transcript:
-  - input: [1]
-  - input: [2]
-  - input: [3]
-  - input: [10]
-  - input: [11]
-  - input: [25]
-  - watermark: 100
-  - expect:
-      - {window: [0, 9], values: [1, 2, 3], timestamp: 1, final: false}
-      - {window: [10, 19], values: [10, 11], timestamp: 10}
-      - {window: [20, 29], values: [25], timestamp: 25, late: false}
-
----
 name: timestamp_combiner_latest
 window_fn: FixedWindows(10)
 trigger_fn: Default
diff --git a/sdks/python/apache_beam/testing/test_stream_test.py b/sdks/python/apache_beam/testing/test_stream_test.py
index ea815c8..15d1770 100644
--- a/sdks/python/apache_beam/testing/test_stream_test.py
+++ b/sdks/python/apache_beam/testing/test_stream_test.py
@@ -169,7 +169,6 @@
     assert_that(
         records,
         equal_to_per_window(expected_window_to_elements),
-        use_global_window=False,
         label='assert per window')
 
     p.run()
@@ -177,8 +176,9 @@
   def test_gbk_execution_after_watermark_trigger(self):
     test_stream = (TestStream()
                    .advance_watermark_to(10)
-                   .add_elements(['a'])
+                   .add_elements([TimestampedValue('a', 11)])
                    .advance_watermark_to(20)
+                   .add_elements([TimestampedValue('b', 21)])
                    .advance_watermark_to_infinity())
 
     options = PipelineOptions()
@@ -199,15 +199,18 @@
 
     # assert per window
     expected_window_to_elements = {
-        window.IntervalWindow(15, 30): [
+        window.IntervalWindow(0, 15): [
             ('k', ['a']),
-            ('k', []),
+            ('k', [])
+        ],
+        window.IntervalWindow(15, 30): [
+            ('k', ['b']),
+            ('k', [])
         ],
     }
     assert_that(
         records,
         equal_to_per_window(expected_window_to_elements),
-        use_global_window=False,
         label='assert per window')
 
     p.run()
@@ -247,7 +250,6 @@
     assert_that(
         records,
         equal_to_per_window(expected_window_to_elements),
-        use_global_window=False,
         label='assert per window')
 
     p.run()
@@ -349,7 +351,6 @@
     assert_that(
         records,
         equal_to_per_window(expected_window_to_elements),
-        use_global_window=False,
         label='assert per window')
 
     p.run()
@@ -403,7 +404,6 @@
     assert_that(
         records,
         equal_to_per_window(expected_window_to_elements),
-        use_global_window=False,
         label='assert per window')
 
     p.run()
diff --git a/sdks/python/apache_beam/testing/util.py b/sdks/python/apache_beam/testing/util.py
index b52e61b..5b6bc85 100644
--- a/sdks/python/apache_beam/testing/util.py
+++ b/sdks/python/apache_beam/testing/util.py
@@ -39,6 +39,7 @@
 __all__ = [
     'assert_that',
     'equal_to',
+    'equal_to_per_window',
     'is_empty',
     'is_not_empty',
     'matches_all',
@@ -85,30 +86,67 @@
 
   return InAnyOrder(iterable)
 
+class _EqualToPerWindowMatcher(object):
+  def __init__(self, expected_window_to_elements):
+    self._expected_window_to_elements = expected_window_to_elements
+
+  def __call__(self, value):
+    # Short-hand.
+    _expected = self._expected_window_to_elements
+
+    # Match the given windowed value to an expected window. Fails if the window
+    # doesn't exist or the element wasn't found in the window.
+    def match(windowed_value):
+      actual = windowed_value.value
+      window_key = windowed_value.windows[0]
+      try:
+        expected = _expected[window_key]
+      except KeyError:
+        raise BeamAssertException(
+            'Failed assert: window {} not found in any expected ' \
+            'windows {}'.format(window_key, list(_expected.keys())))
+
+      # Remove any matched elements from the window. This is used later on to
+      # assert that all elements in the window were matched with actual
+      # elements.
+      try:
+        _expected[window_key].remove(actual)
+      except ValueError:
+        raise BeamAssertException(
+            'Failed assert: element {} not found in window ' \
+            '{}:{}'.format(actual, window_key, _expected[window_key]))
+
+    # Run the matcher for each window and value pair. Fails if the
+    # windowed_value is not a TestWindowedValue.
+    for windowed_value in value:
+      if not isinstance(windowed_value, TestWindowedValue):
+        raise BeamAssertException(
+            'Failed assert: Received element {} is not of type ' \
+            'TestWindowedValue. Did you forget to set reify_windows=True ' \
+            'on the assertion?'.format(windowed_value))
+      match(windowed_value)
+
+    # Finally, some elements may not have been matched. Assert that we removed
+    # all the elements that we received from the expected list. If the list is
+    # non-empty, then there are unmatched elements.
+    for win in _expected:
+      if _expected[win]:
+        raise BeamAssertException(
+            'Failed assert: unmatched elements {} in window {}'.format(
+                _expected[win], win))
 
 def equal_to_per_window(expected_window_to_elements):
-  """Matcher used by assert_that to check on values for specific windows.
+  """Matcher used by assert_that to check to assert expected windows.
+
+  The 'assert_that' statement must have reify_windows=True. This assertion works
+  when elements are emitted and are finally checked at the end of the window.
 
   Arguments:
     expected_window_to_elements: A dictionary where the keys are the windows
       to check and the values are the elements associated with each window.
   """
-  def matcher(elements):
-    actual_elements_in_window, window = elements
-    if window in expected_window_to_elements:
-      expected_elements_in_window = list(
-          expected_window_to_elements[window])
-      sorted_expected = sorted(expected_elements_in_window)
-      sorted_actual = sorted(actual_elements_in_window)
-      if sorted_expected != sorted_actual:
-        # Results for the same window don't necessarily come all
-        # at once. Hence the same actual window may contain only
-        # subsets of the expected elements for the window.
-        # For example, in the presence of early triggers.
-        if all(elem in sorted_expected for elem in sorted_actual) is False:
-          raise BeamAssertException(
-              'Failed assert: %r not in %r' % (sorted_actual, sorted_expected))
-  return matcher
+
+  return _EqualToPerWindowMatcher(expected_window_to_elements)
 
 
 # Note that equal_to checks if expected and actual are permutations of each
@@ -214,6 +252,10 @@
       pvalue.PCollection), ('%s is not a supported type for Beam assert'
                             % type(actual))
 
+  if isinstance(matcher, _EqualToPerWindowMatcher):
+    reify_windows = True
+    use_global_window = True
+
   class ReifyTimestampWindow(DoFn):
     def process(self, element, timestamp=DoFn.TimestampParam,
                 window=DoFn.WindowParam):
@@ -239,6 +281,8 @@
 
       keyed_actual = pcoll | "ToVoidKey" >> Map(lambda v: (None, v))
 
+      # This is a CoGroupByKey so that the matcher always runs, even if the
+      # PCollection is empty.
       plain_actual = ((keyed_singleton, keyed_actual)
                       | "Group" >> CoGroupByKey()
                       | "Unkey" >> Map(lambda k_values: k_values[1][1]))
diff --git a/sdks/python/apache_beam/testing/util_test.py b/sdks/python/apache_beam/testing/util_test.py
index 1fd1da6..72c9205 100644
--- a/sdks/python/apache_beam/testing/util_test.py
+++ b/sdks/python/apache_beam/testing/util_test.py
@@ -21,14 +21,20 @@
 
 import unittest
 
+import apache_beam as beam
 from apache_beam import Create
+from apache_beam.options.pipeline_options import StandardOptions
 from apache_beam.testing.test_pipeline import TestPipeline
 from apache_beam.testing.util import BeamAssertException
 from apache_beam.testing.util import TestWindowedValue
 from apache_beam.testing.util import assert_that
 from apache_beam.testing.util import equal_to
+from apache_beam.testing.util import equal_to_per_window
 from apache_beam.testing.util import is_empty
 from apache_beam.testing.util import is_not_empty
+from apache_beam.transforms import trigger
+from apache_beam.transforms import window
+from apache_beam.transforms.window import FixedWindows
 from apache_beam.transforms.window import GlobalWindow
 from apache_beam.transforms.window import IntervalWindow
 from apache_beam.utils.timestamp import MIN_TIMESTAMP
@@ -110,6 +116,96 @@
       with TestPipeline() as p:
         assert_that(p | Create([]), is_not_empty())
 
+  def test_equal_to_per_window_passes(self):
+    start = int(MIN_TIMESTAMP.micros // 1e6) - 5
+    end = start + 20
+    expected = {
+        window.IntervalWindow(start, end): [('k', [1])],
+    }
+    with TestPipeline(options=StandardOptions(streaming=True)) as p:
+      assert_that((p
+                   | Create([1])
+                   | beam.WindowInto(
+                       FixedWindows(20),
+                       trigger=trigger.AfterWatermark(),
+                       accumulation_mode=trigger.AccumulationMode.DISCARDING)
+                   | beam.Map(lambda x: ('k', x))
+                   | beam.GroupByKey()),
+                  equal_to_per_window(expected),
+                  reify_windows=True)
+
+  def test_equal_to_per_window_fail_unmatched_window(self):
+    with self.assertRaises(BeamAssertException):
+      expected = {
+          window.IntervalWindow(50, 100): [('k', [1])],
+      }
+      with TestPipeline(options=StandardOptions(streaming=True)) as p:
+        assert_that((p
+                     | Create([1])
+                     | beam.WindowInto(
+                         FixedWindows(20),
+                         trigger=trigger.AfterWatermark(),
+                         accumulation_mode=trigger.AccumulationMode.DISCARDING)
+                     | beam.Map(lambda x: ('k', x))
+                     | beam.GroupByKey()),
+                    equal_to_per_window(expected),
+                    reify_windows=True)
+
+  def test_equal_to_per_window_fail_unmatched_element(self):
+    with self.assertRaises(BeamAssertException):
+      start = int(MIN_TIMESTAMP.micros // 1e6) - 5
+      end = start + 20
+      expected = {
+          window.IntervalWindow(start, end): [('k', [1]), ('k', [2])],
+      }
+      with TestPipeline(options=StandardOptions(streaming=True)) as p:
+        assert_that((p
+                     | Create([1])
+                     | beam.WindowInto(
+                         FixedWindows(20),
+                         trigger=trigger.AfterWatermark(),
+                         accumulation_mode=trigger.AccumulationMode.DISCARDING)
+                     | beam.Map(lambda x: ('k', x))
+                     | beam.GroupByKey()),
+                    equal_to_per_window(expected),
+                    reify_windows=True)
+
+  def test_equal_to_per_window_succeeds_no_reify_windows(self):
+    start = int(MIN_TIMESTAMP.micros // 1e6) - 5
+    end = start + 20
+    expected = {
+        window.IntervalWindow(start, end): [('k', [1])],
+    }
+    with TestPipeline(options=StandardOptions(streaming=True)) as p:
+      assert_that((p
+                   | Create([1])
+                   | beam.WindowInto(
+                       FixedWindows(20),
+                       trigger=trigger.AfterWatermark(),
+                       accumulation_mode=trigger.AccumulationMode.DISCARDING)
+                   | beam.Map(lambda x: ('k', x))
+                   | beam.GroupByKey()),
+                  equal_to_per_window(expected))
+
+  def test_equal_to_per_window_fail_unexpected_element(self):
+    with self.assertRaises(BeamAssertException):
+      start = int(MIN_TIMESTAMP.micros // 1e6) - 5
+      end = start + 20
+      expected = {
+          window.IntervalWindow(start, end): [('k', [1])],
+      }
+      with TestPipeline(options=StandardOptions(streaming=True)) as p:
+        assert_that((p
+                     | Create([1, 2])
+                     | beam.WindowInto(
+                         FixedWindows(20),
+                         trigger=trigger.AfterWatermark(),
+                         accumulation_mode=trigger.AccumulationMode.DISCARDING)
+                     | beam.Map(lambda x: ('k', x))
+                     | beam.GroupByKey()),
+                    equal_to_per_window(expected),
+                    reify_windows=True)
+
 
 if __name__ == '__main__':
   unittest.main()
diff --git a/sdks/python/apache_beam/transforms/core.py b/sdks/python/apache_beam/transforms/core.py
index 4ac4613..3169d53 100644
--- a/sdks/python/apache_beam/transforms/core.py
+++ b/sdks/python/apache_beam/transforms/core.py
@@ -1492,8 +1492,7 @@
 
   # Proxy the type-hint information from the original function to this new
   # wrapped function.
-  type_hints = get_type_hints(fn).with_defaults(
-      typehints.decorators.IOTypeHints.from_callable(fn))
+  type_hints = get_type_hints(fn)
   get_type_hints(wrapper).input_types = type_hints.input_types
   output_hint = type_hints.simple_output_type(label)
   if output_hint:
@@ -1563,8 +1562,7 @@
 
   # Proxy the type-hint information from the original function to this new
   # wrapped function.
-  type_hints = get_type_hints(fn).with_defaults(
-      typehints.decorators.IOTypeHints.from_callable(fn))
+  type_hints = get_type_hints(fn)
   get_type_hints(wrapper).input_types = type_hints.input_types
   output_hint = type_hints.simple_output_type(label)
   if output_hint:
@@ -1610,10 +1608,16 @@
   # TODO: What about callable classes?
   if hasattr(fn, '__name__'):
     wrapper.__name__ = fn.__name__
+
+  # Get type hints from this instance or the callable. Do not use output type
+  # hints from the callable (which should be bool if set).
+  fn_type_hints = typehints.decorators.IOTypeHints.from_callable(fn)
+  if fn_type_hints is not None:
+    fn_type_hints.output_types = None
+  type_hints = get_type_hints(fn).with_defaults(fn_type_hints)
+
   # Proxy the type-hint information from the function being wrapped, setting the
   # output type to be the same as the input type.
-  type_hints = get_type_hints(fn).with_defaults(
-      typehints.decorators.IOTypeHints.from_callable(fn))
   get_type_hints(wrapper).input_types = type_hints.input_types
   output_hint = type_hints.simple_output_type(label)
   if (output_hint is None
diff --git a/sdks/python/apache_beam/transforms/external.py b/sdks/python/apache_beam/transforms/external.py
index 75fe766..f1f0c15 100644
--- a/sdks/python/apache_beam/transforms/external.py
+++ b/sdks/python/apache_beam/transforms/external.py
@@ -225,7 +225,6 @@
   _namespace_counter = 0
   _namespace = threading.local()
 
-  _EXPANDED_TRANSFORM_UNIQUE_NAME = 'root'
   _IMPULSE_PREFIX = 'impulse'
 
   def __init__(self, urn, payload, expansion_service=None):
@@ -294,7 +293,7 @@
         else pvalueish.pipeline)
     context = pipeline_context.PipelineContext()
     transform_proto = beam_runner_api_pb2.PTransform(
-        unique_name=self._EXPANDED_TRANSFORM_UNIQUE_NAME,
+        unique_name=pipeline._current_transform().full_label,
         spec=beam_runner_api_pb2.FunctionSpec(
             urn=self._urn, payload=self._payload))
     for tag, pcoll in self._inputs.items():
@@ -398,8 +397,7 @@
         continue
       assert id.startswith(self._namespace), (id, self._namespace)
       new_proto = beam_runner_api_pb2.PTransform(
-          unique_name=full_label + proto.unique_name[
-              len(self._EXPANDED_TRANSFORM_UNIQUE_NAME):],
+          unique_name=proto.unique_name,
           spec=proto.spec,
           subtransforms=proto.subtransforms,
           inputs={tag: pcoll_renames.get(pcoll, pcoll)
diff --git a/sdks/python/apache_beam/transforms/external_test.py b/sdks/python/apache_beam/transforms/external_test.py
index fe26977..7ae2bf5 100644
--- a/sdks/python/apache_beam/transforms/external_test.py
+++ b/sdks/python/apache_beam/transforms/external_test.py
@@ -320,6 +320,17 @@
     with beam.Pipeline() as p:
       assert_that(p | FibTransform(6), equal_to([8]))
 
+  def test_unique_name(self):
+    p = beam.Pipeline()
+    _ = p | FibTransform(6)
+    proto = p.to_runner_api()
+    xforms = [x.unique_name for x in proto.components.transforms.values()]
+    self.assertEqual(
+        len(set(xforms)), len(xforms), msg='Transform names are not unique.')
+    pcolls = [x.unique_name for x in proto.components.pcollections.values()]
+    self.assertEqual(
+        len(set(pcolls)), len(pcolls), msg='PCollection names are not unique.')
+
   def test_java_expansion_portable_runner(self):
     ExternalTransformTest.expansion_service_port = os.environ.get(
         'EXPANSION_PORT')
diff --git a/sdks/python/apache_beam/transforms/ptransform_test.py b/sdks/python/apache_beam/transforms/ptransform_test.py
index f240204..ffb245c 100644
--- a/sdks/python/apache_beam/transforms/ptransform_test.py
+++ b/sdks/python/apache_beam/transforms/ptransform_test.py
@@ -575,7 +575,6 @@
     assert_that(result, equal_to([0, 1, 2, 3, 4, 5, 6, 7]))
     pipeline.run()
 
-  @attr('ValidatesRunner')
   def test_flatten_no_pcollections(self):
     pipeline = TestPipeline()
     with self.assertRaises(ValueError):
diff --git a/sdks/python/apache_beam/transforms/trigger.py b/sdks/python/apache_beam/transforms/trigger.py
index 6f59f21..2a76c2f 100644
--- a/sdks/python/apache_beam/transforms/trigger.py
+++ b/sdks/python/apache_beam/transforms/trigger.py
@@ -215,6 +215,16 @@
     pass
 
   @abstractmethod
+  def has_ontime_pane(self):
+    """Whether this trigger creates an empty pane even if there are no elements.
+
+    Returns:
+      True if this trigger guarantees that there will always be an ON_TIME pane
+      even if there are no elements in that pane.
+    """
+    pass
+
+  @abstractmethod
   def on_fire(self, watermark, window, context):
     """Called when a trigger actually fires.
 
@@ -280,6 +290,10 @@
         context.clear_timer('', TimeDomain.WATERMARK)
 
   def should_fire(self, time_domain, watermark, window, context):
+    if watermark >= window.end:
+      # Explicitly clear the timer so that late elements are not emitted again
+      # when the timer is fired.
+      context.clear_timer('', TimeDomain.WATERMARK)
     return watermark >= window.end
 
   def on_fire(self, watermark, window, context):
@@ -302,6 +316,9 @@
     return beam_runner_api_pb2.Trigger(
         default=beam_runner_api_pb2.Trigger.Default())
 
+  def has_ontime_pane(self):
+    return True
+
 
 class AfterProcessingTime(TriggerFn):
   """Fire exactly once after a specified delay from processing time.
@@ -351,6 +368,9 @@
         after_processing_time=beam_runner_api_pb2.Trigger.AfterProcessingTime(
             timestamp_transforms=[delay_proto]))
 
+  def has_ontime_pane(self):
+    return False
+
 
 class AfterWatermark(TriggerFn):
   """Fire exactly once when the watermark passes the end of the window.
@@ -406,6 +426,9 @@
       return self.late.should_fire(time_domain, watermark,
                                    window, NestedContext(context, 'late'))
     elif watermark >= window.end:
+      # Explicitly clear the timer so that late elements are not emitted again
+      # when the timer is fired.
+      context.clear_timer('', TimeDomain.WATERMARK)
       return True
     elif self.early:
       return self.early.should_fire(time_domain, watermark,
@@ -461,6 +484,9 @@
             early_firings=early_proto,
             late_firings=late_proto))
 
+  def has_ontime_pane(self):
+    return True
+
 
 class AfterCount(TriggerFn):
   """Fire when there are at least count elements in this window pane.
@@ -509,6 +535,8 @@
         element_count=beam_runner_api_pb2.Trigger.ElementCount(
             element_count=self.count))
 
+  def has_ontime_pane(self):
+    return False
 
 class Repeatedly(TriggerFn):
   """Repeatedly invoke the given trigger, never finishing."""
@@ -552,6 +580,9 @@
         repeat=beam_runner_api_pb2.Trigger.Repeat(
             subtrigger=self.underlying.to_runner_api(context)))
 
+  def has_ontime_pane(self):
+    return self.underlying.has_ontime_pane()
+
 
 class _ParallelTriggerFn(with_metaclass(ABCMeta, TriggerFn)):
 
@@ -630,6 +661,8 @@
     else:
       raise NotImplementedError(self)
 
+  def has_ontime_pane(self):
+    return any(t.has_ontime_pane() for t in self.triggers)
 
 class AfterAny(_ParallelTriggerFn):
   """Fires when any subtrigger fires.
@@ -717,6 +750,8 @@
                 subtrigger.to_runner_api(context)
                 for subtrigger in self.triggers]))
 
+  def has_ontime_pane(self):
+    return any(t.has_ontime_pane() for t in self.triggers)
 
 class OrFinally(AfterAny):
 
@@ -968,18 +1003,21 @@
   """Breaks a series of bundle and timer firings into window (pane)s."""
 
   @abstractmethod
-  def process_elements(self, state, windowed_values, output_watermark):
+  def process_elements(self, state, windowed_values, output_watermark,
+                       input_watermark=MIN_TIMESTAMP):
     pass
 
   @abstractmethod
-  def process_timer(self, window_id, name, time_domain, timestamp, state):
+  def process_timer(self, window_id, name, time_domain, timestamp, state,
+                    input_watermark=None):
     pass
 
-  def process_entire_key(
-      self, key, windowed_values, output_watermark=MIN_TIMESTAMP):
+  def process_entire_key(self, key, windowed_values,
+                         unused_output_watermark=None,
+                         unused_input_watermark=None):
     state = InMemoryUnmergedState()
     for wvalue in self.process_elements(
-        state, windowed_values, output_watermark):
+        state, windowed_values, MIN_TIMESTAMP, MIN_TIMESTAMP):
       yield wvalue.with_value((key, wvalue.value))
     while state.timers:
       fired = state.get_and_clear_timers()
@@ -1039,14 +1077,17 @@
       index=0,
       nonspeculative_index=0)
 
-  def process_elements(self, state, windowed_values, unused_output_watermark):
+  def process_elements(self, state, windowed_values,
+                       unused_output_watermark,
+                       unused_input_watermark=MIN_TIMESTAMP):
     yield WindowedValue(
         _UnwindowedValues(windowed_values),
         MIN_TIMESTAMP,
         self.GLOBAL_WINDOW_TUPLE,
         self.ONLY_FIRING)
 
-  def process_timer(self, window_id, name, time_domain, timestamp, state):
+  def process_timer(self, window_id, name, time_domain, timestamp, state,
+                    input_watermark=None):
     raise TypeError('Triggers never set or called for batch default windowing.')
 
 
@@ -1057,15 +1098,19 @@
     self.phased_combine_fn = phased_combine_fn
     self.underlying = underlying
 
-  def process_elements(self, state, windowed_values, output_watermark):
+  def process_elements(self, state, windowed_values, output_watermark,
+                       input_watermark=MIN_TIMESTAMP):
     uncombined = self.underlying.process_elements(state, windowed_values,
-                                                  output_watermark)
+                                                  output_watermark,
+                                                  input_watermark)
     for output in uncombined:
       yield output.with_value(self.phased_combine_fn.apply(output.value))
 
-  def process_timer(self, window_id, name, time_domain, timestamp, state):
+  def process_timer(self, window_id, name, time_domain, timestamp, state,
+                    input_watermark=None):
     uncombined = self.underlying.process_timer(window_id, name, time_domain,
-                                               timestamp, state)
+                                               timestamp, state,
+                                               input_watermark)
     for output in uncombined:
       yield output.with_value(self.phased_combine_fn.apply(output.value))
 
@@ -1094,7 +1139,8 @@
     self.accumulation_mode = windowing.accumulation_mode
     self.is_merging = True
 
-  def process_elements(self, state, windowed_values, output_watermark):
+  def process_elements(self, state, windowed_values, output_watermark,
+                       input_watermark=MIN_TIMESTAMP):
     if self.is_merging:
       state = MergeableStateAdapter(state)
 
@@ -1155,14 +1201,17 @@
         self.trigger_fn.on_element(value, window, context)
 
       # Maybe fire this window.
-      watermark = MIN_TIMESTAMP
-      if self.trigger_fn.should_fire(TimeDomain.WATERMARK, watermark,
+      if self.trigger_fn.should_fire(TimeDomain.WATERMARK, input_watermark,
                                      window, context):
-        finished = self.trigger_fn.on_fire(watermark, window, context)
-        yield self._output(window, finished, state, output_watermark, False)
+        finished = self.trigger_fn.on_fire(input_watermark, window, context)
+        yield self._output(window, finished, state, input_watermark,
+                           output_watermark, False)
 
   def process_timer(self, window_id, unused_name, time_domain, timestamp,
-                    state):
+                    state, input_watermark=None):
+    if input_watermark is None:
+      input_watermark = timestamp
+
     if self.is_merging:
       state = MergeableStateAdapter(state)
     window = state.get_window(window_id)
@@ -1175,16 +1224,17 @@
         if self.trigger_fn.should_fire(time_domain, timestamp,
                                        window, context):
           finished = self.trigger_fn.on_fire(timestamp, window, context)
-          yield self._output(window, finished, state, timestamp,
-                             time_domain == TimeDomain.WATERMARK)
+          yield self._output(window, finished, state, input_watermark,
+                             timestamp, time_domain == TimeDomain.WATERMARK)
     else:
       raise Exception('Unexpected time domain: %s' % time_domain)
 
-  def _output(self, window, finished, state, watermark, maybe_ontime):
+  def _output(self, window, finished, state, input_watermark, output_watermark,
+              maybe_ontime):
     """Output window and clean up if appropriate."""
     index = state.get_state(window, self.INDEX)
     state.add_state(window, self.INDEX, 1)
-    if watermark <= window.max_timestamp():
+    if output_watermark <= window.max_timestamp():
       nonspeculative_index = -1
       timing = windowed_value.PaneInfoTiming.EARLY
       if state.get_state(window, self.NONSPECULATIVE_INDEX):
@@ -1220,6 +1270,10 @@
     if timestamp is None:
       # If no watermark hold was set, output at end of window.
       timestamp = window.max_timestamp()
+    elif input_watermark < window.end and self.trigger_fn.has_ontime_pane():
+      # Hold the watermark in case there is an empty pane that needs to be fired
+      # at the end of the window.
+      pass
     else:
       state.clear_state(window, self.WATERMARK_HOLD)
 
diff --git a/sdks/python/apache_beam/transforms/trigger_test.py b/sdks/python/apache_beam/transforms/trigger_test.py
index d1e5433..58b29e0 100644
--- a/sdks/python/apache_beam/transforms/trigger_test.py
+++ b/sdks/python/apache_beam/transforms/trigger_test.py
@@ -23,6 +23,7 @@
 import json
 import os.path
 import pickle
+import random
 import unittest
 from builtins import range
 from builtins import zip
@@ -122,7 +123,8 @@
     state = InMemoryUnmergedState()
 
     for bundle in bundles:
-      for wvalue in driver.process_elements(state, bundle, MIN_TIMESTAMP):
+      for wvalue in driver.process_elements(state, bundle, MIN_TIMESTAMP,
+                                            MIN_TIMESTAMP):
         window, = wvalue.windows
         self.assertEqual(window.max_timestamp(), wvalue.timestamp)
         actual_panes[window].append(set(wvalue.value))
@@ -131,13 +133,14 @@
       for timer_window, (name, time_domain, timestamp) in (
           state.get_and_clear_timers()):
         for wvalue in driver.process_timer(
-            timer_window, name, time_domain, timestamp, state):
+            timer_window, name, time_domain, timestamp, state, MIN_TIMESTAMP):
           window, = wvalue.windows
           self.assertEqual(window.max_timestamp(), wvalue.timestamp)
           actual_panes[window].append(set(wvalue.value))
 
     for bundle in late_bundles:
-      for wvalue in driver.process_elements(state, bundle, MAX_TIMESTAMP):
+      for wvalue in driver.process_elements(state, bundle, MAX_TIMESTAMP,
+                                            MAX_TIMESTAMP):
         window, = wvalue.windows
         self.assertEqual(window.max_timestamp(), wvalue.timestamp)
         actual_panes[window].append(set(wvalue.value))
@@ -146,7 +149,7 @@
         for timer_window, (name, time_domain, timestamp) in (
             state.get_and_clear_timers()):
           for wvalue in driver.process_timer(
-              timer_window, name, time_domain, timestamp, state):
+              timer_window, name, time_domain, timestamp, state, MAX_TIMESTAMP):
             window, = wvalue.windows
             self.assertEqual(window.max_timestamp(), wvalue.timestamp)
             actual_panes[window].append(set(wvalue.value))
@@ -395,7 +398,7 @@
                    for k in range(10))
     with self.assertRaises(TypeError):
       pickle.dumps(unpicklable)
-    for unwindowed in driver.process_elements(None, unpicklable, None):
+    for unwindowed in driver.process_elements(None, unpicklable, None, None):
       self.assertEqual(pickle.loads(pickle.dumps(unwindowed)).value,
                        list(range(10)))
 
@@ -644,7 +647,9 @@
           vs, windows=[window], timestamp=t, pane_info=p)))
 
 
-def _windowed_value_info_check(actual, expected):
+def _windowed_value_info_check(actual, expected, key=None):
+
+  key_string = ' for %s' % key if key else ''
 
   def format(panes):
     return '\n[%s]\n' % '\n '.join(str(pane) for pane in sorted(
@@ -652,12 +657,12 @@
 
   if len(actual) > len(expected):
     raise AssertionError(
-        'Unexpected output: expected %s but got %s' % (
-            format(expected), format(actual)))
+        'Unexpected output%s: expected %s but got %s' % (
+            key_string, format(expected), format(actual)))
   elif len(expected) > len(actual):
     raise AssertionError(
-        'Unmatched output: expected %s but got %s' % (
-            format(expected), format(actual)))
+        'Unmatched output%s: expected %s but got %s' % (
+            key_string, format(expected), format(actual)))
   else:
 
     def diff(actual, expected):
@@ -670,8 +675,8 @@
       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))
+            'Unmatched output%s: %s not found in %s (diffs in %s)' % (
+                key_string, output, format(expected), diffs))
 
 
 class _ConcatCombineFn(beam.CombineFn):
@@ -757,6 +762,19 @@
     if runner_name in spec.get('broken_on', ()):
       self.skipTest('Known to be broken on %s' % runner_name)
 
+    is_order_agnostic = (
+        isinstance(trigger_fn, DefaultTrigger)
+        and accumulation_mode == AccumulationMode.ACCUMULATING)
+
+    if is_order_agnostic:
+      reshuffle_seed = random.randrange(1 << 20)
+      keys = [
+          u'original', u'reversed', u'reshuffled(%s)' % reshuffle_seed,
+          u'one-element-bundles', u'one-element-bundles-reversed',
+          u'two-element-bundles']
+    else:
+      keys = [u'key1', u'key2']
+
     # 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.
@@ -767,7 +785,28 @@
       else:
         test_stream.add_elements([json.dumps(('expect', []))])
         if action == 'input':
-          test_stream.add_elements([json.dumps(('input', e)) for e in params])
+          def keyed(key, values):
+            return [json.dumps(('input', (key, v))) for v in values]
+          if is_order_agnostic:
+            # Must match keys above.
+            test_stream.add_elements(keyed('original', params))
+            test_stream.add_elements(keyed('reversed', reversed(params)))
+            r = random.Random(reshuffle_seed)
+            reshuffled = list(params)
+            r.shuffle(reshuffled)
+            test_stream.add_elements(keyed(
+                'reshuffled(%s)' % reshuffle_seed, reshuffled))
+            for v in params:
+              test_stream.add_elements(keyed('one-element-bundles', [v]))
+            for v in reversed(params):
+              test_stream.add_elements(
+                  keyed('one-element-bundles-reversed', [v]))
+            for ix in range(0, len(params), 2):
+              test_stream.add_elements(
+                  keyed('two-element-bundles', params[ix:ix+2]))
+          else:
+            for key in keys:
+              test_stream.add_elements(keyed(key, params))
         elif action == 'watermark':
           test_stream.advance_watermark_to(params)
         elif action == 'clock':
@@ -806,7 +845,7 @@
               beam.transforms.userstate.BagStateSpec(
                   'expected',
                   beam.coders.FastPrimitivesCoder()))):
-        _, (action, data) = element
+        key, (action, data) = element
 
         if self.allow_out_of_order:
           if action == 'expect' and not list(seen.read()):
@@ -831,7 +870,7 @@
         elif action == 'expect':
           actual = list(seen.read())
           seen.clear()
-          _windowed_value_info_check(actual, data)
+          _windowed_value_info_check(actual, data, key)
 
         else:
           raise ValueError('Unexpected action: %s' % action)
@@ -842,11 +881,9 @@
       # a branch of expected results.
       inputs, expected = (
           inputs_and_expected
-          | beam.FlatMapTuple(
-              lambda tag, value: [
-                  beam.pvalue.TaggedOutput(tag, ('key1', value)),
-                  beam.pvalue.TaggedOutput(tag, ('key2', value)),
-              ]).with_outputs('input', 'expect'))
+          | beam.MapTuple(
+              lambda tag, value: beam.pvalue.TaggedOutput(tag, value),
+              ).with_outputs('input', 'expect'))
 
       # Process the inputs with the given windowing to produce actual outputs.
       outputs = (
@@ -865,7 +902,8 @@
           | 'Global' >> beam.WindowInto(beam.transforms.window.GlobalWindows()))
       # Feed both the expected and actual outputs to Check() for comparison.
       tagged_expected = (
-          expected | beam.MapTuple(lambda key, value: (key, ('expect', value))))
+          expected | beam.FlatMap(
+              lambda value: [(key, ('expect', value)) for key in keys]))
       tagged_outputs = (
           outputs | beam.MapTuple(lambda key, value: (key, ('actual', value))))
       # pylint: disable=expression-not-assigned
diff --git a/sdks/python/apache_beam/typehints/typed_pipeline_test_py3.py b/sdks/python/apache_beam/typehints/typed_pipeline_test_py3.py
index 988f0c2..e49b11c 100644
--- a/sdks/python/apache_beam/typehints/typed_pipeline_test_py3.py
+++ b/sdks/python/apache_beam/typehints/typed_pipeline_test_py3.py
@@ -215,7 +215,9 @@
     self.assertEqual(th.input_types, ((int,), {}))
     self.assertEqual(th.output_types, ((int,), {}))
 
+  @unittest.skip('BEAM-8662: Py3 annotations not yet supported for MapTuple')
   def test_flat_map_tuple_wrapper(self):
+    # TODO(BEAM-8662): Also test with a fn that accepts default arguments.
     def tuple_map_fn(a: str, b: str, c: str) -> typehints.Iterable[str]:
       return [a, b, c]
 
@@ -231,7 +233,9 @@
     self.assertEqual(th.input_types, ((int,), {}))
     self.assertEqual(th.output_types, ((int,), {}))
 
+  @unittest.skip('BEAM-8662: Py3 annotations not yet supported for MapTuple')
   def test_map_tuple(self):
+    # TODO(BEAM-8662): Also test with a fn that accepts default arguments.
     def tuple_map_fn(a: str, b: str, c: str) -> str:
       return a + b + c
 
@@ -245,7 +249,7 @@
 
     th = beam.Filter(filter_fn).get_type_hints()
     self.assertEqual(th.input_types, ((int,), {}))
-    self.assertEqual(th.output_types, ((bool,), {}))
+    self.assertEqual(th.output_types, ((int,), {}))
 
 
 if __name__ == '__main__':
diff --git a/sdks/python/apache_beam/version.py b/sdks/python/apache_beam/version.py
index f32561a..ba28fb7 100644
--- a/sdks/python/apache_beam/version.py
+++ b/sdks/python/apache_beam/version.py
@@ -18,4 +18,4 @@
 """Apache Beam SDK version information and utilities."""
 
 
-__version__ = '2.18.0.dev'
+__version__ = '2.19.0.dev'
diff --git a/website/src/documentation/dsls/sql/extensions/create-external-table.md b/website/src/documentation/dsls/sql/extensions/create-external-table.md
index 81d7dae..e331eff 100644
--- a/website/src/documentation/dsls/sql/extensions/create-external-table.md
+++ b/website/src/documentation/dsls/sql/extensions/create-external-table.md
@@ -308,6 +308,43 @@
 
 Only simple types are supported.
 
+## MongoDB
+
+### Syntax
+
+```
+CREATE EXTERNAL TABLE [ IF NOT EXISTS ] tableName (tableElement [, tableElement ]*)
+TYPE mongodb
+LOCATION 'mongodb://[HOST]:[PORT]/[DATABASE]/[COLLECTION]'
+```
+*   `LOCATION`: Location of the collection.
+    *   `HOST`: Location of the MongoDB server. Can be localhost or an ip address.
+         When authentication is required username and password can be specified
+         as follows: `username:password@localhost`.
+    *   `PORT`: Port on which MongoDB server is listening.
+    *   `DATABASE`: Database to connect to.
+    *   `COLLECTION`: Collection within the database.
+
+### Read Mode
+
+Read Mode supports reading from a collection.
+
+### Write Mode
+
+Write Mode supports writing to a collection.
+
+### Schema
+
+Only simple types are supported. MongoDB documents are mapped to Beam SQL types via [`JsonToRow`](https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/transforms/JsonToRow.html) transform.
+
+### Example
+
+```
+CREATE EXTERNAL TABLE users (id INTEGER, username VARCHAR)
+TYPE mongodb
+LOCATION 'mongodb://localhost:27017/apache/users'
+```
+
 ## Text
 
 TextIO is experimental in Beam SQL. Read Mode and Write Mode do not currently