Merge pull request #13126 More flexible dataframe conversions.

diff --git a/.mailmap b/.mailmap
index be5e16c..69ed29b 100644
--- a/.mailmap
+++ b/.mailmap
@@ -443,9 +443,6 @@
 Roberto Congiu <rcongiu@agentace.com>
 Robin Qiu <robinyq@rodete-desktop-imager.corp.google.com>
 Rodrigo Benenson <rodrigo.benenson@gmail.com>
-Romain Manni-Bucau <rmannibucau@apache.org>
-Romain manni-Bucau <rmannibucau@gmail.com>
-Romain Manni-Bucau <rmannibucau@gmail.com>
 Romain Yon <yonromai@users.noreply.github.com>
 Rong Ou <rong.ou@gmail.com>
 Roy Lenferink <lenferinkroy@gmail.com>
diff --git a/.test-infra/jenkins/job_PostCommit_Java_ValidatesRunner_Dataflow.groovy b/.test-infra/jenkins/job_PostCommit_Java_ValidatesRunner_Dataflow.groovy
index 5feeb11..0e3e628 100644
--- a/.test-infra/jenkins/job_PostCommit_Java_ValidatesRunner_Dataflow.groovy
+++ b/.test-infra/jenkins/job_PostCommit_Java_ValidatesRunner_Dataflow.groovy
@@ -23,7 +23,7 @@
 // This job runs the suite of ValidatesRunner tests against the Dataflow
 // runner.
 PostcommitJobBuilder.postCommitJob('beam_PostCommit_Java_ValidatesRunner_Dataflow',
-    'Run Dataflow ValidatesRunner', 'Google Cloud Dataflow Runner ValidatesRunner Tests', this) {
+    'Run Dataflow ValidatesRunner', 'Google Cloud Dataflow Runner ValidatesRunner Tests (streaming/batch auto)', this) {
 
       description('Runs the ValidatesRunner suite on the Dataflow runner.')
 
diff --git a/.test-infra/jenkins/job_PostCommit_Java_ValidatesRunner_Dataflow_streaming.groovy b/.test-infra/jenkins/job_PostCommit_Java_ValidatesRunner_Dataflow_streaming.groovy
index 53662a3..8bea6be 100644
--- a/.test-infra/jenkins/job_PostCommit_Java_ValidatesRunner_Dataflow_streaming.groovy
+++ b/.test-infra/jenkins/job_PostCommit_Java_ValidatesRunner_Dataflow_streaming.groovy
@@ -23,7 +23,7 @@
 // This job runs the suite of ValidatesRunner tests against the Dataflow
 // runner.
 PostcommitJobBuilder.postCommitJob('beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming',
-    'Run Dataflow Streaming ValidatesRunner', 'Google Cloud Dataflow Runner ValidatesRunner Tests', this) {
+    'Run Dataflow Streaming ValidatesRunner', 'Google Cloud Dataflow Runner ValidatesRunner Tests (streaming)', this) {
 
       description('Runs the ValidatesRunner suite on the Dataflow runner forcing streaming mode.')
 
diff --git a/.test-infra/jenkins/metrics_report/templates/Metrics_Report.template b/.test-infra/jenkins/metrics_report/templates/Metrics_Report.template
index 2c8c0d3..9492322 100644
--- a/.test-infra/jenkins/metrics_report/templates/Metrics_Report.template
+++ b/.test-infra/jenkins/metrics_report/templates/Metrics_Report.template
@@ -1,20 +1,20 @@
-/*
- * 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.
- */
+{#
+   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.
+#}
 
 <html>
 <body>
diff --git a/.test-infra/metrics/build.gradle b/.test-infra/metrics/build.gradle
index e2e25c1..4dc7157 100644
--- a/.test-infra/metrics/build.gradle
+++ b/.test-infra/metrics/build.gradle
@@ -64,7 +64,7 @@
   doLast {
     assert grgit : 'Cannot use outside of git repository'
 
-    def git = grgit.open()
+    def git = grgit.open(currentDir: project.rootDir)
     def lastCommit = git.log(paths: ['.test-infra/metrics'], maxCommits: 1)[0]
     project.ext.lastCommitId = lastCommit.abbreviatedId
   }
diff --git a/CHANGES.md b/CHANGES.md
index 7293263..10fc2a9 100644
--- a/CHANGES.md
+++ b/CHANGES.md
@@ -77,7 +77,7 @@
 * Fixed X (Java/Python) ([BEAM-X](https://issues.apache.org/jira/browse/BEAM-X)).
 
 
-# [2.25.0] - Unreleased
+# [2.25.0] - 2020-10-23
 
 ## Highlights
 
@@ -92,6 +92,7 @@
 * Added cross-language support to Java's SnowflakeIO.Write, now available in the Python module `apache_beam.io.snowflake` ([BEAM-9898](https://issues.apache.org/jira/browse/BEAM-9898)).
 * Added delete function to Java's `ElasticsearchIO#Write`. Now, Java's ElasticsearchIO can be used to selectively delete documents using `withIsDeleteFn` function ([BEAM-5757](https://issues.apache.org/jira/browse/BEAM-5757)).
 * Java SDK: Added new IO connector for InfluxDB - InfluxDbIO ([BEAM-2546](https://issues.apache.org/jira/browse/BEAM-2546)).
+* Config options added for Python's S3IO ([BEAM-9094](https://issues.apache.org/jira/browse/BEAM-9094))
 
 ## New Features / Improvements
 
diff --git a/build.gradle b/build.gradle
index d7884d8..92ed06f 100644
--- a/build.gradle
+++ b/build.gradle
@@ -161,6 +161,7 @@
   dependsOn ":sdks:java:extensions:zetasketch:postCommit"
   dependsOn ":sdks:java:io:google-cloud-platform:postCommit"
   dependsOn ":sdks:java:io:kinesis:integrationTest"
+  dependsOn ":sdks:java:io:hadoop-format:hadoopFormatIOElasticTest"
 }
 
 task sqlPostCommit() {
diff --git a/buildSrc/build.gradle b/buildSrc/build.gradle
index c932ee5..5ab7120 100644
--- a/buildSrc/build.gradle
+++ b/buildSrc/build.gradle
@@ -57,7 +57,7 @@
   runtime "com.avast.gradle:gradle-docker-compose-plugin:0.13.2"                                       // Enable docker compose tasks
   runtime "ca.cutterslade.gradle:gradle-dependency-analyze:1.4.2"                                     // Enable dep analysis
   runtime "gradle.plugin.net.ossindex:ossindex-gradle-plugin:0.4.11"                                  // Enable dep vulnerability analysis
-  runtime "org.checkerframework:checkerframework-gradle-plugin:0.5.10"                                 // Enable enhanced static checking plugin
+  runtime "org.checkerframework:checkerframework-gradle-plugin:0.5.11"                                 // Enable enhanced static checking plugin
 }
 
 // Because buildSrc is built and tested automatically _before_ gradle
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 b7e1a91..5f599df 100644
--- a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy
+++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy
@@ -389,7 +389,7 @@
     def aws_java_sdk_version = "1.11.718"
     def aws_java_sdk2_version = "2.13.54"
     def cassandra_driver_version = "3.10.2"
-    def checkerframework_version = "3.5.0"
+    def checkerframework_version = "3.7.0"
     def classgraph_version = "4.8.65"
     def gax_version = "1.57.1"
     def generated_grpc_ga_version = "1.85.1"
@@ -397,7 +397,7 @@
     def google_clients_version = "1.30.10"
     def google_cloud_bigdataoss_version = "2.1.5"
     def google_cloud_core_version = "1.93.7"
-    def google_cloud_pubsublite_version = "0.1.6"
+    def google_cloud_pubsublite_version = "0.4.1"
     def google_cloud_spanner_version = "1.59.0"
     def google_cloud_datacatalog_version = "0.32.1"
     def google_code_gson_version = "2.8.6"
@@ -413,7 +413,7 @@
     def kafka_version = "1.0.0"
     def nemo_version = "0.1"
     def netty_version = "4.1.51.Final"
-    def postgres_version = "42.2.2"
+    def postgres_version = "42.2.16"
     def powermock_version = "2.0.2"
     def proto_google_common_protos_version = "1.17.0"
     def protobuf_version = "3.11.1"
@@ -811,9 +811,6 @@
         "com.google.auto.value:auto-value-annotations:1.7",
         "com.google.auto.service:auto-service-annotations:1.0-rc6",
         "com.google.j2objc:j2objc-annotations:1.3",
-        // This contains many improved annotations beyond javax.annotations for enhanced static checking
-        // of the codebase
-        "org.checkerframework:checker-qual:$checkerframework_version",
         // These dependencies are needed to avoid error-prone warnings on package-info.java files,
         // also to include the annotations to suppress warnings.
         //
@@ -857,6 +854,11 @@
           annotationProcessor dep
           testAnnotationProcessor dep
         }
+
+        // This contains many improved annotations beyond javax.annotations for enhanced static checking
+        // of the codebase. It is runtime so users can also take advantage of them. The annotations themselves
+        // are MIT licensed (checkerframework is GPL and cannot be distributed)
+        compile "org.checkerframework:checker-qual:$checkerframework_version"
       }
 
       // Add the optional and provided configurations for dependencies
@@ -1957,10 +1959,9 @@
 
       // Python interpreter version for virtualenv setup and test run. This value can be
       // set from commandline with -PpythonVersion, or in build script of certain project.
-      // If none of them applied, version set here will be used as default value. Currently
-      // the minimum version supported by Beam is 3.6
+      // If none of them applied, version set here will be used as default value.
       project.ext.pythonVersion = project.hasProperty('pythonVersion') ?
-          project.pythonVersion : '3.6'
+          project.pythonVersion : '3.8'
 
       project.task('setupVirtualenv')  {
         doLast {
diff --git a/examples/java/src/main/java/org/apache/beam/examples/common/ExampleUtils.java b/examples/java/src/main/java/org/apache/beam/examples/common/ExampleUtils.java
index 97d3298..bc805fd 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/common/ExampleUtils.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/common/ExampleUtils.java
@@ -62,6 +62,7 @@
  *
  * <p>It is used to run Beam examples.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ExampleUtils {
 
   private static final int SC_NOT_FOUND = 404;
diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/AutoComplete.java b/examples/java/src/main/java/org/apache/beam/examples/complete/AutoComplete.java
index 4fcf282..c4fcefc 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/complete/AutoComplete.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/complete/AutoComplete.java
@@ -95,6 +95,7 @@
  * <p>This will update the Cloud Datastore every 10 seconds based on the last 30 minutes of data
  * received.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class AutoComplete {
 
   /**
diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficMaxLaneFlow.java b/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficMaxLaneFlow.java
index 5388e6e..731ea36 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficMaxLaneFlow.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficMaxLaneFlow.java
@@ -73,6 +73,7 @@
  * <p>The example will try to cancel the pipelines on the signal to terminate the process (CTRL-C)
  * and then exits.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class TrafficMaxLaneFlow {
 
   static final int WINDOW_DURATION = 60; // Default sliding window duration in minutes
diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficRoutes.java b/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficRoutes.java
index 53fa009..048d3f7 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficRoutes.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/complete/TrafficRoutes.java
@@ -77,6 +77,7 @@
  * <p>The example will try to cancel the pipelines on the signal to terminate the process (CTRL-C)
  * and then exits.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class TrafficRoutes {
 
   // Instantiate some small predefined San Diego routes to analyze
diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/game/StatefulTeamScore.java b/examples/java/src/main/java/org/apache/beam/examples/complete/game/StatefulTeamScore.java
index e79e5e4..1af84d0 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/complete/game/StatefulTeamScore.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/complete/game/StatefulTeamScore.java
@@ -76,6 +76,7 @@
  * <p>The BigQuery dataset you specify must already exist. The PubSub topic you specify should be
  * the same topic to which the Injector is publishing.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class StatefulTeamScore extends LeaderBoard {
 
   /** Options supported by {@link StatefulTeamScore}. */
diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/game/UserScore.java b/examples/java/src/main/java/org/apache/beam/examples/complete/game/UserScore.java
index 2938fb0..0d55cab 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/complete/game/UserScore.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/complete/game/UserScore.java
@@ -70,6 +70,7 @@
  * value for example batch data file, or use {@code injector.Injector} to generate your own batch
  * data.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class UserScore {
 
   /** Class to hold info about a game event. */
diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/game/injector/Injector.java b/examples/java/src/main/java/org/apache/beam/examples/complete/game/injector/Injector.java
index 0a779b0..2deef70 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/complete/game/injector/Injector.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/complete/game/injector/Injector.java
@@ -75,6 +75,7 @@
  * Injector <project-name> none <filename>
  * }</pre>
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class Injector {
   private static Pubsub pubsub;
   private static Random random = new Random();
diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/game/utils/WriteToBigQuery.java b/examples/java/src/main/java/org/apache/beam/examples/complete/game/utils/WriteToBigQuery.java
index 0e9b110..2b4e479 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/complete/game/utils/WriteToBigQuery.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/complete/game/utils/WriteToBigQuery.java
@@ -39,6 +39,7 @@
  * Generate, format, and write BigQuery table row information. Use provided information about the
  * field names and types, as well as lambda functions that describe how to generate their values.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class WriteToBigQuery<InputT> extends PTransform<PCollection<InputT>, PDone> {
 
   protected String projectId;
diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/game/utils/WriteToText.java b/examples/java/src/main/java/org/apache/beam/examples/complete/game/utils/WriteToText.java
index e88484e..546d6ca 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/complete/game/utils/WriteToText.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/complete/game/utils/WriteToText.java
@@ -47,6 +47,7 @@
  * Generate, format, and write rows. Use provided information about the field names and types, as
  * well as lambda functions that describe how to generate their values.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class WriteToText<InputT> extends PTransform<PCollection<InputT>, PDone> {
 
   private static final DateTimeFormatter formatter =
diff --git a/examples/java/src/main/java/org/apache/beam/examples/cookbook/TriggerExample.java b/examples/java/src/main/java/org/apache/beam/examples/cookbook/TriggerExample.java
index bc65413..3e24d44 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/cookbook/TriggerExample.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/cookbook/TriggerExample.java
@@ -121,6 +121,7 @@
  * <p>The example will try to cancel the pipelines on the signal to terminate the process (CTRL-C)
  * and then exits.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class TriggerExample {
   // Numeric value of fixed window duration, in minutes
   public static final int WINDOW_DURATION = 30;
diff --git a/examples/java/src/main/java/org/apache/beam/examples/snippets/Snippets.java b/examples/java/src/main/java/org/apache/beam/examples/snippets/Snippets.java
index cb55475..28c6979 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/snippets/Snippets.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/snippets/Snippets.java
@@ -29,6 +29,9 @@
 import com.google.cloud.language.v1.Sentence;
 import com.google.cloud.language.v1.Token;
 import com.google.gson.Gson;
+import java.io.File;
+import java.io.IOException;
+import java.io.RandomAccessFile;
 import java.io.Serializable;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -39,6 +42,7 @@
 import java.util.List;
 import java.util.Map;
 import java.util.stream.Collectors;
+import javax.annotation.Nullable;
 import org.apache.avro.generic.GenericRecord;
 import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.coders.AvroCoder;
@@ -59,6 +63,7 @@
 import org.apache.beam.sdk.io.gcp.bigquery.SchemaAndRecord;
 import org.apache.beam.sdk.io.gcp.bigquery.TableDestination;
 import org.apache.beam.sdk.io.gcp.bigquery.WriteResult;
+import org.apache.beam.sdk.io.range.OffsetRange;
 import org.apache.beam.sdk.options.Default;
 import org.apache.beam.sdk.options.Description;
 import org.apache.beam.sdk.options.PipelineOptions;
@@ -70,6 +75,7 @@
 import org.apache.beam.sdk.transforms.Count;
 import org.apache.beam.sdk.transforms.Create;
 import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.DoFn.BoundedPerElement;
 import org.apache.beam.sdk.transforms.MapElements;
 import org.apache.beam.sdk.transforms.ParDo;
 import org.apache.beam.sdk.transforms.PeriodicImpulse;
@@ -80,6 +86,10 @@
 import org.apache.beam.sdk.transforms.join.CoGbkResult;
 import org.apache.beam.sdk.transforms.join.CoGroupByKey;
 import org.apache.beam.sdk.transforms.join.KeyedPCollectionTuple;
+import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker;
+import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker.TruncateResult;
+import org.apache.beam.sdk.transforms.splittabledofn.TimestampObservingWatermarkEstimator;
+import org.apache.beam.sdk.transforms.splittabledofn.WatermarkEstimator;
 import org.apache.beam.sdk.transforms.windowing.AfterProcessingTime;
 import org.apache.beam.sdk.transforms.windowing.FixedWindows;
 import org.apache.beam.sdk.transforms.windowing.GlobalWindows;
@@ -106,6 +116,7 @@
 import org.slf4j.LoggerFactory;
 
 /** Code snippets used in webdocs. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class Snippets {
 
   @DefaultCoder(AvroCoder.class)
@@ -1158,4 +1169,239 @@
       }
     }
   }
+
+  private static class BundleFinalization {
+    private static class BundleFinalizationDoFn extends DoFn<String, Integer> {
+      // [START BundleFinalize]
+      @ProcessElement
+      public void processElement(ProcessContext c, BundleFinalizer bundleFinalizer) {
+        // ... produce output ...
+
+        bundleFinalizer.afterBundleCommit(
+            Instant.now().plus(Duration.standardMinutes(5)),
+            () -> {
+              // ... perform a side effect ...
+            });
+      }
+      // [END BundleFinalize]
+    }
+  }
+
+  private static class SplittableDoFn {
+
+    private static void seekToNextRecordBoundaryInFile(
+        RandomAccessFile file, long initialPosition) {}
+
+    private static Integer readNextRecord(RandomAccessFile file) {
+      // ... read a record ...
+      return null;
+    }
+
+    // [START SDF_BasicExample]
+    @BoundedPerElement
+    private static class FileToWordsFn extends DoFn<String, Integer> {
+      @GetInitialRestriction
+      public OffsetRange getInitialRestriction(@Element String fileName) throws IOException {
+        return new OffsetRange(0, new File(fileName).length());
+      }
+
+      @ProcessElement
+      public void processElement(
+          @Element String fileName,
+          RestrictionTracker<OffsetRange, Long> tracker,
+          OutputReceiver<Integer> outputReceiver)
+          throws IOException {
+        RandomAccessFile file = new RandomAccessFile(fileName, "r");
+        seekToNextRecordBoundaryInFile(file, tracker.currentRestriction().getFrom());
+        while (tracker.tryClaim(file.getFilePointer())) {
+          outputReceiver.output(readNextRecord(file));
+        }
+      }
+
+      // Providing the coder is only necessary if it can not be inferred at runtime.
+      @GetRestrictionCoder
+      public Coder<OffsetRange> getRestrictionCoder() {
+        return OffsetRange.Coder.of();
+      }
+    }
+    // [END SDF_BasicExample]
+
+    private static class BasicExampleWithInitialSplitting extends FileToWordsFn {
+      // [START SDF_BasicExampleWithSplitting]
+      void splitRestriction(
+          @Restriction OffsetRange restriction, OutputReceiver<OffsetRange> splitReceiver) {
+        long splitSize = 64 * (1 << 20);
+        long i = restriction.getFrom();
+        while (i < restriction.getTo() - splitSize) {
+          // Compute and output 64 MiB size ranges to process in parallel
+          long end = i + splitSize;
+          splitReceiver.output(new OffsetRange(i, end));
+          i = end;
+        }
+        // Output the last range
+        splitReceiver.output(new OffsetRange(i, restriction.getTo()));
+      }
+      // [END SDF_BasicExampleWithSplitting]
+    }
+
+    private static class BasicExampleWithBadTryClaimLoop extends DoFn<String, Integer> {
+      // [START SDF_BadTryClaimLoop]
+      @ProcessElement
+      public void badTryClaimLoop(
+          @Element String fileName,
+          RestrictionTracker<OffsetRange, Long> tracker,
+          OutputReceiver<Integer> outputReceiver)
+          throws IOException {
+        RandomAccessFile file = new RandomAccessFile(fileName, "r");
+        seekToNextRecordBoundaryInFile(file, tracker.currentRestriction().getFrom());
+        // The restriction tracker can be modified by another thread in parallel
+        // so storing state locally is ill advised.
+        long end = tracker.currentRestriction().getTo();
+        while (file.getFilePointer() < end) {
+          // Only after successfully claiming should we produce any output and/or
+          // perform side effects.
+          tracker.tryClaim(file.getFilePointer());
+          outputReceiver.output(readNextRecord(file));
+        }
+      }
+      // [END SDF_BadTryClaimLoop]
+    }
+
+    private static class CustomWatermarkEstimatorExample extends DoFn<String, Integer> {
+      private static Instant currentWatermark = Instant.now();
+
+      // [START SDF_CustomWatermarkEstimator]
+
+      // (Optional) Define a custom watermark state type to save information between bundle
+      // processing rounds.
+      public static class MyCustomWatermarkState {
+        public MyCustomWatermarkState(String element, OffsetRange restriction) {
+          // Store data necessary for future watermark computations
+        }
+      }
+
+      // (Optional) Choose which coder to use to encode the watermark estimator state.
+      @GetWatermarkEstimatorStateCoder
+      public Coder<MyCustomWatermarkState> getWatermarkEstimatorStateCoder() {
+        return AvroCoder.of(MyCustomWatermarkState.class);
+      }
+
+      // Define a WatermarkEstimator
+      public static class MyCustomWatermarkEstimator
+          implements TimestampObservingWatermarkEstimator<MyCustomWatermarkState> {
+
+        public MyCustomWatermarkEstimator(MyCustomWatermarkState type) {
+          // Initialize watermark estimator state
+        }
+
+        @Override
+        public void observeTimestamp(Instant timestamp) {
+          // Will be invoked on each output from the SDF
+        }
+
+        @Override
+        public Instant currentWatermark() {
+          // Return a monotonically increasing value
+          return currentWatermark;
+        }
+
+        @Override
+        public MyCustomWatermarkState getState() {
+          // Return state to resume future watermark estimation after a checkpoint/split
+          return null;
+        }
+      }
+
+      // Then, update the DoFn to generate the initial watermark estimator state for all new element
+      // and restriction pairs and to create a new instance given watermark estimator state.
+
+      @GetInitialWatermarkEstimatorState
+      public MyCustomWatermarkState getInitialWatermarkEstimatorState(
+          @Element String element, @Restriction OffsetRange restriction) {
+        // Compute and return the initial watermark estimator state for each element and
+        // restriction. All subsequent processing of an element and restriction will be restored
+        // from the existing state.
+        return new MyCustomWatermarkState(element, restriction);
+      }
+
+      @NewWatermarkEstimator
+      public WatermarkEstimator<MyCustomWatermarkState> newWatermarkEstimator(
+          @WatermarkEstimatorState MyCustomWatermarkState oldState) {
+        return new MyCustomWatermarkEstimator(oldState);
+      }
+    }
+    // [END SDF_CustomWatermarkEstimator]
+
+    private static class UserInitiatedCheckpointExample extends DoFn<String, Integer> {
+      public static class ThrottlingException extends Exception {}
+
+      public static class ElementNotReadyException extends Exception {}
+
+      private Service initializeService() {
+        return null;
+      }
+
+      public interface Service {
+        List<Record> readNextRecords(long position) throws ThrottlingException;
+      }
+
+      public interface Record {
+        long getPosition();
+      }
+
+      // [START SDF_UserInitiatedCheckpoint]
+      @ProcessElement
+      public ProcessContinuation processElement(
+          RestrictionTracker<OffsetRange, Long> tracker, OutputReceiver<Record> outputReceiver) {
+        long currentPosition = tracker.currentRestriction().getFrom();
+        Service service = initializeService();
+        try {
+          while (true) {
+            List<Record> records = service.readNextRecords(currentPosition);
+            if (records.isEmpty()) {
+              // Return a short delay if there is no data to process at the moment.
+              return ProcessContinuation.resume().withResumeDelay(Duration.standardSeconds(10));
+            }
+            for (Record record : records) {
+              if (!tracker.tryClaim(record.getPosition())) {
+                return ProcessContinuation.stop();
+              }
+              currentPosition = record.getPosition() + 1;
+
+              outputReceiver.output(record);
+            }
+          }
+        } catch (ThrottlingException exception) {
+          // Return a longer delay in case we are being throttled.
+          return ProcessContinuation.resume().withResumeDelay(Duration.standardSeconds(60));
+        }
+      }
+      // [END SDF_UserInitiatedCheckpoint]
+    }
+
+    private static class TruncateExample extends DoFn<String, Integer> {
+      // [START SDF_Truncate]
+      @TruncateRestriction
+      @Nullable
+      TruncateResult<OffsetRange> truncateRestriction(
+          @Element String fileName, @Restriction OffsetRange restriction) {
+        if (fileName.contains("optional")) {
+          // Skip optional files
+          return null;
+        }
+        return TruncateResult.of(restriction);
+      }
+      // [END SDF_Truncate]
+    }
+
+    private static class GetSizeExample extends DoFn<String, Integer> {
+      // [START SDF_GetSize]
+      @GetSize
+      double getSize(@Element String fileName, @Restriction OffsetRange restriction) {
+        return (fileName.contains("expensiveRecords") ? 2 : 1) * restriction.getTo()
+            - restriction.getFrom();
+      }
+      // [END SDF_GetSize]
+    }
+  }
 }
diff --git a/examples/java/src/main/java/org/apache/beam/examples/snippets/transforms/io/gcp/bigquery/BigQueryMyData.java b/examples/java/src/main/java/org/apache/beam/examples/snippets/transforms/io/gcp/bigquery/BigQueryMyData.java
index 43d6e1a..9901753 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/snippets/transforms/io/gcp/bigquery/BigQueryMyData.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/snippets/transforms/io/gcp/bigquery/BigQueryMyData.java
@@ -33,6 +33,7 @@
 import org.apache.beam.sdk.coders.AvroCoder;
 import org.apache.beam.sdk.coders.DefaultCoder;
 
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class BigQueryMyData {
 
   @SuppressFBWarnings("URF_UNREAD_FIELD")
diff --git a/examples/java/src/main/java/org/apache/beam/examples/subprocess/configuration/SubProcessConfiguration.java b/examples/java/src/main/java/org/apache/beam/examples/subprocess/configuration/SubProcessConfiguration.java
index b55da9a..2da9815 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/subprocess/configuration/SubProcessConfiguration.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/subprocess/configuration/SubProcessConfiguration.java
@@ -23,7 +23,7 @@
  * Configuration file used to setup the Process kernel for execution of the external library Values
  * are copied from the Options to all them to be Serializable.
  */
-@SuppressWarnings("serial")
+@SuppressWarnings({"serial", "nullness"}) // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SubProcessConfiguration implements Serializable {
 
   // Source GCS directory where the C++ library is located gs://bucket/tests
diff --git a/examples/java/src/main/java/org/apache/beam/examples/subprocess/kernel/SubProcessCommandLineArgs.java b/examples/java/src/main/java/org/apache/beam/examples/subprocess/kernel/SubProcessCommandLineArgs.java
index b29b3c2..6462496 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/subprocess/kernel/SubProcessCommandLineArgs.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/subprocess/kernel/SubProcessCommandLineArgs.java
@@ -21,6 +21,7 @@
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Lists;
 
 /** Parameters to the sub-process, has tuple of ordinal position and the value. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SubProcessCommandLineArgs {
 
   // Parameters to pass to the sub-process
diff --git a/examples/java/src/main/java/org/apache/beam/examples/subprocess/kernel/SubProcessKernel.java b/examples/java/src/main/java/org/apache/beam/examples/subprocess/kernel/SubProcessKernel.java
index 3621f19..13b6236 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/subprocess/kernel/SubProcessKernel.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/subprocess/kernel/SubProcessKernel.java
@@ -33,6 +33,7 @@
 /**
  * This is the process kernel which deals with exec of the subprocess. It also deals with all I/O.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SubProcessKernel {
 
   private static final Logger LOG = LoggerFactory.getLogger(SubProcessKernel.class);
diff --git a/examples/java/src/main/java/org/apache/beam/examples/subprocess/utils/ExecutableFile.java b/examples/java/src/main/java/org/apache/beam/examples/subprocess/utils/ExecutableFile.java
index 675d81a..c350cf5 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/subprocess/utils/ExecutableFile.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/subprocess/utils/ExecutableFile.java
@@ -25,6 +25,7 @@
 
 /** Contains the configuration for the external library. */
 @DefaultCoder(AvroCoder.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ExecutableFile {
 
   String fileName;
diff --git a/examples/java/src/main/java/org/apache/beam/examples/subprocess/utils/FileUtils.java b/examples/java/src/main/java/org/apache/beam/examples/subprocess/utils/FileUtils.java
index cac539d..feabc6e 100644
--- a/examples/java/src/main/java/org/apache/beam/examples/subprocess/utils/FileUtils.java
+++ b/examples/java/src/main/java/org/apache/beam/examples/subprocess/utils/FileUtils.java
@@ -36,6 +36,7 @@
 import org.slf4j.LoggerFactory;
 
 /** Utilities for dealing with movement of files from object stores and workers. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class FileUtils {
 
   private static final Logger LOG = LoggerFactory.getLogger(FileUtils.class);
diff --git a/examples/java/src/test/java/org/apache/beam/examples/WindowedWordCountIT.java b/examples/java/src/test/java/org/apache/beam/examples/WindowedWordCountIT.java
index bfb11ae..3b973f2 100644
--- a/examples/java/src/test/java/org/apache/beam/examples/WindowedWordCountIT.java
+++ b/examples/java/src/test/java/org/apache/beam/examples/WindowedWordCountIT.java
@@ -62,6 +62,7 @@
 
 /** End-to-end integration test of {@link WindowedWordCount}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class WindowedWordCountIT {
 
   @Rule public TestName testName = new TestName();
diff --git a/examples/java/src/test/java/org/apache/beam/examples/complete/TrafficMaxLaneFlowIT.java b/examples/java/src/test/java/org/apache/beam/examples/complete/TrafficMaxLaneFlowIT.java
index 0ee34be..ac9129c 100644
--- a/examples/java/src/test/java/org/apache/beam/examples/complete/TrafficMaxLaneFlowIT.java
+++ b/examples/java/src/test/java/org/apache/beam/examples/complete/TrafficMaxLaneFlowIT.java
@@ -42,6 +42,7 @@
 
 /** End-to-end tests of TrafficMaxLaneFlowIT. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class TrafficMaxLaneFlowIT {
   private static final Logger LOG = LoggerFactory.getLogger(TrafficMaxLaneFlowIT.class);
   private TrafficMaxLaneFlowOptions options;
diff --git a/examples/java/src/test/java/org/apache/beam/examples/complete/TrafficRoutesIT.java b/examples/java/src/test/java/org/apache/beam/examples/complete/TrafficRoutesIT.java
index 3fde9d8..245dfcb 100644
--- a/examples/java/src/test/java/org/apache/beam/examples/complete/TrafficRoutesIT.java
+++ b/examples/java/src/test/java/org/apache/beam/examples/complete/TrafficRoutesIT.java
@@ -42,6 +42,7 @@
 
 /** End-to-end tests of TrafficRoutes. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class TrafficRoutesIT {
   private static final Logger LOG = LoggerFactory.getLogger(TrafficRoutesIT.class);
   private TrafficRoutesOptions options;
diff --git a/learning/katas/go/course-remote-info.yaml b/learning/katas/go/course-remote-info.yaml
index 938ab3f..ce4c7cc 100644
--- a/learning/katas/go/course-remote-info.yaml
+++ b/learning/katas/go/course-remote-info.yaml
@@ -1,2 +1,2 @@
 id: 70387
-update_date: Wed, 30 Sep 2020 15:56:05 UTC
+update_date: Sat, 03 Oct 2020 19:32:12 UTC
diff --git a/learning/katas/go/windowing/adding_timestamp/lesson-info.yaml b/learning/katas/go/windowing/adding_timestamp/lesson-info.yaml
new file mode 100644
index 0000000..eae93f7
--- /dev/null
+++ b/learning/katas/go/windowing/adding_timestamp/lesson-info.yaml
@@ -0,0 +1,21 @@
+#
+# 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.
+#
+
+content:
+- pardo
diff --git a/learning/katas/go/windowing/adding_timestamp/lesson-remote-info.yaml b/learning/katas/go/windowing/adding_timestamp/lesson-remote-info.yaml
new file mode 100644
index 0000000..299b639
--- /dev/null
+++ b/learning/katas/go/windowing/adding_timestamp/lesson-remote-info.yaml
@@ -0,0 +1,3 @@
+id: 422629
+update_date: Wed, 21 Oct 2020 17:54:40 UTC
+unit: 412306
diff --git a/learning/katas/go/windowing/adding_timestamp/pardo/cmd/main.go b/learning/katas/go/windowing/adding_timestamp/pardo/cmd/main.go
new file mode 100644
index 0000000..586d102
--- /dev/null
+++ b/learning/katas/go/windowing/adding_timestamp/pardo/cmd/main.go
@@ -0,0 +1,47 @@
+// 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 main
+
+import (
+	"beam.apache.org/learning/katas/windowing/adding_timestamp/pardo/pkg/common"
+	"beam.apache.org/learning/katas/windowing/adding_timestamp/pardo/pkg/task"
+	"context"
+	"github.com/apache/beam/sdks/go/pkg/beam"
+	"github.com/apache/beam/sdks/go/pkg/beam/log"
+	"github.com/apache/beam/sdks/go/pkg/beam/x/beamx"
+	"time"
+)
+
+func main() {
+	ctx := context.Background()
+
+	p, s := beam.NewPipelineWithRoot()
+
+	input := common.CreateCommits(s)
+
+	result := task.ApplyTransform(s, input)
+
+	beam.ParDo0(s, func(et beam.EventTime, commit task.Commit){
+		t := time.Unix(0, int64(et.Milliseconds()) * 1e6)
+		log.Infof(ctx, "time: %s, message: %s", t.Format("03:04"), commit)
+	}, result)
+
+	err := beamx.Run(ctx, p)
+
+	if err != nil {
+		log.Exitf(context.Background(), "Failed to execute job: %v", err)
+	}
+}
diff --git a/learning/katas/go/windowing/adding_timestamp/pardo/pkg/common/data.go b/learning/katas/go/windowing/adding_timestamp/pardo/pkg/common/data.go
new file mode 100644
index 0000000..07474e2
--- /dev/null
+++ b/learning/katas/go/windowing/adding_timestamp/pardo/pkg/common/data.go
@@ -0,0 +1,51 @@
+// 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 common
+
+import (
+	"beam.apache.org/learning/katas/windowing/adding_timestamp/pardo/pkg/task"
+	"github.com/apache/beam/sdks/go/pkg/beam"
+	"time"
+)
+
+var (
+	commits = []task.Commit{
+		{
+			Datetime: time.Date(2020, 7, 31, 15, 52, 5, 0, time.UTC),
+			Message:  "3c6c45924a Remove trailing whitespace from README",
+		},
+		{
+			Datetime: time.Date(2020, 7, 31, 15, 59, 40, 0, time.UTC),
+			Message:  "a52be99b62 Merge pull request #12443 from KevinGG/whitespace",
+		},
+		{
+			Datetime: time.Date(2020, 7, 31, 16, 7, 36, 0, time.UTC),
+			Message:  "7c1772d13f Merge pull request #12439 from ibzib/beam-9199-1",
+		},
+		{
+			Datetime: time.Date(2020, 7, 31, 16, 35, 41, 0, time.UTC),
+			Message:  "d971ba13b8 Widen ranges for GCP libraries (#12198)",
+		},
+		{
+			Datetime: time.Date(2020, 8, 1, 0, 7, 25, 0, time.UTC),
+			Message:  "875620111b Enable all Jenkins jobs triggering for committers (#12407)",
+		},
+	}
+)
+
+func CreateCommits(s beam.Scope) beam.PCollection {
+	return beam.CreateList(s, commits)
+}
diff --git a/learning/katas/go/windowing/adding_timestamp/pardo/pkg/task/task.go b/learning/katas/go/windowing/adding_timestamp/pardo/pkg/task/task.go
new file mode 100644
index 0000000..be59311
--- /dev/null
+++ b/learning/katas/go/windowing/adding_timestamp/pardo/pkg/task/task.go
@@ -0,0 +1,35 @@
+// 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 task
+
+import (
+	"github.com/apache/beam/sdks/go/pkg/beam"
+	"github.com/apache/beam/sdks/go/pkg/beam/core/graph/mtime"
+	"time"
+)
+
+// Commit represents data about a git commit message.
+type Commit struct {
+	Datetime time.Time
+	Message string
+}
+
+// ApplyTransform applies a beam.EventTime timestamp to PCollection<Commit> elements.
+func ApplyTransform(s beam.Scope, input beam.PCollection) beam.PCollection {
+	return beam.ParDo(s, func(element Commit) (beam.EventTime, Commit) {
+		return mtime.FromTime(element.Datetime), element
+	}, input)
+}
diff --git a/learning/katas/go/windowing/adding_timestamp/pardo/task-info.yaml b/learning/katas/go/windowing/adding_timestamp/pardo/task-info.yaml
new file mode 100644
index 0000000..1d82b8e
--- /dev/null
+++ b/learning/katas/go/windowing/adding_timestamp/pardo/task-info.yaml
@@ -0,0 +1,33 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#  http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+
+type: edu
+files:
+- name: test/task_test.go
+  visible: false
+- name: cmd/main.go
+  visible: true
+- name: pkg/task/task.go
+  visible: true
+  placeholders:
+  - offset: 1217
+    length: 123
+    placeholder_text: TODO()
+- name: pkg/common/data.go
+  visible: true
diff --git a/learning/katas/go/windowing/adding_timestamp/pardo/task-remote-info.yaml b/learning/katas/go/windowing/adding_timestamp/pardo/task-remote-info.yaml
new file mode 100644
index 0000000..922d44a
--- /dev/null
+++ b/learning/katas/go/windowing/adding_timestamp/pardo/task-remote-info.yaml
@@ -0,0 +1,2 @@
+id: 1641937
+update_date: Wed, 21 Oct 2020 17:54:44 UTC
diff --git a/learning/katas/go/windowing/adding_timestamp/pardo/task.md b/learning/katas/go/windowing/adding_timestamp/pardo/task.md
new file mode 100644
index 0000000..01ffce1
--- /dev/null
+++ b/learning/katas/go/windowing/adding_timestamp/pardo/task.md
@@ -0,0 +1,43 @@
+<!--
+  ~ 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.
+  -->
+
+# Assigning Timestamps
+
+A DoFn can assign timestamps to elements of a PCollection.  We will see the use these timestamps in the next lesson
+on windowing.  A simple dataset in this lesson has five git commit messages and their timestamps from the
+[Apache Beam public repository](https://github.com/apache/beam).
+
+**Kata:** Assign a timestamp to PCollection&lt;Commit&gt; elements based on the datetime of the commit message.
+
+<div class="hint">
+    Use <a href="https://godoc.org/github.com/apache/beam/sdks/go/pkg/beam#ParDo">
+    beam.ParDo</a>
+    with a <a href="https://godoc.org/github.com/apache/beam/sdks/go/pkg/beam#hdr-DoFns">
+    DoFn</a> to accomplish this lesson.
+</div>
+
+<div class="hint">
+    Use <a href="https://godoc.org/github.com/apache/beam/sdks/go/pkg/beam/core/graph/mtime#FromTime">
+    mtime.FromTime</a> to assign a timestamp to the PCollection element.
+</div>
+
+<div class="hint">
+  Refer to the Beam Programming Guide
+  <a href="https://beam.apache.org/documentation/programming-guide/#adding-timestamps-to-a-pcollections-elements">
+    "Adding timestamps to a PCollection’s elements"</a> section for more information.
+</div>
diff --git a/learning/katas/go/windowing/adding_timestamp/pardo/test/task_test.go b/learning/katas/go/windowing/adding_timestamp/pardo/test/task_test.go
new file mode 100644
index 0000000..bfda2c4
--- /dev/null
+++ b/learning/katas/go/windowing/adding_timestamp/pardo/test/task_test.go
@@ -0,0 +1,48 @@
+// 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 test
+
+import (
+	"beam.apache.org/learning/katas/windowing/adding_timestamp/pardo/pkg/common"
+	"beam.apache.org/learning/katas/windowing/adding_timestamp/pardo/pkg/task"
+	"github.com/apache/beam/sdks/go/pkg/beam"
+	"github.com/apache/beam/sdks/go/pkg/beam/core/graph/mtime"
+	"github.com/apache/beam/sdks/go/pkg/beam/testing/ptest"
+	"testing"
+)
+
+func TestApplyTransform(t *testing.T) {
+	p, s := beam.NewPipelineWithRoot()
+	tests := []struct {
+		input beam.PCollection
+	}{
+		{
+			input: common.CreateCommits(s),
+		},
+	}
+	for _, tt := range tests {
+		got := task.ApplyTransform(s, tt.input)
+		beam.ParDo0(s, func(et beam.EventTime, commit task.Commit) {
+			// assert whether student assigned the correct timestamp
+			if et.Milliseconds()*1e6 != commit.Datetime.UnixNano() {
+				t.Errorf("ApplyTransform() = %v , want %v", et, mtime.FromTime(commit.Datetime))
+			}
+		}, got)
+		if err := ptest.Run(p); err != nil {
+			t.Error(err)
+		}
+	}
+}
diff --git a/learning/katas/go/windowing/fixed_time_window/lesson-remote-info.yaml b/learning/katas/go/windowing/fixed_time_window/lesson-remote-info.yaml
index b967c97..935f5d3 100644
--- a/learning/katas/go/windowing/fixed_time_window/lesson-remote-info.yaml
+++ b/learning/katas/go/windowing/fixed_time_window/lesson-remote-info.yaml
@@ -1,3 +1,3 @@
 id: 414186
-update_date: Sat, 03 Oct 2020 19:31:08 UTC
+update_date: Wed, 21 Oct 2020 17:54:47 UTC
 unit: 403690
diff --git a/learning/katas/go/windowing/section-info.yaml b/learning/katas/go/windowing/section-info.yaml
index 1bfd258..b73a58d 100644
--- a/learning/katas/go/windowing/section-info.yaml
+++ b/learning/katas/go/windowing/section-info.yaml
@@ -18,4 +18,5 @@
 #
 
 content:
+- adding_timestamp
 - fixed_time_window
diff --git a/model/fn-execution/src/main/proto/beam_fn_api.proto b/model/fn-execution/src/main/proto/beam_fn_api.proto
index 439b534..eaf5dc3 100644
--- a/model/fn-execution/src/main/proto/beam_fn_api.proto
+++ b/model/fn-execution/src/main/proto/beam_fn_api.proto
@@ -34,7 +34,7 @@
 
 package org.apache.beam.model.fn_execution.v1;
 
-option go_package = "fnexecution_v1";
+option go_package = "github.com/apache/beam/sdks/go/pkg/beam/model/fnexecution_v1;fnexecution_v1";
 option java_package = "org.apache.beam.model.fnexecution.v1";
 option java_outer_classname = "BeamFnApi";
 
@@ -43,7 +43,6 @@
 import "google/protobuf/descriptor.proto";
 import "google/protobuf/timestamp.proto";
 import "google/protobuf/duration.proto";
-import "google/protobuf/wrappers.proto";
 import "metrics.proto";
 
 // A descriptor for connecting to a remote port using the Beam Fn Data API.
@@ -108,7 +107,8 @@
     ProcessBundleProgressRequest process_bundle_progress = 1002;
     ProcessBundleSplitRequest process_bundle_split = 1003;
     FinalizeBundleRequest finalize_bundle = 1004;
-    ProcessBundleProgressMetadataRequest process_bundle_progress_metadata = 1005;
+    MonitoringInfosMetadataRequest monitoring_infos = 1005;
+    HarnessMonitoringInfosRequest harness_monitoring_infos = 1006;
 
     // DEPRECATED
     RegisterRequest register = 1000;
@@ -135,13 +135,45 @@
     ProcessBundleProgressResponse process_bundle_progress = 1002;
     ProcessBundleSplitResponse process_bundle_split = 1003;
     FinalizeBundleResponse finalize_bundle = 1004;
-    ProcessBundleProgressMetadataResponse process_bundle_progress_metadata = 1005;
+    MonitoringInfosMetadataResponse monitoring_infos = 1005;
+    HarnessMonitoringInfosResponse harness_monitoring_infos = 1006;
 
     // DEPRECATED
     RegisterResponse register = 1000;
   }
 }
 
+// A request to provide full MonitoringInfo associated with the entire SDK
+// harness process, not specific to a bundle.
+//
+// An SDK can report metrics using an identifier that only contains the
+// associated payload. A runner who wants to receive the full metrics
+// information can request all the monitoring metadata via a
+// MonitoringInfosMetadataRequest providing a list of ids as necessary.
+//
+// The SDK is allowed to reuse the identifiers
+// for the lifetime of the associated control connection as long
+// as the MonitoringInfo could be reconstructed fully by overwriting its
+// payload field with the bytes specified here.
+message HarnessMonitoringInfosRequest {
+}
+
+message HarnessMonitoringInfosResponse {
+  // An identifier to MonitoringInfo.payload mapping containing
+  // Metrics associated with the SDK harness, not a specific bundle.
+  //
+  // An SDK can report metrics using an identifier that only contains the
+  // associated payload. A runner who wants to receive the full metrics
+  // information can request all the monitoring metadata via a
+  // MonitoringInfosMetadataRequest providing a list of ids as necessary.
+  //
+  // The SDK is allowed to reuse the identifiers
+  // for the lifetime of the associated control connection as long
+  // as the MonitoringInfo could be reconstructed fully by overwriting its
+  // payload field with the bytes specified here.
+  map<string, bytes> monitoring_data = 1;
+}
+
 // A list of objects which can be referred to by the runner in
 // future requests.
 // Stable
@@ -288,7 +320,7 @@
   // An SDK can report metrics using an identifier that only contains the
   // associated payload. A runner who wants to receive the full metrics
   // information can request all the monitoring metadata via a
-  // ProcessBundleProgressMetadataRequest providing a list of ids as necessary.
+  // MonitoringInfosMetadataRequest providing a list of ids as necessary.
   //
   // The SDK is allowed to reuse the identifiers across multiple bundles as long
   // as the MonitoringInfo could be reconstructed fully by overwriting its
@@ -307,17 +339,18 @@
   string instruction_id = 1;
 }
 
-// A request to provide full MonitoringInfo for a given bundle.
+// A request to provide full MonitoringInfo for a set of provided ids.
 //
 // An SDK can report metrics using an identifier that only contains the
 // associated payload. A runner who wants to receive the full metrics
 // information can request all the monitoring metadata via a
-// ProcessBundleProgressMetadataRequest providing a list of ids as necessary.
+// MonitoringInfosMetadataRequest providing a list of ids as necessary.
 //
-// The SDK is allowed to reuse the identifiers across multiple bundles as long
-// as the MonitoringInfo could be reconstructed fully by overwriting its
-// payload field with the bytes specified here.
-message ProcessBundleProgressMetadataRequest {
+// The SDK is allowed to reuse the identifiers for the lifetime of the
+// associated control connection as long as the MonitoringInfo could be
+// reconstructed fully by overwriting its payload field with the bytes specified
+// here.
+message MonitoringInfosMetadataRequest {
   // A list of ids for which the full MonitoringInfo is requested for.
   repeated string monitoring_info_id = 1;
 }
@@ -332,9 +365,10 @@
   // An SDK can report metrics using an identifier that only contains the
   // associated payload. A runner who wants to receive the full metrics
   // information can request all the monitoring metadata via a
-  // ProcessBundleProgressMetadataRequest providing a list of ids as necessary.
+  // MonitoringInfosMetadataRequest providing a list of ids as necessary.
   //
-  // The SDK is allowed to reuse the identifiers across multiple bundles as long
+  // The SDK is allowed to reuse the identifiers
+  // for the lifetime of the associated control connection as long
   // as the MonitoringInfo could be reconstructed fully by overwriting its
   // payload field with the bytes specified here.
   map<string, bytes> monitoring_data = 5;
@@ -348,12 +382,13 @@
 // An SDK can report metrics using an identifier that only contains the
 // associated payload. A runner who wants to receive the full metrics
 // information can request all the monitoring metadata via a
-// ProcessBundleProgressMetadataRequest providing a list of ids as necessary.
+// MonitoringInfosMetadataRequest providing a list of ids as necessary.
 //
-// The SDK is allowed to reuse the identifiers across multiple bundles as long
+// The SDK is allowed to reuse the identifiers
+// for the lifetime of the associated control connection as long
 // as the MonitoringInfo could be reconstructed fully by overwriting its
 // payload field with the bytes specified here.
-message ProcessBundleProgressMetadataResponse {
+message MonitoringInfosMetadataResponse {
   // A mapping from an identifier to the full metrics information.
   map<string, org.apache.beam.model.pipeline.v1.MonitoringInfo> monitoring_info = 1;
 }
@@ -375,9 +410,6 @@
 
     // A set of allowed element indices where the SDK may split. When this is
     // empty, there are no constraints on where to split.
-    // Specifically, the first_residual_element of a split result must be an
-    // allowed split point, and the last_primary_element must immediately
-    // preceded an allowed split point.
     repeated int64 allowed_split_points = 3;
 
     // (Required for GrpcRead operations) Number of total elements expected
diff --git a/model/fn-execution/src/main/proto/beam_provision_api.proto b/model/fn-execution/src/main/proto/beam_provision_api.proto
index 7deeadc..cef769a 100644
--- a/model/fn-execution/src/main/proto/beam_provision_api.proto
+++ b/model/fn-execution/src/main/proto/beam_provision_api.proto
@@ -25,7 +25,7 @@
 
 package org.apache.beam.model.fn_execution.v1;
 
-option go_package = "fnexecution_v1";
+option go_package = "github.com/apache/beam/sdks/go/pkg/beam/model/fnexecution_v1;fnexecution_v1";
 option java_package = "org.apache.beam.model.fnexecution.v1";
 option java_outer_classname = "ProvisionApi";
 
diff --git a/model/fn-execution/src/main/resources/org/apache/beam/model/fnexecution/v1/standard_coders.yaml b/model/fn-execution/src/main/resources/org/apache/beam/model/fnexecution/v1/standard_coders.yaml
index 528ad73..db7021d 100644
--- a/model/fn-execution/src/main/resources/org/apache/beam/model/fnexecution/v1/standard_coders.yaml
+++ b/model/fn-execution/src/main/resources/org/apache/beam/model/fnexecution/v1/standard_coders.yaml
@@ -417,3 +417,26 @@
 examples:
     "\x03\x00\x02\x00\xb6\x95\xd5\xf9\x05\xc0\xc4\x07\x1b2020-08-13T14:14:14.123456Z\xc0\xf7\x85\xda\xae\x98\xeb\x02": {f_timestamp: {seconds: 1597328054, micros: 123456}, f_string: "2020-08-13T14:14:14.123456Z", f_int: 1597328054123456}
 
+---
+
+coder:
+  urn: "beam:coder:sharded_key:v1"
+  components: [{urn: "beam:coder:string_utf8:v1"}]
+
+examples:
+  "\u0000\u0000": {
+    shardId: "",
+    key: ""
+  }
+  "\u0008\u0073\u0068\u0061\u0072\u0064\u005f\u0069\u0064\u0000": {
+    shardId: shard_id,
+    key: ""
+  }
+  "\u0008\u0073\u0068\u0061\u0072\u0064\u005f\u0069\u0064\u0003\u006b\u0065\u0079": {
+    shardId: shard_id,
+    key: "key"
+  }
+  "\u0000\u0003\u006b\u0065\u0079": {
+    shardId: "",
+    key: "key"
+  }
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 dcdf151..dc27fc6 100644
--- a/model/job-management/src/main/proto/beam_artifact_api.proto
+++ b/model/job-management/src/main/proto/beam_artifact_api.proto
@@ -25,7 +25,7 @@
 
 package org.apache.beam.model.job_management.v1;
 
-option go_package = "jobmanagement_v1";
+option go_package = "github.com/apache/beam/sdks/go/pkg/beam/model/jobmanagement_v1;jobmanagement_v1";
 option java_package = "org.apache.beam.model.jobmanagement.v1";
 option java_outer_classname = "ArtifactApi";
 
diff --git a/model/job-management/src/main/proto/beam_expansion_api.proto b/model/job-management/src/main/proto/beam_expansion_api.proto
index e358c56..a888741 100644
--- a/model/job-management/src/main/proto/beam_expansion_api.proto
+++ b/model/job-management/src/main/proto/beam_expansion_api.proto
@@ -25,7 +25,7 @@
 
 package org.apache.beam.model.expansion.v1;
 
-option go_package = "jobmanagement_v1";
+option go_package = "github.com/apache/beam/sdks/go/pkg/beam/model/jobmanagement_v1;jobmanagement_v1";
 option java_package = "org.apache.beam.model.expansion.v1";
 option java_outer_classname = "ExpansionApi";
 
diff --git a/model/job-management/src/main/proto/beam_job_api.proto b/model/job-management/src/main/proto/beam_job_api.proto
index 6a72f5d..0a05738 100644
--- a/model/job-management/src/main/proto/beam_job_api.proto
+++ b/model/job-management/src/main/proto/beam_job_api.proto
@@ -25,7 +25,7 @@
 
 package org.apache.beam.model.job_management.v1;
 
-option go_package = "jobmanagement_v1";
+option go_package = "github.com/apache/beam/sdks/go/pkg/beam/model/jobmanagement_v1;jobmanagement_v1";
 option java_package = "org.apache.beam.model.jobmanagement.v1";
 option java_outer_classname = "JobApi";
 
diff --git a/model/pipeline/src/main/proto/beam_runner_api.proto b/model/pipeline/src/main/proto/beam_runner_api.proto
index 6009154..3a43ef6 100644
--- a/model/pipeline/src/main/proto/beam_runner_api.proto
+++ b/model/pipeline/src/main/proto/beam_runner_api.proto
@@ -25,14 +25,12 @@
 
 package org.apache.beam.model.pipeline.v1;
 
-option go_package = "pipeline_v1";
+option go_package = "github.com/apache/beam/sdks/go/pkg/beam/model/pipeline_v1;pipeline_v1";
 option java_package = "org.apache.beam.model.pipeline.v1";
 option java_outer_classname = "RunnerApi";
 
 import "endpoints.proto";
-import "google/protobuf/any.proto";
 import "google/protobuf/descriptor.proto";
-import "google/protobuf/timestamp.proto";
 
 message BeamConstants {
   enum Constants {
@@ -931,6 +929,28 @@
     // Components: None
     // Experimental.
     ROW = 13 [(beam_urn) = "beam:coder:row:v1"];
+
+    // Encodes a user key and a shard id which is an opaque byte string.
+    //
+    // The encoding for a sharded key consists of a shard id byte string and the
+    // encoded user key in the following order:
+    //
+    //     - shard id using beam:coder:bytes:v1
+    //     - encoded user key
+    //
+    // Examples:
+    // user key with an empty shard id
+    //     0x00
+    //     encode(user_key)
+    //
+    // user key with a shard id taking up two bytes.
+    //     0x02
+    //     0x11 0x22
+    //     encode(user_key)
+    //
+    // Components: the user key coder.
+    // Experimental.
+    SHARDED_KEY = 15 [(beam_urn) = "beam:coder:sharded_key:v1"];
   }
 }
 
diff --git a/model/pipeline/src/main/proto/endpoints.proto b/model/pipeline/src/main/proto/endpoints.proto
index 063a48f..e757a82 100644
--- a/model/pipeline/src/main/proto/endpoints.proto
+++ b/model/pipeline/src/main/proto/endpoints.proto
@@ -24,7 +24,7 @@
 
 package org.apache.beam.model.pipeline.v1;
 
-option go_package = "pipeline_v1";
+option go_package = "github.com/apache/beam/sdks/go/pkg/beam/model/pipeline_v1;pipeline_v1";
 option java_package = "org.apache.beam.model.pipeline.v1";
 option java_outer_classname = "Endpoints";
 
diff --git a/model/pipeline/src/main/proto/external_transforms.proto b/model/pipeline/src/main/proto/external_transforms.proto
index 2e5e166..54a5548 100644
--- a/model/pipeline/src/main/proto/external_transforms.proto
+++ b/model/pipeline/src/main/proto/external_transforms.proto
@@ -24,11 +24,10 @@
 
 package org.apache.beam.model.pipeline.v1;
 
-option go_package = "pipeline_v1";
+option go_package = "github.com/apache/beam/sdks/go/pkg/beam/model/pipeline_v1;pipeline_v1";
 option java_package = "org.apache.beam.model.pipeline.v1";
 option java_outer_classname = "ExternalTransforms";
 
-import "beam_runner_api.proto";
 import "schema.proto";
 
 // A configuration payload for an external transform.
diff --git a/model/pipeline/src/main/proto/metrics.proto b/model/pipeline/src/main/proto/metrics.proto
index fe59266..86114a8 100644
--- a/model/pipeline/src/main/proto/metrics.proto
+++ b/model/pipeline/src/main/proto/metrics.proto
@@ -24,14 +24,13 @@
 
 package org.apache.beam.model.pipeline.v1;
 
-option go_package = "pipeline_v1";
+option go_package = "github.com/apache/beam/sdks/go/pkg/beam/model/pipeline_v1;pipeline_v1";
 option java_package = "org.apache.beam.model.pipeline.v1";
 option java_outer_classname = "MetricsApi";
 
 
 import "beam_runner_api.proto";
 import "google/protobuf/descriptor.proto";
-import "google/protobuf/timestamp.proto";
 
 // A specification for describing a well known MonitoringInfo.
 //
@@ -307,6 +306,29 @@
         value: "The read index of the data channel."
       }]
     }];
+
+    API_REQUEST_COUNT = 19 [(monitoring_info_spec) = {
+      urn: "beam:metric:io:api_request_count:v1",
+      type: "beam:metrics:sum_int64:v1",
+      required_labels: [
+        "SERVICE",
+        "METHOD",
+        "RESOURCE",
+        "PTRANSFORM",
+        "STATUS"
+      ],
+      annotations: [
+        {
+          key: "description",
+          value: "Request counts with status made to an IOs service APIs to batch read or write elements."
+        },
+        {
+          key: "process_metric",  // Should be reported as a process metric
+                                  // instead of a bundle metric
+          value: "true"
+        }
+      ]
+    }];
   }
 }
 
@@ -355,6 +377,15 @@
     ENVIRONMENT = 4 [(label_props) = { name: "ENVIRONMENT" }];
     NAMESPACE = 5 [(label_props) = { name: "NAMESPACE" }];
     NAME = 6 [(label_props) = { name: "NAME" }];
+    SERVICE = 7 [(label_props) = { name: "SERVICE" }];
+    METHOD = 8 [(label_props) = { name: "METHOD" }];
+    RESOURCE = 9 [(label_props) = { name: "RESOURCE" }];
+    STATUS = 10 [(label_props) = { name: "STATUS" }];
+    BIGQUERY_PROJECT_ID = 11 [(label_props) = { name: "BIGQUERY_PROJECT_ID" }];
+    BIGQUERY_DATASET = 12 [(label_props) = { name: "BIGQUERY_DATASET" }];
+    BIGQUERY_TABLE = 13 [(label_props) = { name: "BIGQUERY_TABLE" }];
+    BIGQUERY_VIEW = 14 [(label_props) = { name: "BIGQUERY_VIEW" }];
+    BIGQUERY_QUERY_NAME = 15 [(label_props) = { name: "BIGQUERY_QUERY_NAME" }];
   }
 
   // A set of key and value labels which define the scope of the metric. For
diff --git a/model/pipeline/src/main/proto/schema.proto b/model/pipeline/src/main/proto/schema.proto
index 60e1c38..a40087c 100644
--- a/model/pipeline/src/main/proto/schema.proto
+++ b/model/pipeline/src/main/proto/schema.proto
@@ -27,7 +27,7 @@
 
 package org.apache.beam.model.pipeline.v1;
 
-option go_package = "pipeline_v1";
+option go_package = "github.com/apache/beam/sdks/go/pkg/beam/model/pipeline_v1;pipeline_v1";
 option java_package = "org.apache.beam.model.pipeline.v1";
 option java_outer_classname = "SchemaApi";
 
diff --git a/model/pipeline/src/main/proto/standard_window_fns.proto b/model/pipeline/src/main/proto/standard_window_fns.proto
index da4d53b..adb357d 100644
--- a/model/pipeline/src/main/proto/standard_window_fns.proto
+++ b/model/pipeline/src/main/proto/standard_window_fns.proto
@@ -25,7 +25,7 @@
 
 package org.apache.beam.model.pipeline.v1;
 
-option go_package = "pipeline_v1";
+option go_package = "github.com/apache/beam/sdks/go/pkg/beam/model/pipeline_v1;pipeline_v1";
 option java_package = "org.apache.beam.model.pipeline.v1";
 option java_outer_classname = "StandardWindowFns";
 
diff --git a/release/src/main/scripts/build_release_candidate.sh b/release/src/main/scripts/build_release_candidate.sh
index 7a16104..074b7ff 100755
--- a/release/src/main/scripts/build_release_candidate.sh
+++ b/release/src/main/scripts/build_release_candidate.sh
@@ -25,6 +25,17 @@
 
 set -e
 
+if [[ "$JAVA_HOME" ]]; then
+  version=$("$JAVA_HOME/bin/java" -version 2>&1 | awk -F '"' '/version/ {print $2}')
+  if [[ ! `echo $version | sed "s/1\.8\..*/1.8/"` == "1.8" ]]; then
+    echo "Java version $version detected. Set \$JAVA_HOME to point to a JDK 8 installation."
+    exit 1
+  fi
+else
+  echo "\$JAVA_HOME must be set."
+  exit 1
+fi
+
 LOCAL_CLONE_DIR=build_release_candidate
 LOCAL_JAVA_STAGING_DIR=java_staging_dir
 LOCAL_PYTHON_STAGING_DIR=python_staging_dir
diff --git a/release/src/main/scripts/preparation_before_release.sh b/release/src/main/scripts/preparation_before_release.sh
index 9d77d3a..88e9ac6 100755
--- a/release/src/main/scripts/preparation_before_release.sh
+++ b/release/src/main/scripts/preparation_before_release.sh
@@ -34,7 +34,7 @@
 read confirmation
 if [[ $confirmation = "y" ]]; then
   echo "===============Generating new GPG key================"
-  sudo apt-get install rng-tools
+  sudo apt-get install rng-tools # Get more entropy for creating a GPG key.
   sudo rngd -r /dev/urandom
   echo "NOTE: When creating the key, please select the type to be RSA and RSA (default), and the size to be 4096 bit long."
   gpg --full-generate-key
@@ -54,32 +54,32 @@
 echo "Have you put your key in KEYS? [y|N]"
 read confirmation
 if [[ $confirmation != "y" ]]; then
-  echo "Only PMC member can write into dist.apache.org. Are you a PMC member? [y|N]"
+  echo "Please input your name: "
+  read name
+  echo "======Starting updating KEYS file in dev repo===="
+  if [[ -d ${LOCAL_SVN_DIR} ]]; then
+    rm -rf ${LOCAL_SVN_DIR}
+  fi
+  mkdir ${LOCAL_SVN_DIR}
+  cd ${LOCAL_SVN_DIR}
+  svn co ${ROOT_SVN_URL}/${DEV_REPO}/${BEAM_REPO}
+  cd ${BEAM_REPO}
+  (gpg --list-sigs ${name} && gpg --armor --export ${name}) >> KEYS
+  svn status
+  echo "Please review all changes. Do you confirm to commit? [y|N]"
+  read commit_confirmation
+  if [[ $commit_confirmation = "y" ]]; then
+    svn commit --no-auth-cache KEYS
+  else
+    echo "Not commit new changes into ${ROOT_SVN_URL}${DEV_REPO}/${BEAM_REPO}/KEYS"
+  fi
+
+  echo "Only a PMC member can write into dist.apache.org's release KEYS. Are you a PMC member? [y|N]"
   read pmc_permission
   if [[ $pmc_permission != "y" ]]; then
     echo "Please ask a PMC member to help you add your key in dev@ list."
-    echo "Skip adding key into dist.apache.org/KEYS file."
+    echo "Skip adding key into ${ROOT_SVN_URL}${RELEASE_REPO}/${BEAM_REPO}/KEYS"
   else
-    echo "Please input your name: "
-    read name
-    echo "======Starting updating KEYS file in dev repo===="
-    if [[ -d ${LOCAL_SVN_DIR} ]]; then
-      rm -rf ${LOCAL_SVN_DIR}
-    fi
-    mkdir ${LOCAL_SVN_DIR}
-    cd ${LOCAL_SVN_DIR}
-    svn co ${ROOT_SVN_URL}/${DEV_REPO}/${BEAM_REPO}
-    cd ${BEAM_REPO}
-    (gpg --list-sigs ${name} && gpg --armor --export ${name}) >> KEYS
-    svn status
-    echo "Please review all changes. Do you confirm to commit? [y|N]"
-    read commit_confirmation
-    if [[ $commit_confirmation = "y" ]]; then
-      svn commit --no-auth-cache KEYS
-    else
-      echo "Not commit new changes into ${ROOT_SVN_URL}/${DEV_REPO}/${BEAM_REPO}${DEV_REPO}/KEYS"
-    fi
-
     cd ~/${LOCAL_SVN_DIR}
     echo "===Starting updating KEYS file in release repo==="
     svn co ${ROOT_SVN_URL}/${RELEASE_REPO}/${BEAM_REPO}
@@ -93,10 +93,10 @@
     else
       echo "Not commit new changes into ${ROOT_SVN_URL}/${DEV_REPO}/${BEAM_REPO}${RELEASE_REPO}/KEYS"
     fi
-
-    cd ~
-    rm -rf ${LOCAL_SVN_DIR}
   fi
+
+  cd ~
+  rm -rf ${LOCAL_SVN_DIR}
 fi
 
 echo "================Setting up gpg agent================="
diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CoderTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CoderTranslation.java
index 6f6c59b..d77dc03 100644
--- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CoderTranslation.java
+++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CoderTranslation.java
@@ -35,6 +35,7 @@
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
 
 /** Converts to and from Beam Runner API representations of {@link Coder Coders}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class CoderTranslation {
 
   /**
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 c4f4502..574b26e 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
@@ -32,6 +32,7 @@
 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.ShardedKey;
 import org.apache.beam.sdk.util.WindowedValue;
 import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder;
 import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.InvalidProtocolBufferException;
@@ -174,6 +175,20 @@
     };
   }
 
+  static CoderTranslator<ShardedKey.Coder<?>> shardedKey() {
+    return new SimpleStructuredCoderTranslator<ShardedKey.Coder<?>>() {
+      @Override
+      public List<? extends Coder<?>> getComponents(ShardedKey.Coder<?> from) {
+        return Collections.singletonList(from.getKeyCoder());
+      }
+
+      @Override
+      public ShardedKey.Coder<?> fromComponents(List<Coder<?>> components) {
+        return ShardedKey.Coder.of(components.get(0));
+      }
+    };
+  }
+
   public abstract static class SimpleStructuredCoderTranslator<T extends Coder<?>>
       implements CoderTranslator<T> {
     @Override
diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/DeduplicatedFlattenFactory.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/DeduplicatedFlattenFactory.java
index 20f16c2..de25ef7 100644
--- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/DeduplicatedFlattenFactory.java
+++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/DeduplicatedFlattenFactory.java
@@ -60,7 +60,7 @@
    *
    * <p>The input {@link PCollectionList} that is constructed will have the same values in the same
    */
-  private PCollectionList<T> getInput(Map<TupleTag<?>, PValue> inputs, Pipeline p) {
+  private PCollectionList<T> getInput(Map<TupleTag<?>, PCollection<?>> inputs, Pipeline p) {
     PCollectionList<T> pCollections = PCollectionList.empty(p);
     for (PValue input : inputs.values()) {
       PCollection<T> pcollection = (PCollection<T>) input;
@@ -70,8 +70,8 @@
   }
 
   @Override
-  public Map<PValue, ReplacementOutput> mapOutputs(
-      Map<TupleTag<?>, PValue> outputs, PCollection<T> newOutput) {
+  public Map<PCollection<?>, ReplacementOutput> mapOutputs(
+      Map<TupleTag<?>, PCollection<?>> outputs, PCollection<T> newOutput) {
     return ReplacementOutputs.singleton(outputs, newOutput);
   }
 
diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/DefaultArtifactResolver.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/DefaultArtifactResolver.java
index 6406e57..2a83694 100644
--- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/DefaultArtifactResolver.java
+++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/DefaultArtifactResolver.java
@@ -34,6 +34,7 @@
  * they registered i.e. the function registered later overrides the earlier one if they resolve the
  * same artifact.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DefaultArtifactResolver implements ArtifactResolver {
   public static final ArtifactResolver INSTANCE = new DefaultArtifactResolver();
 
diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/DisplayDataTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/DisplayDataTranslation.java
index 99bc1a8..8be3c26 100644
--- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/DisplayDataTranslation.java
+++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/DisplayDataTranslation.java
@@ -30,6 +30,7 @@
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
 
 /** Utilities for going to/from DisplayData protos. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DisplayDataTranslation {
   public static final String LABELLED_STRING = "beam:display_data:labelled_string:v1";
 
diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/EmptyFlattenAsCreateFactory.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/EmptyFlattenAsCreateFactory.java
index 82d0ace..5462909 100644
--- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/EmptyFlattenAsCreateFactory.java
+++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/EmptyFlattenAsCreateFactory.java
@@ -29,7 +29,6 @@
 import org.apache.beam.sdk.transforms.PTransform;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PCollectionList;
-import org.apache.beam.sdk.values.PValue;
 import org.apache.beam.sdk.values.TupleTag;
 
 /**
@@ -61,8 +60,8 @@
   }
 
   @Override
-  public Map<PValue, ReplacementOutput> mapOutputs(
-      Map<TupleTag<?>, PValue> outputs, PCollection<T> newOutput) {
+  public Map<PCollection<?>, ReplacementOutput> mapOutputs(
+      Map<TupleTag<?>, PCollection<?>> outputs, PCollection<T> newOutput) {
     return ReplacementOutputs.singleton(outputs, newOutput);
   }
 
diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/Environments.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/Environments.java
index 1e4b897..9f45c7b 100644
--- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/Environments.java
+++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/Environments.java
@@ -60,6 +60,7 @@
 import org.slf4j.LoggerFactory;
 
 /** Utilities for interacting with portability {@link Environment environments}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class Environments {
   private static final Logger LOG = LoggerFactory.getLogger(Environments.class);
 
diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/External.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/External.java
index e8b181b..46c7277 100644
--- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/External.java
+++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/External.java
@@ -48,6 +48,7 @@
 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.PValues;
 import org.apache.beam.sdk.values.TupleTag;
 import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
 import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.ManagedChannel;
@@ -67,6 +68,7 @@
  * high-level wrapper classes rather than this one.
  */
 @Experimental(Kind.PORTABILITY)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class External {
   private static final String EXPANDED_TRANSFORM_BASE_NAME = "external";
   private static final String IMPULSE_PREFIX = "IMPULSE";
@@ -186,8 +188,8 @@
             AppliedPTransform<?, ?, ?> fakeImpulse =
                 AppliedPTransform.of(
                     String.format("%s_%s", IMPULSE_PREFIX, entry.getKey().getId()),
-                    PBegin.in(p).expand(),
-                    ImmutableMap.of(entry.getKey(), entry.getValue()),
+                    PValues.expandInput(PBegin.in(p)),
+                    ImmutableMap.of(entry.getKey(), (PCollection<?>) entry.getValue()),
                     Impulse.create(),
                     p);
             // using fake Impulses to provide inputs
diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ExternalTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ExternalTranslation.java
index 2884fa7..36d6515 100644
--- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ExternalTranslation.java
+++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ExternalTranslation.java
@@ -42,9 +42,8 @@
       return new ExternalTranslator();
     }
 
-    @Nullable
     @Override
-    public String getUrn(External.ExpandableTransform transform) {
+    public @Nullable String getUrn(External.ExpandableTransform transform) {
       return EXTERNAL_TRANSFORM_URN;
     }
 
diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ForwardingPTransform.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ForwardingPTransform.java
index 1bf6f95..d804041 100644
--- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ForwardingPTransform.java
+++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ForwardingPTransform.java
@@ -33,6 +33,7 @@
  * delegate transform but with overridden methods. Implementors are required to implement {@link
  * #delegate()}, which returns the object to forward calls to, and {@link #expand(PInput)}.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public abstract class ForwardingPTransform<InputT extends PInput, OutputT extends POutput>
     extends PTransform<InputT, OutputT> {
   protected abstract PTransform<InputT, OutputT> delegate();
diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ModelCoderRegistrar.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ModelCoderRegistrar.java
index 501a823..beeca85 100644
--- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ModelCoderRegistrar.java
+++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ModelCoderRegistrar.java
@@ -35,6 +35,7 @@
 import org.apache.beam.sdk.coders.VarLongCoder;
 import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
 import org.apache.beam.sdk.transforms.windowing.IntervalWindow.IntervalWindowCoder;
+import org.apache.beam.sdk.util.ShardedKey;
 import org.apache.beam.sdk.util.WindowedValue;
 import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
@@ -45,6 +46,7 @@
 
 /** The {@link CoderTranslatorRegistrar} for coders which are shared across languages. */
 @AutoService(CoderTranslatorRegistrar.class)
+@SuppressWarnings({"nullness", "keyfor"}) // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ModelCoderRegistrar implements CoderTranslatorRegistrar {
 
   // The URNs for coders which are shared across languages
@@ -67,6 +69,7 @@
               ModelCoders.PARAM_WINDOWED_VALUE_CODER_URN)
           .put(DoubleCoder.class, ModelCoders.DOUBLE_CODER_URN)
           .put(RowCoder.class, ModelCoders.ROW_CODER_URN)
+          .put(ShardedKey.Coder.class, ModelCoders.SHARDED_KEY_CODER_URN)
           .build();
 
   public static final Set<String> WELL_KNOWN_CODER_URNS = BEAM_MODEL_CODER_URNS.values();
@@ -88,6 +91,7 @@
           .put(WindowedValue.ParamWindowedValueCoder.class, CoderTranslators.paramWindowedValue())
           .put(DoubleCoder.class, CoderTranslators.atomic(DoubleCoder.class))
           .put(RowCoder.class, CoderTranslators.row())
+          .put(ShardedKey.Coder.class, CoderTranslators.shardedKey())
           .build();
 
   static {
diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ModelCoders.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ModelCoders.java
index e1c2047..ca69be9 100644
--- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ModelCoders.java
+++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ModelCoders.java
@@ -63,6 +63,8 @@
   public static final String STATE_BACKED_ITERABLE_CODER_URN =
       "beam:coder:state_backed_iterable:v1";
 
+  public static final String SHARDED_KEY_CODER_URN = getUrn(StandardCoders.Enum.SHARDED_KEY);
+
   static {
     checkState(
         STATE_BACKED_ITERABLE_CODER_URN.equals(getUrn(StandardCoders.Enum.STATE_BACKED_ITERABLE)));
@@ -84,7 +86,8 @@
           DOUBLE_CODER_URN,
           ROW_CODER_URN,
           PARAM_WINDOWED_VALUE_CODER_URN,
-          STATE_BACKED_ITERABLE_CODER_URN);
+          STATE_BACKED_ITERABLE_CODER_URN,
+          SHARDED_KEY_CODER_URN);
 
   public static Set<String> urns() {
     return MODEL_CODER_URNS;
diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PTransformReplacements.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PTransformReplacements.java
index 39b586e..0ecb0f8 100644
--- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PTransformReplacements.java
+++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PTransformReplacements.java
@@ -24,11 +24,11 @@
 import org.apache.beam.sdk.runners.AppliedPTransform;
 import org.apache.beam.sdk.transforms.PTransform;
 import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PValue;
 import org.apache.beam.sdk.values.TupleTag;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables;
 
 /** */
+@SuppressWarnings({"nullness", "keyfor"}) // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class PTransformReplacements {
   /**
    * Gets the singleton input of an {@link AppliedPTransform}, ignoring any additional inputs
@@ -41,9 +41,9 @@
   }
 
   private static <T> PCollection<T> getSingletonMainInput(
-      Map<TupleTag<?>, PValue> inputs, Set<TupleTag<?>> ignoredTags) {
+      Map<TupleTag<?>, PCollection<?>> inputs, Set<TupleTag<?>> ignoredTags) {
     PCollection<T> mainInput = null;
-    for (Map.Entry<TupleTag<?>, PValue> input : inputs.entrySet()) {
+    for (Map.Entry<TupleTag<?>, PCollection<?>> input : inputs.entrySet()) {
       if (!ignoredTags.contains(input.getKey())) {
         checkArgument(
             mainInput == null,
diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PTransformTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PTransformTranslation.java
index 01e4068..4ff4e40 100644
--- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PTransformTranslation.java
+++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PTransformTranslation.java
@@ -18,7 +18,6 @@
 package org.apache.beam.runners.core.construction;
 
 import static org.apache.beam.runners.core.construction.BeamUrns.getUrn;
-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.checkState;
 
 import java.io.IOException;
@@ -46,7 +45,6 @@
 import org.apache.beam.sdk.values.PCollection;
 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.TupleTag;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Joiner;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
@@ -61,6 +59,7 @@
  * Utilities for converting {@link PTransform PTransforms} to {@link RunnerApi Runner API protocol
  * buffers}.
  */
+@SuppressWarnings({"nullness", "keyfor"}) // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class PTransformTranslation {
   // We specifically copy the values here so that they can be used in switch case statements
   // and we validate that the value matches the actual URN in the static block below.
@@ -464,26 +463,15 @@
       SdkComponents components)
       throws IOException {
     RunnerApi.PTransform.Builder transformBuilder = RunnerApi.PTransform.newBuilder();
-    for (Map.Entry<TupleTag<?>, PValue> taggedInput : appliedPTransform.getInputs().entrySet()) {
-      checkArgument(
-          taggedInput.getValue() instanceof PCollection,
-          "Unexpected input type %s",
-          taggedInput.getValue().getClass());
+    for (Map.Entry<TupleTag<?>, PCollection<?>> taggedInput :
+        appliedPTransform.getInputs().entrySet()) {
       transformBuilder.putInputs(
-          toProto(taggedInput.getKey()),
-          components.registerPCollection((PCollection<?>) taggedInput.getValue()));
+          toProto(taggedInput.getKey()), components.registerPCollection(taggedInput.getValue()));
     }
-    for (Map.Entry<TupleTag<?>, PValue> taggedOutput : appliedPTransform.getOutputs().entrySet()) {
-      // TODO: Remove gating
-      if (taggedOutput.getValue() instanceof PCollection) {
-        checkArgument(
-            taggedOutput.getValue() instanceof PCollection,
-            "Unexpected output type %s",
-            taggedOutput.getValue().getClass());
-        transformBuilder.putOutputs(
-            toProto(taggedOutput.getKey()),
-            components.registerPCollection((PCollection<?>) taggedOutput.getValue()));
-      }
+    for (Map.Entry<TupleTag<?>, PCollection<?>> taggedOutput :
+        appliedPTransform.getOutputs().entrySet()) {
+      transformBuilder.putOutputs(
+          toProto(taggedOutput.getKey()), components.registerPCollection(taggedOutput.getValue()));
     }
     for (AppliedPTransform<?, ?, ?> subtransform : subtransforms) {
       transformBuilder.addSubtransforms(components.getExistingPTransformId(subtransform));
diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PrimitiveCreate.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PrimitiveCreate.java
index 3ffee9e..2cd62d5 100644
--- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PrimitiveCreate.java
+++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PrimitiveCreate.java
@@ -27,7 +27,6 @@
 import org.apache.beam.sdk.values.PBegin;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PCollection.IsBounded;
-import org.apache.beam.sdk.values.PValue;
 import org.apache.beam.sdk.values.TupleTag;
 import org.apache.beam.sdk.values.WindowingStrategy;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables;
@@ -67,8 +66,8 @@
     }
 
     @Override
-    public Map<PValue, ReplacementOutput> mapOutputs(
-        Map<TupleTag<?>, PValue> outputs, PCollection<T> newOutput) {
+    public Map<PCollection<?>, ReplacementOutput> mapOutputs(
+        Map<TupleTag<?>, PCollection<?>> outputs, PCollection<T> newOutput) {
       return ReplacementOutputs.singleton(outputs, newOutput);
     }
   }
diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/RehydratedComponents.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/RehydratedComponents.java
index 68ef4e1..7f7fc13 100644
--- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/RehydratedComponents.java
+++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/RehydratedComponents.java
@@ -42,6 +42,7 @@
  * <p>This ensures maximum memoization of rehydrated components, which is semantically necessary for
  * {@link PCollection} and nice-to-have for other objects.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class RehydratedComponents {
   private final Components components;
 
diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ReplacementOutputs.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ReplacementOutputs.java
index 6f48fb7..8fbf837 100644
--- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ReplacementOutputs.java
+++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ReplacementOutputs.java
@@ -21,44 +21,49 @@
 
 import java.util.Collections;
 import java.util.HashMap;
-import java.util.HashSet;
 import java.util.Map;
 import java.util.Map.Entry;
-import java.util.Set;
 import org.apache.beam.sdk.runners.PTransformOverrideFactory.ReplacementOutput;
+import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.POutput;
 import org.apache.beam.sdk.values.PValue;
+import org.apache.beam.sdk.values.PValues;
 import org.apache.beam.sdk.values.TaggedPValue;
 import org.apache.beam.sdk.values.TupleTag;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables;
 
 /** Utility methods for creating {@link ReplacementOutput} for known styles of {@link POutput}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ReplacementOutputs {
   private ReplacementOutputs() {}
 
-  public static Map<PValue, ReplacementOutput> singleton(
-      Map<TupleTag<?>, PValue> original, PValue replacement) {
-    Entry<TupleTag<?>, PValue> originalElement = Iterables.getOnlyElement(original.entrySet());
+  public static Map<PCollection<?>, ReplacementOutput> singleton(
+      Map<TupleTag<?>, PCollection<?>> original, PValue replacement) {
+    Entry<TupleTag<?>, PCollection<?>> originalElement =
+        Iterables.getOnlyElement(original.entrySet());
     TupleTag<?> replacementTag = Iterables.getOnlyElement(replacement.expand().entrySet()).getKey();
+    PCollection<?> replacementCollection =
+        (PCollection<?>) Iterables.getOnlyElement(replacement.expand().entrySet()).getValue();
     return Collections.singletonMap(
-        replacement,
+        replacementCollection,
         ReplacementOutput.of(
             TaggedPValue.of(originalElement.getKey(), originalElement.getValue()),
-            TaggedPValue.of(replacementTag, replacement)));
+            TaggedPValue.of(replacementTag, replacementCollection)));
   }
 
-  public static Map<PValue, ReplacementOutput> tagged(
-      Map<TupleTag<?>, PValue> original, POutput replacement) {
+  public static Map<PCollection<?>, ReplacementOutput> tagged(
+      Map<TupleTag<?>, PCollection<?>> original, POutput replacement) {
     Map<TupleTag<?>, TaggedPValue> originalTags = new HashMap<>();
-    for (Map.Entry<TupleTag<?>, PValue> originalValue : original.entrySet()) {
+    for (Map.Entry<TupleTag<?>, PCollection<?>> originalValue : original.entrySet()) {
       originalTags.put(
           originalValue.getKey(),
           TaggedPValue.of(originalValue.getKey(), originalValue.getValue()));
     }
-    ImmutableMap.Builder<PValue, ReplacementOutput> resultBuilder = ImmutableMap.builder();
-    Set<TupleTag<?>> missingTags = new HashSet<>(originalTags.keySet());
-    for (Map.Entry<TupleTag<?>, PValue> replacementValue : replacement.expand().entrySet()) {
+    ImmutableMap.Builder<PCollection<?>, ReplacementOutput> resultBuilder = ImmutableMap.builder();
+    Map<TupleTag<?>, PCollection<?>> remainingTaggedOriginals = new HashMap<>(original);
+    Map<TupleTag<?>, PCollection<?>> taggedReplacements = PValues.expandOutput(replacement);
+    for (Map.Entry<TupleTag<?>, PCollection<?>> replacementValue : taggedReplacements.entrySet()) {
       TaggedPValue mapped = originalTags.get(replacementValue.getKey());
       checkArgument(
           mapped != null,
@@ -70,15 +75,16 @@
       resultBuilder.put(
           replacementValue.getValue(),
           ReplacementOutput.of(
-              mapped, TaggedPValue.of(replacementValue.getKey(), replacementValue.getValue())));
-      missingTags.remove(replacementValue.getKey());
+              mapped,
+              TaggedPValue.of(
+                  replacementValue.getKey(), (PCollection<?>) replacementValue.getValue())));
+      remainingTaggedOriginals.remove(replacementValue.getKey());
     }
-    ImmutableMap<PValue, ReplacementOutput> result = resultBuilder.build();
     checkArgument(
-        missingTags.isEmpty(),
-        "Missing replacement for tags %s. Encountered tags: %s",
-        missingTags,
-        result.keySet());
-    return result;
+        remainingTaggedOriginals.isEmpty(),
+        "Missing replacement for tagged values %s. Replacement was: %s",
+        remainingTaggedOriginals,
+        taggedReplacements);
+    return resultBuilder.build();
   }
 }
diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/RunnerPCollectionView.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/RunnerPCollectionView.java
index e9e22d4..b54028a 100644
--- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/RunnerPCollectionView.java
+++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/RunnerPCollectionView.java
@@ -33,6 +33,7 @@
 import org.checkerframework.checker.nullness.qual.Nullable;
 
 /** A {@link PCollectionView} created from the components of a {@link SideInput}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class RunnerPCollectionView<T> extends PValueBase implements PCollectionView<T> {
   private final TupleTag<Iterable<WindowedValue<?>>> tag;
   private final ViewFn<Iterable<WindowedValue<?>>, T> viewFn;
diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SdkComponents.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SdkComponents.java
index 395839c..123a95e 100644
--- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SdkComponents.java
+++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SdkComponents.java
@@ -44,6 +44,7 @@
 import org.checkerframework.checker.nullness.qual.Nullable;
 
 /** SDK objects that will be represented at some later point within a {@link Components} object. */
+@SuppressWarnings({"nullness", "keyfor"}) // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SdkComponents {
   private final String newIdPrefix;
   private final RunnerApi.Components.Builder componentsBuilder = RunnerApi.Components.newBuilder();
diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SingleInputOutputOverrideFactory.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SingleInputOutputOverrideFactory.java
index cd67a18..fae483d 100644
--- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SingleInputOutputOverrideFactory.java
+++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SingleInputOutputOverrideFactory.java
@@ -20,6 +20,7 @@
 import java.util.Map;
 import org.apache.beam.sdk.runners.PTransformOverrideFactory;
 import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PValue;
 import org.apache.beam.sdk.values.TupleTag;
 
@@ -33,8 +34,8 @@
         TransformT extends PTransform<InputT, OutputT>>
     implements PTransformOverrideFactory<InputT, OutputT, TransformT> {
   @Override
-  public final Map<PValue, ReplacementOutput> mapOutputs(
-      Map<TupleTag<?>, PValue> outputs, OutputT newOutput) {
+  public final Map<PCollection<?>, ReplacementOutput> mapOutputs(
+      Map<TupleTag<?>, PCollection<?>> outputs, OutputT newOutput) {
     return ReplacementOutputs.singleton(outputs, newOutput);
   }
 }
diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SplittableParDo.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SplittableParDo.java
index a3e8b12..edc3f78 100644
--- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SplittableParDo.java
+++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SplittableParDo.java
@@ -117,8 +117,8 @@
     }
 
     @Override
-    public Map<PValue, ReplacementOutput> mapOutputs(
-        Map<TupleTag<?>, PValue> outputs, PCollectionTuple newOutput) {
+    public Map<PCollection<?>, ReplacementOutput> mapOutputs(
+        Map<TupleTag<?>, PCollection<?>> outputs, PCollectionTuple newOutput) {
       return ReplacementOutputs.tagged(outputs, newOutput);
     }
   }
@@ -164,7 +164,7 @@
 
     try {
       Map<TupleTag<?>, Coder<?>> outputTagsToCoders = Maps.newHashMap();
-      for (Map.Entry<TupleTag<?>, PValue> entry : parDo.getOutputs().entrySet()) {
+      for (Map.Entry<TupleTag<?>, PCollection<?>> entry : parDo.getOutputs().entrySet()) {
         outputTagsToCoders.put(entry.getKey(), ((PCollection) entry.getValue()).getCoder());
       }
       return new SplittableParDo(
@@ -694,8 +694,8 @@
     }
 
     @Override
-    public Map<PValue, ReplacementOutput> mapOutputs(
-        Map<TupleTag<?>, PValue> outputs, PCollection<T> newOutput) {
+    public Map<PCollection<?>, ReplacementOutput> mapOutputs(
+        Map<TupleTag<?>, PCollection<?>> outputs, PCollection<T> newOutput) {
       return ReplacementOutputs.singleton(outputs, newOutput);
     }
   }
@@ -710,8 +710,8 @@
     }
 
     @Override
-    public Map<PValue, ReplacementOutput> mapOutputs(
-        Map<TupleTag<?>, PValue> outputs, PCollection<T> newOutput) {
+    public Map<PCollection<?>, ReplacementOutput> mapOutputs(
+        Map<TupleTag<?>, PCollection<?>> outputs, PCollection<T> newOutput) {
       return ReplacementOutputs.singleton(outputs, newOutput);
     }
   }
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 46e5f8b..2836cb7 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
@@ -51,7 +51,6 @@
 import org.apache.beam.sdk.values.PCollection.IsBounded;
 import org.apache.beam.sdk.values.PCollectionTuple;
 import org.apache.beam.sdk.values.PCollectionView;
-import org.apache.beam.sdk.values.PValue;
 import org.apache.beam.sdk.values.Row;
 import org.apache.beam.sdk.values.TupleTag;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.util.concurrent.Uninterruptibles;
@@ -62,6 +61,7 @@
  * Utility transforms and overrides for running bounded splittable DoFn's naively, by implementing
  * {@link ProcessKeyedElements} using a simple {@link Reshuffle} and {@link ParDo}.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SplittableParDoNaiveBounded {
   /** Overrides a {@link ProcessKeyedElements} into {@link SplittableProcessNaive}. */
   public static class OverrideFactory<InputT, OutputT, RestrictionT, WatermarkEstimatorStateT>
@@ -88,8 +88,8 @@
     }
 
     @Override
-    public Map<PValue, ReplacementOutput> mapOutputs(
-        Map<TupleTag<?>, PValue> outputs, PCollectionTuple newOutput) {
+    public Map<PCollection<?>, ReplacementOutput> mapOutputs(
+        Map<TupleTag<?>, PCollection<?>> outputs, PCollectionTuple newOutput) {
       return ReplacementOutputs.tagged(outputs, newOutput);
     }
   }
diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/Timer.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/Timer.java
index a588ab0..ea25a56 100644
--- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/Timer.java
+++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/Timer.java
@@ -49,6 +49,7 @@
  * org.apache.beam.sdk.state.Timer}.
  */
 @AutoValue
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public abstract class Timer<K> {
 
   /**
diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TransformInputs.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TransformInputs.java
index 6fefe87..850b882 100644
--- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TransformInputs.java
+++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TransformInputs.java
@@ -23,6 +23,7 @@
 import java.util.Map;
 import org.apache.beam.sdk.runners.AppliedPTransform;
 import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PValue;
 import org.apache.beam.sdk.values.TupleTag;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
@@ -36,7 +37,7 @@
   public static Collection<PValue> nonAdditionalInputs(AppliedPTransform<?, ?, ?> application) {
     ImmutableList.Builder<PValue> mainInputs = ImmutableList.builder();
     PTransform<?, ?> transform = application.getTransform();
-    for (Map.Entry<TupleTag<?>, PValue> input : application.getInputs().entrySet()) {
+    for (Map.Entry<TupleTag<?>, PCollection<?>> input : application.getInputs().entrySet()) {
       if (!transform.getAdditionalInputs().containsKey(input.getKey())) {
         mainInputs.add(input.getValue());
       }
diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TriggerTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TriggerTranslation.java
index 416792f..b58c098 100644
--- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TriggerTranslation.java
+++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TriggerTranslation.java
@@ -47,6 +47,7 @@
 import org.joda.time.Instant;
 
 /** Utilities for working with {@link TriggerTranslation Triggers}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class TriggerTranslation implements Serializable {
 
   @VisibleForTesting static final ProtoConverter CONVERTER = new ProtoConverter();
diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/UnboundedReadFromBoundedSource.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/UnboundedReadFromBoundedSource.java
index 5f10506..d27ce2e 100644
--- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/UnboundedReadFromBoundedSource.java
+++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/UnboundedReadFromBoundedSource.java
@@ -70,6 +70,7 @@
  * <p>This transform is intended to be used by a runner during pipeline translation to convert a
  * Read.Bounded into a Read.Unbounded.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class UnboundedReadFromBoundedSource<T> extends PTransform<PBegin, PCollection<T>> {
 
   private static final Logger LOG = LoggerFactory.getLogger(UnboundedReadFromBoundedSource.class);
diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/UnconsumedReads.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/UnconsumedReads.java
index 5426c4e..c905e09 100644
--- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/UnconsumedReads.java
+++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/UnconsumedReads.java
@@ -33,6 +33,7 @@
  * Utilities for ensuring that all {@link Read} {@link PTransform PTransforms} are consumed by some
  * {@link PTransform}.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class UnconsumedReads {
   public static void ensureAllReadsConsumed(Pipeline pipeline) {
     final Set<PCollection<?>> unconsumed = new HashSet<>();
diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/UnsupportedOverrideFactory.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/UnsupportedOverrideFactory.java
index e27aa6d..18bbae7 100644
--- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/UnsupportedOverrideFactory.java
+++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/UnsupportedOverrideFactory.java
@@ -21,9 +21,9 @@
 import org.apache.beam.sdk.runners.AppliedPTransform;
 import org.apache.beam.sdk.runners.PTransformOverrideFactory;
 import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PCollection;
 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.TupleTag;
 
 /**
@@ -59,8 +59,8 @@
   }
 
   @Override
-  public Map<PValue, ReplacementOutput> mapOutputs(
-      Map<TupleTag<?>, PValue> outputs, OutputT newOutput) {
+  public Map<PCollection<?>, ReplacementOutput> mapOutputs(
+      Map<TupleTag<?>, PCollection<?>> outputs, OutputT newOutput) {
     throw new UnsupportedOperationException(message);
   }
 }
diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WriteFilesTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WriteFilesTranslation.java
index 986a585..5a69e58 100644
--- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WriteFilesTranslation.java
+++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WriteFilesTranslation.java
@@ -54,6 +54,7 @@
  * Utility methods for translating a {@link WriteFiles} to and from {@link RunnerApi}
  * representations.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class WriteFilesTranslation {
 
   /** The URN for an unknown Java {@link FileBasedSink}. */
diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/GreedyPCollectionFusers.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/GreedyPCollectionFusers.java
index d004de6..9923fd3 100644
--- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/GreedyPCollectionFusers.java
+++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/GreedyPCollectionFusers.java
@@ -40,6 +40,7 @@
 /**
  * A Fuser that constructs a fused pipeline by fusing as many PCollections into a stage as possible.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class GreedyPCollectionFusers {
   private static final Logger LOG = LoggerFactory.getLogger(GreedyPCollectionFusers.class);
 
diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/GreedyPipelineFuser.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/GreedyPipelineFuser.java
index dbfe6c6..dc805ba 100644
--- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/GreedyPipelineFuser.java
+++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/GreedyPipelineFuser.java
@@ -55,6 +55,7 @@
 /** Fuses a {@link Pipeline} into some set of single-environment executable transforms. */
 // The use of NavigableSets everywhere provides consistent ordering but may be overkill for this
 // cause.
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class GreedyPipelineFuser {
   private static final Logger LOG = LoggerFactory.getLogger(GreedyPipelineFuser.class);
 
diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/GreedyStageFuser.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/GreedyStageFuser.java
index 6ec3da2..1f5534f 100644
--- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/GreedyStageFuser.java
+++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/GreedyStageFuser.java
@@ -46,6 +46,7 @@
  * <p>A {@link PCollectionNode} with consumers that execute in an environment other than a stage is
  * materialized, and its consumers execute in independent stages.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class GreedyStageFuser {
   // TODO: Provide a way to merge in a compatible subgraph (e.g. one where all of the siblings
   // consume a PCollection materialized by this subgraph and can be fused into it).
diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/Networks.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/Networks.java
index fd90cb2..bd9cd85 100644
--- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/Networks.java
+++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/Networks.java
@@ -46,6 +46,7 @@
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.graph.NetworkBuilder;
 
 /** Static utility methods for {@link Network} instances that are directed. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class Networks {
   /**
    * An abstract class that can be extended to apply a function in a type safe manner.
diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/OutputDeduplicator.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/OutputDeduplicator.java
index 71dffa7..a45a0b2 100644
--- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/OutputDeduplicator.java
+++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/OutputDeduplicator.java
@@ -46,6 +46,7 @@
  * Utilities to insert synthetic {@link PCollectionNode PCollections} for {@link PCollection
  * PCollections} which are produced by multiple independently executable stages.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class OutputDeduplicator {
 
   /**
diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/QueryablePipeline.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/QueryablePipeline.java
index 6e44ad8..e0cb973 100644
--- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/QueryablePipeline.java
+++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/QueryablePipeline.java
@@ -74,6 +74,7 @@
  * A {@link Pipeline} which has additional methods to relate nodes in the graph relative to each
  * other.
  */
+@SuppressWarnings({"nullness", "keyfor"}) // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class QueryablePipeline {
   // TODO: Is it better to have the signatures here require nodes in almost all contexts, or should
   // they all take strings? Nodes gives some degree of type signalling that names might not, but
@@ -386,7 +387,11 @@
   }
 
   private Set<String> getLocalSideInputNames(PTransform transform) {
-    if (PAR_DO_TRANSFORM_URN.equals(transform.getSpec().getUrn())) {
+    if (PAR_DO_TRANSFORM_URN.equals(transform.getSpec().getUrn())
+        || SPLITTABLE_PAIR_WITH_RESTRICTION_URN.equals(transform.getSpec().getUrn())
+        || SPLITTABLE_SPLIT_AND_SIZE_RESTRICTIONS_URN.equals(transform.getSpec().getUrn())
+        || SPLITTABLE_PROCESS_SIZED_ELEMENTS_AND_RESTRICTIONS_URN.equals(
+            transform.getSpec().getUrn())) {
       try {
         return ParDoPayload.parseFrom(transform.getSpec().getPayload()).getSideInputsMap().keySet();
       } catch (InvalidProtocolBufferException e) {
diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/SplittableParDoExpander.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/SplittableParDoExpander.java
index 4e8088f..277ed9e 100644
--- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/SplittableParDoExpander.java
+++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/SplittableParDoExpander.java
@@ -42,6 +42,7 @@
  * href="https://github.com/apache/beam/blob/cb15994d5228f729dda922419b08520c8be8804e/model/pipeline/src/main/proto/beam_runner_api.proto#L279"
  * />
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SplittableParDoExpander {
 
   /**
diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/TrivialNativeTransformExpander.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/TrivialNativeTransformExpander.java
index 8ad95e8..3fccedb 100644
--- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/TrivialNativeTransformExpander.java
+++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/TrivialNativeTransformExpander.java
@@ -27,6 +27,7 @@
  * TrivialNativeTransformExpander is used to replace transforms with known URNs with their native
  * equivalent.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class TrivialNativeTransformExpander {
   private static final Logger LOG = LoggerFactory.getLogger(TrivialNativeTransformExpander.class);
 
diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/renderer/PipelineDotRenderer.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/renderer/PipelineDotRenderer.java
index bb30dae..3f48dc7 100644
--- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/renderer/PipelineDotRenderer.java
+++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/renderer/PipelineDotRenderer.java
@@ -25,6 +25,7 @@
 import org.apache.beam.sdk.values.PValue;
 
 /** A DOT renderer for BEAM {@link Pipeline} DAG. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class PipelineDotRenderer implements Pipeline.PipelineVisitor {
   public static String toDotString(Pipeline pipeline) {
     final PipelineDotRenderer visitor = new PipelineDotRenderer();
diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/renderer/PortablePipelineDotRenderer.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/renderer/PortablePipelineDotRenderer.java
index 415ef66..67e704c 100644
--- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/renderer/PortablePipelineDotRenderer.java
+++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/renderer/PortablePipelineDotRenderer.java
@@ -26,6 +26,7 @@
 /**
  * A DOT renderer for BEAM portable {@link org.apache.beam.model.pipeline.v1.RunnerApi.Pipeline}.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class PortablePipelineDotRenderer {
   private final StringBuilder dotBuilder = new StringBuilder();
   private final Map<String, Integer> valueToProducerNodeId = new HashMap<>();
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 8bc6772..7174d00 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
@@ -52,6 +52,7 @@
 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.ShardedKey;
 import org.apache.beam.sdk.util.WindowedValue;
 import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
@@ -65,6 +66,7 @@
 import org.junit.runners.Parameterized.Parameters;
 
 /** Tests for {@link CoderTranslation}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class CoderTranslationTest {
   private static final Set<Coder<?>> KNOWN_CODERS =
       ImmutableSet.<Coder<?>>builder()
@@ -90,6 +92,7 @@
                       Field.of("array", FieldType.array(FieldType.STRING)),
                       Field.of("map", FieldType.map(FieldType.STRING, FieldType.INT32)),
                       Field.of("bar", FieldType.logicalType(FixedBytes.of(123))))))
+          .add(ShardedKey.Coder.of(StringUtf8Coder.of()))
           .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 b8d58c3..d1bee27 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
@@ -68,6 +68,7 @@
 import org.apache.beam.sdk.transforms.windowing.IntervalWindow.IntervalWindowCoder;
 import org.apache.beam.sdk.transforms.windowing.PaneInfo;
 import org.apache.beam.sdk.util.CoderUtils;
+import org.apache.beam.sdk.util.ShardedKey;
 import org.apache.beam.sdk.util.WindowedValue;
 import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.Row;
@@ -88,6 +89,7 @@
 
 /** Tests that Java SDK coders standardized by the Fn API meet the common spec. */
 @RunWith(Parameterized.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class CommonCoderTest {
   private static final String STANDARD_CODERS_YAML_PATH =
       "/org/apache/beam/model/fnexecution/v1/standard_coders.yaml";
@@ -111,6 +113,7 @@
               getUrn(StandardCoders.Enum.PARAM_WINDOWED_VALUE),
               WindowedValue.ParamWindowedValueCoder.class)
           .put(getUrn(StandardCoders.Enum.ROW), RowCoder.class)
+          .put(getUrn(StandardCoders.Enum.SHARDED_KEY), ShardedKey.Coder.class)
           .build();
 
   @AutoValue
@@ -334,6 +337,12 @@
       }
 
       return parseField(value, Schema.FieldType.row(schema));
+    } else if (s.equals(getUrn(StandardCoders.Enum.SHARDED_KEY))) {
+      Map<String, Object> kvMap = (Map<String, Object>) value;
+      Coder<?> keyCoder = ((ShardedKey.Coder) coder).getKeyCoder();
+      byte[] shardId = ((String) kvMap.get("shardId")).getBytes(StandardCharsets.ISO_8859_1);
+      return ShardedKey.of(
+          convertValue(kvMap.get("key"), coderSpec.getComponents().get(0), keyCoder), shardId);
     } else {
       throw new IllegalStateException("Unknown coder URN: " + coderSpec.getUrn());
     }
@@ -489,6 +498,8 @@
       assertEquals(expectedValue, actualValue);
     } else if (s.equals(getUrn(StandardCoders.Enum.ROW))) {
       assertEquals(expectedValue, actualValue);
+    } else if (s.equals(getUrn(StandardCoders.Enum.SHARDED_KEY))) {
+      assertEquals(expectedValue, actualValue);
     } else {
       throw new IllegalStateException("Unknown coder URN: " + coder.getUrn());
     }
diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/CreatePCollectionViewTranslationTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/CreatePCollectionViewTranslationTest.java
index 29b857b..35d51ee 100644
--- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/CreatePCollectionViewTranslationTest.java
+++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/CreatePCollectionViewTranslationTest.java
@@ -36,6 +36,7 @@
 import org.apache.beam.sdk.values.PCollectionViews.TypeDescriptorSupplier;
 import org.apache.beam.sdk.values.PCollectionViews.ValueOrMetadata;
 import org.apache.beam.sdk.values.PCollectionViews.ValueOrMetadataCoder;
+import org.apache.beam.sdk.values.PValues;
 import org.apache.beam.sdk.values.TypeDescriptors;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
 import org.hamcrest.Matchers;
@@ -102,8 +103,8 @@
     AppliedPTransform<?, ?, ?> appliedPTransform =
         AppliedPTransform.of(
             "foo",
-            testPCollection.expand(),
-            createViewTransform.getView().expand(),
+            PValues.expandInput(testPCollection),
+            PValues.expandOutput(createViewTransform.getView()),
             createViewTransform,
             p);
 
@@ -127,8 +128,8 @@
     AppliedPTransform<?, ?, ?> appliedPTransform =
         AppliedPTransform.of(
             "foo",
-            testPCollection.expand(),
-            createViewTransform.getView().expand(),
+            PValues.expandInput(testPCollection),
+            PValues.expandOutput(createViewTransform.getView()),
             createViewTransform,
             p);
 
diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/DeduplicatedFlattenFactoryTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/DeduplicatedFlattenFactoryTest.java
index 044b543..ff01bff9 100644
--- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/DeduplicatedFlattenFactoryTest.java
+++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/DeduplicatedFlattenFactoryTest.java
@@ -33,6 +33,7 @@
 import org.apache.beam.sdk.transforms.PTransform;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PCollectionList;
+import org.apache.beam.sdk.values.PValues;
 import org.apache.beam.sdk.values.TaggedPValue;
 import org.hamcrest.Matchers;
 import org.junit.Rule;
@@ -89,7 +90,7 @@
     PCollection<String> replacement = inputList.apply(new FlattenWithoutDuplicateInputs<>());
 
     assertThat(
-        factory.mapOutputs(original.expand(), replacement),
+        factory.mapOutputs(PValues.expandOutput(original), replacement),
         Matchers.hasEntry(
             replacement,
             ReplacementOutput.of(
diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/DefaultArtifactResolverTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/DefaultArtifactResolverTest.java
index 3249430..3f65b11 100644
--- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/DefaultArtifactResolverTest.java
+++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/DefaultArtifactResolverTest.java
@@ -30,6 +30,7 @@
 import org.junit.runners.JUnit4;
 
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DefaultArtifactResolverTest {
   private RunnerApi.Pipeline createEmptyPipeline(
       Iterable<RunnerApi.ArtifactInformation> dependencies) {
diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/EmptyFlattenAsCreateFactoryTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/EmptyFlattenAsCreateFactoryTest.java
index 3d56d4a..309f9a3 100644
--- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/EmptyFlattenAsCreateFactoryTest.java
+++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/EmptyFlattenAsCreateFactoryTest.java
@@ -32,7 +32,7 @@
 import org.apache.beam.sdk.transforms.Flatten;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PCollectionList;
-import org.apache.beam.sdk.values.PValue;
+import org.apache.beam.sdk.values.PValues;
 import org.apache.beam.sdk.values.TaggedPValue;
 import org.hamcrest.Matchers;
 import org.junit.Rule;
@@ -74,7 +74,7 @@
     factory.getReplacementTransform(
         AppliedPTransform.of(
             "nonEmptyInput",
-            nonEmpty.expand(),
+            PValues.expandInput(nonEmpty),
             Collections.emptyMap(),
             Flatten.pCollections(),
             pipeline));
@@ -84,7 +84,8 @@
   public void mapOutputsSucceeds() {
     PCollection<Long> original = pipeline.apply("Original", GenerateSequence.from(0));
     PCollection<Long> replacement = pipeline.apply("Replacement", GenerateSequence.from(0));
-    Map<PValue, ReplacementOutput> mapping = factory.mapOutputs(original.expand(), replacement);
+    Map<PCollection<?>, ReplacementOutput> mapping =
+        factory.mapOutputs(PValues.expandOutput(original), replacement);
 
     assertThat(
         mapping,
diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ForwardingPTransformTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ForwardingPTransformTest.java
index ca61ae1..acc6156 100644
--- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ForwardingPTransformTest.java
+++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ForwardingPTransformTest.java
@@ -47,6 +47,7 @@
 
 /** Tests for {@link ForwardingPTransform}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ForwardingPTransformTest {
   @Rule public ExpectedException thrown = ExpectedException.none();
 
diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/MorePipelineTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/MorePipelineTest.java
new file mode 100644
index 0000000..a5b4a7c
--- /dev/null
+++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/MorePipelineTest.java
@@ -0,0 +1,164 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.runners.core.construction;
+
+import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkState;
+import static org.hamcrest.Matchers.anyOf;
+import static org.hamcrest.Matchers.not;
+import static org.junit.Assert.assertThat;
+
+import java.util.List;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.beam.sdk.Pipeline.PipelineVisitor;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.io.GenerateSequence;
+import org.apache.beam.sdk.runners.AppliedPTransform;
+import org.apache.beam.sdk.runners.PTransformOverride;
+import org.apache.beam.sdk.runners.TransformHierarchy.Node;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.transforms.Materializations;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.View;
+import org.apache.beam.sdk.values.KV;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PCollectionView;
+import org.apache.beam.sdk.values.PCollectionViews;
+import org.apache.beam.sdk.values.TupleTag;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
+import org.hamcrest.Matchers;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/** Tests for Pipeline. */
+@RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
+public class MorePipelineTest {
+
+  @Rule public final TestPipeline pipeline = TestPipeline.create();
+
+  @Test
+  public void testReplaceAllPCollectionView() {
+    pipeline.enableAbandonedNodeEnforcement(false);
+    pipeline.apply(GenerateSequence.from(0).to(100)).apply(View.asList());
+
+    pipeline.replaceAll(
+        ImmutableList.of(
+            PTransformOverride.of(
+                application -> application.getTransform() instanceof View.AsList,
+                new ViewAsListOverride())));
+    pipeline.traverseTopologically(
+        new PipelineVisitor.Defaults() {
+          @Override
+          public CompositeBehavior enterCompositeTransform(Node node) {
+            if (!node.isRootNode()) {
+              assertThat(
+                  node.getTransform().getClass(), not(anyOf(Matchers.equalTo(View.AsList.class))));
+            }
+            return CompositeBehavior.ENTER_TRANSFORM;
+          }
+        });
+  }
+
+  static class ViewAsListOverride<T>
+      extends SingleInputOutputOverrideFactory<
+          PCollection<T>, PCollectionView<List<T>>, View.AsList<T>> {
+    @Override
+    public PTransformReplacement<PCollection<T>, PCollectionView<List<T>>> getReplacementTransform(
+        AppliedPTransform<PCollection<T>, PCollectionView<List<T>>, View.AsList<T>> transform) {
+      return PTransformReplacement.of(
+          PTransformReplacements.getSingletonMainInput(transform),
+          new FakeViewAsList<>(findPCollectionView(transform)));
+    }
+  }
+
+  static class FakeViewAsList<T> extends PTransform<PCollection<T>, PCollectionView<List<T>>> {
+    private final PCollectionView<List<T>> originalView;
+
+    FakeViewAsList(PCollectionView<List<T>> originalView) {
+      this.originalView = originalView;
+    }
+
+    @Override
+    public PCollectionView<List<T>> expand(PCollection<T> input) {
+      PCollection<KV<Void, T>> materializationInput =
+          input.apply(new View.VoidKeyToMultimapMaterialization<>());
+      Coder<T> inputCoder = input.getCoder();
+      PCollectionView<List<T>> view =
+          PCollectionViews.listViewUsingVoidKey(
+              materializationInput,
+              (TupleTag<Materializations.MultimapView<Void, T>>) originalView.getTagInternal(),
+              (PCollectionViews.TypeDescriptorSupplier<T>) inputCoder::getEncodedTypeDescriptor,
+              materializationInput.getWindowingStrategy());
+      materializationInput.apply(View.CreatePCollectionView.of(view));
+      return view;
+    }
+  }
+
+  private static <InputT, ViewT> PCollectionView<ViewT> findPCollectionView(
+      final AppliedPTransform<
+              PCollection<InputT>,
+              PCollectionView<ViewT>,
+              ? extends PTransform<PCollection<InputT>, PCollectionView<ViewT>>>
+          transform) {
+    final AtomicReference<PCollectionView<ViewT>> viewRef = new AtomicReference<>();
+    transform
+        .getPipeline()
+        .traverseTopologically(
+            new PipelineVisitor.Defaults() {
+              // Stores whether we have entered the expected composite view transform.
+              private boolean tracking = false;
+
+              @Override
+              public CompositeBehavior enterCompositeTransform(Node node) {
+                if (transform.getTransform() == node.getTransform()) {
+                  tracking = true;
+                }
+                return super.enterCompositeTransform(node);
+              }
+
+              @Override
+              public void visitPrimitiveTransform(Node node) {
+                if (tracking && node.getTransform() instanceof View.CreatePCollectionView) {
+                  View.CreatePCollectionView createViewTransform =
+                      (View.CreatePCollectionView) node.getTransform();
+                  checkState(
+                      viewRef.compareAndSet(null, createViewTransform.getView()),
+                      "Found more than one instance of a CreatePCollectionView when"
+                          + "attempting to replace %s, found [%s, %s]",
+                      transform.getTransform(),
+                      viewRef.get(),
+                      createViewTransform.getView());
+                }
+              }
+
+              @Override
+              public void leaveCompositeTransform(Node node) {
+                if (transform.getTransform() == node.getTransform()) {
+                  tracking = false;
+                }
+              }
+            });
+    checkState(
+        viewRef.get() != null,
+        "Expected to find CreatePCollectionView contained within %s",
+        transform.getTransform());
+    return viewRef.get();
+  }
+}
diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PCollectionTranslationTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PCollectionTranslationTest.java
index 73d93ba..ad9b463 100644
--- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PCollectionTranslationTest.java
+++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PCollectionTranslationTest.java
@@ -63,6 +63,7 @@
 
 /** Tests for {@link PCollectionTranslation}. */
 @RunWith(Parameterized.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class PCollectionTranslationTest {
   // Each spec activates tests of all subsets of its fields
   @Parameters(name = "{index}: {0}")
diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PTransformMatchersTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PTransformMatchersTest.java
index f67a1bc..fc45dde 100644
--- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PTransformMatchersTest.java
+++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PTransformMatchersTest.java
@@ -66,7 +66,7 @@
 import org.apache.beam.sdk.values.PCollectionView;
 import org.apache.beam.sdk.values.PCollectionViews;
 import org.apache.beam.sdk.values.PCollectionViews.IterableViewFn;
-import org.apache.beam.sdk.values.PValue;
+import org.apache.beam.sdk.values.PValues;
 import org.apache.beam.sdk.values.TupleTag;
 import org.apache.beam.sdk.values.TupleTagList;
 import org.apache.beam.sdk.values.TypeDescriptors;
@@ -83,6 +83,7 @@
 
 /** Tests for {@link PTransformMatcher}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class PTransformMatchersTest implements Serializable {
   @Rule
   public transient TestPipeline p = TestPipeline.create().enableAbandonedNodeEnforcement(false);
@@ -105,7 +106,8 @@
             p, WindowingStrategy.globalDefault(), IsBounded.BOUNDED, VarIntCoder.of());
     output.setName("dummy output");
 
-    return AppliedPTransform.of("pardo", input.expand(), output.expand(), pardo, p);
+    return AppliedPTransform.of(
+        "pardo", PValues.expandInput(input), PValues.expandOutput(output), pardo, p);
   }
 
   @Test
@@ -504,7 +506,7 @@
     AppliedPTransform application =
         AppliedPTransform.of(
             "Flatten",
-            ImmutableMap.<TupleTag<?>, PValue>builder()
+            ImmutableMap.<TupleTag<?>, PCollection<?>>builder()
                 .put(new TupleTag<Integer>(), duplicate)
                 .put(new TupleTag<Integer>(), duplicate)
                 .build(),
@@ -588,9 +590,8 @@
       throw new UnsupportedOperationException("should not be called");
     }
 
-    @Nullable
     @Override
-    public ResourceId unwindowedFilename(
+    public @Nullable ResourceId unwindowedFilename(
         int shardNumber, int numShards, FileBasedSink.OutputFileHints outputFileHints) {
       throw new UnsupportedOperationException("should not be called");
     }
diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PTransformReplacementsTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PTransformReplacementsTest.java
index bc26aa3..dfdf524 100644
--- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PTransformReplacementsTest.java
+++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PTransformReplacementsTest.java
@@ -30,7 +30,7 @@
 import org.apache.beam.sdk.transforms.View;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PCollectionView;
-import org.apache.beam.sdk.values.PValue;
+import org.apache.beam.sdk.values.PValues;
 import org.apache.beam.sdk.values.TupleTag;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
 import org.junit.Rule;
@@ -41,6 +41,7 @@
 
 /** Tests for {@link PTransformReplacements}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class PTransformReplacementsTest {
   @Rule public TestPipeline pipeline = TestPipeline.create().enableAbandonedNodeEnforcement(false);
   @Rule public ExpectedException thrown = ExpectedException.none();
@@ -68,7 +69,7 @@
     AppliedPTransform<PCollection<Long>, ?, ?> application =
         AppliedPTransform.of(
             "application",
-            ImmutableMap.<TupleTag<?>, PValue>builder()
+            ImmutableMap.<TupleTag<?>, PCollection<?>>builder()
                 .put(new TupleTag<Long>(), mainInput)
                 .put(sideInput.getTagInternal(), sideInput.getPCollection())
                 .build(),
@@ -82,9 +83,9 @@
   @Test
   public void getMainInputExtraMainInputsThrows() {
     PCollection<Long> notInParDo = pipeline.apply("otherPCollection", Create.of(1L, 2L, 3L));
-    ImmutableMap<TupleTag<?>, PValue> inputs =
-        ImmutableMap.<TupleTag<?>, PValue>builder()
-            .putAll(mainInput.expand())
+    ImmutableMap<TupleTag<?>, PCollection<?>> inputs =
+        ImmutableMap.<TupleTag<?>, PCollection<?>>builder()
+            .putAll(PValues.expandInput(mainInput))
             // Not represnted as an input
             .put(new TupleTag<Long>(), notInParDo)
             .put(sideInput.getTagInternal(), sideInput.getPCollection())
@@ -106,8 +107,8 @@
 
   @Test
   public void getMainInputNoMainInputsThrows() {
-    ImmutableMap<TupleTag<?>, PValue> inputs =
-        ImmutableMap.<TupleTag<?>, PValue>builder()
+    ImmutableMap<TupleTag<?>, PCollection<?>> inputs =
+        ImmutableMap.<TupleTag<?>, PCollection<?>>builder()
             .put(sideInput.getTagInternal(), sideInput.getPCollection())
             .build();
     AppliedPTransform<PCollection<Long>, ?, ?> application =
diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PTransformTranslationTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PTransformTranslationTest.java
index 49b1182..d59793d 100644
--- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PTransformTranslationTest.java
+++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PTransformTranslationTest.java
@@ -49,6 +49,7 @@
 import org.apache.beam.sdk.values.PCollectionView;
 import org.apache.beam.sdk.values.PDone;
 import org.apache.beam.sdk.values.PValue;
+import org.apache.beam.sdk.values.PValues;
 import org.apache.beam.sdk.values.TupleTag;
 import org.apache.beam.sdk.values.TupleTagList;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
@@ -61,6 +62,7 @@
 
 /** Tests for {@link PTransformTranslation}. */
 @RunWith(Parameterized.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class PTransformTranslationTest {
 
   @Parameters(name = "{index}: {0}")
@@ -164,14 +166,22 @@
     GenerateSequence sequence = GenerateSequence.from(0);
     PCollection<Long> pcollection = pipeline.apply(sequence);
     return AppliedPTransform.of(
-        "Count", pipeline.begin().expand(), pcollection.expand(), sequence, pipeline);
+        "Count",
+        PValues.expandInput(pipeline.begin()),
+        PValues.expandOutput(pcollection),
+        sequence,
+        pipeline);
   }
 
   private static AppliedPTransform<?, ?, ?> read(Pipeline pipeline) {
     Read.Unbounded<Long> transform = Read.from(CountingSource.unbounded());
     PCollection<Long> pcollection = pipeline.apply(transform);
     return AppliedPTransform.of(
-        "ReadTheCount", pipeline.begin().expand(), pcollection.expand(), transform, pipeline);
+        "ReadTheCount",
+        PValues.expandInput(pipeline.begin()),
+        PValues.expandOutput(pcollection),
+        transform,
+        pipeline);
   }
 
   private static AppliedPTransform<?, ?, ?> rawPTransformWithNullSpec(Pipeline pipeline) {
@@ -189,8 +199,8 @@
         };
     return AppliedPTransform.<PBegin, PDone, PTransform<PBegin, PDone>>of(
         "RawPTransformWithNoSpec",
-        pipeline.begin().expand(),
-        PDone.in(pipeline).expand(),
+        PValues.expandInput(pipeline.begin()),
+        PValues.expandOutput(PDone.in(pipeline)),
         rawPTransform,
         pipeline);
   }
@@ -206,12 +216,12 @@
                 TupleTagList.of(new TupleTag<KV<String, Long>>() {}));
     PCollectionTuple output = input.apply(parDo);
 
-    Map<TupleTag<?>, PValue> inputs = new HashMap<>();
-    inputs.putAll(parDo.getAdditionalInputs());
-    inputs.putAll(input.expand());
+    Map<TupleTag<?>, PCollection<?>> inputs = new HashMap<>();
+    inputs.putAll(PValues.fullyExpand(parDo.getAdditionalInputs()));
+    inputs.putAll(PValues.expandInput(input));
 
     return AppliedPTransform
         .<PCollection<Long>, PCollectionTuple, ParDo.MultiOutput<Long, KV<Long, String>>>of(
-            "MultiParDoInAndOut", inputs, output.expand(), parDo, pipeline);
+            "MultiParDoInAndOut", inputs, PValues.expandOutput(output), parDo, pipeline);
   }
 }
diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ParDoTranslationTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ParDoTranslationTest.java
index ee8e52a..f1feb5c 100644
--- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ParDoTranslationTest.java
+++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ParDoTranslationTest.java
@@ -62,7 +62,7 @@
 import org.apache.beam.sdk.values.PCollection.IsBounded;
 import org.apache.beam.sdk.values.PCollectionTuple;
 import org.apache.beam.sdk.values.PCollectionView;
-import org.apache.beam.sdk.values.PValue;
+import org.apache.beam.sdk.values.PValues;
 import org.apache.beam.sdk.values.TupleTag;
 import org.apache.beam.sdk.values.TupleTagList;
 import org.apache.beam.sdk.values.WindowingStrategy;
@@ -150,9 +150,9 @@
 
     @Test
     public void toTransformProto() throws Exception {
-      Map<TupleTag<?>, PValue> inputs = new HashMap<>();
+      Map<TupleTag<?>, PCollection<?>> inputs = new HashMap<>();
       inputs.put(new TupleTag<KV<Long, String>>("mainInputName") {}, mainInput);
-      inputs.putAll(parDo.getAdditionalInputs());
+      inputs.putAll(PValues.fullyExpand(parDo.getAdditionalInputs()));
       PCollectionTuple output = mainInput.apply(parDo);
 
       SdkComponents sdkComponents = SdkComponents.create();
@@ -162,7 +162,7 @@
       RunnerApi.PTransform protoTransform =
           PTransformTranslation.toProto(
               AppliedPTransform.<PCollection<KV<Long, String>>, PCollection<Void>, MultiOutput>of(
-                  "foo", inputs, output.expand(), parDo, p),
+                  "foo", inputs, PValues.expandOutput(output), parDo, p),
               sdkComponents);
       RunnerApi.Components components = sdkComponents.toComponents();
       RehydratedComponents rehydratedComponents = RehydratedComponents.forComponents(components);
diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PipelineOptionsTranslationTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PipelineOptionsTranslationTest.java
index 14d8c1c..346a427 100644
--- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PipelineOptionsTranslationTest.java
+++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PipelineOptionsTranslationTest.java
@@ -42,6 +42,7 @@
 import org.junit.runners.Parameterized.Parameters;
 
 /** Tests for {@link PipelineOptionsTranslation}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class PipelineOptionsTranslationTest {
   /** Tests that translations can round-trip through the proto format. */
   @RunWith(Parameterized.class)
diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PipelineTranslationTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PipelineTranslationTest.java
index 52b9868..e9ed159 100644
--- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PipelineTranslationTest.java
+++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PipelineTranslationTest.java
@@ -70,6 +70,7 @@
 
 /** Tests for {@link PipelineTranslation}. */
 @RunWith(Parameterized.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class PipelineTranslationTest {
   @Parameter(0)
   public Pipeline pipeline;
diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ReplacementOutputsTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ReplacementOutputsTest.java
index 6ca2fe4..4df6cfc 100644
--- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ReplacementOutputsTest.java
+++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ReplacementOutputsTest.java
@@ -28,7 +28,9 @@
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PCollection.IsBounded;
 import org.apache.beam.sdk.values.PCollectionTuple;
+import org.apache.beam.sdk.values.POutput;
 import org.apache.beam.sdk.values.PValue;
+import org.apache.beam.sdk.values.PValues;
 import org.apache.beam.sdk.values.TaggedPValue;
 import org.apache.beam.sdk.values.TupleTag;
 import org.apache.beam.sdk.values.WindowingStrategy;
@@ -43,6 +45,7 @@
 
 /** Tests for {@link ReplacementOutputs}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ReplacementOutputsTest {
   @Rule public ExpectedException thrown = ExpectedException.none();
   private TestPipeline p = TestPipeline.create();
@@ -69,8 +72,8 @@
 
   @Test
   public void singletonSucceeds() {
-    Map<PValue, ReplacementOutput> replacements =
-        ReplacementOutputs.singleton(ints.expand(), replacementInts);
+    Map<PCollection<?>, ReplacementOutput> replacements =
+        ReplacementOutputs.singleton(PValues.expandValue(ints), replacementInts);
 
     assertThat(replacements, Matchers.hasKey(replacementInts));
 
@@ -85,9 +88,9 @@
   public void singletonMultipleOriginalsThrows() {
     thrown.expect(IllegalArgumentException.class);
     ReplacementOutputs.singleton(
-        ImmutableMap.<TupleTag<?>, PValue>builder()
-            .putAll(ints.expand())
-            .putAll(moreInts.expand())
+        ImmutableMap.<TupleTag<?>, PCollection<?>>builder()
+            .putAll(PValues.expandValue(ints))
+            .putAll(PValues.fullyExpand(moreInts.expand()))
             .build(),
         replacementInts);
   }
@@ -101,9 +104,9 @@
     PCollectionTuple original =
         PCollectionTuple.of(intsTag, ints).and(strsTag, strs).and(moreIntsTag, moreInts);
 
-    Map<PValue, ReplacementOutput> replacements =
+    Map<PCollection<?>, ReplacementOutput> replacements =
         ReplacementOutputs.tagged(
-            original.expand(),
+            PValues.expandOutput((POutput) original),
             PCollectionTuple.of(strsTag, replacementStrs)
                 .and(moreIntsTag, moreReplacementInts)
                 .and(intsTag, replacementInts));
@@ -142,7 +145,7 @@
     thrown.expectMessage(intsTag.toString());
     thrown.expectMessage(ints.toString());
     ReplacementOutputs.tagged(
-        original.expand(),
+        PValues.expandOutput(original),
         PCollectionTuple.of(strsTag, replacementStrs).and(moreIntsTag, moreReplacementInts));
   }
 
@@ -155,7 +158,7 @@
     thrown.expectMessage(moreIntsTag.toString());
     thrown.expectMessage(moreReplacementInts.toString());
     ReplacementOutputs.tagged(
-        original.expand(),
+        PValues.expandOutput(original),
         PCollectionTuple.of(strsTag, replacementStrs)
             .and(moreIntsTag, moreReplacementInts)
             .and(intsTag, replacementInts));
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 735af68..e60ea6d 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
@@ -42,6 +42,7 @@
 
 /** Tests for {@link SchemaTranslation}. */
 @RunWith(Enclosed.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SchemaTranslationTest {
 
   /** Tests round-trip proto encodings for {@link Schema}. */
diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/SdkComponentsTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/SdkComponentsTest.java
index 9c813d9..4eebecf 100644
--- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/SdkComponentsTest.java
+++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/SdkComponentsTest.java
@@ -38,6 +38,7 @@
 import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.transforms.Create;
 import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PValues;
 import org.apache.beam.sdk.values.WindowingStrategy;
 import org.apache.beam.sdk.values.WindowingStrategy.AccumulationMode;
 import org.junit.Before;
@@ -49,6 +50,7 @@
 
 /** Tests for {@link SdkComponents}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SdkComponentsTest {
   @Rule public TestPipeline pipeline = TestPipeline.create().enableAbandonedNodeEnforcement(false);
   @Rule public ExpectedException thrown = ExpectedException.none();
@@ -84,7 +86,12 @@
     PCollection<Integer> pt = pipeline.apply(create);
     String userName = "my_transform/my_nesting";
     AppliedPTransform<?, ?, ?> transform =
-        AppliedPTransform.of(userName, pipeline.begin().expand(), pt.expand(), create, pipeline);
+        AppliedPTransform.of(
+            userName,
+            PValues.expandInput(pipeline.begin()),
+            PValues.expandOutput(pt),
+            create,
+            pipeline);
     String componentName = components.registerPTransform(transform, Collections.emptyList());
     assertThat(componentName, equalTo(userName));
     assertThat(components.getExistingPTransformId(transform), equalTo(componentName));
@@ -99,10 +106,19 @@
     String userName = "my_transform";
     String childUserName = "my_transform/my_nesting";
     AppliedPTransform<?, ?, ?> transform =
-        AppliedPTransform.of(userName, pipeline.begin().expand(), pt.expand(), create, pipeline);
+        AppliedPTransform.of(
+            userName,
+            PValues.expandInput(pipeline.begin()),
+            PValues.expandOutput(pt),
+            create,
+            pipeline);
     AppliedPTransform<?, ?, ?> childTransform =
         AppliedPTransform.of(
-            childUserName, pipeline.begin().expand(), pt.expand(), createChild, pipeline);
+            childUserName,
+            PValues.expandInput(pipeline.begin()),
+            PValues.expandOutput(pt),
+            createChild,
+            pipeline);
 
     String childId = components.registerPTransform(childTransform, Collections.emptyList());
     String parentId =
@@ -117,7 +133,8 @@
     Create.Values<Integer> create = Create.of(1, 2, 3);
     PCollection<Integer> pt = pipeline.apply(create);
     AppliedPTransform<?, ?, ?> transform =
-        AppliedPTransform.of("", pipeline.begin().expand(), pt.expand(), create, pipeline);
+        AppliedPTransform.of(
+            "", PValues.expandInput(pipeline.begin()), PValues.expandOutput(pt), create, pipeline);
 
     thrown.expect(IllegalArgumentException.class);
     thrown.expectMessage(transform.toString());
@@ -130,7 +147,12 @@
     PCollection<Integer> pt = pipeline.apply(create);
     String userName = "my_transform/my_nesting";
     AppliedPTransform<?, ?, ?> transform =
-        AppliedPTransform.of(userName, pipeline.begin().expand(), pt.expand(), create, pipeline);
+        AppliedPTransform.of(
+            userName,
+            PValues.expandInput(pipeline.begin()),
+            PValues.expandOutput(pt),
+            create,
+            pipeline);
     thrown.expect(NullPointerException.class);
     thrown.expectMessage("child nodes may not be null");
     components.registerPTransform(transform, null);
@@ -146,10 +168,19 @@
     String userName = "my_transform";
     String childUserName = "my_transform/my_nesting";
     AppliedPTransform<?, ?, ?> transform =
-        AppliedPTransform.of(userName, pipeline.begin().expand(), pt.expand(), create, pipeline);
+        AppliedPTransform.of(
+            userName,
+            PValues.expandInput(pipeline.begin()),
+            PValues.expandOutput(pt),
+            create,
+            pipeline);
     AppliedPTransform<?, ?, ?> childTransform =
         AppliedPTransform.of(
-            childUserName, pipeline.begin().expand(), pt.expand(), createChild, pipeline);
+            childUserName,
+            PValues.expandInput(pipeline.begin()),
+            PValues.expandOutput(pt),
+            createChild,
+            pipeline);
 
     thrown.expect(IllegalArgumentException.class);
     thrown.expectMessage(childTransform.toString());
diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/SingleInputOutputOverrideFactoryTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/SingleInputOutputOverrideFactoryTest.java
index 13a9549..11bbefca 100644
--- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/SingleInputOutputOverrideFactoryTest.java
+++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/SingleInputOutputOverrideFactoryTest.java
@@ -29,7 +29,7 @@
 import org.apache.beam.sdk.transforms.SimpleFunction;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PCollectionList;
-import org.apache.beam.sdk.values.PValue;
+import org.apache.beam.sdk.values.PValues;
 import org.apache.beam.sdk.values.TaggedPValue;
 import org.hamcrest.Matchers;
 import org.junit.Rule;
@@ -81,8 +81,8 @@
     PCollection<Integer> input = pipeline.apply(Create.of(1, 2, 3));
     PCollection<Integer> output = input.apply("Map", MapElements.via(fn));
     PCollection<Integer> reappliedOutput = input.apply("ReMap", MapElements.via(fn));
-    Map<PValue, ReplacementOutput> replacementMap =
-        factory.mapOutputs(output.expand(), reappliedOutput);
+    Map<PCollection<?>, ReplacementOutput> replacementMap =
+        factory.mapOutputs(PValues.expandOutput(output), reappliedOutput);
     assertThat(
         replacementMap,
         Matchers.hasEntry(
@@ -99,6 +99,7 @@
     PCollection<Integer> reappliedOutput = input.apply("ReMap", MapElements.via(fn));
     thrown.expect(IllegalArgumentException.class);
     factory.mapOutputs(
-        PCollectionList.of(output).and(input).and(reappliedOutput).expand(), reappliedOutput);
+        PValues.expandOutput(PCollectionList.of(output).and(input).and(reappliedOutput)),
+        reappliedOutput);
   }
 }
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 3e75f1a..635d665 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
@@ -51,6 +51,7 @@
 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.PValues;
 import org.apache.beam.sdk.values.TupleTag;
 import org.apache.beam.sdk.values.TupleTagList;
 import org.junit.Rule;
@@ -146,7 +147,12 @@
     PCollectionTuple output = multiOutput.expand(input);
     output.get(MAIN_OUTPUT_TAG).setName("main");
     AppliedPTransform<PCollection<Integer>, PCollectionTuple, ?> transform =
-        AppliedPTransform.of("ParDo", input.expand(), output.expand(), multiOutput, pipeline);
+        AppliedPTransform.of(
+            "ParDo",
+            PValues.expandInput(input),
+            PValues.expandOutput(output),
+            multiOutput,
+            pipeline);
     return input.apply(name, SplittableParDo.forAppliedParDo(transform)).get(MAIN_OUTPUT_TAG);
   }
 
diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/TestStreamTranslationTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/TestStreamTranslationTest.java
index 98960b6..703f65d 100644
--- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/TestStreamTranslationTest.java
+++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/TestStreamTranslationTest.java
@@ -30,6 +30,7 @@
 import org.apache.beam.sdk.testing.TestStream;
 import org.apache.beam.sdk.values.PBegin;
 import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PValues;
 import org.apache.beam.sdk.values.TimestampedValue;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
 import org.joda.time.Duration;
@@ -79,7 +80,12 @@
     PCollection<String> output = p.apply(testStream);
 
     AppliedPTransform<PBegin, PCollection<String>, TestStream<String>> appliedTestStream =
-        AppliedPTransform.of("fakeName", PBegin.in(p).expand(), output.expand(), testStream, p);
+        AppliedPTransform.of(
+            "fakeName",
+            PValues.expandInput(PBegin.in(p)),
+            PValues.expandOutput(output),
+            testStream,
+            p);
 
     SdkComponents components = SdkComponents.create();
     components.registerEnvironment(Environments.createDockerEnvironment("java"));
diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/TimerTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/TimerTest.java
index f3d12e8..a88e923 100644
--- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/TimerTest.java
+++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/TimerTest.java
@@ -34,6 +34,7 @@
 
 /** Tests for {@link Timer}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class TimerTest {
   private static final Instant FIRE_TIME = new Instant(123L);
   private static final Instant HOLD_TIME = new Instant(456L);
diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/TransformInputsTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/TransformInputsTest.java
index e7edda7..2533849 100644
--- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/TransformInputsTest.java
+++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/TransformInputsTest.java
@@ -76,7 +76,7 @@
 
   @Test
   public void nonAdditionalInputsWithMultipleNonAdditionalInputsSucceeds() {
-    Map<TupleTag<?>, PValue> allInputs = new HashMap<>();
+    Map<TupleTag<?>, PCollection<?>> allInputs = new HashMap<>();
     PCollection<Integer> mainInts = pipeline.apply("MainInput", Create.of(12, 3));
     allInputs.put(new TupleTag<Integer>() {}, mainInts);
     PCollection<Void> voids = pipeline.apply("VoidInput", Create.empty(VoidCoder.of()));
@@ -96,12 +96,12 @@
     additionalInputs.put(new TupleTag<String>() {}, pipeline.apply(Create.of("1, 2", "3")));
     additionalInputs.put(new TupleTag<Long>() {}, pipeline.apply(GenerateSequence.from(3L)));
 
-    Map<TupleTag<?>, PValue> allInputs = new HashMap<>();
+    Map<TupleTag<?>, PCollection<?>> allInputs = new HashMap<>();
     PCollection<Integer> mainInts = pipeline.apply("MainInput", Create.of(12, 3));
     allInputs.put(new TupleTag<Integer>() {}, mainInts);
     PCollection<Void> voids = pipeline.apply("VoidInput", Create.empty(VoidCoder.of()));
     allInputs.put(new TupleTag<Void>() {}, voids);
-    allInputs.putAll(additionalInputs);
+    allInputs.putAll((Map) additionalInputs);
 
     AppliedPTransform<PInput, POutput, TestTransform> transform =
         AppliedPTransform.of(
@@ -118,7 +118,7 @@
 
   @Test
   public void nonAdditionalInputsWithOnlyAdditionalInputsThrows() {
-    Map<TupleTag<?>, PValue> additionalInputs = new HashMap<>();
+    Map<TupleTag<?>, PCollection<?>> additionalInputs = new HashMap<>();
     additionalInputs.put(new TupleTag<String>() {}, pipeline.apply(Create.of("1, 2", "3")));
     additionalInputs.put(new TupleTag<Long>() {}, pipeline.apply(GenerateSequence.from(3L)));
 
@@ -127,7 +127,7 @@
             "additional-only",
             additionalInputs,
             Collections.emptyMap(),
-            new TestTransform(additionalInputs),
+            new TestTransform((Map) additionalInputs),
             pipeline);
 
     thrown.expect(IllegalArgumentException.class);
diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/TriggerTranslationTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/TriggerTranslationTest.java
index 3dfc233..3ed054f 100644
--- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/TriggerTranslationTest.java
+++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/TriggerTranslationTest.java
@@ -43,6 +43,7 @@
 
 /** Tests for utilities in {@link TriggerTranslation}. */
 @RunWith(Parameterized.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class TriggerTranslationTest {
 
   @AutoValue
diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/UnboundedReadFromBoundedSourceTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/UnboundedReadFromBoundedSourceTest.java
index 57aadc7..9d0fcdc 100644
--- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/UnboundedReadFromBoundedSourceTest.java
+++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/UnboundedReadFromBoundedSourceTest.java
@@ -72,6 +72,7 @@
 
 /** Unit tests for {@link UnboundedReadFromBoundedSource}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class UnboundedReadFromBoundedSourceTest {
 
   @Rule public TemporaryFolder tmpFolder = new TemporaryFolder();
diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/UnsupportedOverrideFactoryTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/UnsupportedOverrideFactoryTest.java
index 18016ff..8b3e37c 100644
--- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/UnsupportedOverrideFactoryTest.java
+++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/UnsupportedOverrideFactoryTest.java
@@ -28,6 +28,7 @@
 
 /** Tests for {@link UnsupportedOverrideFactory}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class UnsupportedOverrideFactoryTest {
   @Rule public ExpectedException thrown = ExpectedException.none();
 
diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ValidateRunnerXlangTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ValidateRunnerXlangTest.java
index 1ae2993..3790d7c 100644
--- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ValidateRunnerXlangTest.java
+++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ValidateRunnerXlangTest.java
@@ -85,6 +85,7 @@
  * details.
  */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ValidateRunnerXlangTest implements Serializable {
   @Rule public transient TestPipeline testPipeline = TestPipeline.create();
   private PipelineResult pipelineResult;
diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/WindowingStrategyTranslationTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/WindowingStrategyTranslationTest.java
index a588607..c8eaecd 100644
--- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/WindowingStrategyTranslationTest.java
+++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/WindowingStrategyTranslationTest.java
@@ -42,6 +42,7 @@
 
 /** Unit tests for {@link WindowingStrategy}. */
 @RunWith(Parameterized.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class WindowingStrategyTranslationTest {
 
   // Each spec activates tests of all subsets of its fields
diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/WriteFilesTranslationTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/WriteFilesTranslationTest.java
index c072c84..ca02743 100644
--- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/WriteFilesTranslationTest.java
+++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/WriteFilesTranslationTest.java
@@ -38,6 +38,7 @@
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
 import org.apache.beam.sdk.transforms.windowing.PaneInfo;
 import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PValues;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
 import org.checkerframework.checker.nullness.qual.Nullable;
 import org.junit.Test;
@@ -90,7 +91,8 @@
 
     AppliedPTransform<PCollection<String>, WriteFilesResult<Void>, WriteFiles<String, Void, String>>
         appliedPTransform =
-            AppliedPTransform.of("foo", input.expand(), output.expand(), writeFiles, p);
+            AppliedPTransform.of(
+                "foo", PValues.expandInput(input), PValues.expandOutput(output), writeFiles, p);
 
     assertThat(
         WriteFilesTranslation.isRunnerDeterminedSharding(appliedPTransform),
diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/GreedyPipelineFuserTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/GreedyPipelineFuserTest.java
index 78cba31..fa92ea1 100644
--- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/GreedyPipelineFuserTest.java
+++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/GreedyPipelineFuserTest.java
@@ -60,6 +60,7 @@
 
 /** Tests for {@link GreedyPipelineFuser}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class GreedyPipelineFuserTest {
   // Contains the 'go' and 'py' environments, and a default 'impulse' step and output.
   private Components partialComponents;
diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/GreedyStageFuserTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/GreedyStageFuserTest.java
index c60f5e1..33b26ae 100644
--- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/GreedyStageFuserTest.java
+++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/GreedyStageFuserTest.java
@@ -54,6 +54,7 @@
 
 /** Tests for {@link GreedyStageFuser}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class GreedyStageFuserTest {
   @Rule public ExpectedException thrown = ExpectedException.none();
 
diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/OutputDeduplicatorTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/OutputDeduplicatorTest.java
index 8d45782..c311614 100644
--- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/OutputDeduplicatorTest.java
+++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/OutputDeduplicatorTest.java
@@ -49,6 +49,7 @@
 
 /** Tests for {@link OutputDeduplicator}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class OutputDeduplicatorTest {
   @Test
   public void unchangedWithNoDuplicates() {
diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/resources/ClasspathScanningResourcesDetectorTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/resources/ClasspathScanningResourcesDetectorTest.java
index afd0472..f3758b7 100644
--- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/resources/ClasspathScanningResourcesDetectorTest.java
+++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/resources/ClasspathScanningResourcesDetectorTest.java
@@ -40,6 +40,7 @@
 import org.junit.rules.TemporaryFolder;
 import org.mockito.Mockito;
 
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ClasspathScanningResourcesDetectorTest {
 
   @Rule public transient TemporaryFolder tmpFolder = new TemporaryFolder();
diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/resources/PipelineResourcesTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/resources/PipelineResourcesTest.java
index b84d3e7..8d2ff5a 100644
--- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/resources/PipelineResourcesTest.java
+++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/resources/PipelineResourcesTest.java
@@ -41,6 +41,7 @@
 
 /** Tests for PipelineResources. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class PipelineResourcesTest {
 
   @Rule public transient TemporaryFolder tmpFolder = new TemporaryFolder();
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunners.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunners.java
index 0b8d181..a90e241 100644
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunners.java
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunners.java
@@ -37,6 +37,7 @@
 import org.checkerframework.checker.nullness.qual.Nullable;
 
 /** Static utility methods that provide {@link DoFnRunner} implementations. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DoFnRunners {
   /** Information about how to create output receivers and output to them. */
   public interface OutputManager {
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/GlobalCombineFnRunners.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/GlobalCombineFnRunners.java
index 94e427e..0c8f5cc 100644
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/GlobalCombineFnRunners.java
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/GlobalCombineFnRunners.java
@@ -31,6 +31,7 @@
  * Static utility methods that provide {@link GlobalCombineFnRunner} implementations for different
  * combine functions.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class GlobalCombineFnRunners {
   /** Returns a {@link GlobalCombineFnRunner} from a {@link GlobalCombineFn}. */
   public static <InputT, AccumT, OutputT> GlobalCombineFnRunner<InputT, AccumT, OutputT> create(
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/InMemoryMultimapSideInputView.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/InMemoryMultimapSideInputView.java
index d7a484c..2e8f85f 100644
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/InMemoryMultimapSideInputView.java
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/InMemoryMultimapSideInputView.java
@@ -31,6 +31,7 @@
 import org.checkerframework.checker.nullness.qual.Nullable;
 
 /** An in-memory representation of {@link MultimapView}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class InMemoryMultimapSideInputView<K, V> implements Materializations.MultimapView<K, V> {
   /** An empty {@link MultimapView}. */
   private static final MultimapView EMPTY =
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/InMemoryStateInternals.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/InMemoryStateInternals.java
index 999042f..d540849 100644
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/InMemoryStateInternals.java
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/InMemoryStateInternals.java
@@ -61,6 +61,7 @@
  * In-memory implementation of {@link StateInternals}. Used in {@code BatchModeExecutionContext} and
  * for running tests that need state.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class InMemoryStateInternals<K> implements StateInternals {
 
   public static <K> InMemoryStateInternals<K> forKey(@Nullable K key) {
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/InMemoryTimerInternals.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/InMemoryTimerInternals.java
index a3f2acf..8218905 100644
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/InMemoryTimerInternals.java
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/InMemoryTimerInternals.java
@@ -34,6 +34,7 @@
 import org.joda.time.Instant;
 
 /** {@link TimerInternals} with all watermarks and processing clock simulated in-memory. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class InMemoryTimerInternals implements TimerInternals {
 
   /** The current set timers by namespace and ID. */
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/LateDataDroppingDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/LateDataDroppingDoFnRunner.java
index fee1bfa..7bac3d1 100644
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/LateDataDroppingDoFnRunner.java
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/LateDataDroppingDoFnRunner.java
@@ -45,6 +45,7 @@
  * @param <OutputT> output value element type
  * @param <W> window type
  */
+@SuppressWarnings({"nullness", "keyfor"}) // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class LateDataDroppingDoFnRunner<K, InputT, OutputT, W extends BoundedWindow>
     implements DoFnRunner<KeyedWorkItem<K, InputT>, KV<K, OutputT>> {
   private final DoFnRunner<KeyedWorkItem<K, InputT>, KV<K, OutputT>> doFnRunner;
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/LateDataUtils.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/LateDataUtils.java
index 0e0eab7..76f0d16 100644
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/LateDataUtils.java
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/LateDataUtils.java
@@ -29,6 +29,7 @@
 import org.joda.time.Instant;
 
 /** Utils to handle late data. */
+@SuppressWarnings({"nullness", "keyfor"}) // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class LateDataUtils {
   private LateDataUtils() {}
 
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/MergingActiveWindowSet.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/MergingActiveWindowSet.java
index e8ea758..380d0ca7 100644
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/MergingActiveWindowSet.java
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/MergingActiveWindowSet.java
@@ -40,6 +40,7 @@
 import org.checkerframework.checker.nullness.qual.Nullable;
 
 /** An {@link ActiveWindowSet} for merging {@link WindowFn} implementations. */
+@SuppressWarnings({"nullness", "keyfor"}) // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class MergingActiveWindowSet<W extends BoundedWindow> implements ActiveWindowSet<W> {
   private final WindowFn<Object, W> windowFn;
 
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 c8822de..c8596bb 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
@@ -58,6 +58,7 @@
  * DoFn.ProcessElement} call either outputs at least a given number of elements (in total over all
  * outputs), or runs for the given duration.
  */
+@SuppressWarnings({"nullness", "keyfor"}) // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class OutputAndTimeBoundedSplittableProcessElementInvoker<
         InputT, OutputT, RestrictionT, PositionT, WatermarkEstimatorStateT>
     extends SplittableProcessElementInvoker<
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/PaneInfoTracker.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/PaneInfoTracker.java
index 6c216f5..8e922c5 100644
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/PaneInfoTracker.java
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/PaneInfoTracker.java
@@ -36,6 +36,7 @@
  * AfterWatermark} trigger firing, and the relation between the element's timestamp and the current
  * output watermark.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class PaneInfoTracker {
   private TimerInternals timerInternals;
 
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/PeekingReiterator.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/PeekingReiterator.java
index 376e70f..36b8bc9 100644
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/PeekingReiterator.java
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/PeekingReiterator.java
@@ -29,6 +29,7 @@
  *
  * @param <T> the type of elements returned by this iterator
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public final class PeekingReiterator<T> implements Reiterator<T> {
   private @Nullable T nextElement;
   private boolean nextElementComputed;
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/ProcessFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/ProcessFnRunner.java
index 707eda7..4b6ed65 100644
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/ProcessFnRunner.java
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/ProcessFnRunner.java
@@ -37,6 +37,7 @@
  * Runs a {@link SplittableParDoViaKeyedWorkItems.ProcessFn} by constructing the appropriate
  * contexts and passing them in.
  */
+@SuppressWarnings({"nullness", "keyfor"}) // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ProcessFnRunner<InputT, OutputT, RestrictionT>
     implements PushbackSideInputDoFnRunner<
         KeyedWorkItem<byte[], KV<InputT, RestrictionT>>, OutputT> {
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnContextFactory.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnContextFactory.java
index e786b5e..96c259e 100644
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnContextFactory.java
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnContextFactory.java
@@ -40,6 +40,7 @@
 import org.joda.time.Instant;
 
 /** Factory for creating instances of the various {@link ReduceFn} contexts. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class ReduceFnContextFactory<K, InputT, OutputT, W extends BoundedWindow> {
   public interface OnTriggerCallbacks<OutputT> {
     void output(OutputT toOutput);
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnRunner.java
index ff41b4d..8814408 100644
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnRunner.java
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnRunner.java
@@ -83,6 +83,7 @@
  * @param <OutputT> The output type that will be produced for each key.
  * @param <W> The type of windows this operates on.
  */
+@SuppressWarnings({"nullness", "keyfor"}) // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ReduceFnRunner<K, InputT, OutputT, W extends BoundedWindow> {
 
   /**
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/SideInputHandler.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/SideInputHandler.java
index 07346a7..cea7f9b 100644
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/SideInputHandler.java
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/SideInputHandler.java
@@ -58,6 +58,7 @@
  * data. For now, this will never clean up side-input data because we have no way of knowing when we
  * reach the GC horizon.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SideInputHandler implements ReadyCheckingSideInputReader {
   private static final Set<String> SUPPORTED_MATERIALIZATIONS =
       ImmutableSet.of(
@@ -149,9 +150,8 @@
     }
   }
 
-  @Nullable
   @Override
-  public <T> T get(PCollectionView<T> view, BoundedWindow window) {
+  public <T> @Nullable T get(PCollectionView<T> view, BoundedWindow window) {
     Iterable<?> elements = getIterable(view, window);
     switch (view.getViewFn().getMaterialization().getUrn()) {
       case Materializations.ITERABLE_MATERIALIZATION_URN:
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java
index 0e69c2b..e287b09 100644
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java
@@ -76,6 +76,7 @@
  * @param <InputT> the type of the {@link DoFn} (main) input elements
  * @param <OutputT> the type of the {@link DoFn} (main) output elements
  */
+@SuppressWarnings({"nullness", "keyfor"}) // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SimpleDoFnRunner<InputT, OutputT> implements DoFnRunner<InputT, OutputT> {
 
   private final PipelineOptions options;
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimplePushbackSideInputDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimplePushbackSideInputDoFnRunner.java
index 69fd03b..5004337 100644
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimplePushbackSideInputDoFnRunner.java
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimplePushbackSideInputDoFnRunner.java
@@ -35,6 +35,7 @@
  * A {@link DoFnRunner} that can refuse to process elements that are not ready, instead returning
  * them via the {@link #processElementInReadyWindows(WindowedValue)}.
  */
+@SuppressWarnings({"nullness", "keyfor"}) // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SimplePushbackSideInputDoFnRunner<InputT, OutputT>
     implements PushbackSideInputDoFnRunner<InputT, OutputT> {
   private final DoFnRunner<InputT, OutputT> underlying;
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableParDoViaKeyedWorkItems.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableParDoViaKeyedWorkItems.java
index 3d9e196..a9856f0 100644
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableParDoViaKeyedWorkItems.java
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableParDoViaKeyedWorkItems.java
@@ -51,7 +51,6 @@
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PCollectionTuple;
 import org.apache.beam.sdk.values.PCollectionView;
-import org.apache.beam.sdk.values.PValue;
 import org.apache.beam.sdk.values.TupleTag;
 import org.apache.beam.sdk.values.TupleTagList;
 import org.apache.beam.sdk.values.WindowingStrategy;
@@ -125,8 +124,8 @@
     }
 
     @Override
-    public Map<PValue, ReplacementOutput> mapOutputs(
-        Map<TupleTag<?>, PValue> outputs, PCollectionTuple newOutput) {
+    public Map<PCollection<?>, ReplacementOutput> mapOutputs(
+        Map<TupleTag<?>, PCollection<?>> outputs, PCollectionTuple newOutput) {
       return ReplacementOutputs.tagged(outputs, newOutput);
     }
   }
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/StateMerging.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/StateMerging.java
index f667bd4..eaacab1 100644
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/StateMerging.java
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/StateMerging.java
@@ -31,6 +31,7 @@
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
 
 /** Helpers for merging state. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class StateMerging {
   /**
    * Clear all state in {@code address} in all windows under merge (even result windows) in {@code
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/StateTable.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/StateTable.java
index f8c2aa6..1bccc91 100644
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/StateTable.java
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/StateTable.java
@@ -35,6 +35,7 @@
  * information carried by the {@link StateTag} is used to configure an empty state cell if it is not
  * yet initialized.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public abstract class StateTable {
 
   private final Table<StateNamespace, Equivalence.Wrapper<StateTag>, State> stateTable =
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/StatefulDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/StatefulDoFnRunner.java
index 7d62fe3..a8f1bc8 100644
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/StatefulDoFnRunner.java
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/StatefulDoFnRunner.java
@@ -53,6 +53,7 @@
  * @param <InputT> the type of the {@link DoFn} (main) input elements
  * @param <OutputT> the type of the {@link DoFn} (main) output elements
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class StatefulDoFnRunner<InputT, OutputT, W extends BoundedWindow>
     implements DoFnRunner<InputT, OutputT> {
 
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/SystemReduceFn.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/SystemReduceFn.java
index 89e8a63..d4d06a1 100644
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/SystemReduceFn.java
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/SystemReduceFn.java
@@ -38,6 +38,7 @@
  * @param <OutputT> The output type that will be produced for each key.
  * @param <W> The type of windows this operates on.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public abstract class SystemReduceFn<K, InputT, AccumT, OutputT, W extends BoundedWindow>
     extends ReduceFn<K, InputT, OutputT, W> {
   private static final String BUFFER_NAME = "buf";
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/TestInMemoryStateInternals.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/TestInMemoryStateInternals.java
index 7e2f62c..a6b2ecd 100644
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/TestInMemoryStateInternals.java
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/TestInMemoryStateInternals.java
@@ -25,6 +25,7 @@
 import org.joda.time.Instant;
 
 /** Simulates state like {@link InMemoryStateInternals} and provides some extra helper methods. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class TestInMemoryStateInternals<K> extends InMemoryStateInternals<K> {
   public TestInMemoryStateInternals(K key) {
     super(key);
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/WatermarkHold.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/WatermarkHold.java
index 1498e77..72b874e 100644
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/WatermarkHold.java
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/WatermarkHold.java
@@ -50,6 +50,7 @@
  *
  * @param <W> The kind of {@link BoundedWindow} the hold is for.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class WatermarkHold<W extends BoundedWindow> implements Serializable {
   /** Return tag for state containing the output watermark hold used for elements. */
   public static <W extends BoundedWindow>
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/DefaultMetricResults.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/DefaultMetricResults.java
index f2a553b..0ad06a5 100644
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/DefaultMetricResults.java
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/DefaultMetricResults.java
@@ -32,6 +32,7 @@
  * {@link Iterable}s of counters, distributions, and gauges, and serves queries by applying {@link
  * org.apache.beam.sdk.metrics.MetricsFilter}s linearly to them.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DefaultMetricResults extends MetricResults {
 
   private final Iterable<MetricResult<Long>> counters;
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/ExecutionStateTracker.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/ExecutionStateTracker.java
index a8e805d..111b0d4 100644
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/ExecutionStateTracker.java
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/ExecutionStateTracker.java
@@ -31,6 +31,7 @@
 
 /** Tracks the current state of a single execution thread. */
 @SuppressFBWarnings(value = "IS2_INCONSISTENT_SYNC", justification = "Intentional for performance.")
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ExecutionStateTracker implements Comparable<ExecutionStateTracker> {
 
   /**
@@ -112,7 +113,7 @@
    * <p>This variable is written by the Execution thread, and read by the sampling and progress
    * reporting threads, thus it being marked volatile.
    */
-  private @Nullable volatile ExecutionState currentState;
+  private volatile @Nullable ExecutionState currentState;
 
   /**
    * The current number of times that this {@link ExecutionStateTracker} has transitioned state.
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/MetricsContainerImpl.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/MetricsContainerImpl.java
index a64af89..ce266cd 100644
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/MetricsContainerImpl.java
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/MetricsContainerImpl.java
@@ -55,6 +55,7 @@
  * <p>For consistency, all threads that update metrics should finish before getting the final
  * cumulative values/updates.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class MetricsContainerImpl implements Serializable, MetricsContainer {
   private static final Logger LOG = LoggerFactory.getLogger(MetricsContainerImpl.class);
 
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/MetricsMap.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/MetricsMap.java
index eb7d1df..1807485 100644
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/MetricsMap.java
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/MetricsMap.java
@@ -30,6 +30,7 @@
  * A map from {@code K} to {@code T} that supports getting or creating values associated with a key
  * in a thread-safe manner.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class MetricsMap<K, T> implements Serializable {
 
   /** Interface for creating instances to populate the {@link MetricsMap}. */
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/MetricsPusher.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/MetricsPusher.java
index a04a209..5d378e9 100644
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/MetricsPusher.java
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/MetricsPusher.java
@@ -35,6 +35,7 @@
 import org.checkerframework.checker.nullness.qual.Nullable;
 
 /** Component that regularly merges metrics and pushes them to a metrics sink. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class MetricsPusher implements Serializable {
 
   private MetricsSink metricsSink;
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/MonitoringInfoMetricName.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/MonitoringInfoMetricName.java
index a3548b8..bcbd63a 100644
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/MonitoringInfoMetricName.java
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/MonitoringInfoMetricName.java
@@ -33,6 +33,7 @@
  * key instead of only a name+namespace. This is useful when defining system defined metrics with a
  * specific urn via a {@code CounterContainer}.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class MonitoringInfoMetricName extends MetricName {
 
   private String urn;
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/SimpleExecutionState.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/SimpleExecutionState.java
index 881261c..d784db4 100644
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/SimpleExecutionState.java
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/SimpleExecutionState.java
@@ -33,6 +33,7 @@
  * set of key value labels in the object which can be retrieved later for reporting purposes via
  * getLabels().
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SimpleExecutionState extends ExecutionState {
   private long totalMillis = 0;
   private HashMap<String, String> labelsMetadata;
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/SimpleStateRegistry.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/SimpleStateRegistry.java
index 1d65e31..15e39aa 100644
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/SimpleStateRegistry.java
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/SimpleStateRegistry.java
@@ -25,6 +25,7 @@
 /**
  * A Class for registering SimpleExecutionStates with and extracting execution time MonitoringInfos.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SimpleStateRegistry {
   private List<SimpleExecutionState> executionStates = new ArrayList<SimpleExecutionState>();
 
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterAllStateMachine.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterAllStateMachine.java
index 33f35b9..70d10ff 100644
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterAllStateMachine.java
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterAllStateMachine.java
@@ -28,6 +28,7 @@
  * A {@link TriggerStateMachine} that fires and finishes once after all of its sub-triggers have
  * fired.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class AfterAllStateMachine extends TriggerStateMachine {
 
   private AfterAllStateMachine(List<TriggerStateMachine> subTriggers) {
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterDelayFromFirstElementStateMachine.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterDelayFromFirstElementStateMachine.java
index b980a96..291c9e2 100644
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterDelayFromFirstElementStateMachine.java
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterDelayFromFirstElementStateMachine.java
@@ -47,6 +47,7 @@
  */
 // This class should be inlined to subclasses and deleted, simplifying them too
 // https://issues.apache.org/jira/browse/BEAM-1486
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public abstract class AfterDelayFromFirstElementStateMachine extends TriggerStateMachine {
 
   protected static final List<SerializableFunction<Instant, Instant>> IDENTITY = ImmutableList.of();
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterEachStateMachine.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterEachStateMachine.java
index 5f039d8..0dc063e 100644
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterEachStateMachine.java
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterEachStateMachine.java
@@ -41,6 +41,7 @@
  *       Repeatedly.forever(a)}, since the repeated trigger never finishes.
  * </ul>
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class AfterEachStateMachine extends TriggerStateMachine {
 
   private AfterEachStateMachine(List<TriggerStateMachine> subTriggers) {
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterFirstStateMachine.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterFirstStateMachine.java
index 038f072..a724700 100644
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterFirstStateMachine.java
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterFirstStateMachine.java
@@ -28,6 +28,7 @@
  * Create a composite {@link TriggerStateMachine} that fires once after at least one of its
  * sub-triggers have fired.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class AfterFirstStateMachine extends TriggerStateMachine {
 
   AfterFirstStateMachine(List<TriggerStateMachine> subTriggers) {
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterWatermarkStateMachine.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterWatermarkStateMachine.java
index b7be021..1ffa7e9 100644
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterWatermarkStateMachine.java
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/AfterWatermarkStateMachine.java
@@ -51,6 +51,7 @@
  * AfterWatermark.pastEndOfWindow.withEarlyFirings(OnceTrigger)} or {@code
  * AfterWatermark.pastEndOfWindow.withEarlyFirings(OnceTrigger)}.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class AfterWatermarkStateMachine {
 
   private static final String TO_STRING = "AfterWatermark.pastEndOfWindow()";
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/DefaultTriggerStateMachine.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/DefaultTriggerStateMachine.java
index 719172d..c1396d7 100644
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/DefaultTriggerStateMachine.java
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/DefaultTriggerStateMachine.java
@@ -24,6 +24,7 @@
  * {@link RepeatedlyStateMachine#forever} and {@link AfterWatermarkStateMachine#pastEndOfWindow} for
  * more details.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DefaultTriggerStateMachine extends TriggerStateMachine {
 
   private DefaultTriggerStateMachine() {
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/ExecutableTriggerStateMachine.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/ExecutableTriggerStateMachine.java
index 9c14e5a..2df2012 100644
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/ExecutableTriggerStateMachine.java
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/ExecutableTriggerStateMachine.java
@@ -30,6 +30,7 @@
  * times (both in the same trigger expression and in other trigger expressions), the {@code
  * ExecutableTrigger} wrapped around them forms a tree (only one occurrence).
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ExecutableTriggerStateMachine implements Serializable {
 
   /** Store the index assigned to this trigger. */
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/OrFinallyStateMachine.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/OrFinallyStateMachine.java
index 3a6c1cd..bc42443 100644
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/OrFinallyStateMachine.java
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/OrFinallyStateMachine.java
@@ -23,6 +23,7 @@
 /**
  * Executes the {@code actual} trigger until it finishes or until the {@code until} trigger fires.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class OrFinallyStateMachine extends TriggerStateMachine {
 
   private static final int ACTUAL = 0;
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/RepeatedlyStateMachine.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/RepeatedlyStateMachine.java
index 7089503..ab1fd8b 100644
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/RepeatedlyStateMachine.java
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/RepeatedlyStateMachine.java
@@ -31,6 +31,7 @@
  * <p>{@code Repeatedly.forever(someTrigger)} behaves like an infinite {@code
  * AfterEach.inOrder(someTrigger, someTrigger, someTrigger, ...)}.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class RepeatedlyStateMachine extends TriggerStateMachine {
 
   private static final int REPEATED = 0;
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/TriggerStateMachine.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/TriggerStateMachine.java
index ebf5c7a..7d2ad9b 100644
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/TriggerStateMachine.java
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/TriggerStateMachine.java
@@ -93,6 +93,7 @@
  * invocations of the callbacks. All important values should be persisted using state before the
  * callback returns.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public abstract class TriggerStateMachine implements Serializable {
 
   /**
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/TriggerStateMachineContextFactory.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/TriggerStateMachineContextFactory.java
index 06e1e1d..ca5477d 100644
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/TriggerStateMachineContextFactory.java
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/triggers/TriggerStateMachineContextFactory.java
@@ -46,6 +46,7 @@
  * <p>These contexts are highly interdependent and share many fields; it is inadvisable to create
  * them via any means other than this factory class.
  */
+@SuppressWarnings({"nullness", "keyfor"}) // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class TriggerStateMachineContextFactory<W extends BoundedWindow> {
 
   private final WindowFn<?, W> windowFn;
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/InMemoryTimerInternalsTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/InMemoryTimerInternalsTest.java
index bc997c3..669fe70 100644
--- a/runners/core-java/src/test/java/org/apache/beam/runners/core/InMemoryTimerInternalsTest.java
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/InMemoryTimerInternalsTest.java
@@ -30,6 +30,7 @@
 
 /** Tests for {@link InMemoryTimerInternals}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class InMemoryTimerInternalsTest {
 
   private static final StateNamespace NS1 = new StateNamespaceForTest("NS1");
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/LateDataDroppingDoFnRunnerTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/LateDataDroppingDoFnRunnerTest.java
index 9ae6bbe..8b61cec 100644
--- a/runners/core-java/src/test/java/org/apache/beam/runners/core/LateDataDroppingDoFnRunnerTest.java
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/LateDataDroppingDoFnRunnerTest.java
@@ -44,6 +44,7 @@
 
 /** Unit tests for {@link LateDataDroppingDoFnRunner}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class LateDataDroppingDoFnRunnerTest {
   private static final FixedWindows WINDOW_FN = FixedWindows.of(Duration.millis(10));
 
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/OutputAndTimeBoundedSplittableProcessElementInvokerTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/OutputAndTimeBoundedSplittableProcessElementInvokerTest.java
index 645e1e6..15614a2 100644
--- a/runners/core-java/src/test/java/org/apache/beam/runners/core/OutputAndTimeBoundedSplittableProcessElementInvokerTest.java
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/OutputAndTimeBoundedSplittableProcessElementInvokerTest.java
@@ -50,6 +50,7 @@
 import org.junit.rules.ExpectedException;
 
 /** Tests for {@link OutputAndTimeBoundedSplittableProcessElementInvoker}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class OutputAndTimeBoundedSplittableProcessElementInvokerTest {
   @Rule public transient ExpectedException e = ExpectedException.none();
 
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnTester.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnTester.java
index eeba452..435d0fb 100644
--- a/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnTester.java
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnTester.java
@@ -83,6 +83,7 @@
  *     Iterable<InputT>})
  * @param <W> The type of windows being used.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ReduceFnTester<InputT, OutputT, W extends BoundedWindow> {
   private static final String KEY = "TEST_KEY";
 
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/SideInputHandlerTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/SideInputHandlerTest.java
index 5b63fe5..3641e8c 100644
--- a/runners/core-java/src/test/java/org/apache/beam/runners/core/SideInputHandlerTest.java
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/SideInputHandlerTest.java
@@ -45,6 +45,7 @@
 
 /** Unit tests for {@link SideInputHandler}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SideInputHandlerTest {
 
   private static final long WINDOW_MSECS_1 = 100;
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/SimpleDoFnRunnerTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/SimpleDoFnRunnerTest.java
index 9daf372..5d40e10 100644
--- a/runners/core-java/src/test/java/org/apache/beam/runners/core/SimpleDoFnRunnerTest.java
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/SimpleDoFnRunnerTest.java
@@ -61,6 +61,7 @@
 
 /** Tests for {@link SimpleDoFnRunner}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SimpleDoFnRunnerTest {
   @Rule public ExpectedException thrown = ExpectedException.none();
 
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/SimplePushbackSideInputDoFnRunnerTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/SimplePushbackSideInputDoFnRunnerTest.java
index da4d482..b5940de 100644
--- a/runners/core-java/src/test/java/org/apache/beam/runners/core/SimplePushbackSideInputDoFnRunnerTest.java
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/SimplePushbackSideInputDoFnRunnerTest.java
@@ -77,6 +77,7 @@
 
 /** Tests for {@link SimplePushbackSideInputDoFnRunner}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SimplePushbackSideInputDoFnRunnerTest {
   @Mock StepContext mockStepContext;
   @Mock private ReadyCheckingSideInputReader reader;
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 b3d085c..9167454 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
@@ -77,6 +77,7 @@
 
 /** Tests for {@link SplittableParDoViaKeyedWorkItems.ProcessFn}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SplittableParDoProcessFnTest {
   private static final int MAX_OUTPUTS_PER_BUNDLE = 10000;
   private static final Duration MAX_BUNDLE_DURATION = Duration.standardSeconds(5);
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/StateInternalsTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/StateInternalsTest.java
index 3d18405..5216ad8 100644
--- a/runners/core-java/src/test/java/org/apache/beam/runners/core/StateInternalsTest.java
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/StateInternalsTest.java
@@ -60,6 +60,7 @@
 import org.junit.Test;
 
 /** Tests for {@link StateInternals}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public abstract class StateInternalsTest {
 
   private static final BoundedWindow WINDOW_1 = new IntervalWindow(new Instant(0), new Instant(10));
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/StatefulDoFnRunnerTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/StatefulDoFnRunnerTest.java
index ccf9d98..30206c3 100644
--- a/runners/core-java/src/test/java/org/apache/beam/runners/core/StatefulDoFnRunnerTest.java
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/StatefulDoFnRunnerTest.java
@@ -61,6 +61,7 @@
 
 /** Tests for {@link StatefulDoFnRunnerTest}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class StatefulDoFnRunnerTest {
 
   private static final long WINDOW_SIZE = 10;
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/WindowMatchers.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/WindowMatchers.java
index d934f72..8d6edef 100644
--- a/runners/core-java/src/test/java/org/apache/beam/runners/core/WindowMatchers.java
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/WindowMatchers.java
@@ -31,6 +31,7 @@
 import org.joda.time.Instant;
 
 /** Matchers that are useful for working with Windowing, Timestamps, etc. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class WindowMatchers {
 
   public static <T> Matcher<WindowedValue<? extends T>> isWindowedValue(
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/metrics/ExecutionStateSamplerTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/metrics/ExecutionStateSamplerTest.java
index 6ce515f..dcb2b48 100644
--- a/runners/core-java/src/test/java/org/apache/beam/runners/core/metrics/ExecutionStateSamplerTest.java
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/metrics/ExecutionStateSamplerTest.java
@@ -29,6 +29,7 @@
 import org.junit.Test;
 
 /** Tests for {@link org.apache.beam.runners.core.metrics.ExecutionStateSampler}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ExecutionStateSamplerTest {
 
   private MillisProvider clock;
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/metrics/ExecutionStateTrackerTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/metrics/ExecutionStateTrackerTest.java
index 5bc4f04..4cdf85e 100644
--- a/runners/core-java/src/test/java/org/apache/beam/runners/core/metrics/ExecutionStateTrackerTest.java
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/metrics/ExecutionStateTrackerTest.java
@@ -29,6 +29,7 @@
 import org.junit.Test;
 
 /** Tests for {@link ExecutionStateTracker}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ExecutionStateTrackerTest {
 
   private MillisProvider clock;
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/metrics/LabeledMetricsTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/metrics/LabeledMetricsTest.java
index 0e6f692..598bdb8 100644
--- a/runners/core-java/src/test/java/org/apache/beam/runners/core/metrics/LabeledMetricsTest.java
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/metrics/LabeledMetricsTest.java
@@ -32,6 +32,7 @@
 import org.mockito.Mockito;
 
 /** Tests for {@link LabeledMetrics}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class LabeledMetricsTest implements Serializable {
 
   @Rule public final transient ExpectedException thrown = ExpectedException.none();
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/metrics/MetricUpdateMatchers.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/metrics/MetricUpdateMatchers.java
index 9b8a436..a37ac44 100644
--- a/runners/core-java/src/test/java/org/apache/beam/runners/core/metrics/MetricUpdateMatchers.java
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/metrics/MetricUpdateMatchers.java
@@ -24,6 +24,7 @@
 import org.hamcrest.TypeSafeMatcher;
 
 /** Matchers for {@link MetricUpdate}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class MetricUpdateMatchers {
 
   /**
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/metrics/MetricsContainerImplTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/metrics/MetricsContainerImplTest.java
index e0cdb5f1..1cf7c4a 100644
--- a/runners/core-java/src/test/java/org/apache/beam/runners/core/metrics/MetricsContainerImplTest.java
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/metrics/MetricsContainerImplTest.java
@@ -35,6 +35,7 @@
 
 /** Tests for {@link MetricsContainerImpl}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class MetricsContainerImplTest {
 
   @Test
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/metrics/MetricsContainerStepMapTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/metrics/MetricsContainerStepMapTest.java
index 626b3ae..d343380 100644
--- a/runners/core-java/src/test/java/org/apache/beam/runners/core/metrics/MetricsContainerStepMapTest.java
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/metrics/MetricsContainerStepMapTest.java
@@ -50,6 +50,7 @@
 import org.slf4j.LoggerFactory;
 
 /** Tests for {@link MetricsContainerStepMap}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class MetricsContainerStepMapTest {
   private static final Logger LOG = LoggerFactory.getLogger(MetricsContainerStepMapTest.class);
 
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 f36fa20..7a2397c 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
@@ -45,6 +45,7 @@
 /** A test that verifies that metrics push system works. */
 @Category({UsesMetricsPusher.class, ValidatesRunner.class})
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class MetricsPusherTest {
   private static final Logger LOG = LoggerFactory.getLogger(MetricsPusherTest.class);
 
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/metrics/MonitoringInfoMetricNameTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/metrics/MonitoringInfoMetricNameTest.java
index 21f0993..b7aa76d 100644
--- a/runners/core-java/src/test/java/org/apache/beam/runners/core/metrics/MonitoringInfoMetricNameTest.java
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/metrics/MonitoringInfoMetricNameTest.java
@@ -27,6 +27,7 @@
 import org.junit.rules.ExpectedException;
 
 /** Tests for {@link MonitoringInfoMetricName}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class MonitoringInfoMetricNameTest implements Serializable {
 
   @Rule public final transient ExpectedException thrown = ExpectedException.none();
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/metrics/MonitoringInfoTestUtil.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/metrics/MonitoringInfoTestUtil.java
index 0f628bf..c2a4e2d 100644
--- a/runners/core-java/src/test/java/org/apache/beam/runners/core/metrics/MonitoringInfoTestUtil.java
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/metrics/MonitoringInfoTestUtil.java
@@ -23,6 +23,7 @@
 /**
  * Provides convenient one line factories for unit tests that need to generate test MonitoringInfos.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class MonitoringInfoTestUtil {
   /** @return A basic MonitoringInfoMetricName to test. */
   public static MonitoringInfoMetricName testElementCountName() {
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/metrics/SimpleExecutionStateTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/metrics/SimpleExecutionStateTest.java
index f774930..a9d6db1 100644
--- a/runners/core-java/src/test/java/org/apache/beam/runners/core/metrics/SimpleExecutionStateTest.java
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/metrics/SimpleExecutionStateTest.java
@@ -30,6 +30,7 @@
 
 /** Tests for {@link SimpleExecutionState}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SimpleExecutionStateTest {
 
   @Test
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/metrics/SimpleMonitoringInfoBuilderTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/metrics/SimpleMonitoringInfoBuilderTest.java
index c1d2016..d8a1998 100644
--- a/runners/core-java/src/test/java/org/apache/beam/runners/core/metrics/SimpleMonitoringInfoBuilderTest.java
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/metrics/SimpleMonitoringInfoBuilderTest.java
@@ -29,6 +29,7 @@
 
 /** Tests for {@link SimpleMonitoringInfoBuilder}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SimpleMonitoringInfoBuilderTest {
 
   @Test
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/metrics/SimpleStateRegistryTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/metrics/SimpleStateRegistryTest.java
index b6aa6c4..14c3451 100644
--- a/runners/core-java/src/test/java/org/apache/beam/runners/core/metrics/SimpleStateRegistryTest.java
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/metrics/SimpleStateRegistryTest.java
@@ -31,6 +31,7 @@
 import org.junit.Test;
 
 /** Tests for {@link SimpleStateRegistryTest}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SimpleStateRegistryTest {
 
   @Test
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/metrics/TestMetricsSink.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/metrics/TestMetricsSink.java
index 29daa84..2c9aee8 100644
--- a/runners/core-java/src/test/java/org/apache/beam/runners/core/metrics/TestMetricsSink.java
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/metrics/TestMetricsSink.java
@@ -29,6 +29,7 @@
  * This sink just stores in a static field the first counter (if it exists) attempted value. This is
  * useful for tests.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class TestMetricsSink implements MetricsSink {
 
   private static MetricQueryResults metricQueryResults;
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/AfterAllStateMachineTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/AfterAllStateMachineTest.java
index b087492..73123a2 100644
--- a/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/AfterAllStateMachineTest.java
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/AfterAllStateMachineTest.java
@@ -33,6 +33,7 @@
 
 /** Tests for {@link AfterAllStateMachine}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class AfterAllStateMachineTest {
 
   private SimpleTriggerStateMachineTester<IntervalWindow> tester;
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/AfterEachStateMachineTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/AfterEachStateMachineTest.java
index 301cf7d..2c95cc8 100644
--- a/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/AfterEachStateMachineTest.java
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/AfterEachStateMachineTest.java
@@ -34,6 +34,7 @@
 
 /** Tests for {@link AfterEachStateMachine}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class AfterEachStateMachineTest {
 
   private SimpleTriggerStateMachineTester<IntervalWindow> tester;
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/AfterFirstStateMachineTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/AfterFirstStateMachineTest.java
index 37855b2..3bfce9d 100644
--- a/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/AfterFirstStateMachineTest.java
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/AfterFirstStateMachineTest.java
@@ -37,6 +37,7 @@
 
 /** Tests for {@link AfterFirstStateMachine}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class AfterFirstStateMachineTest {
 
   @Mock private TriggerStateMachine mockTrigger1;
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/AfterPaneStateMachineTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/AfterPaneStateMachineTest.java
index 8c1f947..cc6bb5b 100644
--- a/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/AfterPaneStateMachineTest.java
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/AfterPaneStateMachineTest.java
@@ -33,6 +33,7 @@
 
 /** Tests for {@link AfterPaneStateMachine}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class AfterPaneStateMachineTest {
 
   SimpleTriggerStateMachineTester<IntervalWindow> tester;
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/AfterWatermarkStateMachineTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/AfterWatermarkStateMachineTest.java
index 7f60100..3965f51 100644
--- a/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/AfterWatermarkStateMachineTest.java
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/AfterWatermarkStateMachineTest.java
@@ -45,6 +45,7 @@
 
 /** Tests the {@link AfterWatermarkStateMachine} triggers. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class AfterWatermarkStateMachineTest {
 
   @Mock private TriggerStateMachine mockEarly;
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/DefaultTriggerStateMachineTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/DefaultTriggerStateMachineTest.java
index f0dc347..f75ba81 100644
--- a/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/DefaultTriggerStateMachineTest.java
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/DefaultTriggerStateMachineTest.java
@@ -36,6 +36,7 @@
  * Repeatedly.forever(AfterWatermark.pastEndOfWindow())}.
  */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DefaultTriggerStateMachineTest {
 
   SimpleTriggerStateMachineTester<IntervalWindow> tester;
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/NeverStateMachineTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/NeverStateMachineTest.java
index 98c6136..af966c6 100644
--- a/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/NeverStateMachineTest.java
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/NeverStateMachineTest.java
@@ -34,6 +34,7 @@
 
 /** Tests for {@link NeverStateMachine}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class NeverStateMachineTest {
   private SimpleTriggerStateMachineTester<IntervalWindow> triggerTester;
 
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/OrFinallyStateMachineTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/OrFinallyStateMachineTest.java
index dcf0b96..8b7a8a4 100644
--- a/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/OrFinallyStateMachineTest.java
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/OrFinallyStateMachineTest.java
@@ -33,6 +33,7 @@
 
 /** Tests for {@link OrFinallyStateMachine}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class OrFinallyStateMachineTest {
 
   private SimpleTriggerStateMachineTester<IntervalWindow> tester;
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/RepeatedlyStateMachineTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/RepeatedlyStateMachineTest.java
index 556fe68..2ae122e 100644
--- a/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/RepeatedlyStateMachineTest.java
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/RepeatedlyStateMachineTest.java
@@ -41,6 +41,7 @@
 
 /** Tests for {@link RepeatedlyStateMachine}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class RepeatedlyStateMachineTest {
 
   @Mock private TriggerStateMachine mockTrigger;
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/TriggerStateMachineTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/TriggerStateMachineTest.java
index 269dfde..7de94ee 100644
--- a/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/TriggerStateMachineTest.java
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/TriggerStateMachineTest.java
@@ -29,6 +29,7 @@
 
 /** Tests for {@link TriggerStateMachine}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class TriggerStateMachineTest {
 
   @Test
diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/TriggerStateMachineTester.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/TriggerStateMachineTester.java
index c74cb0d..378a597 100644
--- a/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/TriggerStateMachineTester.java
+++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/TriggerStateMachineTester.java
@@ -63,6 +63,7 @@
  *
  * @param <W> The type of windows being used.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class TriggerStateMachineTester<InputT, W extends BoundedWindow> {
 
   /**
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactory.java
index 10f5d7e..973f37e 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactory.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactory.java
@@ -51,6 +51,7 @@
  * A {@link TransformEvaluatorFactory} that produces {@link TransformEvaluator TransformEvaluators}
  * for the {@link PrimitiveBoundedRead SplittableParDo.PrimitiveBoundedRead} {@link PTransform}.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 final class BoundedReadEvaluatorFactory implements TransformEvaluatorFactory {
   /**
    * The required minimum size of a source to dynamically split. Produced {@link TransformEvaluator
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CloningBundleFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CloningBundleFactory.java
index 253c390..e240c64 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CloningBundleFactory.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CloningBundleFactory.java
@@ -31,6 +31,7 @@
  * A {@link BundleFactory} where a created {@link UncommittedBundle} clones all elements added to it
  * using the coder of the {@link PCollection}.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class CloningBundleFactory implements BundleFactory {
   private static final CloningBundleFactory INSTANCE = new CloningBundleFactory();
 
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CommittedResult.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CommittedResult.java
index 0a58fdf..e4ef139 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CommittedResult.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CommittedResult.java
@@ -24,6 +24,7 @@
 
 /** A {@link TransformResult} that has been committed. */
 @AutoValue
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 abstract class CommittedResult<ExecutableT> {
   /** Returns the {@link AppliedPTransform} that produced this result. */
   public abstract ExecutableT getExecutable();
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CopyOnAccessInMemoryStateInternals.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CopyOnAccessInMemoryStateInternals.java
index 362ae34..97b63a9 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CopyOnAccessInMemoryStateInternals.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CopyOnAccessInMemoryStateInternals.java
@@ -62,6 +62,7 @@
  * of {@link InMemoryState}. Whenever state that exists in the underlying {@link StateTable} is
  * accessed, an independent copy will be created within this table.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class CopyOnAccessInMemoryStateInternals<K> implements StateInternals {
   private final CopyOnAccessInMemoryStateTable table;
 
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectExecutionContext.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectExecutionContext.java
index 3ff2dcd..b7142f6 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectExecutionContext.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectExecutionContext.java
@@ -37,6 +37,7 @@
  * <p>This implementation is not thread safe. A new {@link DirectExecutionContext} must be created
  * for each thread that requires it.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class DirectExecutionContext {
   private static final Logger LOG = LoggerFactory.getLogger(DirectExecutionContext.class);
   private final Clock clock;
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGraph.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGraph.java
index 1681f7a..9426286 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGraph.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGraph.java
@@ -34,6 +34,7 @@
  * Methods for interacting with the underlying structure of a {@link Pipeline} that is being
  * executed with the {@link DirectRunner}.
  */
+@SuppressWarnings({"keyfor", "nullness"}) // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class DirectGraph implements ExecutableGraph<AppliedPTransform<?, ?, ?>, PValue> {
   private final Map<PCollection<?>, AppliedPTransform<?, ?, ?>> producers;
   private final Map<PCollectionView<?>, AppliedPTransform<?, ?, ?>> viewWriters;
@@ -84,7 +85,7 @@
   public Collection<PValue> getProduced(AppliedPTransform<?, ?, ?> producer) {
     // TODO: This must only be called on primitive transforms; composites should return empty
     // values.
-    return producer.getOutputs().values();
+    return (Collection) producer.getOutputs().values();
   }
 
   @Override
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGraphVisitor.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGraphVisitor.java
index 552122f..3df64bf 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGraphVisitor.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectGraphVisitor.java
@@ -47,6 +47,7 @@
  * {@link Pipeline}. This is used to schedule consuming {@link PTransform PTransforms} to consume
  * input after the upstream transform has produced and committed output.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class DirectGraphVisitor extends PipelineVisitor.Defaults {
   private static final Logger LOG = LoggerFactory.getLogger(DirectGraphVisitor.class);
 
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectMetrics.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectMetrics.java
index b65ec26..c5817f0 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectMetrics.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectMetrics.java
@@ -45,6 +45,7 @@
 import org.checkerframework.checker.nullness.qual.Nullable;
 
 /** Implementation of {@link MetricResults} for the Direct Runner. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class DirectMetrics extends MetricResults {
 
   private interface MetricAggregation<UpdateT, ResultT> {
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java
index 2a8e3b1..c9f317b 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRunner.java
@@ -64,6 +64,7 @@
  * contained within a {@link Pipeline} does not break assumptions within the Beam model, to improve
  * the ability to execute a {@link Pipeline} at scale on a distributed backend.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DirectRunner extends PipelineRunner<DirectPipelineResult> {
 
   enum Enforcement {
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectWriteViewVisitor.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectWriteViewVisitor.java
index 54d55b0..8bafa4e 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectWriteViewVisitor.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectWriteViewVisitor.java
@@ -36,6 +36,7 @@
  * Adds a {@link DirectRunner}-specific {@link WriteView} step for each {@link PCollectionView} for
  * scheduling materialization of side inputs.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class DirectWriteViewVisitor extends PipelineVisitor.Defaults {
 
   /** Private URN for identifying {@link DirectRunner}-specific view writing transform. */
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DisplayDataValidator.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DisplayDataValidator.java
index 0f846a2..a36046b 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DisplayDataValidator.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DisplayDataValidator.java
@@ -27,6 +27,7 @@
  * Validate correct implementation of {@link DisplayData} by evaluating {@link
  * HasDisplayData#populateDisplayData(DisplayData.Builder)} during pipeline construction.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class DisplayDataValidator {
   // Do not instantiate
   private DisplayDataValidator() {}
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EvaluationContext.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EvaluationContext.java
index f7c2006..c0af527 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EvaluationContext.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/EvaluationContext.java
@@ -71,6 +71,7 @@
  * per-{@link StepAndKey} state, updating global watermarks, and executing any callbacks that can be
  * executed.
  */
+@SuppressWarnings({"keyfor", "nullness"}) // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class EvaluationContext {
   private static final Logger LOG = LoggerFactory.getLogger(EvaluationContext.class);
 
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ExecutorServiceParallelExecutor.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ExecutorServiceParallelExecutor.java
index b75f031..eeb14f1 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ExecutorServiceParallelExecutor.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ExecutorServiceParallelExecutor.java
@@ -56,6 +56,7 @@
  * An {@link PipelineExecutor} that uses an underlying {@link ExecutorService} and {@link
  * EvaluationContext} to execute a {@link Pipeline}.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 final class ExecutorServiceParallelExecutor
     implements PipelineExecutor,
         BundleProcessor<PCollection<?>, CommittedBundle<?>, AppliedPTransform<?, ?, ?>> {
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupAlsoByWindowEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupAlsoByWindowEvaluatorFactory.java
index 55028ec..491a123 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupAlsoByWindowEvaluatorFactory.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupAlsoByWindowEvaluatorFactory.java
@@ -58,6 +58,7 @@
  * The {@link DirectRunner} {@link TransformEvaluatorFactory} for the {@link
  * DirectGroupAlsoByWindow} {@link PTransform}.
  */
+@SuppressWarnings({"keyfor", "nullness"}) // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class GroupAlsoByWindowEvaluatorFactory implements TransformEvaluatorFactory {
   private final EvaluationContext evaluationContext;
   private final PipelineOptions options;
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupByKeyOnlyEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupByKeyOnlyEvaluatorFactory.java
index c4e2e80..2622099 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupByKeyOnlyEvaluatorFactory.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupByKeyOnlyEvaluatorFactory.java
@@ -42,6 +42,7 @@
  * The {@link DirectRunner} {@link TransformEvaluatorFactory} for the {@link GroupByKeyOnly} {@link
  * PTransform}.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class GroupByKeyOnlyEvaluatorFactory implements TransformEvaluatorFactory {
   private final EvaluationContext evaluationContext;
 
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactory.java
index 08430ce..d9ee25e 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactory.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactory.java
@@ -43,6 +43,7 @@
  * <p>This catches errors during the execution of a {@link DoFn} caused by modifying an element
  * after it is added to an output {@link PCollection}.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class ImmutabilityCheckingBundleFactory implements BundleFactory {
   /**
    * Create a new {@link ImmutabilityCheckingBundleFactory} that uses the underlying {@link
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutabilityEnforcementFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutabilityEnforcementFactory.java
index 6dd4c18..314b996 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutabilityEnforcementFactory.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutabilityEnforcementFactory.java
@@ -36,6 +36,7 @@
  * {@link ModelEnforcement} that enforces elements are not modified over the course of processing an
  * element.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class ImmutabilityEnforcementFactory implements ModelEnforcementFactory {
   public static ModelEnforcementFactory create() {
     return new ImmutabilityEnforcementFactory();
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutableListBundleFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutableListBundleFactory.java
index 715ccd1..476eac7 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutableListBundleFactory.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutableListBundleFactory.java
@@ -33,6 +33,7 @@
 import org.joda.time.Instant;
 
 /** A factory that produces bundles that perform no additional validation. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class ImmutableListBundleFactory implements BundleFactory {
   private static final ImmutableListBundleFactory FACTORY = new ImmutableListBundleFactory();
 
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitor.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitor.java
index 6ba09f5..a17bdc6 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitor.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitor.java
@@ -30,6 +30,7 @@
 import org.apache.beam.sdk.transforms.GroupByKey;
 import org.apache.beam.sdk.transforms.PTransform;
 import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PValue;
 import org.apache.beam.sdk.values.TupleTag;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableSet;
@@ -44,6 +45,7 @@
  */
 // TODO: Handle Key-preserving transforms when appropriate and more aggressively make PTransforms
 // unkeyed
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class KeyedPValueTrackingVisitor extends PipelineVisitor.Defaults {
 
   private static final Set<Class<? extends PTransform>> PRODUCES_KEYED_OUTPUTS =
@@ -83,7 +85,7 @@
     if (node.isRootNode()) {
       finalized = true;
     } else if (PRODUCES_KEYED_OUTPUTS.contains(node.getTransform().getClass())) {
-      Map<TupleTag<?>, PValue> outputs = node.getOutputs();
+      Map<TupleTag<?>, PCollection<?>> outputs = node.getOutputs();
       for (PValue output : outputs.values()) {
         keyedValues.add(output);
       }
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/MultiStepCombine.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/MultiStepCombine.java
index b25b497..8b87b31 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/MultiStepCombine.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/MultiStepCombine.java
@@ -52,7 +52,6 @@
 import org.apache.beam.sdk.util.WindowedValue;
 import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PValue;
 import org.apache.beam.sdk.values.TupleTag;
 import org.apache.beam.sdk.values.WindowingStrategy;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables;
@@ -64,6 +63,7 @@
 import org.joda.time.Instant;
 
 /** A {@link Combine} that performs the combine in multiple steps. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class MultiStepCombine<
         K extends @Nullable Object,
         InputT extends @Nullable Object,
@@ -83,7 +83,7 @@
       }
 
       private <K, InputT> boolean isApplicable(
-          Map<TupleTag<?>, PValue> inputs, GlobalCombineFn<InputT, ?, ?> fn) {
+          Map<TupleTag<?>, PCollection<?>> inputs, GlobalCombineFn<InputT, ?, ?> fn) {
         if (!(fn instanceof CombineFn)) {
           return false;
         }
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluator.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluator.java
index c6d967f..4642e14 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluator.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluator.java
@@ -49,6 +49,7 @@
 import org.apache.beam.sdk.values.WindowingStrategy;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
 
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class ParDoEvaluator<InputT> implements TransformEvaluator<InputT> {
 
   public interface DoFnRunnerFactory<InputT, OutputT> {
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluatorFactory.java
index 9d25215..82151e8 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluatorFactory.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluatorFactory.java
@@ -17,7 +17,6 @@
  */
 package org.apache.beam.runners.direct;
 
-import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import org.apache.beam.runners.core.construction.ParDoTranslation;
@@ -32,7 +31,6 @@
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PCollectionTuple;
 import org.apache.beam.sdk.values.PCollectionView;
-import org.apache.beam.sdk.values.PValue;
 import org.apache.beam.sdk.values.TupleTag;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.cache.CacheBuilder;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.cache.CacheLoader;
@@ -41,6 +39,7 @@
 import org.slf4j.LoggerFactory;
 
 /** A {@link TransformEvaluatorFactory} for {@link ParDo.MultiOutput}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 final class ParDoEvaluatorFactory<InputT, OutputT> implements TransformEvaluatorFactory {
 
   private static final Logger LOG = LoggerFactory.getLogger(ParDoEvaluatorFactory.class);
@@ -159,7 +158,7 @@
           sideInputs,
           mainOutputTag,
           additionalOutputTags,
-          pcollections(application.getOutputs()),
+          application.getOutputs(),
           doFnSchemaInformation,
           sideInputMapping,
           runnerFactory);
@@ -175,12 +174,4 @@
       throw e;
     }
   }
-
-  static Map<TupleTag<?>, PCollection<?>> pcollections(Map<TupleTag<?>, PValue> outputs) {
-    Map<TupleTag<?>, PCollection<?>> pcs = new HashMap<>();
-    for (Map.Entry<TupleTag<?>, PValue> output : outputs.entrySet()) {
-      pcs.put(output.getKey(), (PCollection<?>) output.getValue());
-    }
-    return pcs;
-  }
 }
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiOverrideFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiOverrideFactory.java
index e3ff795..5f35e01 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiOverrideFactory.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiOverrideFactory.java
@@ -119,8 +119,8 @@
   }
 
   @Override
-  public Map<PValue, ReplacementOutput> mapOutputs(
-      Map<TupleTag<?>, PValue> outputs, PCollectionTuple newOutput) {
+  public Map<PCollection<?>, ReplacementOutput> mapOutputs(
+      Map<TupleTag<?>, PCollection<?>> outputs, PCollectionTuple newOutput) {
     return ReplacementOutputs.tagged(outputs, newOutput);
   }
 
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/QuiescenceDriver.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/QuiescenceDriver.java
index de13435..a667ee2 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/QuiescenceDriver.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/QuiescenceDriver.java
@@ -46,6 +46,7 @@
  * Pushes additional work onto a {@link BundleProcessor} based on the fact that a pipeline has
  * quiesced.
  */
+@SuppressWarnings({"keyfor", "nullness"}) // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class QuiescenceDriver implements ExecutionDriver {
   private static final Logger LOG = LoggerFactory.getLogger(QuiescenceDriver.class);
 
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ReadEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ReadEvaluatorFactory.java
index 00514c1..20b984f 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ReadEvaluatorFactory.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ReadEvaluatorFactory.java
@@ -41,9 +41,8 @@
     unboundedFactory = new UnboundedReadEvaluatorFactory(context, options);
   }
 
-  @Nullable
   @Override
-  public <InputT> TransformEvaluator<InputT> forApplication(
+  public <InputT> @Nullable TransformEvaluator<InputT> forApplication(
       AppliedPTransform<?, ?, ?> application, CommittedBundle<?> inputBundle) throws Exception {
     switch (ReadTranslation.sourceIsBounded(application)) {
       case BOUNDED:
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/RootProviderRegistry.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/RootProviderRegistry.java
index 42f8513..0a04b79 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/RootProviderRegistry.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/RootProviderRegistry.java
@@ -35,6 +35,7 @@
  * A {@link RootInputProvider} that delegates to primitive {@link RootInputProvider} implementations
  * based on the type of {@link PTransform} of the application.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class RootProviderRegistry {
   /** Returns a {@link RootProviderRegistry} that supports the Java SDK root transforms. */
   public static RootProviderRegistry javaNativeRegistry(
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SideInputContainer.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SideInputContainer.java
index e75a5f2..a1c458d 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SideInputContainer.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SideInputContainer.java
@@ -56,6 +56,7 @@
  * constructing {@link SideInputReader SideInputReaders} which block until a side input is available
  * and writing to a {@link PCollectionView}.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class SideInputContainer {
   private static final Set<String> SUPPORTED_MATERIALIZATIONS =
       ImmutableSet.of(
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SplittableProcessElementsEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SplittableProcessElementsEvaluatorFactory.java
index cd1421e..f928cc9 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SplittableProcessElementsEvaluatorFactory.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SplittableProcessElementsEvaluatorFactory.java
@@ -47,6 +47,7 @@
 import org.joda.time.Duration;
 import org.joda.time.Instant;
 
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class SplittableProcessElementsEvaluatorFactory<
         InputT, OutputT, RestrictionT, PositionT, WatermarkEstimatorStateT>
     implements TransformEvaluatorFactory {
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactory.java
index d7b7ef1..5f238d0 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactory.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactory.java
@@ -50,6 +50,7 @@
 import org.joda.time.Instant;
 
 /** A {@link TransformEvaluatorFactory} for stateful {@link ParDo}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 final class StatefulParDoEvaluatorFactory<K, InputT, OutputT> implements TransformEvaluatorFactory {
 
   private final ParDoEvaluatorFactory<KV<K, InputT>, OutputT> delegateFactory;
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StepTransformResult.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StepTransformResult.java
index d13b33c..6760846 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StepTransformResult.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StepTransformResult.java
@@ -37,6 +37,7 @@
 
 /** An immutable {@link TransformResult}. */
 @AutoValue
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 abstract class StepTransformResult<InputT> implements TransformResult<InputT> {
 
   public static <InputT> Builder<InputT> withHold(
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactory.java
index 8c50d42..136eb8d 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactory.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactory.java
@@ -40,7 +40,6 @@
 import org.apache.beam.sdk.values.PBegin;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PCollection.IsBounded;
-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;
@@ -52,6 +51,7 @@
 import org.joda.time.Instant;
 
 /** The {@link TransformEvaluatorFactory} for the {@link TestStream} primitive. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class TestStreamEvaluatorFactory implements TransformEvaluatorFactory {
   private final EvaluationContext evaluationContext;
 
@@ -59,9 +59,8 @@
     this.evaluationContext = evaluationContext;
   }
 
-  @Nullable
   @Override
-  public <InputT> TransformEvaluator<InputT> forApplication(
+  public <InputT> @Nullable TransformEvaluator<InputT> forApplication(
       AppliedPTransform<?, ?, ?> application, CommittedBundle<?> inputBundle) {
     return createEvaluator((AppliedPTransform) application);
   }
@@ -184,8 +183,8 @@
     }
 
     @Override
-    public Map<PValue, ReplacementOutput> mapOutputs(
-        Map<TupleTag<?>, PValue> outputs, PCollection<T> newOutput) {
+    public Map<PCollection<?>, ReplacementOutput> mapOutputs(
+        Map<TupleTag<?>, PCollection<?>> outputs, PCollection<T> newOutput) {
       return ReplacementOutputs.singleton(outputs, newOutput);
     }
 
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorRegistry.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorRegistry.java
index c806594..9de6d49 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorRegistry.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorRegistry.java
@@ -57,6 +57,7 @@
  * A {@link TransformEvaluatorFactory} that delegates to primitive {@link TransformEvaluatorFactory}
  * implementations based on the type of {@link PTransform} of the application.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class TransformEvaluatorRegistry {
   private static final Logger LOG = LoggerFactory.getLogger(TransformEvaluatorRegistry.class);
 
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformExecutorServices.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformExecutorServices.java
index 39499c8..b1306c4 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformExecutorServices.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformExecutorServices.java
@@ -28,6 +28,7 @@
 import org.slf4j.LoggerFactory;
 
 /** Static factory methods for constructing instances of {@link TransformExecutorService}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 final class TransformExecutorServices {
   private TransformExecutorServices() {
     // Do not instantiate
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactory.java
index 4f3520f..7e7834f 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactory.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactory.java
@@ -50,6 +50,7 @@
  * for the {@link PrimitiveUnboundedRead SplittableParDo.PrimitiveUnboundedRead} primitive {@link
  * PTransform}.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class UnboundedReadEvaluatorFactory implements TransformEvaluatorFactory {
   // Occasionally close an existing reader and resume from checkpoint, to exercise close-and-resume
   private static final double DEFAULT_READER_REUSE_CHANCE = 0.95;
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkCallbackExecutor.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkCallbackExecutor.java
index 1ca90db..b2d48e7 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkCallbackExecutor.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkCallbackExecutor.java
@@ -49,6 +49,7 @@
  * followed by a call to {@link #fireForWatermark(AppliedPTransform, Instant)} for the same
  * transform with the current value of the watermark.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class WatermarkCallbackExecutor {
   /** Create a new {@link WatermarkCallbackExecutor}. */
   public static WatermarkCallbackExecutor create(Executor executor) {
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java
index e42d009..9e12f05 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WatermarkManager.java
@@ -136,6 +136,7 @@
  * Watermark_PCollection = Watermark_Out_ProducingPTransform
  * </pre>
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class WatermarkManager<ExecutableT, CollectionT> {
   // The number of updates to apply in #tryApplyPendingUpdates
   private static final int MAX_INCREMENTAL_UPDATES = 10;
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WindowEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WindowEvaluatorFactory.java
index 475f2e2..eda32ac 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WindowEvaluatorFactory.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WindowEvaluatorFactory.java
@@ -34,6 +34,7 @@
  * The {@link DirectRunner} {@link TransformEvaluatorFactory} for the {@link Window.Assign}
  * primitive {@link PTransform}.
  */
+@SuppressWarnings({"keyfor", "nullness"}) // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class WindowEvaluatorFactory implements TransformEvaluatorFactory {
   private final EvaluationContext evaluationContext;
 
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WriteWithShardingFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WriteWithShardingFactory.java
index 4fd898e..f570b04 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WriteWithShardingFactory.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WriteWithShardingFactory.java
@@ -37,7 +37,6 @@
 import org.apache.beam.sdk.transforms.windowing.Window;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PCollectionView;
-import org.apache.beam.sdk.values.PValue;
 import org.apache.beam.sdk.values.TupleTag;
 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;
@@ -81,8 +80,8 @@
   }
 
   @Override
-  public Map<PValue, ReplacementOutput> mapOutputs(
-      Map<TupleTag<?>, PValue> outputs, WriteFilesResult<DestinationT> newOutput) {
+  public Map<PCollection<?>, ReplacementOutput> mapOutputs(
+      Map<TupleTag<?>, PCollection<?>> outputs, WriteFilesResult<DestinationT> newOutput) {
     // We must connect the new output from WriteFilesResult to the outputs provided by the original
     // transform.
     return ReplacementOutputs.tagged(outputs, newOutput);
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactoryTest.java
index b934639..59c0964 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactoryTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactoryTest.java
@@ -70,6 +70,7 @@
 
 /** Tests for {@link BoundedReadEvaluatorFactory}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BoundedReadEvaluatorFactoryTest {
   private BoundedSource<Long> source;
   private PCollection<Long> longs;
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CloningBundleFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CloningBundleFactoryTest.java
index 5d36b92..bcce809 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CloningBundleFactoryTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CloningBundleFactoryTest.java
@@ -55,6 +55,7 @@
 
 /** Tests for {@link CloningBundleFactory}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class CloningBundleFactoryTest {
   @Rule public ExpectedException thrown = ExpectedException.none();
   @Rule public final TestPipeline p = TestPipeline.create().enableAbandonedNodeEnforcement(false);
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CommittedResultTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CommittedResultTest.java
index 16cb694..5a437ba 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CommittedResultTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CommittedResultTest.java
@@ -34,6 +34,7 @@
 import org.apache.beam.sdk.values.PBegin;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PDone;
+import org.apache.beam.sdk.values.PValues;
 import org.apache.beam.sdk.values.WindowingStrategy;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
 import org.hamcrest.Matchers;
@@ -54,8 +55,8 @@
   private transient AppliedPTransform<?, ?, ?> transform =
       AppliedPTransform.<PBegin, PDone, PTransform<PBegin, PDone>>of(
           "foo",
-          p.begin().expand(),
-          PDone.in(p).expand(),
+          PValues.expandInput(p.begin()),
+          PValues.expandOutput(PDone.in(p)),
           new PTransform<PBegin, PDone>() {
             @Override
             public PDone expand(PBegin begin) {
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CopyOnAccessInMemoryStateInternalsTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CopyOnAccessInMemoryStateInternalsTest.java
index 5e53d58..b63cc36 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CopyOnAccessInMemoryStateInternalsTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CopyOnAccessInMemoryStateInternalsTest.java
@@ -60,6 +60,7 @@
 
 /** Tests for {@link CopyOnAccessInMemoryStateInternals}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class CopyOnAccessInMemoryStateInternalsTest {
 
   @Rule public final TestPipeline pipeline = TestPipeline.create();
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectMetricsTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectMetricsTest.java
index 7957d52..fe6e7bc 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectMetricsTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectMetricsTest.java
@@ -48,6 +48,7 @@
 
 /** Tests for {@link DirectMetrics}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DirectMetricsTest {
 
   @Mock private CommittedBundle<Object> bundle1;
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectRunnerApiSurfaceTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectRunnerApiSurfaceTest.java
index 7651e45..a9f5714 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectRunnerApiSurfaceTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectRunnerApiSurfaceTest.java
@@ -35,6 +35,7 @@
 
 /** API surface verification for {@link org.apache.beam.runners.direct}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DirectRunnerApiSurfaceTest {
   @Test
   public void testDirectRunnerApiSurface() throws Exception {
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectRunnerTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectRunnerTest.java
index b493d2f..8894e69 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectRunnerTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectRunnerTest.java
@@ -106,6 +106,7 @@
 
 /** Tests for basic {@link DirectRunner} functionality. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DirectRunnerTest implements Serializable {
   @Rule public transient ExpectedException thrown = ExpectedException.none();
 
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectTimerInternalsTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectTimerInternalsTest.java
index 5bd0d39..26a4ae6 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectTimerInternalsTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectTimerInternalsTest.java
@@ -41,6 +41,7 @@
 
 /** Tests for {@link DirectTimerInternals}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DirectTimerInternalsTest {
   private MockClock clock;
   @Mock private TransformWatermarks watermarks;
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectTransformExecutorTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectTransformExecutorTest.java
index b28333b..a86ac8e 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectTransformExecutorTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectTransformExecutorTest.java
@@ -58,6 +58,7 @@
 
 /** Tests for {@link DirectTransformExecutor}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DirectTransformExecutorTest {
   @Rule public ExpectedException thrown = ExpectedException.none();
   private PCollection<String> created;
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagerRemovingTransformEvaluatorTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagerRemovingTransformEvaluatorTest.java
index 3e7871b..e36f493 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagerRemovingTransformEvaluatorTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagerRemovingTransformEvaluatorTest.java
@@ -42,6 +42,7 @@
 
 /** Tests for {@link DoFnLifecycleManagerRemovingTransformEvaluator}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DoFnLifecycleManagerRemovingTransformEvaluatorTest {
   private DoFnLifecycleManager lifecycleManager;
 
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagersTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagersTest.java
index dfa24af..1c285c2 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagersTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagersTest.java
@@ -38,6 +38,7 @@
 
 /** Tests for {@link DoFnLifecycleManagers}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DoFnLifecycleManagersTest {
   @Rule public ExpectedException thrown = ExpectedException.none();
 
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EvaluationContextTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EvaluationContextTest.java
index 5cb6782..db44130 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EvaluationContextTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EvaluationContextTest.java
@@ -78,6 +78,7 @@
 
 /** Tests for {@link EvaluationContext}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class EvaluationContextTest implements Serializable {
   private transient EvaluationContext context;
 
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/FlattenEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/FlattenEvaluatorFactoryTest.java
index ba9e7bc..40290b7 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/FlattenEvaluatorFactoryTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/FlattenEvaluatorFactoryTest.java
@@ -43,6 +43,7 @@
 
 /** Tests for {@link FlattenEvaluatorFactory}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings({"keyfor", "nullness"}) // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class FlattenEvaluatorFactoryTest {
   private BundleFactory bundleFactory = ImmutableListBundleFactory.create();
 
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/GroupByKeyEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/GroupByKeyEvaluatorFactoryTest.java
index a9c6524..eb451bf 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/GroupByKeyEvaluatorFactoryTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/GroupByKeyEvaluatorFactoryTest.java
@@ -47,6 +47,7 @@
 
 /** Tests for {@link GroupByKeyEvaluatorFactory}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class GroupByKeyEvaluatorFactoryTest {
   private BundleFactory bundleFactory = ImmutableListBundleFactory.create();
 
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/GroupByKeyOnlyEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/GroupByKeyOnlyEvaluatorFactoryTest.java
index 9847dae..f11ebea 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/GroupByKeyOnlyEvaluatorFactoryTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/GroupByKeyOnlyEvaluatorFactoryTest.java
@@ -47,6 +47,7 @@
 
 /** Tests for {@link GroupByKeyOnlyEvaluatorFactory}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class GroupByKeyOnlyEvaluatorFactoryTest {
   private BundleFactory bundleFactory = ImmutableListBundleFactory.create();
 
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactoryTest.java
index ccdaa00..2a67e41 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactoryTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactoryTest.java
@@ -42,6 +42,7 @@
 
 /** Tests for {@link ImmutabilityCheckingBundleFactory}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ImmutabilityCheckingBundleFactoryTest {
 
   @Rule public final TestPipeline p = TestPipeline.create().enableAbandonedNodeEnforcement(false);
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityEnforcementFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityEnforcementFactoryTest.java
index 3a57410..d198d1a 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityEnforcementFactoryTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityEnforcementFactoryTest.java
@@ -40,6 +40,7 @@
 
 /** Tests for {@link ImmutabilityEnforcementFactory}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ImmutabilityEnforcementFactoryTest implements Serializable {
   @Rule
   public transient TestPipeline p = TestPipeline.create().enableAbandonedNodeEnforcement(false);
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutableListBundleFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutableListBundleFactoryTest.java
index 21d6e65..19d8bfa 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutableListBundleFactoryTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutableListBundleFactoryTest.java
@@ -53,6 +53,7 @@
 
 /** Tests for {@link ImmutableListBundleFactory}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ImmutableListBundleFactoryTest {
   @Rule public final TestPipeline p = TestPipeline.create().enableAbandonedNodeEnforcement(false);
   @Rule public ExpectedException thrown = ExpectedException.none();
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitorTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitorTest.java
index 9d7df5d..eb9d4d5 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitorTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitorTest.java
@@ -56,6 +56,7 @@
 
 /** Tests for {@link KeyedPValueTrackingVisitor}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class KeyedPValueTrackingVisitorTest {
   @Rule public ExpectedException thrown = ExpectedException.none();
 
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoEvaluatorTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoEvaluatorTest.java
index 8a96fb3..32b9726 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoEvaluatorTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoEvaluatorTest.java
@@ -67,6 +67,7 @@
 
 /** Tests for {@link ParDoEvaluator}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings({"keyfor", "nullness"}) // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ParDoEvaluatorTest {
   @Mock private EvaluationContext evaluationContext;
   private PCollection<Integer> inputPc;
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/SideInputContainerTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/SideInputContainerTest.java
index 5db9fe5..ac1e0cc 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/SideInputContainerTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/SideInputContainerTest.java
@@ -64,6 +64,7 @@
 
 /** Tests for {@link SideInputContainer}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SideInputContainerTest {
   private static final BoundedWindow FIRST_WINDOW =
       new BoundedWindow() {
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactoryTest.java
index 32b4549..d29239b 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactoryTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactoryTest.java
@@ -86,6 +86,7 @@
 
 /** Tests for {@link StatefulParDoEvaluatorFactory}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class StatefulParDoEvaluatorFactoryTest implements Serializable {
   @Mock private transient EvaluationContext mockEvaluationContext;
   @Mock private transient DirectExecutionContext mockExecutionContext;
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StepTransformResultTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StepTransformResultTest.java
index 6e6a68e..86438b6 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StepTransformResultTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StepTransformResultTest.java
@@ -35,6 +35,7 @@
 
 /** Tests for {@link StepTransformResult}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class StepTransformResultTest {
   private AppliedPTransform<?, ?, ?> transform;
   private BundleFactory bundleFactory;
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactoryTest.java
index 5def3b8..bf44d1d 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactoryTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactoryTest.java
@@ -47,6 +47,7 @@
 
 /** Tests for {@link TestStreamEvaluatorFactory}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings({"keyfor", "nullness"}) // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class TestStreamEvaluatorFactoryTest {
   private TestStreamEvaluatorFactory factory;
   private BundleFactory bundleFactory;
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TransformExecutorServicesTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TransformExecutorServicesTest.java
index 4105616..e595ff1 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TransformExecutorServicesTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TransformExecutorServicesTest.java
@@ -32,6 +32,7 @@
 
 /** Tests for {@link TransformExecutorServices}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class TransformExecutorServicesTest {
   @Rule public ExpectedException thrown = ExpectedException.none();
 
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactoryTest.java
index d4072d2..75a8aa9 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactoryTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactoryTest.java
@@ -95,6 +95,7 @@
 
 /** Tests for {@link UnboundedReadEvaluatorFactory}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings({"keyfor", "nullness"}) // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class UnboundedReadEvaluatorFactoryTest {
   private PCollection<Long> longs;
   private UnboundedReadEvaluatorFactory factory;
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ViewEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ViewEvaluatorFactoryTest.java
index c733cc0..eb136fb 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ViewEvaluatorFactoryTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ViewEvaluatorFactoryTest.java
@@ -45,6 +45,7 @@
 
 /** Tests for {@link ViewEvaluatorFactory}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ViewEvaluatorFactoryTest {
   private BundleFactory bundleFactory = ImmutableListBundleFactory.create();
 
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkCallbackExecutorTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkCallbackExecutorTest.java
index cb0f697..d3c8b60 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkCallbackExecutorTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkCallbackExecutorTest.java
@@ -44,6 +44,7 @@
 
 /** Tests for {@link WatermarkCallbackExecutor}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class WatermarkCallbackExecutorTest {
   private WatermarkCallbackExecutor executor =
       WatermarkCallbackExecutor.create(Executors.newSingleThreadExecutor());
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkManagerTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkManagerTest.java
index a18b339..53ec939 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkManagerTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkManagerTest.java
@@ -87,6 +87,7 @@
  * tracking while impulse would normally only output a single empty byte array in the global window.
  */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class WatermarkManagerTest implements Serializable {
   @Rule public transient ExpectedException thrown = ExpectedException.none();
 
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WindowEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WindowEvaluatorFactoryTest.java
index c9cb86c..5abd6a3 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WindowEvaluatorFactoryTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WindowEvaluatorFactoryTest.java
@@ -59,6 +59,7 @@
 
 /** Tests for {@link WindowEvaluatorFactory}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class WindowEvaluatorFactoryTest {
   private static final Instant EPOCH = new Instant(0);
 
diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WriteWithShardingFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WriteWithShardingFactoryTest.java
index 4232e6e4..df3abed 100644
--- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WriteWithShardingFactoryTest.java
+++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WriteWithShardingFactoryTest.java
@@ -60,6 +60,7 @@
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
 import org.apache.beam.sdk.transforms.windowing.PaneInfo;
 import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PValues;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Splitter;
 import org.checkerframework.checker.nullness.qual.Nullable;
 import org.junit.Rule;
@@ -155,7 +156,8 @@
             WriteFilesResult<Void>,
             PTransform<PCollection<Object>, WriteFilesResult<Void>>>
         originalApplication =
-            AppliedPTransform.of("write", objs.expand(), Collections.emptyMap(), original, p);
+            AppliedPTransform.of(
+                "write", PValues.expandInput(objs), Collections.emptyMap(), original, p);
 
     assertThat(
         factory.getReplacementTransform(originalApplication).getTransform(),
@@ -251,9 +253,8 @@
       throw new IllegalArgumentException("Should not be used");
     }
 
-    @Nullable
     @Override
-    public ResourceId unwindowedFilename(
+    public @Nullable ResourceId unwindowedFilename(
         int shardNumber, int numShards, FileBasedSink.OutputFileHints outputFileHints) {
       throw new IllegalArgumentException("Should not be used");
     }
diff --git a/runners/extensions-java/metrics/src/main/java/org/apache/beam/runners/extensions/metrics/MetricsGraphiteSink.java b/runners/extensions-java/metrics/src/main/java/org/apache/beam/runners/extensions/metrics/MetricsGraphiteSink.java
index 865f890..c940910 100644
--- a/runners/extensions-java/metrics/src/main/java/org/apache/beam/runners/extensions/metrics/MetricsGraphiteSink.java
+++ b/runners/extensions-java/metrics/src/main/java/org/apache/beam/runners/extensions/metrics/MetricsGraphiteSink.java
@@ -40,6 +40,7 @@
  * {@code beam.counter.throughput.nbRecords.attempted.value} Or {@code
  * beam.distribution.throughput.nbRecordsPerSec.attempted.mean}
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class MetricsGraphiteSink implements MetricsSink {
   private static final Charset UTF_8 = Charset.forName("UTF-8");
   private static final Pattern WHITESPACE = Pattern.compile("[\\s]+");
diff --git a/runners/extensions-java/metrics/src/main/java/org/apache/beam/runners/extensions/metrics/MetricsHttpSink.java b/runners/extensions-java/metrics/src/main/java/org/apache/beam/runners/extensions/metrics/MetricsHttpSink.java
index de5b383..d6ff651 100644
--- a/runners/extensions-java/metrics/src/main/java/org/apache/beam/runners/extensions/metrics/MetricsHttpSink.java
+++ b/runners/extensions-java/metrics/src/main/java/org/apache/beam/runners/extensions/metrics/MetricsHttpSink.java
@@ -41,6 +41,7 @@
 import org.apache.beam.sdk.metrics.MetricsSink;
 
 /** HTTP Sink to push metrics in a POST HTTP request. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class MetricsHttpSink implements MetricsSink {
   private final String urlString;
   private final ObjectMapper objectMapper = new ObjectMapper();
diff --git a/runners/extensions-java/metrics/src/test/java/org/apache/beam/runners/extensions/metrics/MetricsGraphiteSinkTest.java b/runners/extensions-java/metrics/src/test/java/org/apache/beam/runners/extensions/metrics/MetricsGraphiteSinkTest.java
index 0e4f5e8..86b08a1 100644
--- a/runners/extensions-java/metrics/src/test/java/org/apache/beam/runners/extensions/metrics/MetricsGraphiteSinkTest.java
+++ b/runners/extensions-java/metrics/src/test/java/org/apache/beam/runners/extensions/metrics/MetricsGraphiteSinkTest.java
@@ -31,6 +31,7 @@
 import org.junit.Test;
 
 /** Test class for MetricsGraphiteSink. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class MetricsGraphiteSinkTest {
   private static NetworkMockServer graphiteServer;
   private static int port;
diff --git a/runners/extensions-java/metrics/src/test/java/org/apache/beam/runners/extensions/metrics/MetricsHttpSinkTest.java b/runners/extensions-java/metrics/src/test/java/org/apache/beam/runners/extensions/metrics/MetricsHttpSinkTest.java
index afbe77b..782df7f 100644
--- a/runners/extensions-java/metrics/src/test/java/org/apache/beam/runners/extensions/metrics/MetricsHttpSinkTest.java
+++ b/runners/extensions-java/metrics/src/test/java/org/apache/beam/runners/extensions/metrics/MetricsHttpSinkTest.java
@@ -39,6 +39,7 @@
 import org.junit.Test;
 
 /** Test class for MetricsHttpSink. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class MetricsHttpSinkTest {
   private static int port;
   private static List<String> messages = new ArrayList<>();
diff --git a/runners/extensions-java/metrics/src/test/java/org/apache/beam/runners/extensions/metrics/NetworkMockServer.java b/runners/extensions-java/metrics/src/test/java/org/apache/beam/runners/extensions/metrics/NetworkMockServer.java
index 5b60837..f2c4205 100644
--- a/runners/extensions-java/metrics/src/test/java/org/apache/beam/runners/extensions/metrics/NetworkMockServer.java
+++ b/runners/extensions-java/metrics/src/test/java/org/apache/beam/runners/extensions/metrics/NetworkMockServer.java
@@ -30,6 +30,7 @@
 import java.util.concurrent.atomic.AtomicBoolean;
 
 /** Mock of a network server. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class NetworkMockServer {
   private final int port;
   private ServerSocket serverSocket;
diff --git a/runners/flink/1.11/src/test/java/org/apache/beam/runners/flink/FlinkRunnerTestCompat.java b/runners/flink/1.11/src/test/java/org/apache/beam/runners/flink/FlinkRunnerTestCompat.java
index b49f323..05e0b95 100644
--- a/runners/flink/1.11/src/test/java/org/apache/beam/runners/flink/FlinkRunnerTestCompat.java
+++ b/runners/flink/1.11/src/test/java/org/apache/beam/runners/flink/FlinkRunnerTestCompat.java
@@ -27,6 +27,7 @@
  * Compatibility layer for {@link PackagedProgram} and {@link OptimizerPlanEnvironment} breaking
  * changes.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public abstract class FlinkRunnerTestCompat {
   public PackagedProgram getPackagedProgram() throws ProgramInvocationException {
     return PackagedProgram.newBuilder().setEntryPointClassName(getClass().getName()).build();
diff --git a/runners/flink/1.8/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeSerializer.java b/runners/flink/1.8/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeSerializer.java
index 303ed9c..04f4bae 100644
--- a/runners/flink/1.8/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeSerializer.java
+++ b/runners/flink/1.8/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeSerializer.java
@@ -39,6 +39,7 @@
  * Flink {@link org.apache.flink.api.common.typeutils.TypeSerializer} for Beam {@link
  * org.apache.beam.sdk.coders.Coder Coders}.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class CoderTypeSerializer<T> extends TypeSerializer<T> {
 
   private final Coder<T> coder;
diff --git a/runners/flink/1.8/src/test/java/org/apache/beam/runners/flink/translation/types/CoderTypeSerializerTest.java b/runners/flink/1.8/src/test/java/org/apache/beam/runners/flink/translation/types/CoderTypeSerializerTest.java
index 3fdf462..e88f5cc 100644
--- a/runners/flink/1.8/src/test/java/org/apache/beam/runners/flink/translation/types/CoderTypeSerializerTest.java
+++ b/runners/flink/1.8/src/test/java/org/apache/beam/runners/flink/translation/types/CoderTypeSerializerTest.java
@@ -32,6 +32,7 @@
 import org.junit.Test;
 
 /** Tests {@link CoderTypeSerializer}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class CoderTypeSerializerTest implements Serializable {
 
   @Test
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/CreateStreamingFlinkView.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/CreateStreamingFlinkView.java
index c5ef3b6..76b364f 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/CreateStreamingFlinkView.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/CreateStreamingFlinkView.java
@@ -32,7 +32,6 @@
 import org.apache.beam.sdk.transforms.PTransform;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PCollectionView;
-import org.apache.beam.sdk.values.PValue;
 import org.apache.beam.sdk.values.TupleTag;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables;
 
@@ -163,8 +162,8 @@
     }
 
     @Override
-    public Map<PValue, ReplacementOutput> mapOutputs(
-        Map<TupleTag<?>, PValue> outputs, PCollection<ElemT> newOutput) {
+    public Map<PCollection<?>, ReplacementOutput> mapOutputs(
+        Map<TupleTag<?>, PCollection<?>> outputs, PCollection<ElemT> newOutput) {
       return ReplacementOutputs.singleton(outputs, newOutput);
     }
   }
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchPipelineTranslator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchPipelineTranslator.java
index 669583d..0c21bd2 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchPipelineTranslator.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchPipelineTranslator.java
@@ -30,6 +30,7 @@
 import org.slf4j.LoggerFactory;
 
 /** {@link Pipeline.PipelineVisitor} for executing a {@link Pipeline} as a Flink batch job. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class FlinkBatchPipelineTranslator extends FlinkPipelineTranslator {
 
   private static final Logger LOG = LoggerFactory.getLogger(FlinkBatchPipelineTranslator.class);
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchPortablePipelineTranslator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchPortablePipelineTranslator.java
index e867335..d8989d4 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchPortablePipelineTranslator.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchPortablePipelineTranslator.java
@@ -115,6 +115,7 @@
  * <p>After translation the {@link ExecutionEnvironment} in the translation context will contain the
  * full not-yet-executed pipeline DAG corresponding to the input pipeline.
  */
+@SuppressWarnings({"keyfor", "nullness"}) // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class FlinkBatchPortablePipelineTranslator
     implements FlinkPortablePipelineTranslator<
         FlinkBatchPortablePipelineTranslator.BatchTranslationContext> {
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchTransformTranslators.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchTransformTranslators.java
index 40c240f..aac9c94 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchTransformTranslators.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchTransformTranslators.java
@@ -106,6 +106,7 @@
 /**
  * Translators for transforming {@link PTransform PTransforms} to Flink {@link DataSet DataSets}.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class FlinkBatchTransformTranslators {
 
   // --------------------------------------------------------------------------------------------
@@ -638,7 +639,7 @@
       DataSet<WindowedValue<InputT>> inputDataSet =
           context.getInputDataSet(context.getInput(transform));
 
-      Map<TupleTag<?>, PValue> outputs = context.getOutputs(transform);
+      Map<TupleTag<?>, PCollection<?>> outputs = context.getOutputs(transform);
 
       final TupleTag<OutputT> mainOutputTag;
       DoFnSchemaInformation doFnSchemaInformation;
@@ -778,7 +779,7 @@
 
       transformSideInputs(sideInputs, outputDataSet, context);
 
-      for (Entry<TupleTag<?>, PValue> output : outputs.entrySet()) {
+      for (Entry<TupleTag<?>, PCollection<?>> output : outputs.entrySet()) {
         pruneOutput(
             outputDataSet,
             context,
@@ -823,7 +824,7 @@
         PTransform<PCollectionList<T>, PCollection<T>> transform,
         FlinkBatchTranslationContext context) {
 
-      Map<TupleTag<?>, PValue> allInputs = context.getInputs(transform);
+      Map<TupleTag<?>, PCollection<?>> allInputs = context.getInputs(transform);
       DataSet<WindowedValue<T>> result = null;
 
       if (allInputs.isEmpty()) {
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchTranslationContext.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchTranslationContext.java
index 1af8020..75ee568 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchTranslationContext.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchTranslationContext.java
@@ -41,6 +41,7 @@
  * Helper for {@link FlinkBatchPipelineTranslator} and translators in {@link
  * FlinkBatchTransformTranslators}.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class FlinkBatchTranslationContext {
 
   private final Map<PValue, DataSet<?>> dataSets;
@@ -145,7 +146,7 @@
     return new CoderTypeInformation<>(windowedValueCoder);
   }
 
-  Map<TupleTag<?>, PValue> getInputs(PTransform<?, ?> transform) {
+  Map<TupleTag<?>, PCollection<?>> getInputs(PTransform<?, ?> transform) {
     return lookupPipelineVisitor.getInputs(transform);
   }
 
@@ -153,7 +154,7 @@
     return lookupPipelineVisitor.getInput(transform);
   }
 
-  Map<TupleTag<?>, PValue> getOutputs(PTransform<?, ?> transform) {
+  Map<TupleTag<?>, PCollection<?>> getOutputs(PTransform<?, ?> transform) {
     return lookupPipelineVisitor.getOutputs(transform);
   }
 
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkExecutionEnvironments.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkExecutionEnvironments.java
index dca436c..0ac4733 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkExecutionEnvironments.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkExecutionEnvironments.java
@@ -45,6 +45,7 @@
 import org.slf4j.LoggerFactory;
 
 /** Utilities for Flink execution environments. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class FlinkExecutionEnvironments {
 
   private static final Logger LOG = LoggerFactory.getLogger(FlinkExecutionEnvironments.class);
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkJobInvoker.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkJobInvoker.java
index 0d24654..e297c89 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkJobInvoker.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkJobInvoker.java
@@ -36,6 +36,7 @@
 import org.slf4j.LoggerFactory;
 
 /** Job Invoker for the {@link FlinkRunner}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class FlinkJobInvoker extends JobInvoker {
   private static final Logger LOG = LoggerFactory.getLogger(FlinkJobInvoker.class);
 
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkJobServerDriver.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkJobServerDriver.java
index bd6637a..8afbf26 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkJobServerDriver.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkJobServerDriver.java
@@ -31,6 +31,7 @@
 import org.slf4j.LoggerFactory;
 
 /** Driver program that starts a job server for the Flink runner. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class FlinkJobServerDriver extends JobServerDriver {
 
   private static final Logger LOG = LoggerFactory.getLogger(FlinkJobServerDriver.class);
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineExecutionEnvironment.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineExecutionEnvironment.java
index d99aba3..f9eb155 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineExecutionEnvironment.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineExecutionEnvironment.java
@@ -39,6 +39,7 @@
  * FlinkStreamingPipelineTranslator}) to transform the Beam job into a Flink one, and executes the
  * (translated) job.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class FlinkPipelineExecutionEnvironment {
 
   private static final Logger LOG =
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineRunner.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineRunner.java
index ebc3d04..0adcd35 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineRunner.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineRunner.java
@@ -54,6 +54,7 @@
 import org.slf4j.LoggerFactory;
 
 /** Runs a Pipeline on Flink via {@link FlinkRunner}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class FlinkPipelineRunner implements PortablePipelineRunner {
   private static final Logger LOG = LoggerFactory.getLogger(FlinkPipelineRunner.class);
 
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPortableClientEntryPoint.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPortableClientEntryPoint.java
index 4794240..70abedb 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPortableClientEntryPoint.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPortableClientEntryPoint.java
@@ -67,6 +67,7 @@
  *
  * <p>Finally Flink launches the job.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class FlinkPortableClientEntryPoint {
   private static final Logger LOG = LoggerFactory.getLogger(FlinkPortableClientEntryPoint.class);
   private static final String JOB_ENDPOINT_FLAG = "--job_endpoint";
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkRunner.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkRunner.java
index 186efe1..3635954 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkRunner.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkRunner.java
@@ -45,6 +45,7 @@
  * to a Flink Plan and then executing them either locally or on a Flink cluster, depending on the
  * configuration.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class FlinkRunner extends PipelineRunner<PipelineResult> {
 
   private static final Logger LOG = LoggerFactory.getLogger(FlinkRunner.class);
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkRunnerResult.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkRunnerResult.java
index b0d7faf..4a6da65 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkRunnerResult.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkRunnerResult.java
@@ -31,6 +31,7 @@
  * Result of executing a {@link org.apache.beam.sdk.Pipeline} with Flink. This has methods to query
  * to job runtime and the final values of the accumulators.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class FlinkRunnerResult implements PipelineResult {
 
   private final Map<String, Object> accumulators;
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingPipelineTranslator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingPipelineTranslator.java
index e5ff563..033d8d2 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingPipelineTranslator.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingPipelineTranslator.java
@@ -68,6 +68,7 @@
  * user-provided {@link org.apache.beam.sdk.values.PCollection}-based job into a {@link
  * org.apache.flink.streaming.api.datastream.DataStream} one.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class FlinkStreamingPipelineTranslator extends FlinkPipelineTranslator {
 
   private static final Logger LOG = LoggerFactory.getLogger(FlinkStreamingPipelineTranslator.class);
@@ -290,8 +291,8 @@
     }
 
     @Override
-    public Map<PValue, ReplacementOutput> mapOutputs(
-        Map<TupleTag<?>, PValue> outputs, WriteFilesResult<DestinationT> newOutput) {
+    public Map<PCollection<?>, ReplacementOutput> mapOutputs(
+        Map<TupleTag<?>, PCollection<?>> outputs, WriteFilesResult<DestinationT> newOutput) {
       return ReplacementOutputs.tagged(outputs, newOutput);
     }
   }
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingPortablePipelineTranslator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingPortablePipelineTranslator.java
index 9f6ce67..6eabacb 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingPortablePipelineTranslator.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingPortablePipelineTranslator.java
@@ -117,6 +117,7 @@
 import org.apache.flink.util.OutputTag;
 
 /** Translate an unbounded portable pipeline representation into a Flink pipeline representation. */
+@SuppressWarnings({"keyfor", "nullness"}) // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class FlinkStreamingPortablePipelineTranslator
     implements FlinkPortablePipelineTranslator<
         FlinkStreamingPortablePipelineTranslator.StreamingTranslationContext> {
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTransformTranslators.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTransformTranslators.java
index cc82ee5..68af4fb 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTransformTranslators.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTransformTranslators.java
@@ -126,6 +126,7 @@
  * {@link FlinkStreamingPipelineTranslator} traverses the Beam job and comes here to translate the
  * encountered Beam transformations into Flink one, based on the mapping available in this class.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class FlinkStreamingTransformTranslators {
 
   // --------------------------------------------------------------------------------------------
@@ -493,7 +494,7 @@
         DoFn<InputT, OutputT> doFn,
         PCollection<InputT> input,
         List<PCollectionView<?>> sideInputs,
-        Map<TupleTag<?>, PValue> outputs,
+        Map<TupleTag<?>, PCollection<?>> outputs,
         TupleTag<OutputT> mainOutputTag,
         List<TupleTag<?>> additionalOutputTags,
         DoFnSchemaInformation doFnSchemaInformation,
@@ -514,7 +515,7 @@
       Map<TupleTag<?>, Integer> tagsToIds = Maps.newHashMap();
       int idCount = 0;
       tagsToIds.put(mainOutputTag, idCount++);
-      for (Map.Entry<TupleTag<?>, PValue> entry : outputs.entrySet()) {
+      for (Map.Entry<TupleTag<?>, PCollection<?>> entry : outputs.entrySet()) {
         if (!tagsToOutputTags.containsKey(entry.getKey())) {
           tagsToOutputTags.put(
               entry.getKey(),
@@ -641,7 +642,7 @@
       outputStream.uid(transformName);
       context.setOutputDataStream(outputs.get(mainOutputTag), outputStream);
 
-      for (Map.Entry<TupleTag<?>, PValue> entry : outputs.entrySet()) {
+      for (Map.Entry<TupleTag<?>, PCollection<?>> entry : outputs.entrySet()) {
         if (!entry.getKey().equals(mainOutputTag)) {
           context.setOutputDataStream(
               entry.getValue(), outputStream.getSideOutput(tagsToOutputTags.get(entry.getKey())));
@@ -1200,7 +1201,7 @@
     public void translateNode(
         PTransform<PCollection<T>, PCollection<T>> transform,
         FlinkStreamingTranslationContext context) {
-      Map<TupleTag<?>, PValue> allInputs = context.getInputs(transform);
+      Map<TupleTag<?>, PCollection<?>> allInputs = context.getInputs(transform);
 
       if (allInputs.isEmpty()) {
 
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTranslationContext.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTranslationContext.java
index 7fc7ce0..75c2a20 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTranslationContext.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTranslationContext.java
@@ -43,6 +43,7 @@
  * Helper for keeping track of which {@link DataStream DataStreams} map to which {@link PTransform
  * PTransforms}.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class FlinkStreamingTranslationContext {
 
   private final StreamExecutionEnvironment env;
@@ -126,7 +127,7 @@
     return (T) Iterables.getOnlyElement(TransformInputs.nonAdditionalInputs(currentTransform));
   }
 
-  public <T extends PInput> Map<TupleTag<?>, PValue> getInputs(PTransform<T, ?> transform) {
+  public <T extends PInput> Map<TupleTag<?>, PCollection<?>> getInputs(PTransform<T, ?> transform) {
     return currentTransform.getInputs();
   }
 
@@ -135,7 +136,7 @@
     return (T) Iterables.getOnlyElement(currentTransform.getOutputs().values());
   }
 
-  public <OutputT extends POutput> Map<TupleTag<?>, PValue> getOutputs(
+  public <OutputT extends POutput> Map<TupleTag<?>, PCollection<?>> getOutputs(
       PTransform<?, OutputT> transform) {
     return currentTransform.getOutputs();
   }
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/PipelineTranslationModeOptimizer.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/PipelineTranslationModeOptimizer.java
index bc62c14..06102cb 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/PipelineTranslationModeOptimizer.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/PipelineTranslationModeOptimizer.java
@@ -31,6 +31,7 @@
  * Traverses the Pipeline to determine the translation mode (i.e. streaming or batch) for this
  * pipeline.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class PipelineTranslationModeOptimizer extends FlinkPipelineTranslator {
 
   private static final Logger LOG = LoggerFactory.getLogger(PipelineTranslationModeOptimizer.class);
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/TestFlinkRunner.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/TestFlinkRunner.java
index eef7d2d..d9a99a6 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/TestFlinkRunner.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/TestFlinkRunner.java
@@ -26,6 +26,7 @@
 import org.apache.beam.sdk.util.UserCodeException;
 
 /** Test Flink runner. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class TestFlinkRunner extends PipelineRunner<PipelineResult> {
 
   private FlinkRunner delegate;
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/metrics/FileReporter.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/metrics/FileReporter.java
index d6262e7..dd6fe9e 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/metrics/FileReporter.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/metrics/FileReporter.java
@@ -31,6 +31,7 @@
  * metrics to a file specified via the "metrics.reporter.file.path" config key (assuming an alias of
  * "file" for this reporter in the "metrics.reporters" setting).
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class FileReporter extends AbstractReporter {
   @Override
   public String filterCharacters(String input) {
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkAssignContext.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkAssignContext.java
index 137ff23..b6c956f 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkAssignContext.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkAssignContext.java
@@ -24,6 +24,7 @@
 import org.joda.time.Instant;
 
 /** {@link org.apache.beam.sdk.transforms.windowing.WindowFn.AssignContext} for Flink functions. */
+@SuppressWarnings({"keyfor", "nullness"}) // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class FlinkAssignContext<InputT, W extends BoundedWindow>
     extends WindowFn<InputT, W>.AssignContext {
   private final WindowedValue<InputT> value;
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkDoFnFunction.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkDoFnFunction.java
index e799909..0e12b53 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkDoFnFunction.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkDoFnFunction.java
@@ -57,6 +57,7 @@
  * all outputs with the output number. Afterwards a filter will filter out those elements that are
  * not to be in a specific output.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class FlinkDoFnFunction<InputT, OutputT> extends AbstractRichFunction
     implements FlatMapFunction<WindowedValue<InputT>, WindowedValue<RawUnionValue>>,
         MapPartitionFunction<WindowedValue<InputT>, WindowedValue<RawUnionValue>> {
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkExecutableStageFunction.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkExecutableStageFunction.java
index 5081951..b2b96c6 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkExecutableStageFunction.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkExecutableStageFunction.java
@@ -74,6 +74,7 @@
  * coder. The coder's tags are determined by the output coder map. The resulting data set should be
  * further processed by a {@link FlinkExecutableStagePruningFunction}.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class FlinkExecutableStageFunction<InputT> extends AbstractRichFunction
     implements MapPartitionFunction<WindowedValue<InputT>, RawUnionValue>,
         GroupReduceFunction<WindowedValue<InputT>, RawUnionValue> {
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkExecutableStagePruningFunction.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkExecutableStagePruningFunction.java
index 639f297..b29eed7 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkExecutableStagePruningFunction.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkExecutableStagePruningFunction.java
@@ -27,6 +27,7 @@
 import org.apache.flink.util.Collector;
 
 /** A Flink function that demultiplexes output from a {@link FlinkExecutableStageFunction}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class FlinkExecutableStagePruningFunction
     extends RichFlatMapFunction<RawUnionValue, WindowedValue<?>> {
 
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkNonMergingReduceFunction.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkNonMergingReduceFunction.java
index 21a9cac..b1ad8d0 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkNonMergingReduceFunction.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkNonMergingReduceFunction.java
@@ -41,6 +41,7 @@
  * @param <K> Key type.
  * @param <InputT> Input type.
  */
+@SuppressWarnings({"keyfor", "nullness"}) // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class FlinkNonMergingReduceFunction<K, InputT>
     implements GroupReduceFunction<
         WindowedValue<KV<K, InputT>>, WindowedValue<KV<K, Iterable<InputT>>>> {
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkSideInputReader.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkSideInputReader.java
index d5eed87..2facde3 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkSideInputReader.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkSideInputReader.java
@@ -67,9 +67,8 @@
     this.runtimeContext = runtimeContext;
   }
 
-  @Nullable
   @Override
-  public <T> T get(PCollectionView<T> view, BoundedWindow window) {
+  public <T> @Nullable T get(PCollectionView<T> view, BoundedWindow window) {
     checkNotNull(view, "View passed to sideInput cannot be null");
     TupleTag<?> tag = view.getTagInternal();
     checkNotNull(sideInputs.get(tag), "Side input for " + view + " not available.");
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkStatefulDoFnFunction.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkStatefulDoFnFunction.java
index fdc0df2..cd16449 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkStatefulDoFnFunction.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkStatefulDoFnFunction.java
@@ -59,6 +59,7 @@
 import org.joda.time.Instant;
 
 /** A {@link RichGroupReduceFunction} for stateful {@link ParDo} in Flink Batch Runner. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class FlinkStatefulDoFnFunction<K, V, OutputT>
     extends RichGroupReduceFunction<WindowedValue<KV<K, V>>, WindowedValue<RawUnionValue>> {
 
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkStreamingSideInputHandlerFactory.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkStreamingSideInputHandlerFactory.java
index 5a7f1d9..3c5ac61 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkStreamingSideInputHandlerFactory.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkStreamingSideInputHandlerFactory.java
@@ -42,6 +42,7 @@
  * {@link StateRequestHandler} that uses {@link org.apache.beam.runners.core.SideInputHandler} to
  * access the Flink broadcast state that represents side inputs.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class FlinkStreamingSideInputHandlerFactory implements SideInputHandlerFactory {
 
   // Map from side input id to global PCollection id.
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/ImpulseSourceFunction.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/ImpulseSourceFunction.java
index 34af8d2..7455c9d 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/ImpulseSourceFunction.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/ImpulseSourceFunction.java
@@ -33,6 +33,7 @@
  * source alive although its work is already done. It will only shutdown when the streaming job is
  * cancelled.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ImpulseSourceFunction
     implements SourceFunction<WindowedValue<byte[]>>, CheckpointedFunction {
 
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/SingleWindowFlinkCombineRunner.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/SingleWindowFlinkCombineRunner.java
index 976ba35..e597738 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/SingleWindowFlinkCombineRunner.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/SingleWindowFlinkCombineRunner.java
@@ -40,6 +40,7 @@
  * A Flink combine runner takes elements pre-grouped by window and produces output after seeing all
  * input.
  */
+@SuppressWarnings({"keyfor", "nullness"}) // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SingleWindowFlinkCombineRunner<K, InputT, AccumT, OutputT, W extends BoundedWindow>
     extends AbstractFlinkCombineRunner<K, InputT, AccumT, OutputT, W> {
 
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/SortingFlinkCombineRunner.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/SortingFlinkCombineRunner.java
index 283adc5..6bbc847 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/SortingFlinkCombineRunner.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/SortingFlinkCombineRunner.java
@@ -39,6 +39,7 @@
  * A Flink combine runner that first sorts the elements by window and then does one pass that merges
  * windows and outputs results.
  */
+@SuppressWarnings({"keyfor", "nullness"}) // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SortingFlinkCombineRunner<K, InputT, AccumT, OutputT, W extends BoundedWindow>
     extends AbstractFlinkCombineRunner<K, InputT, AccumT, OutputT, W> {
 
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/EncodedValueComparator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/EncodedValueComparator.java
index 7c55def..a4371dd 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/EncodedValueComparator.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/EncodedValueComparator.java
@@ -29,6 +29,7 @@
  * Flink {@link org.apache.flink.api.common.typeutils.TypeComparator} for Beam values that have been
  * encoded to byte data by a {@link Coder}.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class EncodedValueComparator extends TypeComparator<byte[]> {
 
   /** For storing the Reference in encoded form. */
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/utils/CountingPipelineVisitor.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/utils/CountingPipelineVisitor.java
index 5f92d9f..fabcd49 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/utils/CountingPipelineVisitor.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/utils/CountingPipelineVisitor.java
@@ -24,6 +24,7 @@
 import org.apache.beam.sdk.values.PValue;
 
 /** Pipeline visitors that fills a lookup table of {@link PValue} to number of consumers. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class CountingPipelineVisitor extends Pipeline.PipelineVisitor.Defaults {
 
   private final Map<PValue, Integer> numConsumers = new HashMap<>();
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/utils/LookupPipelineVisitor.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/utils/LookupPipelineVisitor.java
index 662e4d6..17a2766 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/utils/LookupPipelineVisitor.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/utils/LookupPipelineVisitor.java
@@ -71,7 +71,7 @@
     return applied;
   }
 
-  public Map<TupleTag<?>, PValue> getInputs(PTransform<?, ?> transform) {
+  public Map<TupleTag<?>, PCollection<?>> getInputs(PTransform<?, ?> transform) {
     return applied(transform).getInputs();
   }
 
@@ -80,7 +80,7 @@
     return (T) Iterables.getOnlyElement(TransformInputs.nonAdditionalInputs(applied(transform)));
   }
 
-  public Map<TupleTag<?>, PValue> getOutputs(PTransform<?, ?> transform) {
+  public Map<TupleTag<?>, PCollection<?>> getOutputs(PTransform<?, ?> transform) {
     return applied(transform).getOutputs();
   }
 
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputFormat.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputFormat.java
index e64c6d3..c4ccc3d 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputFormat.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputFormat.java
@@ -39,6 +39,7 @@
 import org.slf4j.LoggerFactory;
 
 /** Wrapper for executing a {@link Source} as a Flink {@link InputFormat}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SourceInputFormat<T> extends RichInputFormat<WindowedValue<T>, SourceInputSplit<T>> {
   private static final Logger LOG = LoggerFactory.getLogger(SourceInputFormat.class);
 
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputSplit.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputSplit.java
index 97e1402..e6d2d2e 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputSplit.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputSplit.java
@@ -26,6 +26,7 @@
  * around in the input split because Sources simply split up into several Sources for sharding. This
  * is different to how Flink creates a separate InputSplit from an InputFormat.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SourceInputSplit<T> implements InputSplit {
 
   private Source<T> source;
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java
index a48da2c..448dacb 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java
@@ -132,6 +132,7 @@
  */
 // We use Flink's lifecycle methods to initialize transient fields
 @SuppressFBWarnings("SE_TRANSIENT_FIELD_NOT_RESTORED")
+@SuppressWarnings({"keyfor", "nullness"}) // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DoFnOperator<InputT, OutputT>
     extends AbstractStreamOperatorCompat<WindowedValue<OutputT>>
     implements OneInputStreamOperator<WindowedValue<InputT>, WindowedValue<OutputT>>,
@@ -1500,9 +1501,8 @@
       return new Instant(timerService.currentProcessingTime());
     }
 
-    @Nullable
     @Override
-    public Instant currentSynchronizedProcessingTime() {
+    public @Nullable Instant currentSynchronizedProcessingTime() {
       return new Instant(timerService.currentProcessingTime());
     }
 
@@ -1511,9 +1511,8 @@
       return new Instant(getEffectiveInputWatermark());
     }
 
-    @Nullable
     @Override
-    public Instant currentOutputWatermarkTime() {
+    public @Nullable Instant currentOutputWatermarkTime() {
       return new Instant(currentOutputWatermark);
     }
 
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/ExecutableStageDoFnOperator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/ExecutableStageDoFnOperator.java
index 3a25474..08d7a20 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/ExecutableStageDoFnOperator.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/ExecutableStageDoFnOperator.java
@@ -120,6 +120,7 @@
  */
 // We use Flink's lifecycle methods to initialize transient fields
 @SuppressFBWarnings("SE_TRANSIENT_FIELD_NOT_RESTORED")
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ExecutableStageDoFnOperator<InputT, OutputT> extends DoFnOperator<InputT, OutputT> {
 
   private static final Logger LOG = LoggerFactory.getLogger(ExecutableStageDoFnOperator.class);
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SplittableDoFnOperator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SplittableDoFnOperator.java
index 8e4ed88..53b1782 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SplittableDoFnOperator.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SplittableDoFnOperator.java
@@ -60,6 +60,7 @@
  * Flink operator for executing splittable {@link DoFn DoFns}. Specifically, for executing the
  * {@code @ProcessElement} method of a splittable {@link DoFn}.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SplittableDoFnOperator<InputT, OutputT, RestrictionT>
     extends DoFnOperator<KeyedWorkItem<byte[], KV<InputT, RestrictionT>>, OutputT> {
 
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperator.java
index d8c7651..6d263d9 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperator.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperator.java
@@ -46,6 +46,7 @@
 import org.apache.flink.api.java.functions.KeySelector;
 
 /** Flink operator for executing window {@link DoFn DoFns}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class WindowDoFnOperator<K, InputT, OutputT>
     extends DoFnOperator<KeyedWorkItem<K, InputT>, KV<K, OutputT>> {
 
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/DedupingOperator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/DedupingOperator.java
index 5677e1a..5ce41ab 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/DedupingOperator.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/DedupingOperator.java
@@ -38,6 +38,7 @@
 import org.joda.time.Duration;
 
 /** Remove values with duplicate ids. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DedupingOperator<T> extends AbstractStreamOperator<WindowedValue<T>>
     implements OneInputStreamOperator<WindowedValue<ValueWithRecordId<T>>, WindowedValue<T>>,
         Triggerable<ByteBuffer, VoidNamespace> {
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSourceWrapper.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSourceWrapper.java
index d4011d5..06729cd 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSourceWrapper.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSourceWrapper.java
@@ -61,6 +61,7 @@
 import org.slf4j.LoggerFactory;
 
 /** Wrapper for executing {@link UnboundedSource UnboundedSources} as a Flink Source. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class UnboundedSourceWrapper<OutputT, CheckpointMarkT extends UnboundedSource.CheckpointMark>
     extends RichParallelSourceFunction<WindowedValue<ValueWithRecordId<OutputT>>>
     implements ProcessingTimeCallback,
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/stableinput/BufferingDoFnRunner.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/stableinput/BufferingDoFnRunner.java
index 29f3690..31709cd 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/stableinput/BufferingDoFnRunner.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/stableinput/BufferingDoFnRunner.java
@@ -45,6 +45,7 @@
  * after a checkpoint has completed. This ensures that the input is stable and we produce idempotent
  * results on failures.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BufferingDoFnRunner<InputT, OutputT> implements DoFnRunner<InputT, OutputT> {
 
   public static <InputT, OutputT> BufferingDoFnRunner<InputT, OutputT> create(
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkBroadcastStateInternals.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkBroadcastStateInternals.java
index bd6cca4..794a0e4 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkBroadcastStateInternals.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkBroadcastStateInternals.java
@@ -60,6 +60,7 @@
  *
  * <p>Note: Ignore index of key. Mainly for SideInputs.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class FlinkBroadcastStateInternals<K> implements StateInternals {
 
   private int indexInSubtaskGroup;
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkStateInternals.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkStateInternals.java
index e4a3c7b..e2d85af 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkStateInternals.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkStateInternals.java
@@ -77,6 +77,7 @@
  * <p>Note: In the Flink streaming runner the key is always encoded using an {@link Coder} and
  * stored in a {@link ByteBuffer}.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class FlinkStateInternals<K> implements StateInternals {
 
   private static final StateNamespace globalWindowNamespace =
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkExecutionEnvironmentsTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkExecutionEnvironmentsTest.java
index 9698892..f2eb3b1 100644
--- a/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkExecutionEnvironmentsTest.java
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkExecutionEnvironmentsTest.java
@@ -46,6 +46,7 @@
 import org.powermock.reflect.exceptions.FieldNotFoundException;
 
 /** Tests for {@link FlinkExecutionEnvironments}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class FlinkExecutionEnvironmentsTest {
 
   @Rule public TemporaryFolder temporaryFolder = new TemporaryFolder();
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkPipelineExecutionEnvironmentTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkPipelineExecutionEnvironmentTest.java
index 37a49c1..4785f4e 100644
--- a/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkPipelineExecutionEnvironmentTest.java
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkPipelineExecutionEnvironmentTest.java
@@ -77,6 +77,7 @@
 
 /** Tests for {@link FlinkPipelineExecutionEnvironment}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class FlinkPipelineExecutionEnvironmentTest implements Serializable {
 
   @Rule public transient TemporaryFolder tmpFolder = new TemporaryFolder();
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkPipelineOptionsTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkPipelineOptionsTest.java
index 13c04aa..f99a41e 100644
--- a/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkPipelineOptionsTest.java
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkPipelineOptionsTest.java
@@ -52,6 +52,7 @@
 /**
  * Tests for serialization and deserialization of {@link PipelineOptions} in {@link DoFnOperator}.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class FlinkPipelineOptionsTest {
 
   /** Pipeline options. */
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkRequiresStableInputTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkRequiresStableInputTest.java
index 058e3bf..6baf700 100644
--- a/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkRequiresStableInputTest.java
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkRequiresStableInputTest.java
@@ -56,6 +56,7 @@
 import org.junit.rules.TemporaryFolder;
 
 /** Tests {@link DoFn.RequiresStableInput} with Flink. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class FlinkRequiresStableInputTest {
 
   @ClassRule public static TemporaryFolder tempFolder = new TemporaryFolder();
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkRunnerTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkRunnerTest.java
index 0285e74..4cda4cf 100644
--- a/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkRunnerTest.java
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkRunnerTest.java
@@ -33,6 +33,7 @@
 import org.junit.Test;
 
 /** Test for {@link FlinkRunner}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class FlinkRunnerTest extends FlinkRunnerTestCompat {
 
   @Test
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkSavepointTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkSavepointTest.java
index e650d49..9b928d1 100644
--- a/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkSavepointTest.java
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkSavepointTest.java
@@ -81,6 +81,7 @@
  * running pipeline, shutting down the pipeline, and restarting the pipeline from the savepoint with
  * a different parallelism.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class FlinkSavepointTest implements Serializable {
 
   private static final Logger LOG = LoggerFactory.getLogger(FlinkSavepointTest.class);
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkStreamingTransformTranslatorsTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkStreamingTransformTranslatorsTest.java
index b497102..7465c80 100644
--- a/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkStreamingTransformTranslatorsTest.java
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkStreamingTransformTranslatorsTest.java
@@ -43,6 +43,7 @@
 import org.apache.beam.sdk.transforms.PTransform;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PValue;
+import org.apache.beam.sdk.values.PValues;
 import org.apache.beam.sdk.values.TupleTag;
 import org.apache.beam.sdk.values.WindowingStrategy;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
@@ -51,6 +52,7 @@
 import org.junit.Test;
 
 /** Tests for Flink streaming transform translators. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class FlinkStreamingTransformTranslatorsTest {
 
   @Test
@@ -158,7 +160,11 @@
     outputs.put(new TupleTag<>(), pc);
     AppliedPTransform<?, ?, ?> appliedTransform =
         AppliedPTransform.of(
-            "test-transform", Collections.emptyMap(), outputs, transform, Pipeline.create());
+            "test-transform",
+            Collections.emptyMap(),
+            PValues.fullyExpand(outputs),
+            transform,
+            Pipeline.create());
 
     ctx.setCurrentTransform(appliedTransform);
     translator.translateNode(transform, ctx);
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkSubmissionTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkSubmissionTest.java
index 0d6c0ba..9b39917 100644
--- a/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkSubmissionTest.java
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkSubmissionTest.java
@@ -51,6 +51,7 @@
 import org.junit.rules.Timeout;
 
 /** End-to-end submission test of Beam jobs on a Flink cluster. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class FlinkSubmissionTest {
 
   @ClassRule public static final TemporaryFolder TEMP_FOLDER = new TemporaryFolder();
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkTransformOverridesTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkTransformOverridesTest.java
index 6792f1b..5436137 100644
--- a/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkTransformOverridesTest.java
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkTransformOverridesTest.java
@@ -41,12 +41,14 @@
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
 import org.apache.beam.sdk.transforms.windowing.PaneInfo;
 import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.PValues;
 import org.checkerframework.checker.nullness.qual.Nullable;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.TemporaryFolder;
 
 /** Tests if overrides are properly applied. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class FlinkTransformOverridesTest {
 
   @Rule public transient TemporaryFolder tmpFolder = new TemporaryFolder();
@@ -68,7 +70,8 @@
     PCollection<Object> objs = (PCollection) p.apply(Create.empty(VoidCoder.of()));
     AppliedPTransform<PCollection<Object>, WriteFilesResult<Void>, WriteFiles<Object, Void, Object>>
         originalApplication =
-            AppliedPTransform.of("writefiles", objs.expand(), Collections.emptyMap(), original, p);
+            AppliedPTransform.of(
+                "writefiles", PValues.expandInput(objs), Collections.emptyMap(), original, p);
 
     WriteFiles<Object, Void, Object> replacement =
         (WriteFiles<Object, Void, Object>)
@@ -94,9 +97,8 @@
             throw new UnsupportedOperationException("should not be called");
           }
 
-          @Nullable
           @Override
-          public ResourceId unwindowedFilename(
+          public @Nullable ResourceId unwindowedFilename(
               int shardNumber, int numShards, OutputFileHints outputFileHints) {
             throw new UnsupportedOperationException("should not be called");
           }
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/PortableExecutionTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/PortableExecutionTest.java
index 00ca07c..7b85b4b 100644
--- a/runners/flink/src/test/java/org/apache/beam/runners/flink/PortableExecutionTest.java
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/PortableExecutionTest.java
@@ -61,6 +61,7 @@
  * batch and streaming.
  */
 @RunWith(Parameterized.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class PortableExecutionTest implements Serializable {
 
   private static final Logger LOG = LoggerFactory.getLogger(PortableExecutionTest.class);
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/PortableStateExecutionTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/PortableStateExecutionTest.java
index bff62b3..7d6a0bc 100644
--- a/runners/flink/src/test/java/org/apache/beam/runners/flink/PortableStateExecutionTest.java
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/PortableStateExecutionTest.java
@@ -58,6 +58,7 @@
  * org.apache.beam.runners.flink.translation.wrappers.streaming.ExecutableStageDoFnOperator}.
  */
 @RunWith(Parameterized.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class PortableStateExecutionTest implements Serializable {
 
   private static final Logger LOG = LoggerFactory.getLogger(PortableStateExecutionTest.class);
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/PortableTimersExecutionTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/PortableTimersExecutionTest.java
index 18f6dd7..26ef0b1 100644
--- a/runners/flink/src/test/java/org/apache/beam/runners/flink/PortableTimersExecutionTest.java
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/PortableTimersExecutionTest.java
@@ -71,6 +71,7 @@
  * of a given timer is run.
  */
 @RunWith(Parameterized.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class PortableTimersExecutionTest implements Serializable {
 
   private static final Logger LOG = LoggerFactory.getLogger(PortableTimersExecutionTest.class);
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/ReadSourcePortableTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/ReadSourcePortableTest.java
index 2a24072..eae4aad 100644
--- a/runners/flink/src/test/java/org/apache/beam/runners/flink/ReadSourcePortableTest.java
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/ReadSourcePortableTest.java
@@ -49,6 +49,7 @@
 
 /** Tests that Read translation is supported in portable pipelines. */
 @RunWith(Parameterized.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ReadSourcePortableTest implements Serializable {
 
   private static final Logger LOG = LoggerFactory.getLogger(ReadSourcePortableTest.class);
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/ReadSourceStreamingTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/ReadSourceStreamingTest.java
index 5f2434d..e6edf8b 100644
--- a/runners/flink/src/test/java/org/apache/beam/runners/flink/ReadSourceStreamingTest.java
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/ReadSourceStreamingTest.java
@@ -30,6 +30,7 @@
 import org.junit.Test;
 
 /** Reads from a bounded source in streaming. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ReadSourceStreamingTest extends AbstractTestBase {
 
   protected String resultDir;
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/ReadSourceTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/ReadSourceTest.java
index 96d45dd..8a21c4b 100644
--- a/runners/flink/src/test/java/org/apache/beam/runners/flink/ReadSourceTest.java
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/ReadSourceTest.java
@@ -29,6 +29,7 @@
 import org.apache.flink.test.util.JavaProgramTestBase;
 
 /** Reads from a bounded source in batch execution. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ReadSourceTest extends JavaProgramTestBase {
 
   protected String resultPath;
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/batch/NonMergingGroupByKeyTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/batch/NonMergingGroupByKeyTest.java
index 935ca21..60928b4 100644
--- a/runners/flink/src/test/java/org/apache/beam/runners/flink/batch/NonMergingGroupByKeyTest.java
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/batch/NonMergingGroupByKeyTest.java
@@ -34,6 +34,7 @@
 import org.junit.Assume;
 import org.junit.Test;
 
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class NonMergingGroupByKeyTest extends AbstractTestBase {
 
   private static class ReiterateDoFn<K, V> extends DoFn<KV<K, Iterable<V>>, Void> {
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/batch/ReshuffleTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/batch/ReshuffleTest.java
index bf0d739..4dca884 100644
--- a/runners/flink/src/test/java/org/apache/beam/runners/flink/batch/ReshuffleTest.java
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/batch/ReshuffleTest.java
@@ -36,6 +36,7 @@
 import org.junit.Assert;
 import org.junit.Test;
 
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ReshuffleTest {
 
   private static class WithBundleIdFn extends DoFn<String, String> {
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/metrics/FlinkMetricContainerTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/metrics/FlinkMetricContainerTest.java
index 9f8046a..72fa43c 100644
--- a/runners/flink/src/test/java/org/apache/beam/runners/flink/metrics/FlinkMetricContainerTest.java
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/metrics/FlinkMetricContainerTest.java
@@ -57,6 +57,7 @@
 import org.mockito.MockitoAnnotations;
 
 /** Tests for {@link FlinkMetricContainer}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class FlinkMetricContainerTest {
 
   @Mock private RuntimeContext runtimeContext;
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/FlinkBroadcastStateInternalsTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/FlinkBroadcastStateInternalsTest.java
index 2dfc85e..0011db9 100644
--- a/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/FlinkBroadcastStateInternalsTest.java
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/FlinkBroadcastStateInternalsTest.java
@@ -35,6 +35,7 @@
  * <p>Just test value, bag and combining.
  */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class FlinkBroadcastStateInternalsTest extends StateInternalsTest {
 
   @Override
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/FlinkStateInternalsTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/FlinkStateInternalsTest.java
index ede3c05..c92c893 100644
--- a/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/FlinkStateInternalsTest.java
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/FlinkStateInternalsTest.java
@@ -55,6 +55,7 @@
 
 /** Tests for {@link FlinkStateInternals}. This is based on {@link StateInternalsTest}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class FlinkStateInternalsTest extends StateInternalsTest {
 
   @Override
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/GroupByNullKeyTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/GroupByNullKeyTest.java
index 6d08906..b0d6c4e 100644
--- a/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/GroupByNullKeyTest.java
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/GroupByNullKeyTest.java
@@ -42,6 +42,7 @@
 import org.junit.Test;
 
 /** Test for GroupByNullKey. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class GroupByNullKeyTest extends AbstractTestBase implements Serializable {
 
   protected String resultDir;
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/GroupByWithNullValuesTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/GroupByWithNullValuesTest.java
index 19e544e..d783801 100644
--- a/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/GroupByWithNullValuesTest.java
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/GroupByWithNullValuesTest.java
@@ -39,6 +39,7 @@
 import org.junit.Test;
 
 /** Tests grouping with null values. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class GroupByWithNullValuesTest implements Serializable {
 
   @Test
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/TopWikipediaSessionsTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/TopWikipediaSessionsTest.java
index 63abfa5..13c05d1 100644
--- a/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/TopWikipediaSessionsTest.java
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/TopWikipediaSessionsTest.java
@@ -41,6 +41,7 @@
 import org.junit.Test;
 
 /** Session window test. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class TopWikipediaSessionsTest extends AbstractTestBase implements Serializable {
 
   protected String resultDir;
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/functions/FlinkDoFnFunctionTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/functions/FlinkDoFnFunctionTest.java
index 779f9c1..998d44c 100644
--- a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/functions/FlinkDoFnFunctionTest.java
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/functions/FlinkDoFnFunctionTest.java
@@ -34,6 +34,7 @@
 import org.powermock.reflect.Whitebox;
 
 /** Tests for {@link FlinkDoFnFunction}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class FlinkDoFnFunctionTest {
 
   @Test
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/functions/FlinkExecutableStageFunctionTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/functions/FlinkExecutableStageFunctionTest.java
index 20f6f64..126810f 100644
--- a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/functions/FlinkExecutableStageFunctionTest.java
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/functions/FlinkExecutableStageFunctionTest.java
@@ -69,6 +69,7 @@
 
 /** Tests for {@link FlinkExecutableStageFunction}. */
 @RunWith(Parameterized.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class FlinkExecutableStageFunctionTest {
 
   @Parameterized.Parameters
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/functions/FlinkStatefulDoFnFunctionTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/functions/FlinkStatefulDoFnFunctionTest.java
index 4f9707c..c76b771 100644
--- a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/functions/FlinkStatefulDoFnFunctionTest.java
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/functions/FlinkStatefulDoFnFunctionTest.java
@@ -34,6 +34,7 @@
 import org.powermock.reflect.Whitebox;
 
 /** Tests for {@link FlinkStatefulDoFnFunction}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class FlinkStatefulDoFnFunctionTest {
 
   @Test
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperatorTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperatorTest.java
index 83d84c2..06dfb53 100644
--- a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperatorTest.java
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperatorTest.java
@@ -101,6 +101,7 @@
 
 /** Tests for {@link DoFnOperator}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings({"keyfor", "nullness"}) // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DoFnOperatorTest {
 
   // views and windows for testing side inputs
@@ -2121,10 +2122,9 @@
         .filter(o -> o instanceof StreamRecord)
         .transform(
             new Function<Object, WindowedValue<String>>() {
-              @Nullable
               @Override
               @SuppressWarnings({"unchecked", "rawtypes"})
-              public WindowedValue<String> apply(@Nullable Object o) {
+              public @Nullable WindowedValue<String> apply(@Nullable Object o) {
                 if (o instanceof StreamRecord) {
                   return (WindowedValue<String>) ((StreamRecord) o).getValue();
                 }
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/ExecutableStageDoFnOperatorTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/ExecutableStageDoFnOperatorTest.java
index 6a29dd7..02684bc 100644
--- a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/ExecutableStageDoFnOperatorTest.java
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/ExecutableStageDoFnOperatorTest.java
@@ -130,6 +130,7 @@
 
 /** Tests for {@link ExecutableStageDoFnOperator}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ExecutableStageDoFnOperatorTest {
   @Rule public ExpectedException thrown = ExpectedException.none();
 
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/StreamRecordStripper.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/StreamRecordStripper.java
index cc52ff9..0832b45 100644
--- a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/StreamRecordStripper.java
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/StreamRecordStripper.java
@@ -23,6 +23,7 @@
 import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
 import org.checkerframework.checker.nullness.qual.Nullable;
 
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class StreamRecordStripper {
 
   @SuppressWarnings("Guava")
@@ -33,10 +34,9 @@
                 o instanceof StreamRecord && ((StreamRecord) o).getValue() instanceof WindowedValue)
         .transform(
             new Function<Object, WindowedValue<T>>() {
-              @Nullable
               @Override
               @SuppressWarnings({"unchecked", "rawtypes"})
-              public WindowedValue<T> apply(@Nullable Object o) {
+              public @Nullable WindowedValue<T> apply(@Nullable Object o) {
                 if (o instanceof StreamRecord
                     && ((StreamRecord) o).getValue() instanceof WindowedValue) {
                   return (WindowedValue) ((StreamRecord) o).getValue();
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperatorTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperatorTest.java
index 5aeddcf..371cbd0 100644
--- a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperatorTest.java
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperatorTest.java
@@ -64,6 +64,7 @@
 
 /** Tests for {@link WindowDoFnOperator}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class WindowDoFnOperatorTest {
 
   @Test
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/TestCountingSource.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/TestCountingSource.java
index 92bf298..abd6b92 100644
--- a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/TestCountingSource.java
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/TestCountingSource.java
@@ -44,6 +44,7 @@
  * The reader will occasionally return false from {@code advance}, in order to simulate a source
  * where not all the data is available immediately.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class TestCountingSource
     extends UnboundedSource<KV<Integer, Integer>, TestCountingSource.CounterMark> {
   private static final Logger LOG = LoggerFactory.getLogger(TestCountingSource.class);
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSourceWrapperTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSourceWrapperTest.java
index a7aff37..60204c7 100644
--- a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSourceWrapperTest.java
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSourceWrapperTest.java
@@ -87,6 +87,7 @@
 
 /** Tests for {@link UnboundedSourceWrapper}. */
 @RunWith(Enclosed.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class UnboundedSourceWrapperTest {
 
   private static final Logger LOG = LoggerFactory.getLogger(UnboundedSourceWrapperTest.class);
diff --git a/runners/google-cloud-dataflow-java/build.gradle b/runners/google-cloud-dataflow-java/build.gradle
index fb2553b..c0d7d67 100644
--- a/runners/google-cloud-dataflow-java/build.gradle
+++ b/runners/google-cloud-dataflow-java/build.gradle
@@ -147,6 +147,8 @@
   'org.apache.beam.sdk.testing.LargeKeys$Above10MB',
   'org.apache.beam.sdk.testing.UsesAttemptedMetrics',
   'org.apache.beam.sdk.testing.UsesCrossLanguageTransforms',
+  'org.apache.beam.sdk.testing.UsesDistributionMetrics',
+  'org.apache.beam.sdk.testing.UsesGaugeMetrics',
   'org.apache.beam.sdk.testing.UsesSetState',
   'org.apache.beam.sdk.testing.UsesMapState',
   'org.apache.beam.sdk.testing.UsesOrderedListState',
@@ -361,6 +363,9 @@
     excludeTestsMatching 'org.apache.beam.sdk.metrics.MetricsTest$AttemptedMetricTests.testAttemptedGaugeMetrics'
     excludeTestsMatching 'org.apache.beam.sdk.metrics.MetricsTest$CommittedMetricTests.testAllCommittedMetrics'
     excludeTestsMatching 'org.apache.beam.sdk.metrics.MetricsTest$CommittedMetricTests.testCommittedGaugeMetrics'
+
+    // TODO(BEAM-11130): support OrderedListState in Google Cloud Dataflow Runner V2
+    excludeTestsMatching 'org.apache.beam.sdk.transforms.ParDoTest$StateTests.testOrderedListState*'
   }
 }
 
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BatchStatefulParDoOverrides.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BatchStatefulParDoOverrides.java
index d4e9485..e93a328 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BatchStatefulParDoOverrides.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BatchStatefulParDoOverrides.java
@@ -45,7 +45,6 @@
 import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PCollectionTuple;
-import org.apache.beam.sdk.values.PValue;
 import org.apache.beam.sdk.values.TupleTag;
 import org.apache.beam.sdk.values.TypeDescriptor;
 import org.apache.beam.sdk.values.WindowingStrategy;
@@ -121,8 +120,8 @@
     }
 
     @Override
-    public Map<PValue, ReplacementOutput> mapOutputs(
-        Map<TupleTag<?>, PValue> outputs, PCollection<OutputT> newOutput) {
+    public Map<PCollection<?>, ReplacementOutput> mapOutputs(
+        Map<TupleTag<?>, PCollection<?>> outputs, PCollection<OutputT> newOutput) {
       return ReplacementOutputs.singleton(outputs, newOutput);
     }
   }
@@ -151,8 +150,8 @@
     }
 
     @Override
-    public Map<PValue, ReplacementOutput> mapOutputs(
-        Map<TupleTag<?>, PValue> outputs, PCollectionTuple newOutput) {
+    public Map<PCollection<?>, ReplacementOutput> mapOutputs(
+        Map<TupleTag<?>, PCollection<?>> outputs, PCollectionTuple newOutput) {
       return ReplacementOutputs.tagged(outputs, newOutput);
     }
   }
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BatchViewOverrides.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BatchViewOverrides.java
index 82a0089..bf07965 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BatchViewOverrides.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BatchViewOverrides.java
@@ -90,6 +90,7 @@
 /**
  * Dataflow batch overrides for {@link CreatePCollectionView}, specialized for different view types.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class BatchViewOverrides {
   /**
    * Specialized implementation for {@link org.apache.beam.sdk.transforms.View.AsMap View.AsMap} for
@@ -125,7 +126,8 @@
    * org.apache.beam.sdk.transforms.View.AsSingleton View.AsSingleton} printing a warning to users
    * to specify a deterministic key coder.
    */
-  static class BatchViewAsMap<K, V> extends PTransform<PCollection<KV<K, V>>, PCollection<?>> {
+  static class BatchViewAsMap<K, V>
+      extends PTransform<PCollection<KV<K, V>>, PCollectionView<Map<K, V>>> {
 
     /**
      * A {@link DoFn} which groups elements by window boundaries. For each group, the group of
@@ -205,11 +207,12 @@
     }
 
     @Override
-    public PCollection<?> expand(PCollection<KV<K, V>> input) {
+    public PCollectionView<Map<K, V>> expand(PCollection<KV<K, V>> input) {
       return this.applyInternal(input);
     }
 
-    private <W extends BoundedWindow> PCollection<?> applyInternal(PCollection<KV<K, V>> input) {
+    private <W extends BoundedWindow> PCollectionView<Map<K, V>> applyInternal(
+        PCollection<KV<K, V>> input) {
       try {
         return BatchViewAsMultimap.applyForMapLike(runner, input, view, true /* unique keys */);
       } catch (NonDeterministicException e) {
@@ -227,7 +230,7 @@
     }
 
     /** Transforms the input {@link PCollection} into a singleton {@link Map} per window. */
-    private <W extends BoundedWindow> PCollection<?> applyForSingletonFallback(
+    private <W extends BoundedWindow> PCollectionView<Map<K, V>> applyForSingletonFallback(
         PCollection<KV<K, V>> input) {
       @SuppressWarnings("unchecked")
       Coder<W> windowCoder = (Coder<W>) input.getWindowingStrategy().getWindowFn().windowCoder();
@@ -285,7 +288,8 @@
    * org.apache.beam.sdk.transforms.View.AsSingleton View.AsSingleton} printing a warning to users
    * to specify a deterministic key coder.
    */
-  static class BatchViewAsMultimap<K, V> extends PTransform<PCollection<KV<K, V>>, PCollection<?>> {
+  static class BatchViewAsMultimap<K, V>
+      extends PTransform<PCollection<KV<K, V>>, PCollectionView<Map<K, Iterable<V>>>> {
     /**
      * A {@link PTransform} that groups elements by the hash of window's byte representation if the
      * input {@link PCollection} is not within the global window. Otherwise by the hash of the
@@ -689,11 +693,12 @@
     }
 
     @Override
-    public PCollection<?> expand(PCollection<KV<K, V>> input) {
+    public PCollectionView<Map<K, Iterable<V>>> expand(PCollection<KV<K, V>> input) {
       return this.applyInternal(input);
     }
 
-    private <W extends BoundedWindow> PCollection<?> applyInternal(PCollection<KV<K, V>> input) {
+    private <W extends BoundedWindow> PCollectionView<Map<K, Iterable<V>>> applyInternal(
+        PCollection<KV<K, V>> input) {
       try {
         return applyForMapLike(runner, input, view, false /* unique keys not expected */);
       } catch (NonDeterministicException e) {
@@ -706,8 +711,9 @@
     }
 
     /** Transforms the input {@link PCollection} into a singleton {@link Map} per window. */
-    private <W extends BoundedWindow> PCollection<?> applyForSingletonFallback(
-        PCollection<KV<K, V>> input) {
+    private <W extends BoundedWindow>
+        PCollectionView<Map<K, Iterable<V>>> applyForSingletonFallback(
+            PCollection<KV<K, V>> input) {
       @SuppressWarnings("unchecked")
       Coder<W> windowCoder = (Coder<W>) input.getWindowingStrategy().getWindowFn().windowCoder();
 
@@ -730,7 +736,7 @@
           runner, input, new ToMultimapDoFn<>(windowCoder), finalValueCoder, view);
     }
 
-    private static <K, V, W extends BoundedWindow, ViewT> PCollection<?> applyForMapLike(
+    private static <K, V, W extends BoundedWindow, ViewT> PCollectionView<ViewT> applyForMapLike(
         DataflowRunner runner,
         PCollection<KV<K, V>> input,
         PCollectionView<ViewT> view,
@@ -819,7 +825,7 @@
       PCollection<IsmRecord<WindowedValue<V>>> flattenedOutputs =
           Pipeline.applyTransform(outputs, Flatten.pCollections());
       flattenedOutputs.apply(CreateDataflowView.forBatch(view));
-      return flattenedOutputs;
+      return view;
     }
 
     @Override
@@ -852,7 +858,7 @@
    *   <li>Value: Windowed value
    * </ul>
    */
-  static class BatchViewAsSingleton<T> extends PTransform<PCollection<T>, PCollection<?>> {
+  static class BatchViewAsSingleton<T> extends PTransform<PCollection<T>, PCollectionView<T>> {
 
     /**
      * A {@link DoFn} that outputs {@link IsmRecord}s. These records are structured as follows:
@@ -914,7 +920,7 @@
     }
 
     @Override
-    public PCollection<?> expand(PCollection<T> input) {
+    public PCollectionView<T> expand(PCollection<T> input) {
       input = input.apply(Combine.globally(combineFn).withoutDefaults().withFanout(fanout));
       @SuppressWarnings("unchecked")
       Coder<BoundedWindow> windowCoder =
@@ -928,7 +934,7 @@
           view);
     }
 
-    static <T, FinalT, ViewT, W extends BoundedWindow> PCollection<?> applyForSingleton(
+    static <T, FinalT, ViewT, W extends BoundedWindow> PCollectionView<ViewT> applyForSingleton(
         DataflowRunner runner,
         PCollection<T> input,
         DoFn<KV<Integer, Iterable<KV<W, WindowedValue<T>>>>, IsmRecord<WindowedValue<FinalT>>> doFn,
@@ -949,7 +955,7 @@
 
       runner.addPCollectionRequiringIndexedFormat(reifiedPerWindowAndSorted);
       reifiedPerWindowAndSorted.apply(CreateDataflowView.forBatch(view));
-      return reifiedPerWindowAndSorted;
+      return view;
     }
 
     @Override
@@ -980,7 +986,7 @@
    *   <li>Value: Windowed value
    * </ul>
    */
-  static class BatchViewAsList<T> extends PTransform<PCollection<T>, PCollection<?>> {
+  static class BatchViewAsList<T> extends PTransform<PCollection<T>, PCollectionView<List<T>>> {
     /**
      * A {@link DoFn} which creates {@link IsmRecord}s assuming that each element is within the
      * global window. Each {@link IsmRecord} has
@@ -1061,11 +1067,11 @@
     }
 
     @Override
-    public PCollection<?> expand(PCollection<T> input) {
+    public PCollectionView<List<T>> expand(PCollection<T> input) {
       return applyForIterableLike(runner, input, view);
     }
 
-    static <T, W extends BoundedWindow, ViewT> PCollection<?> applyForIterableLike(
+    static <T, W extends BoundedWindow, ViewT> PCollectionView<ViewT> applyForIterableLike(
         DataflowRunner runner, PCollection<T> input, PCollectionView<ViewT> view) {
 
       @SuppressWarnings("unchecked")
@@ -1085,7 +1091,7 @@
 
         runner.addPCollectionRequiringIndexedFormat(reifiedPerWindowAndSorted);
         reifiedPerWindowAndSorted.apply(CreateDataflowView.forBatch(view));
-        return reifiedPerWindowAndSorted;
+        return view;
       }
 
       PCollection<IsmRecord<WindowedValue<T>>> reifiedPerWindowAndSorted =
@@ -1096,7 +1102,7 @@
 
       runner.addPCollectionRequiringIndexedFormat(reifiedPerWindowAndSorted);
       reifiedPerWindowAndSorted.apply(CreateDataflowView.forBatch(view));
-      return reifiedPerWindowAndSorted;
+      return view;
     }
 
     @Override
@@ -1129,7 +1135,8 @@
    *   <li>Value: Windowed value
    * </ul>
    */
-  static class BatchViewAsIterable<T> extends PTransform<PCollection<T>, PCollection<?>> {
+  static class BatchViewAsIterable<T>
+      extends PTransform<PCollection<T>, PCollectionView<Iterable<T>>> {
 
     private final transient DataflowRunner runner;
     private final PCollectionView<Iterable<T>> view;
@@ -1142,7 +1149,7 @@
     }
 
     @Override
-    public PCollection<?> expand(PCollection<T> input) {
+    public PCollectionView<Iterable<T>> expand(PCollection<T> input) {
       return BatchViewAsList.applyForIterableLike(runner, input, view);
     }
   }
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/CreateDataflowView.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/CreateDataflowView.java
index 272ee26..8d5f07d 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/CreateDataflowView.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/CreateDataflowView.java
@@ -22,6 +22,7 @@
 import org.apache.beam.sdk.values.PCollectionView;
 
 /** A {@link DataflowRunner} marker class for creating a {@link PCollectionView}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class CreateDataflowView<ElemT, ViewT>
     extends PTransform<PCollection<ElemT>, PCollection<ElemT>> {
   public static <ElemT, ViewT> CreateDataflowView<ElemT, ViewT> forBatch(
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowClient.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowClient.java
index 528bcf9..e889b81 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowClient.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowClient.java
@@ -35,6 +35,7 @@
 import org.checkerframework.checker.nullness.qual.Nullable;
 
 /** Wrapper around the generated {@link Dataflow} client to provide common functionality. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DataflowClient {
 
   public static DataflowClient create(DataflowPipelineOptions options) {
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowMetrics.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowMetrics.java
index f64ed72..e2c05fb 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowMetrics.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowMetrics.java
@@ -43,6 +43,7 @@
 import org.slf4j.LoggerFactory;
 
 /** Implementation of {@link MetricResults} for the Dataflow Runner. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class DataflowMetrics extends MetricResults {
   private static final Logger LOG = LoggerFactory.getLogger(DataflowMetrics.class);
   /**
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPTransformMatchers.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPTransformMatchers.java
index a4a4700..281aaa7 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPTransformMatchers.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPTransformMatchers.java
@@ -31,6 +31,7 @@
  * A set of {@link PTransformMatcher PTransformMatchers} that are used in the Dataflow Runner and
  * not general enough to be shared between runners.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class DataflowPTransformMatchers {
   private DataflowPTransformMatchers() {}
 
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineJob.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineJob.java
index 72a8018..5b8d0ed 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineJob.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineJob.java
@@ -53,6 +53,7 @@
 import org.slf4j.LoggerFactory;
 
 /** A DataflowPipelineJob represents a job submitted to Dataflow using {@link DataflowRunner}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DataflowPipelineJob implements PipelineResult {
 
   private static final Logger LOG = LoggerFactory.getLogger(DataflowPipelineJob.class);
@@ -200,9 +201,9 @@
    * @return The final state of the job or null on timeout or if the thread is interrupted.
    * @throws IOException If there is a persistent problem getting job information.
    */
-  @Nullable
   @VisibleForTesting
-  public State waitUntilFinish(Duration duration, MonitoringUtil.JobMessagesHandler messageHandler)
+  public @Nullable State waitUntilFinish(
+      Duration duration, MonitoringUtil.JobMessagesHandler messageHandler)
       throws IOException, InterruptedException {
     // We ignore the potential race condition here (Ctrl-C after job submission but before the
     // shutdown hook is registered). Even if we tried to do something smarter (eg., SettableFuture)
@@ -230,8 +231,8 @@
     }
   }
 
-  @Nullable
   @VisibleForTesting
+  @Nullable
   State waitUntilFinish(
       Duration duration,
       MonitoringUtil.@Nullable JobMessagesHandler messageHandler,
@@ -265,8 +266,8 @@
    * @throws IOException If there is a persistent problem getting job information.
    * @throws InterruptedException if the thread is interrupted.
    */
-  @Nullable
   @VisibleForTesting
+  @Nullable
   State waitUntilFinish(
       Duration duration,
       MonitoringUtil.@Nullable JobMessagesHandler messageHandler,
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 07d0ca1..a5c25e9 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
@@ -119,7 +119,11 @@
  * {@link DataflowPipelineTranslator} knows how to translate {@link Pipeline} objects into Cloud
  * Dataflow Service API {@link Job}s.
  */
-@SuppressWarnings({"rawtypes", "unchecked"})
+@SuppressWarnings({
+  "rawtypes",
+  "unchecked",
+  "nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
+})
 @VisibleForTesting
 public class DataflowPipelineTranslator {
 
@@ -356,7 +360,9 @@
           if (experiments.contains(GcpOptions.STREAMING_ENGINE_EXPERIMENT)
               || experiments.contains(GcpOptions.WINDMILL_SERVICE_EXPERIMENT)) {
             throw new IllegalArgumentException(
-                "Streaming engine both disabled and enabled: enableStreamingEngine is set to false, but enable_windmill_service and/or enable_streaming_engine are present. It is recommended you only set enableStreamingEngine.");
+                "Streaming engine both disabled and enabled: enableStreamingEngine is set to"
+                    + " false, but enable_windmill_service and/or enable_streaming_engine are"
+                    + " present. It is recommended you only set enableStreamingEngine.");
           }
         }
       }
@@ -432,7 +438,7 @@
     }
 
     @Override
-    public <InputT extends PInput> Map<TupleTag<?>, PValue> getInputs(
+    public <InputT extends PInput> Map<TupleTag<?>, PCollection<?>> getInputs(
         PTransform<InputT, ?> transform) {
       return getCurrentTransform(transform).getInputs();
     }
@@ -445,7 +451,7 @@
     }
 
     @Override
-    public <OutputT extends POutput> Map<TupleTag<?>, PValue> getOutputs(
+    public <OutputT extends POutput> Map<TupleTag<?>, PCollection<?>> getOutputs(
         PTransform<?, OutputT> transform) {
       return getCurrentTransform(transform).getOutputs();
     }
@@ -1274,15 +1280,10 @@
   }
 
   private static void translateOutputs(
-      Map<TupleTag<?>, PValue> outputs, StepTranslationContext stepContext) {
-    for (Map.Entry<TupleTag<?>, PValue> taggedOutput : outputs.entrySet()) {
+      Map<TupleTag<?>, PCollection<?>> outputs, StepTranslationContext stepContext) {
+    for (Map.Entry<TupleTag<?>, PCollection<?>> taggedOutput : outputs.entrySet()) {
       TupleTag<?> tag = taggedOutput.getKey();
-      checkArgument(
-          taggedOutput.getValue() instanceof PCollection,
-          "Non %s returned from Multi-output %s",
-          PCollection.class.getSimpleName(),
-          stepContext);
-      stepContext.addOutput(tag.getId(), (PCollection<?>) taggedOutput.getValue());
+      stepContext.addOutput(tag.getId(), taggedOutput.getValue());
     }
   }
 
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java
index 9eb162c..8931143 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java
@@ -121,7 +121,6 @@
 import org.apache.beam.sdk.runners.TransformHierarchy;
 import org.apache.beam.sdk.runners.TransformHierarchy.Node;
 import org.apache.beam.sdk.state.MapState;
-import org.apache.beam.sdk.state.OrderedListState;
 import org.apache.beam.sdk.state.SetState;
 import org.apache.beam.sdk.transforms.Combine;
 import org.apache.beam.sdk.transforms.Combine.CombineFn;
@@ -189,6 +188,7 @@
  * <p>Please see <a href="https://cloud.google.com/dataflow/security-and-permissions">Google Cloud
  * Dataflow Security and Permissions</a> for more details.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DataflowRunner extends PipelineRunner<DataflowPipelineJob> {
 
   private static final Logger LOG = LoggerFactory.getLogger(DataflowRunner.class);
@@ -355,7 +355,8 @@
       throw new IllegalArgumentException(
           "Project ID '"
               + project
-              + "' invalid. Please make sure you specified the Project ID, not project description.");
+              + "' invalid. Please make sure you specified the Project ID, not project"
+              + " description.");
     }
 
     DataflowPipelineDebugOptions debugOptions =
@@ -480,16 +481,16 @@
                 PTransformMatchers.classEqualTo(PubsubUnboundedSink.class),
                 new StreamingPubsubIOWriteOverrideFactory(this)));
       }
+      overridesBuilder.add(
+          PTransformOverride.of(
+              PTransformMatchers.writeWithRunnerDeterminedSharding(),
+              new StreamingShardedWriteFactory(options)));
       if (fnApiEnabled) {
         overridesBuilder.add(
             PTransformOverride.of(
                 PTransformMatchers.classEqualTo(Create.Values.class),
                 new StreamingFnApiCreateOverrideFactory()));
       }
-      overridesBuilder.add(
-          PTransformOverride.of(
-              PTransformMatchers.writeWithRunnerDeterminedSharding(),
-              new StreamingShardedWriteFactory(options)));
 
       overridesBuilder.add(
           PTransformOverride.of(
@@ -615,9 +616,13 @@
     }
 
     @Override
-    public PTransformReplacement<PCollection<InputT>, PValue> getReplacementTransform(
-        AppliedPTransform<PCollection<InputT>, PValue, PTransform<PCollection<InputT>, PValue>>
-            transform) {
+    public PTransformReplacement<PCollection<InputT>, PCollectionView<ViewT>>
+        getReplacementTransform(
+            AppliedPTransform<
+                    PCollection<InputT>,
+                    PCollectionView<ViewT>,
+                    PTransform<PCollection<InputT>, PCollectionView<ViewT>>>
+                transform) {
       Combine.GloballyAsSingletonView<?, ?> combineTransform =
           (Combine.GloballyAsSingletonView) transform.getTransform();
       return PTransformReplacement.of(
@@ -637,20 +642,25 @@
    */
   private static class ReflectiveViewOverrideFactory<InputT, ViewT>
       implements PTransformOverrideFactory<
-          PCollection<InputT>, PValue, PTransform<PCollection<InputT>, PValue>> {
+          PCollection<InputT>,
+          PCollectionView<ViewT>,
+          PTransform<PCollection<InputT>, PCollectionView<ViewT>>> {
 
-    final Class<PTransform<PCollection<InputT>, PValue>> replacement;
+    final Class<PTransform<PCollection<InputT>, PCollectionView<ViewT>>> replacement;
     final DataflowRunner runner;
 
     private ReflectiveViewOverrideFactory(
-        Class<PTransform<PCollection<InputT>, PValue>> replacement, DataflowRunner runner) {
+        Class<PTransform<PCollection<InputT>, PCollectionView<ViewT>>> replacement,
+        DataflowRunner runner) {
       this.replacement = replacement;
       this.runner = runner;
     }
 
     CreatePCollectionView<ViewT, ViewT> findCreatePCollectionView(
         final AppliedPTransform<
-                PCollection<InputT>, PValue, PTransform<PCollection<InputT>, PValue>>
+                PCollection<InputT>,
+                PCollectionView<ViewT>,
+                PTransform<PCollection<InputT>, PCollectionView<ViewT>>>
             transform) {
       final AtomicReference<CreatePCollectionView> viewTransformRef = new AtomicReference<>();
       transform
@@ -698,12 +708,15 @@
     }
 
     @Override
-    public PTransformReplacement<PCollection<InputT>, PValue> getReplacementTransform(
-        final AppliedPTransform<
-                PCollection<InputT>, PValue, PTransform<PCollection<InputT>, PValue>>
-            transform) {
+    public PTransformReplacement<PCollection<InputT>, PCollectionView<ViewT>>
+        getReplacementTransform(
+            final AppliedPTransform<
+                    PCollection<InputT>,
+                    PCollectionView<ViewT>,
+                    PTransform<PCollection<InputT>, PCollectionView<ViewT>>>
+                transform) {
 
-      PTransform<PCollection<InputT>, PValue> rep =
+      PTransform<PCollection<InputT>, PCollectionView<ViewT>> rep =
           InstanceBuilder.ofType(replacement)
               .withArg(DataflowRunner.class, runner)
               .withArg(CreatePCollectionView.class, findCreatePCollectionView(transform))
@@ -713,11 +726,15 @@
     }
 
     @Override
-    public Map<PValue, ReplacementOutput> mapOutputs(
-        Map<TupleTag<?>, PValue> outputs, PValue newOutput) {
-      // We do not replace any of the outputs because we expect that the new PTransform will
-      // re-use the original PCollectionView that was returned.
-      return ImmutableMap.of();
+    public Map<PCollection<?>, ReplacementOutput> mapOutputs(
+        Map<TupleTag<?>, PCollection<?>> outputs, PCollectionView<ViewT> newOutput) {
+      /*
+      The output of View.AsXYZ is a PCollectionView that expands to the PCollection to be materialized.
+      The PCollectionView itself must have the same tag since that tag may have been embedded in serialized DoFns
+      previously and cannot easily be rewired. The PCollection may differ, so we rewire it, even if the rewiring
+      is a noop.
+      */
+      return ReplacementOutputs.singleton(outputs, newOutput);
     }
   }
 
@@ -1128,8 +1145,8 @@
         throw new DataflowJobAlreadyExistsException(
             dataflowPipelineJob,
             String.format(
-                "There is already an active job named %s with id: %s. If you want "
-                    + "to submit a second job, try again by setting a different name using --jobName.",
+                "There is already an active job named %s with id: %s. If you want to submit a"
+                    + " second job, try again by setting a different name using --jobName.",
                 newJob.getName(), jobResult.getId()));
       }
     }
@@ -1256,10 +1273,10 @@
           });
 
       LOG.warn(
-          "Unable to use indexed implementation for View.AsMap and View.AsMultimap for {} "
-              + "because the key coder is not deterministic. Falling back to singleton implementation "
-              + "which may cause memory and/or performance problems. Future major versions of "
-              + "Dataflow will require deterministic key coders.",
+          "Unable to use indexed implementation for View.AsMap and View.AsMultimap for {} because"
+              + " the key coder is not deterministic. Falling back to singleton implementation"
+              + " which may cause memory and/or performance problems. Future major versions of"
+              + " Dataflow will require deterministic key coders.",
           ptransformViewNamesWithNonDeterministicKeyCoders);
     }
   }
@@ -1315,8 +1332,8 @@
     }
 
     @Override
-    public Map<PValue, ReplacementOutput> mapOutputs(
-        Map<TupleTag<?>, PValue> outputs, PCollection<PubsubMessage> newOutput) {
+    public Map<PCollection<?>, ReplacementOutput> mapOutputs(
+        Map<TupleTag<?>, PCollection<?>> outputs, PCollection<PubsubMessage> newOutput) {
       return ReplacementOutputs.singleton(outputs, newOutput);
     }
   }
@@ -1515,8 +1532,8 @@
     }
 
     @Override
-    public Map<PValue, ReplacementOutput> mapOutputs(
-        Map<TupleTag<?>, PValue> outputs, PCollection<T> newOutput) {
+    public Map<PCollection<?>, ReplacementOutput> mapOutputs(
+        Map<TupleTag<?>, PCollection<?>> outputs, PCollection<T> newOutput) {
       return ReplacementOutputs.singleton(outputs, newOutput);
     }
   }
@@ -1644,8 +1661,8 @@
     }
 
     @Override
-    public Map<PValue, ReplacementOutput> mapOutputs(
-        Map<TupleTag<?>, PValue> outputs, PCollection<T> newOutput) {
+    public Map<PCollection<?>, ReplacementOutput> mapOutputs(
+        Map<TupleTag<?>, PCollection<?>> outputs, PCollection<T> newOutput) {
       return ReplacementOutputs.singleton(outputs, newOutput);
     }
   }
@@ -1771,8 +1788,8 @@
     }
 
     @Override
-    public Map<PValue, ReplacementOutput> mapOutputs(
-        Map<TupleTag<?>, PValue> outputs, PCollection<T> newOutput) {
+    public Map<PCollection<?>, ReplacementOutput> mapOutputs(
+        Map<TupleTag<?>, PCollection<?>> outputs, PCollection<T> newOutput) {
       return ReplacementOutputs.singleton(outputs, newOutput);
     }
   }
@@ -1910,8 +1927,8 @@
     }
 
     @Override
-    public Map<PValue, ReplacementOutput> mapOutputs(
-        Map<TupleTag<?>, PValue> outputs, PCollection<KV<K, OutputT>> newOutput) {
+    public Map<PCollection<?>, ReplacementOutput> mapOutputs(
+        Map<TupleTag<?>, PCollection<?>> outputs, PCollection<KV<K, OutputT>> newOutput) {
       return ReplacementOutputs.singleton(outputs, newOutput);
     }
   }
@@ -1934,8 +1951,8 @@
     }
 
     @Override
-    public Map<PValue, ReplacementOutput> mapOutputs(
-        Map<TupleTag<?>, PValue> outputs, PDone newOutput) {
+    public Map<PCollection<?>, ReplacementOutput> mapOutputs(
+        Map<TupleTag<?>, PCollection<?>> outputs, PDone newOutput) {
       return Collections.emptyMap();
     }
   }
@@ -1997,8 +2014,8 @@
     }
 
     @Override
-    public Map<PValue, ReplacementOutput> mapOutputs(
-        Map<TupleTag<?>, PValue> outputs, WriteFilesResult<DestinationT> newOutput) {
+    public Map<PCollection<?>, ReplacementOutput> mapOutputs(
+        Map<TupleTag<?>, PCollection<?>> outputs, WriteFilesResult<DestinationT> newOutput) {
       return ReplacementOutputs.tagged(outputs, newOutput);
     }
   }
@@ -2046,12 +2063,6 @@
               "%s does not currently support %s",
               DataflowRunner.class.getSimpleName(), MapState.class.getSimpleName()));
     }
-    if (DoFnSignatures.usesOrderedListState(fn)) {
-      throw new UnsupportedOperationException(
-          String.format(
-              "%s does not currently support %s",
-              DataflowRunner.class.getSimpleName(), OrderedListState.class.getSimpleName()));
-    }
     if (streaming && DoFnSignatures.requiresTimeSortedInput(fn)) {
       throw new UnsupportedOperationException(
           String.format(
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunnerInfo.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunnerInfo.java
index 6aa49d4..08506a0 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunnerInfo.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunnerInfo.java
@@ -29,6 +29,7 @@
 import org.slf4j.LoggerFactory;
 
 /** Populates versioning and other information for {@link DataflowRunner}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public final class DataflowRunnerInfo extends ReleaseInfo {
   private static final Logger LOG = LoggerFactory.getLogger(DataflowRunnerInfo.class);
 
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/GroupIntoBatchesOverride.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/GroupIntoBatchesOverride.java
index b0e5115..b6a13ef 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/GroupIntoBatchesOverride.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/GroupIntoBatchesOverride.java
@@ -31,7 +31,6 @@
 import org.apache.beam.sdk.transforms.ParDo;
 import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PValue;
 import org.apache.beam.sdk.values.TupleTag;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterators;
 
@@ -53,8 +52,8 @@
     }
 
     @Override
-    public Map<PValue, ReplacementOutput> mapOutputs(
-        Map<TupleTag<?>, PValue> outputs, PCollection<KV<K, Iterable<V>>> newOutput) {
+    public Map<PCollection<?>, ReplacementOutput> mapOutputs(
+        Map<TupleTag<?>, PCollection<?>> outputs, PCollection<KV<K, Iterable<V>>> newOutput) {
       return ReplacementOutputs.singleton(outputs, newOutput);
     }
   }
@@ -115,8 +114,8 @@
     }
 
     @Override
-    public Map<PValue, ReplacementOutput> mapOutputs(
-        Map<TupleTag<?>, PValue> outputs, PCollection<KV<K, Iterable<V>>> newOutput) {
+    public Map<PCollection<?>, ReplacementOutput> mapOutputs(
+        Map<TupleTag<?>, PCollection<?>> outputs, PCollection<KV<K, Iterable<V>>> newOutput) {
       return ReplacementOutputs.singleton(outputs, newOutput);
     }
   }
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/PrimitiveParDoSingleFactory.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/PrimitiveParDoSingleFactory.java
index 3256848..a7903f5 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/PrimitiveParDoSingleFactory.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/PrimitiveParDoSingleFactory.java
@@ -69,18 +69,20 @@
  * ParDo.SingleOutput} instances. {@link ParDoSingle} is a primitive {@link PTransform}, to ensure
  * that {@link DisplayData} appears on all {@link ParDo ParDos} in the {@link DataflowRunner}.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class PrimitiveParDoSingleFactory<InputT, OutputT>
     extends SingleInputOutputOverrideFactory<
-        PCollection<? extends InputT>, PCollection<OutputT>, ParDo.SingleOutput<InputT, OutputT>> {
+        PCollection<? extends InputT>, PCollection<OutputT>, SingleOutput<InputT, OutputT>> {
   @Override
-  public PTransformReplacement<PCollection<? extends InputT>, PCollection<OutputT>>
+  public PTransformOverrideFactory.PTransformReplacement<
+          PCollection<? extends InputT>, PCollection<OutputT>>
       getReplacementTransform(
           AppliedPTransform<
                   PCollection<? extends InputT>,
                   PCollection<OutputT>,
                   SingleOutput<InputT, OutputT>>
               transform) {
-    return PTransformReplacement.of(
+    return PTransformOverrideFactory.PTransformReplacement.of(
         PTransformReplacements.getSingletonMainInput(transform),
         new ParDoSingle<>(
             transform.getTransform(),
@@ -195,7 +197,8 @@
       if (signature.usesState() || signature.usesTimers()) {
         checkArgument(
             mainInput.getCoder() instanceof KvCoder,
-            "DoFn's that use state or timers must have an input PCollection with a KvCoder but received %s",
+            "DoFn's that use state or timers must have an input PCollection with a KvCoder but"
+                + " received %s",
             mainInput.getCoder());
         keyCoder = ((KvCoder) mainInput.getCoder()).getKeyCoder();
       } else {
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/RequiresStableInputParDoOverrides.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/RequiresStableInputParDoOverrides.java
index 31b5520..403e618 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/RequiresStableInputParDoOverrides.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/RequiresStableInputParDoOverrides.java
@@ -28,7 +28,6 @@
 import org.apache.beam.sdk.transforms.Reshuffle;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PCollectionTuple;
-import org.apache.beam.sdk.values.PValue;
 import org.apache.beam.sdk.values.TupleTag;
 
 /** Transform overrides for supporting {@link RequiresStableInput} in the Dataflow runner. */
@@ -78,8 +77,8 @@
     }
 
     @Override
-    public Map<PValue, ReplacementOutput> mapOutputs(
-        Map<TupleTag<?>, PValue> outputs, PCollection<OutputT> newOutput) {
+    public Map<PCollection<?>, ReplacementOutput> mapOutputs(
+        Map<TupleTag<?>, PCollection<?>> outputs, PCollection<OutputT> newOutput) {
       return ReplacementOutputs.singleton(outputs, newOutput);
     }
   }
@@ -105,8 +104,8 @@
     }
 
     @Override
-    public Map<PValue, ReplacementOutput> mapOutputs(
-        Map<TupleTag<?>, PValue> outputs, PCollectionTuple newOutput) {
+    public Map<PCollection<?>, ReplacementOutput> mapOutputs(
+        Map<TupleTag<?>, PCollection<?>> outputs, PCollectionTuple newOutput) {
       return ReplacementOutputs.tagged(outputs, newOutput);
     }
   }
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/ReshuffleOverrideFactory.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/ReshuffleOverrideFactory.java
index 17f92bc..cf4d6f4 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/ReshuffleOverrideFactory.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/ReshuffleOverrideFactory.java
@@ -49,7 +49,7 @@
       getReplacementTransform(
           AppliedPTransform<PCollection<KV<K, V>>, PCollection<KV<K, V>>, Reshuffle<K, V>>
               transform) {
-    return PTransformReplacement.of(
+    return PTransformOverrideFactory.PTransformReplacement.of(
         PTransformReplacements.getSingletonMainInput(transform), new ReshuffleWithOnlyTrigger<>());
   }
 
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/SplittableParDoOverrides.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/SplittableParDoOverrides.java
index 7b65950..4105b29 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/SplittableParDoOverrides.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/SplittableParDoOverrides.java
@@ -28,7 +28,6 @@
 import org.apache.beam.sdk.transforms.ParDo;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PCollectionTuple;
-import org.apache.beam.sdk.values.PValue;
 import org.apache.beam.sdk.values.TupleTag;
 import org.apache.beam.sdk.values.TupleTagList;
 
@@ -68,8 +67,8 @@
     }
 
     @Override
-    public Map<PValue, ReplacementOutput> mapOutputs(
-        Map<TupleTag<?>, PValue> outputs, PCollectionTuple newOutput) {
+    public Map<PCollection<?>, ReplacementOutput> mapOutputs(
+        Map<TupleTag<?>, PCollection<?>> outputs, PCollectionTuple newOutput) {
       return ReplacementOutputs.tagged(outputs, newOutput);
     }
   }
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/TestDataflowRunner.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/TestDataflowRunner.java
index 57494d6..d060ed9 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/TestDataflowRunner.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/TestDataflowRunner.java
@@ -52,6 +52,7 @@
  *
  * @see TestPipeline
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class TestDataflowRunner extends PipelineRunner<DataflowPipelineJob> {
   private static final String TENTATIVE_COUNTER = "tentative";
   private static final Logger LOG = LoggerFactory.getLogger(TestDataflowRunner.class);
@@ -288,8 +289,8 @@
     return Optional.absent();
   }
 
-  @Nullable
   @VisibleForTesting
+  @Nullable
   JobMetrics getJobMetrics(DataflowPipelineJob job) {
     JobMetrics metrics = null;
     try {
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/TransformTranslator.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/TransformTranslator.java
index 1289de8..f88dd07 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/TransformTranslator.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/TransformTranslator.java
@@ -62,12 +62,14 @@
     DataflowPipelineOptions getPipelineOptions();
 
     /** Returns the input of the currently being translated transform. */
-    <InputT extends PInput> Map<TupleTag<?>, PValue> getInputs(PTransform<InputT, ?> transform);
+    <InputT extends PInput> Map<TupleTag<?>, PCollection<?>> getInputs(
+        PTransform<InputT, ?> transform);
 
     <InputT extends PValue> InputT getInput(PTransform<InputT, ?> transform);
 
     /** Returns the output of the currently being translated transform. */
-    <OutputT extends POutput> Map<TupleTag<?>, PValue> getOutputs(PTransform<?, OutputT> transform);
+    <OutputT extends POutput> Map<TupleTag<?>, PCollection<?>> getOutputs(
+        PTransform<?, OutputT> transform);
 
     <OutputT extends PValue> OutputT getOutput(PTransform<?, OutputT> transform);
 
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/IsmFormat.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/IsmFormat.java
index c569916..119229c 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/IsmFormat.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/IsmFormat.java
@@ -99,6 +99,7 @@
  * the number of shard index records followed by that many shard index records. See {@link
  * IsmShardCoder} for further details as to its encoding scheme.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class IsmFormat {
   private static final int HASH_SEED = 1225801234;
   private static final HashFunction HASH_FUNCTION = Hashing.murmur3_32(HASH_SEED);
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineOptions.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineOptions.java
index 69cfb0a..daa706a 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineOptions.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineOptions.java
@@ -39,6 +39,7 @@
 
 /** Options that can be used to configure the {@link DataflowRunner}. */
 @Description("Options that configure the Dataflow pipeline.")
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public interface DataflowPipelineOptions
     extends PipelineOptions,
         GcpOptions,
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DefaultGcpRegionFactory.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DefaultGcpRegionFactory.java
index 59435ee..34413c7 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DefaultGcpRegionFactory.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DefaultGcpRegionFactory.java
@@ -37,6 +37,7 @@
  * be found, returns the empty string.
  */
 @VisibleForTesting
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DefaultGcpRegionFactory implements DefaultValueFactory<String> {
   private static final Logger LOG = LoggerFactory.getLogger(DefaultGcpRegionFactory.class);
 
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/CloudKnownType.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/CloudKnownType.java
index a75e3b0..13de98a 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/CloudKnownType.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/CloudKnownType.java
@@ -24,7 +24,10 @@
 import org.checkerframework.checker.nullness.qual.Nullable;
 
 /** A utility for manipulating well-known cloud types. */
-@SuppressWarnings("ImmutableEnumChecker")
+@SuppressWarnings({
+  "ImmutableEnumChecker",
+  "nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
+})
 enum CloudKnownType {
   TEXT("http://schema.org/Text", String.class) {
     @Override
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/CloudObject.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/CloudObject.java
index db0190e..b007d78 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/CloudObject.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/CloudObject.java
@@ -34,6 +34,7 @@
  * add additional properties to be presented during deserialization, representing child objects by
  * building additional {@code CloudObject}s.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public final class CloudObject extends GenericJson implements Cloneable {
   /**
    * Constructs a {@code CloudObject} by copying the supplied serialized object spec, which must
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/CloudObjectTranslators.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/CloudObjectTranslators.java
index 7834784..b8a9876 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/CloudObjectTranslators.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/CloudObjectTranslators.java
@@ -48,6 +48,7 @@
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
 
 /** Utilities for creating {@link CloudObjectTranslator} instances for {@link Coder Coders}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class CloudObjectTranslators {
   private CloudObjectTranslators() {}
 
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/CloudObjects.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/CloudObjects.java
index 8d1b94e..bc5d3de 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/CloudObjects.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/CloudObjects.java
@@ -40,6 +40,7 @@
 import org.checkerframework.checker.nullness.qual.Nullable;
 
 /** Utilities for converting an object to a {@link CloudObject}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class CloudObjects {
   private CloudObjects() {}
 
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/DataflowTemplateJob.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/DataflowTemplateJob.java
index ddced86..076e74a 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/DataflowTemplateJob.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/DataflowTemplateJob.java
@@ -25,6 +25,7 @@
 import org.joda.time.Duration;
 
 /** A {@link DataflowPipelineJob} that is returned when {@code --templateRunner} is set. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DataflowTemplateJob extends DataflowPipelineJob {
   private static final String ERROR = "The result of template creation should not be used.";
 
@@ -47,8 +48,8 @@
     throw new UnsupportedOperationException(ERROR);
   }
 
-  @Nullable
   @VisibleForTesting
+  @Nullable
   State waitUntilFinish(
       Duration duration,
       MonitoringUtil.JobMessagesHandler messageHandler,
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/PackageUtil.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/PackageUtil.java
index ebc6d7c..92eb4e9 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/PackageUtil.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/PackageUtil.java
@@ -66,6 +66,7 @@
 import org.slf4j.LoggerFactory;
 
 /** Helper routines for packages. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class PackageUtil implements Closeable {
 
   private static final Logger LOG = LoggerFactory.getLogger(PackageUtil.class);
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/RandomAccessData.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/RandomAccessData.java
index 8a0452c..bdd89ce 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/RandomAccessData.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/RandomAccessData.java
@@ -50,6 +50,7 @@
  * temporary unused storage.
  */
 @NotThreadSafe
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class RandomAccessData {
   /**
    * A {@link Coder} which encodes the valid parts of this stream. This follows the same encoding
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/Structs.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/Structs.java
index 095bd05..44d5c36 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/Structs.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/Structs.java
@@ -30,6 +30,7 @@
  * A collection of static methods for manipulating datastructure representations transferred via the
  * Dataflow API.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public final class Structs {
   private Structs() {} // Non-instantiable
 
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/TimeUtil.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/TimeUtil.java
index c11556f..8fad3ba 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/TimeUtil.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/TimeUtil.java
@@ -40,6 +40,7 @@
  * <p>In both formats, fractional seconds are either three digits (millisecond resolution), six
  * digits (microsecond resolution), or nine digits (nanosecond resolution).
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public final class TimeUtil {
   private TimeUtil() {} // Non-instantiable.
 
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/BatchStatefulParDoOverridesTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/BatchStatefulParDoOverridesTest.java
index 6c422a1..1751a34 100644
--- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/BatchStatefulParDoOverridesTest.java
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/BatchStatefulParDoOverridesTest.java
@@ -58,6 +58,7 @@
 
 /** Tests for {@link BatchStatefulParDoOverrides}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BatchStatefulParDoOverridesTest implements Serializable {
 
   @Test
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowMetricsTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowMetricsTest.java
index 7ed285d..70a9f25 100644
--- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowMetricsTest.java
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowMetricsTest.java
@@ -59,6 +59,7 @@
 
 /** Tests for {@link DataflowMetrics}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DataflowMetricsTest {
   private static final String PROJECT_ID = "some-project";
   private static final String JOB_ID = "1234";
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPTransformMatchersTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPTransformMatchersTest.java
index fe0ef3f..fb93044 100644
--- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPTransformMatchersTest.java
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPTransformMatchersTest.java
@@ -43,6 +43,7 @@
 
 /** Tests for {@link DataflowPTransformMatchers}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DataflowPTransformMatchersTest {
 
   /**
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineJobTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineJobTest.java
index 1f2a8cc..572b065 100644
--- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineJobTest.java
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineJobTest.java
@@ -75,6 +75,7 @@
 
 /** Tests for DataflowPipelineJob. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DataflowPipelineJobTest {
   private static final String PROJECT_ID = "some-project";
   private static final String REGION_ID = "some-region-2b";
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslatorTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslatorTest.java
index 150b545..202c1e7 100644
--- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslatorTest.java
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslatorTest.java
@@ -130,6 +130,7 @@
 
 /** Tests for DataflowPipelineTranslator. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DataflowPipelineTranslatorTest implements Serializable {
 
   @Rule public transient ExpectedException thrown = ExpectedException.none();
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerInfoTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerInfoTest.java
index bf67979..728b32e 100644
--- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerInfoTest.java
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerInfoTest.java
@@ -31,6 +31,7 @@
  * <p>Note that tests for checking that the Dataflow distribution correctly loads overridden
  * properties is contained within the Dataflow distribution.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DataflowRunnerInfoTest {
 
   @Test
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java
index 9c86f2b..d5e8fa7 100644
--- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java
@@ -141,7 +141,7 @@
 import org.apache.beam.sdk.transforms.windowing.Window;
 import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PValue;
+import org.apache.beam.sdk.values.PValues;
 import org.apache.beam.sdk.values.TimestampedValue;
 import org.apache.beam.sdk.values.WindowingStrategy;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Throwables;
@@ -173,6 +173,7 @@
  * <p>Implements {@link Serializable} because it is caught in closures.
  */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DataflowRunnerTest implements Serializable {
 
   private static final String VALID_BUCKET = "valid-bucket";
@@ -1677,7 +1678,8 @@
     PCollection<Object> objs = (PCollection) p.apply(Create.empty(VoidCoder.of()));
     AppliedPTransform<PCollection<Object>, WriteFilesResult<Void>, WriteFiles<Object, Void, Object>>
         originalApplication =
-            AppliedPTransform.of("writefiles", objs.expand(), Collections.emptyMap(), original, p);
+            AppliedPTransform.of(
+                "writefiles", PValues.expandInput(objs), Collections.emptyMap(), original, p);
 
     WriteFiles<Object, Void, Object> replacement =
         (WriteFiles<Object, Void, Object>)
@@ -1687,8 +1689,8 @@
 
     WriteFilesResult<Void> originalResult = objs.apply(original);
     WriteFilesResult<Void> replacementResult = objs.apply(replacement);
-    Map<PValue, ReplacementOutput> res =
-        factory.mapOutputs(originalResult.expand(), replacementResult);
+    Map<PCollection<?>, ReplacementOutput> res =
+        factory.mapOutputs(PValues.expandOutput(originalResult), replacementResult);
     assertEquals(1, res.size());
     assertEquals(
         originalResult.getPerDestinationOutputFilenames(),
@@ -1715,9 +1717,8 @@
                   throw new UnsupportedOperationException("should not be called");
                 }
 
-                @Nullable
                 @Override
-                public ResourceId unwindowedFilename(
+                public @Nullable ResourceId unwindowedFilename(
                     int shardNumber, int numShards, OutputFileHints outputFileHints) {
                   throw new UnsupportedOperationException("should not be called");
                 }
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/PrimitiveParDoSingleFactoryTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/PrimitiveParDoSingleFactoryTest.java
index 67c6d0b..5d73e16 100644
--- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/PrimitiveParDoSingleFactoryTest.java
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/PrimitiveParDoSingleFactoryTest.java
@@ -37,6 +37,7 @@
 import org.apache.beam.sdk.transforms.display.DisplayDataEvaluator;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PCollectionView;
+import org.apache.beam.sdk.values.PValues;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables;
 import org.checkerframework.checker.nullness.qual.Nullable;
 import org.junit.Rule;
@@ -70,8 +71,8 @@
         application =
             AppliedPTransform.of(
                 "original",
-                input.expand(),
-                input.apply(originalTransform).expand(),
+                PValues.expandInput(input),
+                PValues.expandOutput(input.apply(originalTransform)),
                 originalTransform,
                 pipeline);
 
@@ -107,8 +108,8 @@
         application =
             AppliedPTransform.of(
                 "original",
-                input.expand(),
-                input.apply(originalTransform).expand(),
+                PValues.expandInput(input),
+                PValues.expandOutput(input.apply(originalTransform)),
                 originalTransform,
                 pipeline);
 
@@ -129,8 +130,8 @@
         application =
             AppliedPTransform.of(
                 "original",
-                input.expand(),
-                input.apply(originalTransform).expand(),
+                PValues.expandInput(input),
+                PValues.expandOutput(input.apply(originalTransform)),
                 originalTransform,
                 pipeline);
 
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/RecordingPipelineVisitor.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/RecordingPipelineVisitor.java
index e04a676..93ca383 100644
--- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/RecordingPipelineVisitor.java
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/RecordingPipelineVisitor.java
@@ -28,6 +28,7 @@
  * Provides a simple {@link org.apache.beam.sdk.Pipeline.PipelineVisitor} that records the
  * transformation tree.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class RecordingPipelineVisitor extends Pipeline.PipelineVisitor.Defaults {
 
   public final List<PTransform<?, ?>> transforms = new ArrayList<>();
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/TestDataflowRunnerTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/TestDataflowRunnerTest.java
index 68d8791..c796dfa 100644
--- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/TestDataflowRunnerTest.java
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/TestDataflowRunnerTest.java
@@ -75,6 +75,7 @@
 
 /** Tests for {@link TestDataflowRunner}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class TestDataflowRunnerTest {
   @Rule public ExpectedException expectedException = ExpectedException.none();
   @Mock private DataflowClient mockClient;
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/options/DataflowPipelineOptionsTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/options/DataflowPipelineOptionsTest.java
index 292ce61..eb755fc 100644
--- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/options/DataflowPipelineOptionsTest.java
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/options/DataflowPipelineOptionsTest.java
@@ -43,6 +43,7 @@
 
 /** Tests for {@link DataflowPipelineOptions}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DataflowPipelineOptionsTest {
   @Rule public TestRule restoreSystemProperties = new RestoreSystemProperties();
   @Rule public ResetDateTimeProvider resetDateTimeProviderRule = new ResetDateTimeProvider();
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/options/DataflowWorkerLoggingOptionsTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/options/DataflowWorkerLoggingOptionsTest.java
index 1fa5b14..dba4ce8 100644
--- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/options/DataflowWorkerLoggingOptionsTest.java
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/options/DataflowWorkerLoggingOptionsTest.java
@@ -32,6 +32,7 @@
 
 /** Tests for {@link DataflowWorkerLoggingOptions}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DataflowWorkerLoggingOptionsTest {
   private static final ObjectMapper MAPPER =
       new ObjectMapper()
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/transforms/DataflowGroupByKeyTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/transforms/DataflowGroupByKeyTest.java
index 13b9655..5e40aa5 100644
--- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/transforms/DataflowGroupByKeyTest.java
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/transforms/DataflowGroupByKeyTest.java
@@ -50,6 +50,7 @@
 
 /** Tests for {@link GroupByKey} for the {@link DataflowRunner}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DataflowGroupByKeyTest {
   @Rule public ExpectedException thrown = ExpectedException.none();
 
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/transforms/DataflowViewTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/transforms/DataflowViewTest.java
index 8d4067e..6c7ccc3 100644
--- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/transforms/DataflowViewTest.java
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/transforms/DataflowViewTest.java
@@ -51,6 +51,7 @@
 
 /** Tests for {@link View} for a {@link DataflowRunner}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DataflowViewTest {
   @Rule public transient ExpectedException thrown = ExpectedException.none();
 
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 21fa593..e50511a 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
@@ -80,6 +80,7 @@
 
 /** Tests for {@link CloudObjects}. */
 @RunWith(Enclosed.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class CloudObjectsTest {
   private static final Schema TEST_SCHEMA =
       Schema.builder()
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/MonitoringUtilTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/MonitoringUtilTest.java
index df169b6..dd3b700 100644
--- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/MonitoringUtilTest.java
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/MonitoringUtilTest.java
@@ -45,6 +45,7 @@
 
 /** Tests for MonitoringUtil. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class MonitoringUtilTest {
   private static final String PROJECT_ID = "someProject";
   private static final String REGION_ID = "thatRegion";
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/PackageUtilTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/PackageUtilTest.java
index e1a5f4a..002d0a6 100644
--- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/PackageUtilTest.java
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/PackageUtilTest.java
@@ -104,6 +104,7 @@
 
 /** Tests for {@link PackageUtil}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class PackageUtilTest {
   @Rule public ExpectedLogs logged = ExpectedLogs.none(PackageUtil.class);
   @Rule public TemporaryFolder tmpFolder = new TemporaryFolder();
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/StructsTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/StructsTest.java
index 91804ab..a28df3c 100644
--- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/StructsTest.java
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/StructsTest.java
@@ -48,6 +48,7 @@
 
 /** Tests for Structs. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class StructsTest {
   private List<Map<String, Object>> makeCloudObjects() {
     List<Map<String, Object>> objects = new ArrayList<>();
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/TimeUtilTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/TimeUtilTest.java
index 1ac9fab..0825ebc 100644
--- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/TimeUtilTest.java
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/TimeUtilTest.java
@@ -32,6 +32,7 @@
 
 /** Unit tests for {@link TimeUtil}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public final class TimeUtilTest {
   @Test
   public void toCloudTimeShouldPrintTimeStrings() {
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ApplianceShuffleReader.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ApplianceShuffleReader.java
index a8bf014..d344311 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ApplianceShuffleReader.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ApplianceShuffleReader.java
@@ -28,6 +28,7 @@
  * <p>It is a JNI wrapper of an equivalent C++ class.
  */
 @ThreadSafe
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public final class ApplianceShuffleReader implements ShuffleReader, Closeable {
   static {
     ShuffleLibrary.load();
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ApplianceShuffleWriter.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ApplianceShuffleWriter.java
index 6fb787e..28d7fc0 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ApplianceShuffleWriter.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ApplianceShuffleWriter.java
@@ -27,6 +27,7 @@
  * <p>It is a JNI wrapper of an equivalent C++ class.
  */
 @ThreadSafe
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public final class ApplianceShuffleWriter implements ShuffleWriter {
   static {
     ShuffleLibrary.load();
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/AvroByteReader.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/AvroByteReader.java
index bfeb3ca..1024694 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/AvroByteReader.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/AvroByteReader.java
@@ -46,6 +46,7 @@
  *
  * @param <T> the type of the elements read from the source
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class AvroByteReader<T> extends NativeReader<T> {
   private static final Logger LOG = LoggerFactory.getLogger(AvroByteReader.class);
 
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/AvroByteReaderFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/AvroByteReaderFactory.java
index 55f150d..0e7d584 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/AvroByteReaderFactory.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/AvroByteReaderFactory.java
@@ -32,6 +32,7 @@
 import org.checkerframework.checker.nullness.qual.Nullable;
 
 /** Creates an {@link AvroByteReader} from a CloudObject spec. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class AvroByteReaderFactory implements ReaderFactory {
 
   /** A {@link ReaderFactory.Registrar} for Avro byte sources. */
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/BatchDataflowWorker.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/BatchDataflowWorker.java
index 2780c50..531791a 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/BatchDataflowWorker.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/BatchDataflowWorker.java
@@ -68,6 +68,7 @@
  * <p>DataflowWorker presents one public interface, getAndPerformWork(), which uses the
  * WorkUnitClient to get work, execute it, and update the work.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BatchDataflowWorker implements Closeable {
   private static final Logger LOG = LoggerFactory.getLogger(BatchDataflowWorker.class);
 
@@ -382,7 +383,7 @@
       workItemStatusClient.setWorker(worker, executionContext);
 
       DataflowWorkProgressUpdater progressUpdater =
-          new DataflowWorkProgressUpdater(workItemStatusClient, workItem, worker);
+          new DataflowWorkProgressUpdater(workItemStatusClient, workItem, worker, options);
       executeWork(worker, progressUpdater);
       workItemStatusClient.reportSuccess();
       return true;
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/BatchModeExecutionContext.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/BatchModeExecutionContext.java
index 7e836ff..b9d35cc 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/BatchModeExecutionContext.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/BatchModeExecutionContext.java
@@ -54,6 +54,7 @@
 import org.joda.time.Instant;
 
 /** {@link DataflowExecutionContext} for use in batch mode. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BatchModeExecutionContext
     extends DataflowExecutionContext<BatchModeExecutionContext.StepContext> {
 
@@ -186,9 +187,8 @@
      * <p>Final updates are extracted by the execution thread, and will be reported after all
      * processing has completed and the writer thread has been shutdown.
      */
-    @Nullable
     @Override
-    public CounterUpdate extractUpdate(boolean isFinalUpdate) {
+    public @Nullable CounterUpdate extractUpdate(boolean isFinalUpdate) {
       long millisToReport = totalMillisInState;
       if (millisToReport == lastReportedMillis && !isFinalUpdate) {
         return null;
@@ -445,9 +445,8 @@
       return wrapped.getUserTimerInternals();
     }
 
-    @Nullable
     @Override
-    public <W extends BoundedWindow> TimerData getNextFiredTimer(Coder<W> windowCoder) {
+    public <W extends BoundedWindow> @Nullable TimerData getNextFiredTimer(Coder<W> windowCoder) {
       // Only event time timers fire, as processing time timers are reserved until after the
       // bundle is complete, so they are all delivered droppably late
       //
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/BatchModeUngroupingParDoFn.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/BatchModeUngroupingParDoFn.java
index d099064..2cb0ff7 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/BatchModeUngroupingParDoFn.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/BatchModeUngroupingParDoFn.java
@@ -30,6 +30,7 @@
  * <p>Each input element must be a {@link KV} where the value is an iterable of {@link WindowedValue
  * WindowedValues}.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class BatchModeUngroupingParDoFn<K, V> implements ParDoFn {
 
   private final ParDoFn underlyingParDoFn;
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/BeamFnMapTaskExecutorFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/BeamFnMapTaskExecutorFactory.java
index 33cdb39..699126a 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/BeamFnMapTaskExecutorFactory.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/BeamFnMapTaskExecutorFactory.java
@@ -110,6 +110,7 @@
 import org.slf4j.LoggerFactory;
 
 /** Creates a {@link DataflowMapTaskExecutor} from a {@link MapTask} definition. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BeamFnMapTaskExecutorFactory implements DataflowMapTaskExecutorFactory {
   private static final Logger LOG = LoggerFactory.getLogger(BeamFnMapTaskExecutorFactory.class);
 
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/CombineValuesFnFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/CombineValuesFnFactory.java
index 4ffd851..484d65d 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/CombineValuesFnFactory.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/CombineValuesFnFactory.java
@@ -52,6 +52,7 @@
  * A {@link ParDoFnFactory} to create instances of user {@link CombineFn} according to
  * specifications from the Dataflow service.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class CombineValuesFnFactory implements ParDoFnFactory {
 
   @Override
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ConcatReader.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ConcatReader.java
index ba01d6e..d98c483 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ConcatReader.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ConcatReader.java
@@ -56,6 +56,7 @@
  *
  * @param <T> Type of the elements read by the {@link NativeReader}s.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ConcatReader<T> extends NativeReader<T> {
   private static final Logger LOG = LoggerFactory.getLogger(ConcatReader.class);
 
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ConcatReaderFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ConcatReaderFactory.java
index ea82b4c..03d2afa 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ConcatReaderFactory.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ConcatReaderFactory.java
@@ -36,6 +36,7 @@
 import org.checkerframework.checker.nullness.qual.Nullable;
 
 /** Creates an {@link ConcatReader} from a {@link CloudObject} spec. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ConcatReaderFactory implements ReaderFactory {
 
   private final ReaderRegistry registry;
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/CounterShortIdCache.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/CounterShortIdCache.java
index 2d81036..67eff6f 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/CounterShortIdCache.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/CounterShortIdCache.java
@@ -40,6 +40,7 @@
  * fine because the total number of unique counters is expected to be small and limited by the
  * backend.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class CounterShortIdCache {
   private static final Logger LOG = LoggerFactory.getLogger(CounterShortIdCache.class);
   private Cache cache = new Cache();
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/CreateIsmShardKeyAndSortKeyDoFnFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/CreateIsmShardKeyAndSortKeyDoFnFactory.java
index 3042bc5..f221141 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/CreateIsmShardKeyAndSortKeyDoFnFactory.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/CreateIsmShardKeyAndSortKeyDoFnFactory.java
@@ -47,6 +47,7 @@
  * <p>This {@link ParDoFnFactory} is part of an expansion of steps required to materialize ISM
  * files. See <a href="go/dataflow-side-inputs">go/dataflow-side-inputs</a> for further details.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class CreateIsmShardKeyAndSortKeyDoFnFactory implements ParDoFnFactory {
 
   @Override
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowElementExecutionTracker.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowElementExecutionTracker.java
index 492bd4e..a6606ec 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowElementExecutionTracker.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowElementExecutionTracker.java
@@ -62,6 +62,7 @@
  * Beam/Dataflow-specific classes, such as {@link PipelineOptions} which cannot be shared with
  * Flume.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DataflowElementExecutionTracker extends ElementExecutionTracker {
   @VisibleForTesting
   public static final String TIME_PER_ELEMENT_EXPERIMENT = "time_per_element_counter";
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowExecutionContext.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowExecutionContext.java
index aaf8056..17e0163 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowExecutionContext.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowExecutionContext.java
@@ -49,6 +49,7 @@
 import org.joda.time.Instant;
 
 /** Execution context for the Dataflow worker. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public abstract class DataflowExecutionContext<T extends DataflowStepContext> {
 
   private final CounterFactory counterFactory;
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowExecutionStateRegistry.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowExecutionStateRegistry.java
index a1a4b95..77e02ba 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowExecutionStateRegistry.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowExecutionStateRegistry.java
@@ -30,6 +30,7 @@
 import org.checkerframework.checker.nullness.qual.Nullable;
 
 /** Manages the instances of {@link ExecutionState} */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public abstract class DataflowExecutionStateRegistry {
 
   /**
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowMetricsContainer.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowMetricsContainer.java
index dd29aa8..6de7853 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowMetricsContainer.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowMetricsContainer.java
@@ -34,6 +34,7 @@
 // not clear why the interface extends Serializable
 // https://issues.apache.org/jira/browse/BEAM-6573
 @SuppressFBWarnings("SE_BAD_FIELD")
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DataflowMetricsContainer implements MetricsContainer {
 
   private final ExecutionStateTracker executionStateTracker;
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowOperationContext.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowOperationContext.java
index 7f28512..3177db6 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowOperationContext.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowOperationContext.java
@@ -54,6 +54,7 @@
  * {@link OperationContext} that manages the current {@link ExecutionState} to ensure the
  * start/process/finish/abort states are properly tracked.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DataflowOperationContext implements OperationContext {
 
   private static final Logger LOG = LoggerFactory.getLogger(DataflowOperationContext.class);
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowOutputCounter.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowOutputCounter.java
index ea6b941..67f4809 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowOutputCounter.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowOutputCounter.java
@@ -31,6 +31,7 @@
  * A Dataflow-specific version of {@link ElementCounter}, which specifies the object counter name
  * differently as PhysicalElementCount. Additionally, it counts element windows as ElementCount.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DataflowOutputCounter implements ElementCounter {
   /** Number of physical element and multiple-window assignments that were serialized/processed. */
   private static final String OBJECT_COUNTER_NAME = "-PhysicalElementCount";
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowPortabilityPCollectionView.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowPortabilityPCollectionView.java
index df756a5..1c60836 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowPortabilityPCollectionView.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowPortabilityPCollectionView.java
@@ -70,9 +70,8 @@
     this.coder = coder;
   }
 
-  @Nullable
   @Override
-  public PCollection<?> getPCollection() {
+  public @Nullable PCollection<?> getPCollection() {
     throw new UnsupportedOperationException();
   }
 
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowProcessFnRunner.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowProcessFnRunner.java
index 2b28765..7d9174c 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowProcessFnRunner.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowProcessFnRunner.java
@@ -42,6 +42,7 @@
  * because the code for handling pushback on streaming side inputs in Dataflow is also divergent
  * from the runner-agnostic code in runners-core. If that code is ever unified, so can this class.
  */
+@SuppressWarnings({"keyfor", "nullness"}) // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class DataflowProcessFnRunner<InputT, OutputT, RestrictionT>
     implements DoFnRunner<KeyedWorkItem<byte[], KV<InputT, RestrictionT>>, OutputT> {
   private final DoFnRunner<KeyedWorkItem<byte[], KV<InputT, RestrictionT>>, OutputT> simpleRunner;
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowRunnerHarness.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowRunnerHarness.java
index 071dc7b..d412702 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowRunnerHarness.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowRunnerHarness.java
@@ -53,6 +53,7 @@
  * instructions (such as map tasks) from the Dataflow Service/DFE into Fn API instructions, and vice
  * versa.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DataflowRunnerHarness {
   private static final Logger LOG = LoggerFactory.getLogger(DataflowRunnerHarness.class);
 
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowSideInputReadCounter.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowSideInputReadCounter.java
index fb9f844..fe3a805 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowSideInputReadCounter.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowSideInputReadCounter.java
@@ -79,6 +79,7 @@
  * input, and the AnotherParDo may be the one that actually manipulates that Iterable. This is
  * possible because both ParDos will be fused, so they will simply exchange objects in memory.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DataflowSideInputReadCounter implements SideInputReadCounter {
   private final DataflowExecutionContext executionContext;
 
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowWorkProgressUpdater.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowWorkProgressUpdater.java
index a27fae5..7c0a96f 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowWorkProgressUpdater.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowWorkProgressUpdater.java
@@ -27,6 +27,7 @@
 import com.google.api.services.dataflow.model.WorkItemServiceState;
 import java.util.concurrent.ScheduledExecutorService;
 import javax.annotation.concurrent.NotThreadSafe;
+import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions;
 import org.apache.beam.runners.dataflow.util.TimeUtil;
 import org.apache.beam.runners.dataflow.worker.util.common.worker.WorkExecutor;
 import org.apache.beam.runners.dataflow.worker.util.common.worker.WorkProgressUpdater;
@@ -40,6 +41,7 @@
  * system.
  */
 @NotThreadSafe
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DataflowWorkProgressUpdater extends WorkProgressUpdater {
 
   private static final Logger LOG = LoggerFactory.getLogger(DataflowWorkProgressUpdater.class);
@@ -49,16 +51,22 @@
   /** The WorkItem for which work progress updates are sent. */
   private final WorkItem workItem;
 
+  private final DataflowPipelineOptions options;
+
   private HotKeyLogger hotKeyLogger;
 
   private boolean wasAskedToAbort = false;
 
   public DataflowWorkProgressUpdater(
-      WorkItemStatusClient workItemStatusClient, WorkItem workItem, WorkExecutor worker) {
+      WorkItemStatusClient workItemStatusClient,
+      WorkItem workItem,
+      WorkExecutor worker,
+      DataflowPipelineOptions options) {
     super(worker, Integer.MAX_VALUE);
     this.workItemStatusClient = workItemStatusClient;
     this.workItem = workItem;
     this.hotKeyLogger = new HotKeyLogger();
+    this.options = options;
   }
 
   /**
@@ -73,11 +81,13 @@
       WorkExecutor worker,
       ScheduledExecutorService executor,
       Clock clock,
-      HotKeyLogger hotKeyLogger) {
+      HotKeyLogger hotKeyLogger,
+      DataflowPipelineOptions options) {
     super(worker, Integer.MAX_VALUE, executor, clock);
     this.workItemStatusClient = workItemStatusClient;
     this.workItem = workItem;
     this.hotKeyLogger = hotKeyLogger;
+    this.options = options;
   }
 
   @Override
@@ -117,9 +127,19 @@
       if (result.getHotKeyDetection() != null
           && result.getHotKeyDetection().getUserStepName() != null) {
         HotKeyDetection hotKeyDetection = result.getHotKeyDetection();
-        hotKeyLogger.logHotKeyDetection(
-            hotKeyDetection.getUserStepName(),
-            TimeUtil.fromCloudDuration(hotKeyDetection.getHotKeyAge()));
+
+        // The key set the in BatchModeExecutionContext is only set in the GroupingShuffleReader
+        // which is the correct key. The key is also translated into a Java object in the reader.
+        if (options.isHotKeyLoggingEnabled()) {
+          hotKeyLogger.logHotKeyDetection(
+              hotKeyDetection.getUserStepName(),
+              TimeUtil.fromCloudDuration(hotKeyDetection.getHotKeyAge()),
+              workItemStatusClient.getExecutionContext().getKey());
+        } else {
+          hotKeyLogger.logHotKeyDetection(
+              hotKeyDetection.getUserStepName(),
+              TimeUtil.fromCloudDuration(hotKeyDetection.getHotKeyAge()));
+        }
       }
 
       // Resets state after a successful progress report.
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowWorkUnitClient.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowWorkUnitClient.java
index 5c4a9c0..33534d5 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowWorkUnitClient.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowWorkUnitClient.java
@@ -53,6 +53,7 @@
 
 /** A Dataflow WorkUnit client that fetches WorkItems from the Dataflow service. */
 @ThreadSafe
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class DataflowWorkUnitClient implements WorkUnitClient {
   private final Logger logger;
 
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowWorkerHarnessHelper.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowWorkerHarnessHelper.java
index bda38b3..1954f31 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowWorkerHarnessHelper.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowWorkerHarnessHelper.java
@@ -37,6 +37,7 @@
 import org.slf4j.LoggerFactory;
 
 /** A helper class for initialization of the Dataflow worker harness. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public final class DataflowWorkerHarnessHelper {
   private static final Logger LOG = LoggerFactory.getLogger(DataflowWorkerHarnessHelper.class);
 
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DoFnInstanceManagers.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DoFnInstanceManagers.java
index 93520a1..cf64728 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DoFnInstanceManagers.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DoFnInstanceManagers.java
@@ -24,6 +24,7 @@
 import org.apache.beam.sdk.util.SerializableUtils;
 
 /** Common {@link DoFnInstanceManager} implementations. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DoFnInstanceManagers {
   /**
    * Returns a {@link DoFnInstanceManager} that returns {@link DoFnInfo} instances obtained by
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ExperimentContext.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ExperimentContext.java
index 6749dd7..73e0329 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ExperimentContext.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ExperimentContext.java
@@ -32,6 +32,7 @@
  * <p>To determine if an experiment is enabled, instantiate an {@link ExperimentContext} with the
  * {@link PipelineOptions} and call {@link #isEnabled} to test if it is enabled.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ExperimentContext {
 
   /** Enumeration of all known experiments. */
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/FetchAndFilterStreamingSideInputsOperation.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/FetchAndFilterStreamingSideInputsOperation.java
index 636f186..5bb634d 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/FetchAndFilterStreamingSideInputsOperation.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/FetchAndFilterStreamingSideInputsOperation.java
@@ -56,6 +56,7 @@
  * This {@link ReceivingOperation} is responsible for fetching any ready side inputs and also
  * filtering any input elements that aren't ready by pushing them back into state.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class FetchAndFilterStreamingSideInputsOperation<T, W extends BoundedWindow>
     extends ReceivingOperation {
 
@@ -103,9 +104,8 @@
       this.delegate = delegate;
     }
 
-    @Nullable
     @Override
-    public PCollection<?> getPCollection() {
+    public @Nullable PCollection<?> getPCollection() {
       return delegate.getPCollection();
     }
 
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/Filepatterns.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/Filepatterns.java
index 3c6c356..a5a983c 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/Filepatterns.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/Filepatterns.java
@@ -22,6 +22,7 @@
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
 
 /** Utilities for handling filepatterns. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class Filepatterns {
   private static final Pattern AT_N_SPEC = Pattern.compile("@(?<N>\\d+)");
 
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/FnApiWindowMappingFn.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/FnApiWindowMappingFn.java
index d8889b9..186d438 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/FnApiWindowMappingFn.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/FnApiWindowMappingFn.java
@@ -72,6 +72,7 @@
  * instead of per mapping request. This requires rewriting the {@link StreamingSideInputFetcher} to
  * not be inline calls and process elements over a stream.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class FnApiWindowMappingFn<TargetWindowT extends BoundedWindow>
     extends WindowMappingFn<TargetWindowT> {
   private static final Logger LOG = LoggerFactory.getLogger(FnApiWindowMappingFn.class);
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupAlsoByWindowParDoFnFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupAlsoByWindowParDoFnFactory.java
index 033e7e0..ab398f1 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupAlsoByWindowParDoFnFactory.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupAlsoByWindowParDoFnFactory.java
@@ -67,6 +67,7 @@
  * A {@link ParDoFnFactory} to create GroupAlsoByWindowsDoFn instances according to specifications
  * from the Dataflow service.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class GroupAlsoByWindowParDoFnFactory implements ParDoFnFactory {
 
   private static final Logger LOG = LoggerFactory.getLogger(GroupAlsoByWindowParDoFnFactory.class);
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupAlsoByWindowsParDoFn.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupAlsoByWindowsParDoFn.java
index e005511..8267797 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupAlsoByWindowsParDoFn.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupAlsoByWindowsParDoFn.java
@@ -43,6 +43,7 @@
 import org.checkerframework.checker.nullness.qual.Nullable;
 
 /** A specialized {@link ParDoFn} for GroupAlsoByWindow related {@link GroupAlsoByWindowFn}'s. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class GroupAlsoByWindowsParDoFn<InputT, K, V, W extends BoundedWindow> implements ParDoFn {
   private final PipelineOptions options;
 
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupingShuffleReader.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupingShuffleReader.java
index 55d7b25..a6d5cb6 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupingShuffleReader.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupingShuffleReader.java
@@ -67,6 +67,7 @@
  * @param <K> the type of the keys read from the shuffle
  * @param <V> the type of the values read from the shuffle
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class GroupingShuffleReader<K, V> extends NativeReader<WindowedValue<KV<K, Reiterable<V>>>> {
   private static final Logger LOG = LoggerFactory.getLogger(GroupingShuffleReader.class);
 
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupingShuffleReaderFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupingShuffleReaderFactory.java
index 574c59a..69a1051 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupingShuffleReaderFactory.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupingShuffleReaderFactory.java
@@ -39,6 +39,7 @@
 import org.checkerframework.checker.nullness.qual.Nullable;
 
 /** Creates a GroupingShuffleReader from a CloudObject spec. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class GroupingShuffleReaderFactory implements ReaderFactory {
 
   /** A {@link ReaderFactory.Registrar} for grouping shuffle sources. */
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupingShuffleReaderWithFaultyBytesReadCounter.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupingShuffleReaderWithFaultyBytesReadCounter.java
index 2348840..0abf5e6 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupingShuffleReaderWithFaultyBytesReadCounter.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupingShuffleReaderWithFaultyBytesReadCounter.java
@@ -35,6 +35,7 @@
  * @param <V> the type of the values read from the shuffle
  */
 @VisibleForTesting
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class GroupingShuffleReaderWithFaultyBytesReadCounter<K, V> extends GroupingShuffleReader<K, V> {
   public GroupingShuffleReaderWithFaultyBytesReadCounter(
       PipelineOptions options,
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/InMemoryReader.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/InMemoryReader.java
index 7a37b1c..4daa267 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/InMemoryReader.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/InMemoryReader.java
@@ -43,6 +43,7 @@
  *
  * @param <T> the type of the elements read from the source
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class InMemoryReader<T> extends NativeReader<T> {
   private static final Logger LOG = LoggerFactory.getLogger(InMemoryReader.class);
 
@@ -170,9 +171,8 @@
           SourceTranslationUtils.cloudPositionToReaderPosition(splitPosition));
     }
 
-    @Nullable
     @Override
-    public DynamicSplitResult requestCheckpoint() {
+    public @Nullable DynamicSplitResult requestCheckpoint() {
       if (!tracker.trySplitAtPosition(lastReturnedIndex + 1)) {
         return null;
       }
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/InMemoryReaderFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/InMemoryReaderFactory.java
index 94aedf9..2816411 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/InMemoryReaderFactory.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/InMemoryReaderFactory.java
@@ -32,6 +32,7 @@
 import org.checkerframework.checker.nullness.qual.Nullable;
 
 /** Creates an InMemoryReader from a CloudObject spec. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class InMemoryReaderFactory implements ReaderFactory {
 
   /** A {@link ReaderFactory.Registrar} for in memory sources. */
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/IsmReader.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/IsmReader.java
index 28b9d0e..5af8802 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/IsmReader.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/IsmReader.java
@@ -33,6 +33,7 @@
  *
  * @param <V> the type of the value written to the sink
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public abstract class IsmReader<V> extends NativeReader<WindowedValue<IsmRecord<V>>> {
   private static final ThreadLocal<SideInputReadCounter> CURRENT_SIDE_INPUT_COUNTERS =
       new ThreadLocal<>();
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/IsmReaderFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/IsmReaderFactory.java
index 4bcfcec..3b34c53 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/IsmReaderFactory.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/IsmReaderFactory.java
@@ -46,6 +46,7 @@
  * Creates an {@link IsmReader} from a {@link CloudObject} spec. Note that it is invalid to use a
  * non {@link IsmRecordCoder} with this reader factory.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class IsmReaderFactory implements ReaderFactory {
 
   /** A {@link ReaderFactory.Registrar} for ISM sources. */
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/IsmReaderImpl.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/IsmReaderImpl.java
index d408b25..99c62db 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/IsmReaderImpl.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/IsmReaderImpl.java
@@ -83,6 +83,7 @@
  */
 // Possible real inconsistency - https://issues.apache.org/jira/browse/BEAM-6560
 @SuppressFBWarnings("IS2_INCONSISTENT_SYNC")
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class IsmReaderImpl<V> extends IsmReader<V> {
   /**
    * This constant represents the distance we would rather read and drop bytes for versus doing an
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/IsmSideInputReader.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/IsmSideInputReader.java
index 1cc1d8c..f471b39 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/IsmSideInputReader.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/IsmSideInputReader.java
@@ -94,6 +94,7 @@
  * {@link #getSingletonForWindow} for singleton views, {@link #getListForWindow} for iterable and
  * list views, and {@link #getMapForWindow} for map and multimap views.
  */
+@SuppressWarnings({"keyfor", "nullness"}) // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class IsmSideInputReader implements SideInputReader {
   private static final String SINGLETON_KIND = "singleton";
   private static final String COLLECTION_KIND = "collection";
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/IsmSink.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/IsmSink.java
index 88eb96b..9ef9d58 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/IsmSink.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/IsmSink.java
@@ -55,6 +55,7 @@
  *
  * @param <V> the type of the value written to the sink
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class IsmSink<V> extends Sink<WindowedValue<IsmRecord<V>>> {
   static final int BLOCK_SIZE_BYTES = 1024 * 1024;
   private final ResourceId resourceId;
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/KeyTokenInvalidException.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/KeyTokenInvalidException.java
index a8a065b..3916ded 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/KeyTokenInvalidException.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/KeyTokenInvalidException.java
@@ -18,6 +18,7 @@
 package org.apache.beam.runners.dataflow.worker;
 
 /** Indicates that the key token was invalid when data was attempted to be fetched. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class KeyTokenInvalidException extends RuntimeException {
   public KeyTokenInvalidException(String key) {
     super("Unable to fetch data due to token mismatch for key " + key);
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/LazilyInitializedSideInputReader.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/LazilyInitializedSideInputReader.java
index c0afdaf..5bd34c0 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/LazilyInitializedSideInputReader.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/LazilyInitializedSideInputReader.java
@@ -45,9 +45,8 @@
     lazyInitSideInputReader = Suppliers.memoize(sideInputReader);
   }
 
-  @Nullable
   @Override
-  public <T> T get(PCollectionView<T> view, BoundedWindow window) {
+  public <T> @Nullable T get(PCollectionView<T> view, BoundedWindow window) {
     return lazyInitSideInputReader.get().get(view, window);
   }
 
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/MetricTrackingWindmillServerStub.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/MetricTrackingWindmillServerStub.java
index ebcbad5..61279e6 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/MetricTrackingWindmillServerStub.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/MetricTrackingWindmillServerStub.java
@@ -42,6 +42,7 @@
  * {@link #NUM_THREADS} polling that queue and making requests to WMS in batches of size {@link
  * #MAX_READS_PER_BATCH}.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class MetricTrackingWindmillServerStub {
 
   private final AtomicInteger activeSideInputs = new AtomicInteger();
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/MetricsToCounterUpdateConverter.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/MetricsToCounterUpdateConverter.java
index 710fef1..f2e9517 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/MetricsToCounterUpdateConverter.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/MetricsToCounterUpdateConverter.java
@@ -29,6 +29,7 @@
 import org.apache.beam.sdk.metrics.MetricName;
 
 /** Convertor from Metrics to {@link CounterUpdate} protos. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class MetricsToCounterUpdateConverter {
 
   private MetricsToCounterUpdateConverter() {}
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/NoOpSourceOperationExecutor.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/NoOpSourceOperationExecutor.java
index 4a589ee..d1217fe 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/NoOpSourceOperationExecutor.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/NoOpSourceOperationExecutor.java
@@ -26,6 +26,7 @@
 import org.apache.beam.runners.dataflow.worker.counters.CounterSet;
 
 /** An executor for a source operation which does not perform any splits. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class NoOpSourceOperationExecutor implements SourceOperationExecutor {
 
   private final SourceOperationRequest request;
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PairWithConstantKeyDoFnFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PairWithConstantKeyDoFnFactory.java
index 8a6ab60..15e6d61 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PairWithConstantKeyDoFnFactory.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PairWithConstantKeyDoFnFactory.java
@@ -40,6 +40,7 @@
  * A {@link ParDoFnFactory} which returns a {@link ParDoFn} that transforms all {@code
  * WindowedValue<V>} to {@code WindowedValue<KV<K, V>>} for a constant key {@code K}.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class PairWithConstantKeyDoFnFactory implements ParDoFnFactory {
   @Override
   public ParDoFn create(
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PartialGroupByKeyParDoFns.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PartialGroupByKeyParDoFns.java
index 5b5d902..f227cd9 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PartialGroupByKeyParDoFns.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PartialGroupByKeyParDoFns.java
@@ -54,6 +54,7 @@
 import org.joda.time.Instant;
 
 /** A factory class that creates {@link ParDoFn} for {@link PartialGroupByKeyInstruction}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class PartialGroupByKeyParDoFns {
   public static <K, InputT, AccumT> ParDoFn create(
       PipelineOptions options,
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PartitioningShuffleReader.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PartitioningShuffleReader.java
index b58747c..d796fcc 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PartitioningShuffleReader.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PartitioningShuffleReader.java
@@ -39,6 +39,7 @@
  * @param <K> the type of the keys read from the shuffle
  * @param <V> the type of the values read from the shuffle
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class PartitioningShuffleReader<K, V> extends NativeReader<WindowedValue<KV<K, V>>> {
   final byte[] shuffleReaderConfig;
   final String startShufflePosition;
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PartitioningShuffleReaderFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PartitioningShuffleReaderFactory.java
index ffd0920..6fe272a 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PartitioningShuffleReaderFactory.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PartitioningShuffleReaderFactory.java
@@ -34,6 +34,7 @@
 import org.checkerframework.checker.nullness.qual.Nullable;
 
 /** Creates a PartitioningShuffleReader from a CloudObject spec. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class PartitioningShuffleReaderFactory implements ReaderFactory {
 
   /** A {@link ReaderFactory.Registrar} for partitioning shuffle sources. */
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PubsubReader.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PubsubReader.java
index 8b44d0c..27eb0b0 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PubsubReader.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PubsubReader.java
@@ -40,6 +40,7 @@
 import org.checkerframework.checker.nullness.qual.Nullable;
 
 /** A Reader that receives elements from Pubsub, via a Windmill server. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class PubsubReader<T> extends NativeReader<WindowedValue<T>> {
   private final Coder<T> coder;
   private final StreamingModeExecutionContext context;
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PubsubSink.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PubsubSink.java
index 1b9f919..0a1aa83 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PubsubSink.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PubsubSink.java
@@ -44,6 +44,7 @@
  *
  * @param <T> the type of the elements written to the sink
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class PubsubSink<T> extends Sink<WindowedValue<T>> {
   private final String topic;
   private final String timestampLabel;
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ReaderCache.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ReaderCache.java
index 8b03f16..9464db9 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ReaderCache.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ReaderCache.java
@@ -35,6 +35,7 @@
  * minute expiration timeout and the reader will be closed if it is not used within this period.
  */
 @ThreadSafe
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class ReaderCache {
 
   private static final Logger LOG = LoggerFactory.getLogger(ReaderCache.class);
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ReaderRegistry.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ReaderRegistry.java
index 1dbc752..0a5b55e 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ReaderRegistry.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ReaderRegistry.java
@@ -39,6 +39,7 @@
  * An immutable registry from {@link String} identifiers (provided to the worker by the Dataflow
  * service) to appropriate {@link ReaderFactory} instances.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ReaderRegistry implements ReaderFactory {
 
   /**
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ReifyTimestampAndWindowsParDoFnFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ReifyTimestampAndWindowsParDoFnFactory.java
index 948c811..4ca6e66 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ReifyTimestampAndWindowsParDoFnFactory.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ReifyTimestampAndWindowsParDoFnFactory.java
@@ -32,6 +32,7 @@
 import org.checkerframework.checker.nullness.qual.Nullable;
 
 /** A {@link ParDoFnFactory} to create instances of {@link ReifyTimestampAndWindowsParDoFn}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class ReifyTimestampAndWindowsParDoFnFactory implements ParDoFnFactory {
   @Override
   public ParDoFn create(
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/RunnerHarnessCoderCloudObjectTranslatorRegistrar.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/RunnerHarnessCoderCloudObjectTranslatorRegistrar.java
index fb22bbd..208be62 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/RunnerHarnessCoderCloudObjectTranslatorRegistrar.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/RunnerHarnessCoderCloudObjectTranslatorRegistrar.java
@@ -45,6 +45,7 @@
  * <p>See {@link CoderCloudObjectTranslatorRegistrar} for more details.
  */
 @AutoService(CoderCloudObjectTranslatorRegistrar.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class RunnerHarnessCoderCloudObjectTranslatorRegistrar
     implements CoderCloudObjectTranslatorRegistrar {
 
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/SdkHarnessRegistries.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/SdkHarnessRegistries.java
index 5224029..45440a4 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/SdkHarnessRegistries.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/SdkHarnessRegistries.java
@@ -36,6 +36,7 @@
 import org.slf4j.LoggerFactory;
 
 /** Factory to create SdkHarnessRegistry */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SdkHarnessRegistries {
 
   /** Create a registry which does not require sdkHarness registration for non fnapi worker. */
@@ -224,27 +225,23 @@
 
     private final SdkWorkerHarness sdkWorkerHarness =
         new SdkWorkerHarness() {
-          @Nullable
           @Override
-          public FnApiControlClient getControlClientHandler() {
+          public @Nullable FnApiControlClient getControlClientHandler() {
             return null;
           }
 
-          @Nullable
           @Override
-          public String getWorkerId() {
+          public @Nullable String getWorkerId() {
             return null;
           }
 
-          @Nullable
           @Override
-          public GrpcFnServer<GrpcDataService> getGrpcDataFnServer() {
+          public @Nullable GrpcFnServer<GrpcDataService> getGrpcDataFnServer() {
             return null;
           }
 
-          @Nullable
           @Override
-          public GrpcFnServer<GrpcStateService> getGrpcStateFnServer() {
+          public @Nullable GrpcFnServer<GrpcStateService> getGrpcStateFnServer() {
             return null;
           }
         };
@@ -274,15 +271,13 @@
     @Override
     public void completeWork(SdkWorkerHarness worker) {}
 
-    @Nullable
     @Override
-    public ApiServiceDescriptor beamFnStateApiServiceDescriptor() {
+    public @Nullable ApiServiceDescriptor beamFnStateApiServiceDescriptor() {
       return null;
     }
 
-    @Nullable
     @Override
-    public ApiServiceDescriptor beamFnDataApiServiceDescriptor() {
+    public @Nullable ApiServiceDescriptor beamFnDataApiServiceDescriptor() {
       return null;
     }
   }
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ShuffleLibrary.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ShuffleLibrary.java
index f8570f8..e879676 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ShuffleLibrary.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ShuffleLibrary.java
@@ -24,6 +24,7 @@
 import java.nio.file.StandardCopyOption;
 
 /** Native library used to read from and write to a shuffle dataset. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class ShuffleLibrary {
   /** Loads the native shuffle library. */
   static void load() {
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ShuffleSink.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ShuffleSink.java
index 2ad66b5..7eca856 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ShuffleSink.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ShuffleSink.java
@@ -45,6 +45,7 @@
  *
  * @param <T> the type of the elements written to the sink
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ShuffleSink<T> extends Sink<WindowedValue<T>> {
   enum ShuffleKind {
     UNGROUPED,
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ShuffleSinkFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ShuffleSinkFactory.java
index da27cda..6b9cfd3 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ShuffleSinkFactory.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ShuffleSinkFactory.java
@@ -33,6 +33,7 @@
 import org.checkerframework.checker.nullness.qual.Nullable;
 
 /** Creates a {@link ShuffleSink} from a {@link CloudObject} spec. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ShuffleSinkFactory implements SinkFactory {
 
   /** A {@link SinkFactory.Registrar} for shuffle sinks. */
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/SimpleParDoFn.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/SimpleParDoFn.java
index 10dcf3c..a782f4e 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/SimpleParDoFn.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/SimpleParDoFn.java
@@ -71,6 +71,7 @@
  * <p>Subclasses override just a method to provide a {@link DoFnInfo} for the wrapped {@link
  * GroupAlsoByWindowFn}.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SimpleParDoFn<InputT, OutputT> implements ParDoFn {
   // TODO: Remove once Distributions has shipped.
   @VisibleForTesting
@@ -518,8 +519,8 @@
    * <p>May be null if no element has been processed yet, or if the {@link SimpleParDoFn} has
    * finished.
    */
-  @Nullable
   @VisibleForTesting
+  @Nullable
   DoFnInfo<?, ?> getDoFnInfo() {
     return fnInfo;
   }
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/SinkRegistry.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/SinkRegistry.java
index 26c06af..86000d4 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/SinkRegistry.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/SinkRegistry.java
@@ -35,6 +35,7 @@
  * An immutable registry from {@link String} identifiers (provided to the worker by the Dataflow
  * service) to appropriate {@link SinkFactory} instances.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public final class SinkRegistry implements SinkFactory {
 
   /**
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/SourceTranslationUtils.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/SourceTranslationUtils.java
index 5bf99e3..4b26e05 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/SourceTranslationUtils.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/SourceTranslationUtils.java
@@ -30,6 +30,7 @@
 import org.checkerframework.checker.nullness.qual.Nullable;
 
 /** Utilities for representing input-specific objects using Dataflow model protos. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SourceTranslationUtils {
   public static NativeReader.Progress cloudProgressToReaderProgress(
       @Nullable ApproximateReportedProgress cloudProgress) {
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StateFetcher.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StateFetcher.java
index 15e831f..1ff633c 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StateFetcher.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StateFetcher.java
@@ -50,6 +50,7 @@
 import org.slf4j.LoggerFactory;
 
 /** Class responsible for fetching state from the windmill server. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class StateFetcher {
   private static final Set<String> SUPPORTED_MATERIALIZATIONS =
       ImmutableSet.of(
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java
index ce60eb3..323369e 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java
@@ -153,6 +153,7 @@
 import org.slf4j.LoggerFactory;
 
 /** Implements a Streaming Dataflow worker. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class StreamingDataflowWorker {
 
   private static final Logger LOG = LoggerFactory.getLogger(StreamingDataflowWorker.class);
@@ -1083,16 +1084,6 @@
         outputDataWatermark == null || !outputDataWatermark.isAfter(inputDataWatermark));
     SdkWorkerHarness worker = sdkHarnessRegistry.getAvailableWorkerAndAssignWork();
 
-    if (workItem.hasHotKeyInfo()) {
-      Windmill.HotKeyInfo hotKeyInfo = workItem.getHotKeyInfo();
-      Duration hotKeyAge = Duration.millis(hotKeyInfo.getHotKeyAgeUsec() / 1000);
-
-      // The MapTask instruction is ordered by dependencies, such that the first element is
-      // always going to be the shuffle task.
-      String stepName = computationState.getMapTask().getInstructions().get(0).getName();
-      hotKeyLogger.logHotKeyDetection(stepName, hotKeyAge);
-    }
-
     Work work =
         new Work(workItem) {
           @Override
@@ -1384,6 +1375,20 @@
       Object executionKey =
           keyCoder == null ? null : keyCoder.decode(key.newInput(), Coder.Context.OUTER);
 
+      if (workItem.hasHotKeyInfo()) {
+        Windmill.HotKeyInfo hotKeyInfo = workItem.getHotKeyInfo();
+        Duration hotKeyAge = Duration.millis(hotKeyInfo.getHotKeyAgeUsec() / 1000);
+
+        // The MapTask instruction is ordered by dependencies, such that the first element is
+        // always going to be the shuffle task.
+        String stepName = computationState.getMapTask().getInstructions().get(0).getName();
+        if (options.isHotKeyLoggingEnabled() && keyCoder != null) {
+          hotKeyLogger.logHotKeyDetection(stepName, hotKeyAge, executionKey);
+        } else {
+          hotKeyLogger.logHotKeyDetection(stepName, hotKeyAge);
+        }
+      }
+
       executionState
           .getContext()
           .start(
@@ -1464,33 +1469,36 @@
       boolean retryLocally = false;
       if (KeyTokenInvalidException.isKeyTokenInvalidException(t)) {
         LOG.debug(
-            "Execution of work for {} for key {} failed due to token expiration. "
-                + "Will not retry locally.",
+            "Execution of work for computation '{}' on key '{}' failed due to token expiration. "
+                + "Work will not be retried locally.",
             computationId,
             key.toStringUtf8());
       } else {
-        LOG.error("Uncaught exception: ", t);
         LastExceptionDataProvider.reportException(t);
         LOG.debug("Failed work: {}", work);
         if (!reportFailure(computationId, workItem, t)) {
           LOG.error(
-              "Execution of work for {} for key {} failed, and Windmill "
-                  + "indicated not to retry locally.",
+              "Execution of work for computation '{}' on key '{}' failed with uncaught exception, "
+                  + "and Windmill indicated not to retry locally.",
               computationId,
-              key.toStringUtf8());
+              key.toStringUtf8(),
+              t);
         } else if (isOutOfMemoryError(t)) {
           File heapDump = memoryMonitor.tryToDumpHeap();
           LOG.error(
-              "Execution of work for {} for key {} failed with out-of-memory. "
-                  + "Will not retry locally. Heap dump {}.",
+              "Execution of work for computation '{}' for key '{}' failed with out-of-memory. "
+                  + "Work will not be retried locally. Heap dump {}.",
               computationId,
               key.toStringUtf8(),
-              heapDump == null ? "not written" : ("written to '" + heapDump + "'"));
+              heapDump == null ? "not written" : ("written to '" + heapDump + "'"),
+              t);
         } else {
           LOG.error(
-              "Execution of work for {} for key {} failed. Will retry locally.",
+              "Execution of work for computation '{}' on key '{}' failed with uncaught exception. "
+                  + "Work will be retried locally.",
               computationId,
-              key.toStringUtf8());
+              key.toStringUtf8(),
+              t);
           retryLocally = true;
         }
       }
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingKeyedWorkItemSideInputDoFnRunner.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingKeyedWorkItemSideInputDoFnRunner.java
index b1a5903..34232bd 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingKeyedWorkItemSideInputDoFnRunner.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingKeyedWorkItemSideInputDoFnRunner.java
@@ -54,6 +54,7 @@
  * @param <OutputT> output element type
  * @param <W> window type
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class StreamingKeyedWorkItemSideInputDoFnRunner<K, InputT, OutputT, W extends BoundedWindow>
     implements DoFnRunner<KeyedWorkItem<K, InputT>, OutputT> {
   private final DoFnRunner<KeyedWorkItem<K, InputT>, OutputT> simpleDoFnRunner;
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContext.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContext.java
index 705259f..65b938d 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContext.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContext.java
@@ -67,6 +67,7 @@
 import org.slf4j.LoggerFactory;
 
 /** {@link DataflowExecutionContext} for use in streaming mode. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class StreamingModeExecutionContext extends DataflowExecutionContext<StepContext> {
 
   private static final Logger LOG = LoggerFactory.getLogger(StreamingModeExecutionContext.class);
@@ -167,9 +168,8 @@
      * <p>Final updates should never be requested from a Streaming job since the work unit never
      * completes.
      */
-    @Nullable
     @Override
-    public CounterUpdate extractUpdate(boolean isFinalUpdate) {
+    public @Nullable CounterUpdate extractUpdate(boolean isFinalUpdate) {
       // Streaming reports deltas, so isFinalUpdate doesn't matter, and should never be true.
       long sum = totalMillisInState.getAndSet(0);
       return sum == 0 ? null : createUpdate(false, sum);
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingPCollectionViewWriterParDoFn.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingPCollectionViewWriterParDoFn.java
index 44c2d95..ea27305 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingPCollectionViewWriterParDoFn.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingPCollectionViewWriterParDoFn.java
@@ -32,6 +32,7 @@
  * A {@link ParDoFn} that writes side input data using {@link
  * StreamingModeExecutionContext.StreamingModeStepContext#writePCollectionViewData}.
  */
+@SuppressWarnings({"keyfor", "nullness"}) // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class StreamingPCollectionViewWriterParDoFn implements ParDoFn {
 
   private final StreamingModeExecutionContext.StreamingModeStepContext stepContext;
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputFetcher.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputFetcher.java
index eb95afb..e87855c 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputFetcher.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputFetcher.java
@@ -55,6 +55,7 @@
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Lists;
 
 /** A class that handles streaming side inputs in a {@link DoFnRunner}. */
+@SuppressWarnings({"keyfor", "nullness"}) // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class StreamingSideInputFetcher<InputT, W extends BoundedWindow> {
   private StreamingModeExecutionContext.StreamingModeStepContext stepContext;
   private Map<String, PCollectionView<?>> sideInputViews;
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingStepMetricsContainer.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingStepMetricsContainer.java
index 64680a1..a426af8 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingStepMetricsContainer.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingStepMetricsContainer.java
@@ -40,6 +40,7 @@
  * For Dataflow Streaming, we want to efficiently support many threads report metric updates, and a
  * single total delta being reported periodically as physical counters.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class StreamingStepMetricsContainer implements MetricsContainer {
 
   private final String stepName;
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ToIsmRecordForMultimapDoFnFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ToIsmRecordForMultimapDoFnFactory.java
index 6de9336..ace8dba 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ToIsmRecordForMultimapDoFnFactory.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ToIsmRecordForMultimapDoFnFactory.java
@@ -48,6 +48,7 @@
  * <p>This {@link ParDoFnFactory} is part of an expansion of steps required to materialize ISM
  * files. See <a href="go/dataflow-side-inputs">go/dataflow-side-inputs</a> for further details.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ToIsmRecordForMultimapDoFnFactory implements ParDoFnFactory {
 
   @Override
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/UngroupedShuffleReader.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/UngroupedShuffleReader.java
index 7aeb142..3bf5c7e 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/UngroupedShuffleReader.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/UngroupedShuffleReader.java
@@ -36,6 +36,7 @@
  *
  * @param <T> the type of the elements read from the source
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class UngroupedShuffleReader<T> extends NativeReader<T> {
   final byte[] shuffleReaderConfig;
   final String startShufflePosition;
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/UngroupedShuffleReaderFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/UngroupedShuffleReaderFactory.java
index 48e516d..73d2cc3 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/UngroupedShuffleReaderFactory.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/UngroupedShuffleReaderFactory.java
@@ -31,6 +31,7 @@
 import org.checkerframework.checker.nullness.qual.Nullable;
 
 /** Creates an UngroupedShuffleReader from a CloudObject spec. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class UngroupedShuffleReaderFactory implements ReaderFactory {
 
   /** A {@link ReaderFactory.Registrar} for ungrouped shuffle sources. */
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/UngroupedWindmillReader.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/UngroupedWindmillReader.java
index e5449d4..733dbd4 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/UngroupedWindmillReader.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/UngroupedWindmillReader.java
@@ -42,6 +42,7 @@
 /**
  * A Reader that receives input data from a Windmill server, and returns it as individual elements.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class UngroupedWindmillReader<T> extends NativeReader<WindowedValue<T>> {
   private final Coder<T> valueCoder;
   private final Coder<Collection<? extends BoundedWindow>> windowsCoder;
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/UserParDoFnFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/UserParDoFnFactory.java
index 69f357b..7b5733d 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/UserParDoFnFactory.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/UserParDoFnFactory.java
@@ -46,6 +46,7 @@
  * A {@link ParDoFnFactory} to create instances of user {@link GroupAlsoByWindowFn} according to
  * specifications from the Dataflow service.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class UserParDoFnFactory implements ParDoFnFactory {
   static UserParDoFnFactory createDefault() {
     return new UserParDoFnFactory(new UserDoFnExtractor(), SimpleDoFnRunnerFactory.INSTANCE);
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ValuesDoFnFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ValuesDoFnFactory.java
index db5eeb8..92bfd88 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ValuesDoFnFactory.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ValuesDoFnFactory.java
@@ -35,6 +35,7 @@
  * A {@link ParDoFnFactory} which returns a {@link ParDoFn} with similar behavior to {@link
  * org.apache.beam.sdk.transforms.Values#create()}.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ValuesDoFnFactory implements ParDoFnFactory {
 
   @Override
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillKeyedWorkItem.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillKeyedWorkItem.java
index b50ca3e..bbe1dce 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillKeyedWorkItem.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillKeyedWorkItem.java
@@ -49,6 +49,7 @@
  * @param <K> the key type
  * @param <ElemT> the element type
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class WindmillKeyedWorkItem<K, ElemT> implements KeyedWorkItem<K, ElemT> {
   private static final Predicate<Timer> IS_WATERMARK =
       input -> input.getType() == Timer.Type.WATERMARK;
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillReaderIteratorBase.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillReaderIteratorBase.java
index 79d8b06..ff904c3 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillReaderIteratorBase.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillReaderIteratorBase.java
@@ -27,6 +27,7 @@
 /**
  * Base class for iterators that decode messages from bundles inside a {@link Windmill.WorkItem}.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public abstract class WindmillReaderIteratorBase<T>
     extends NativeReader.NativeReaderIterator<WindowedValue<T>> {
   private Windmill.WorkItem work;
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillSink.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillSink.java
index e89e55e..dc16aec 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillSink.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillSink.java
@@ -43,6 +43,7 @@
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
 import org.checkerframework.checker.nullness.qual.Nullable;
 
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class WindmillSink<T> extends Sink<WindowedValue<T>> {
   private WindmillStreamWriter writer;
   private final Coder<T> valueCoder;
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillStateCache.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillStateCache.java
index e809897..987c607 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillStateCache.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillStateCache.java
@@ -50,6 +50,7 @@
  * StreamingDataflowWorker} ensures that a single computation * processing key is executing on one
  * thread at a time, so this is safe.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class WindmillStateCache implements StatusDataProvider {
   // Convert Megabytes to bytes
   private static final long MEGABYTES = 1024 * 1024;
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillStateInternals.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillStateInternals.java
index 2ad11f4..6c8b2a4 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillStateInternals.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillStateInternals.java
@@ -17,18 +17,25 @@
  */
 package org.apache.beam.runners.dataflow.worker;
 
+import com.google.auto.value.AutoValue;
 import java.io.Closeable;
 import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
 import java.io.OutputStreamWriter;
 import java.nio.charset.StandardCharsets;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
+import java.util.Comparator;
 import java.util.Iterator;
 import java.util.List;
+import java.util.Map;
 import java.util.Random;
+import java.util.SortedSet;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.Future;
+import java.util.function.BiConsumer;
 import javax.annotation.concurrent.NotThreadSafe;
 import org.apache.beam.runners.core.StateInternals;
 import org.apache.beam.runners.core.StateNamespace;
@@ -38,8 +45,22 @@
 import org.apache.beam.runners.core.StateTags;
 import org.apache.beam.runners.dataflow.worker.WindmillStateCache.ForKey;
 import org.apache.beam.runners.dataflow.worker.windmill.Windmill;
+import org.apache.beam.runners.dataflow.worker.windmill.Windmill.SortedListEntry;
+import org.apache.beam.runners.dataflow.worker.windmill.Windmill.SortedListRange;
+import org.apache.beam.runners.dataflow.worker.windmill.Windmill.TagSortedListDeleteRequest;
+import org.apache.beam.runners.dataflow.worker.windmill.Windmill.TagSortedListInsertRequest;
+import org.apache.beam.runners.dataflow.worker.windmill.Windmill.TagSortedListUpdateRequest;
 import org.apache.beam.runners.dataflow.worker.windmill.Windmill.WorkItemCommitRequest;
 import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.Coder.Context;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.CustomCoder;
+import org.apache.beam.sdk.coders.InstantCoder;
+import org.apache.beam.sdk.coders.MapCoder;
+import org.apache.beam.sdk.coders.NullableCoder;
+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.state.BagState;
 import org.apache.beam.sdk.state.CombiningState;
 import org.apache.beam.sdk.state.MapState;
@@ -60,14 +81,25 @@
 import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
 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.Optional;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Supplier;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.BoundType;
+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.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterators;
+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.collect.Range;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.RangeSet;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Sets;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.TreeRangeSet;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.util.concurrent.Futures;
 import org.checkerframework.checker.nullness.qual.Nullable;
+import org.joda.time.Duration;
 import org.joda.time.Instant;
 
 /** Implementation of {@link StateInternals} using Windmill to manage the underlying data. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class WindmillStateInternals<K> implements StateInternals {
 
   /**
@@ -86,22 +118,28 @@
     private final String stateFamily;
     private final WindmillStateReader reader;
     private final WindmillStateCache.ForKey cache;
+    private final boolean isSystemTable;
     boolean isNewKey;
     private final Supplier<Closeable> scopedReadStateSupplier;
+    private final StateTable derivedStateTable;
 
     public CachingStateTable(
         @Nullable K key,
         String stateFamily,
         WindmillStateReader reader,
         WindmillStateCache.ForKey cache,
+        boolean isSystemTable,
         boolean isNewKey,
-        Supplier<Closeable> scopedReadStateSupplier) {
+        Supplier<Closeable> scopedReadStateSupplier,
+        StateTable derivedStateTable) {
       this.key = key;
       this.stateFamily = stateFamily;
       this.reader = reader;
       this.cache = cache;
+      this.isSystemTable = isSystemTable;
       this.isNewKey = isNewKey;
       this.scopedReadStateSupplier = scopedReadStateSupplier;
+      this.derivedStateTable = derivedStateTable != null ? derivedStateTable : this;
     }
 
     @Override
@@ -112,6 +150,9 @@
       return new StateBinder() {
         @Override
         public <T> BagState<T> bindBag(StateTag<BagState<T>> address, Coder<T> elemCoder) {
+          if (isSystemTable) {
+            address = StateTags.makeSystemTagInternal(address);
+          }
           WindmillBag<T> result = (WindmillBag<T>) cache.get(namespace, address);
           if (result == null) {
             result = new WindmillBag<>(namespace, address, stateFamily, elemCoder, isNewKey);
@@ -138,9 +179,14 @@
         @Override
         public <T> OrderedListState<T> bindOrderedList(
             StateTag<OrderedListState<T>> spec, Coder<T> elemCoder) {
+          if (isSystemTable) {
+            spec = StateTags.makeSystemTagInternal(spec);
+          }
           WindmillOrderedList<T> result = (WindmillOrderedList<T>) cache.get(namespace, spec);
           if (result == null) {
-            result = new WindmillOrderedList<>(namespace, spec, stateFamily, elemCoder, isNewKey);
+            result =
+                new WindmillOrderedList<>(
+                    derivedStateTable, namespace, spec, stateFamily, elemCoder, isNewKey);
           }
           result.initializeForWorkItem(reader, scopedReadStateSupplier);
           return result;
@@ -149,6 +195,9 @@
         @Override
         public WatermarkHoldState bindWatermark(
             StateTag<WatermarkHoldState> address, TimestampCombiner timestampCombiner) {
+          if (isSystemTable) {
+            address = StateTags.makeSystemTagInternal(address);
+          }
           WindmillWatermarkHold result = (WindmillWatermarkHold) cache.get(namespace, address);
           if (result == null) {
             result =
@@ -164,8 +213,11 @@
             StateTag<CombiningState<InputT, AccumT, OutputT>> address,
             Coder<AccumT> accumCoder,
             CombineFn<InputT, AccumT, OutputT> combineFn) {
+          if (isSystemTable) {
+            address = StateTags.makeSystemTagInternal(address);
+          }
           WindmillCombiningState<InputT, AccumT, OutputT> result =
-              new WindmillCombiningState<InputT, AccumT, OutputT>(
+              new WindmillCombiningState<>(
                   namespace, address, stateFamily, accumCoder, combineFn, cache, isNewKey);
           result.initializeForWorkItem(reader, scopedReadStateSupplier);
           return result;
@@ -177,11 +229,17 @@
                 StateTag<CombiningState<InputT, AccumT, OutputT>> address,
                 Coder<AccumT> accumCoder,
                 CombineFnWithContext<InputT, AccumT, OutputT> combineFn) {
+          if (isSystemTable) {
+            address = StateTags.makeSystemTagInternal(address);
+          }
           return bindCombiningValue(address, accumCoder, CombineFnUtil.bindContext(combineFn, c));
         }
 
         @Override
         public <T> ValueState<T> bindValue(StateTag<ValueState<T>> address, Coder<T> coder) {
+          if (isSystemTable) {
+            address = StateTags.makeSystemTagInternal(address);
+          }
           WindmillValue<T> result = (WindmillValue<T>) cache.get(namespace, address);
           if (result == null) {
             result = new WindmillValue<>(namespace, address, stateFamily, coder, isNewKey);
@@ -196,6 +254,7 @@
   private WindmillStateCache.ForKey cache;
   Supplier<Closeable> scopedReadStateSupplier;
   private StateTable workItemState;
+  private StateTable workItemDerivedState;
 
   public WindmillStateInternals(
       @Nullable K key,
@@ -207,16 +266,23 @@
     this.key = key;
     this.cache = cache;
     this.scopedReadStateSupplier = scopedReadStateSupplier;
+    this.workItemDerivedState =
+        new CachingStateTable<>(
+            key, stateFamily, reader, cache, true, isNewKey, scopedReadStateSupplier, null);
     this.workItemState =
-        new CachingStateTable<K>(
-            key, stateFamily, reader, cache, isNewKey, scopedReadStateSupplier);
+        new CachingStateTable<>(
+            key,
+            stateFamily,
+            reader,
+            cache,
+            false,
+            isNewKey,
+            scopedReadStateSupplier,
+            workItemDerivedState);
   }
 
-  public void persist(final Windmill.WorkItemCommitRequest.Builder commitBuilder) {
-    List<Future<WorkItemCommitRequest>> commitsToMerge = new ArrayList<>();
-
-    // Call persist on each first, which may schedule some futures for reading.
-    for (State location : workItemState.values()) {
+  private void persist(List<Future<WorkItemCommitRequest>> commitsToMerge, StateTable stateTable) {
+    for (State location : stateTable.values()) {
       if (!(location instanceof WindmillState)) {
         throw new IllegalStateException(
             String.format(
@@ -235,12 +301,20 @@
     // Clear any references to the underlying reader to prevent space leaks.
     // The next work unit to use these cached State objects will reset the
     // reader to a current reader in case those values are modified.
-    for (State location : workItemState.values()) {
+    for (State location : stateTable.values()) {
       ((WindmillState) location).cleanupAfterWorkItem();
     }
 
     // Clear out the map of already retrieved state instances.
-    workItemState.clear();
+    stateTable.clear();
+  }
+
+  public void persist(final Windmill.WorkItemCommitRequest.Builder commitBuilder) {
+    List<Future<WorkItemCommitRequest>> commitsToMerge = new ArrayList<>();
+
+    // Call persist on each first, which may schedule some futures for reading.
+    persist(commitsToMerge, workItemState);
+    persist(commitsToMerge, workItemDerivedState);
 
     try (Closeable scope = scopedReadStateSupplier.get()) {
       for (Future<WorkItemCommitRequest> commitFuture : commitsToMerge) {
@@ -470,16 +544,305 @@
     }
   }
 
-  private static class WindmillOrderedList<T> extends SimpleWindmillState
-      implements OrderedListState<T> {
+  // Coder for closed-open ranges.
+  private static class RangeCoder<T extends Comparable> extends StructuredCoder<Range<T>> {
+    private Coder<T> boundCoder;
 
+    RangeCoder(Coder<T> boundCoder) {
+      this.boundCoder = NullableCoder.of(boundCoder);
+    }
+
+    @Override
+    public List<? extends Coder<?>> getCoderArguments() {
+      return Lists.newArrayList(boundCoder);
+    }
+
+    @Override
+    public void verifyDeterministic() throws NonDeterministicException {
+      boundCoder.verifyDeterministic();
+      ;
+    }
+
+    @Override
+    public void encode(Range<T> value, OutputStream outStream) throws CoderException, IOException {
+      Preconditions.checkState(
+          value.lowerBoundType().equals(BoundType.CLOSED), "unexpected range " + value);
+      Preconditions.checkState(
+          value.upperBoundType().equals(BoundType.OPEN), "unexpected range " + value);
+      boundCoder.encode(value.hasLowerBound() ? value.lowerEndpoint() : null, outStream);
+      boundCoder.encode(value.hasUpperBound() ? value.upperEndpoint() : null, outStream);
+    }
+
+    @Override
+    public Range<T> decode(InputStream inStream) throws CoderException, IOException {
+      @Nullable T lower = boundCoder.decode(inStream);
+      @Nullable T upper = boundCoder.decode(inStream);
+      if (lower == null) {
+        return upper != null ? Range.lessThan(upper) : Range.all();
+      } else if (upper == null) {
+        return Range.atLeast(lower);
+      } else {
+        return Range.closedOpen(lower, upper);
+      }
+    }
+  }
+
+  private static class RangeSetCoder<T extends Comparable> extends CustomCoder<RangeSet<T>> {
+    private SetCoder<Range<T>> rangesCoder;
+
+    RangeSetCoder(Coder<T> boundCoder) {
+      this.rangesCoder = SetCoder.of(new RangeCoder<>(boundCoder));
+    }
+
+    @Override
+    public void encode(RangeSet<T> value, OutputStream outStream) throws IOException {
+      rangesCoder.encode(value.asRanges(), outStream);
+    }
+
+    @Override
+    public RangeSet<T> decode(InputStream inStream) throws CoderException, IOException {
+      return TreeRangeSet.create(rangesCoder.decode(inStream));
+    }
+  }
+
+  /**
+   * Tracker for the ids used in an ordered list.
+   *
+   * <p>Windmill accepts an int64 id for each timestamped-element in the list. Unique elements are
+   * identified by the pair of timestamp and id. This means that tow unique elements e1, e2 must
+   * have different (ts1, id1), (ts2, id2) pairs. To accomplish this we bucket time into five-minute
+   * buckets, and store a free list of ids available for each bucket.
+   *
+   * <p>When a timestamp range is deleted, we remove id tracking for elements in that range. In
+   * order to handle the case where a range is deleted piecemeal, we track sub-range deletions for
+   * each range. For example:
+   *
+   * <p>12:00 - 12:05 ids 12:05 - 12:10 ids
+   *
+   * <p>delete 12:00-12:06
+   *
+   * <p>12:00 - 12:05 *removed* 12:05 - 12:10 ids subranges deleted 12:05-12:06
+   *
+   * <p>delete 12:06 - 12:07
+   *
+   * <p>12:05 - 12:10 ids subranges deleted 12:05-12:07
+   *
+   * <p>delete 12:07 - 12:10
+   *
+   * <p>12:05 - 12:10 *removed*
+   */
+  static final class IdTracker {
+    static final String IDS_AVAILABLE_STR = "IdsAvailable";
+    static final String DELETIONS_STR = "Deletions";
+
+    static final long MIN_ID = Long.MIN_VALUE;
+    static final long MAX_ID = Long.MAX_VALUE;
+
+    // We track ids on five-minute boundaries.
+    private static final Duration RESOLUTION = Duration.standardMinutes(5);
+    static final MapCoder<Range<Instant>, RangeSet<Long>> IDS_AVAILABLE_CODER =
+        MapCoder.of(new RangeCoder<>(InstantCoder.of()), new RangeSetCoder<>(VarLongCoder.of()));
+    static final MapCoder<Range<Instant>, RangeSet<Instant>> SUBRANGE_DELETIONS_CODER =
+        MapCoder.of(new RangeCoder<>(InstantCoder.of()), new RangeSetCoder<>(InstantCoder.of()));
+    private final StateTag<ValueState<Map<Range<Instant>, RangeSet<Long>>>> idsAvailableTag;
+    // A map from five-minute ranges to the set of ids available in that interval.
+    final ValueState<Map<Range<Instant>, RangeSet<Long>>> idsAvailableValue;
+    private final StateTag<ValueState<Map<Range<Instant>, RangeSet<Instant>>>> subRangeDeletionsTag;
+    // If a timestamp-range in the map has been partially cleared, the cleared intervals are stored
+    // here.
+    final ValueState<Map<Range<Instant>, RangeSet<Instant>>> subRangeDeletionsValue;
+
+    IdTracker(
+        StateTable stateTable,
+        StateNamespace namespace,
+        StateTag<?> spec,
+        String stateFamily,
+        boolean complete) {
+      this.idsAvailableTag =
+          StateTags.makeSystemTagInternal(
+              StateTags.value(spec.getId() + IDS_AVAILABLE_STR, IDS_AVAILABLE_CODER));
+      this.idsAvailableValue =
+          stateTable.get(namespace, idsAvailableTag, StateContexts.nullContext());
+      this.subRangeDeletionsTag =
+          StateTags.makeSystemTagInternal(
+              StateTags.value(spec.getId() + DELETIONS_STR, SUBRANGE_DELETIONS_CODER));
+      this.subRangeDeletionsValue =
+          stateTable.get(namespace, subRangeDeletionsTag, StateContexts.nullContext());
+    }
+
+    static <ValueT extends Comparable<? super ValueT>>
+        Map<Range<Instant>, RangeSet<ValueT>> newSortedRangeMap(Class<ValueT> valueClass) {
+      return Maps.newTreeMap(
+          Comparator.<Range<Instant>, Instant>comparing(Range::lowerEndpoint)
+              .thenComparing(Range::upperEndpoint));
+    }
+
+    private Range<Instant> getTrackedRange(Instant ts) {
+      Instant snapped =
+          new Instant(ts.getMillis() - ts.plus(RESOLUTION).getMillis() % RESOLUTION.getMillis());
+      return Range.closedOpen(snapped, snapped.plus(RESOLUTION));
+    }
+
+    @SuppressWarnings("FutureReturnValueIgnored")
+    void readLater() {
+      idsAvailableValue.readLater();
+      subRangeDeletionsValue.readLater();
+    }
+
+    Map<Range<Instant>, RangeSet<Long>> readIdsAvailable() {
+      Map<Range<Instant>, RangeSet<Long>> idsAvailable = idsAvailableValue.read();
+      return idsAvailable != null ? idsAvailable : newSortedRangeMap(Long.class);
+    }
+
+    Map<Range<Instant>, RangeSet<Instant>> readSubRangeDeletions() {
+      Map<Range<Instant>, RangeSet<Instant>> subRangeDeletions = subRangeDeletionsValue.read();
+      return subRangeDeletions != null ? subRangeDeletions : newSortedRangeMap(Instant.class);
+    }
+
+    void clear() throws ExecutionException, InterruptedException {
+      idsAvailableValue.clear();
+      subRangeDeletionsValue.clear();
+    }
+
+    <T> void add(
+        SortedSet<TimestampedValueWithId<T>> elements, BiConsumer<TimestampedValue<T>, Long> output)
+        throws ExecutionException, InterruptedException {
+      Range<Long> currentIdRange = null;
+      long currentId = 0;
+
+      Range<Instant> currentTsRange = null;
+      RangeSet<Instant> currentTsRangeDeletions = null;
+
+      Map<Range<Instant>, RangeSet<Long>> idsAvailable = readIdsAvailable();
+      Map<Range<Instant>, RangeSet<Instant>> subRangeDeletions = readSubRangeDeletions();
+
+      RangeSet<Long> availableIdsForTsRange = null;
+      Iterator<Range<Long>> idRangeIter = null;
+      RangeSet<Long> idsUsed = TreeRangeSet.create();
+      for (TimestampedValueWithId<T> pendingAdd : elements) {
+        // Since elements are in increasing ts order, often we'll be able to reuse the previous
+        // iteration's range.
+        if (currentTsRange == null
+            || !currentTsRange.contains(pendingAdd.getValue().getTimestamp())) {
+          if (availableIdsForTsRange != null) {
+            // We're moving onto a new ts range. Remove all used ids
+            availableIdsForTsRange.removeAll(idsUsed);
+            idsUsed = TreeRangeSet.create();
+          }
+
+          // Lookup the range for the current timestamp.
+          currentTsRange = getTrackedRange(pendingAdd.getValue().getTimestamp());
+          // Lookup available ids for this timestamp range. If nothing there, we default to all ids
+          // available.
+          availableIdsForTsRange =
+              idsAvailable.computeIfAbsent(
+                  currentTsRange,
+                  r -> TreeRangeSet.create(ImmutableList.of(Range.closedOpen(MIN_ID, MAX_ID))));
+          idRangeIter = availableIdsForTsRange.asRanges().iterator();
+          currentIdRange = null;
+          currentTsRangeDeletions = subRangeDeletions.get(currentTsRange);
+        }
+
+        if (currentIdRange == null || currentId >= currentIdRange.upperEndpoint()) {
+          // Move to the next range of free ids, and start assigning ranges from there.
+          currentIdRange = idRangeIter.next();
+          currentId = currentIdRange.lowerEndpoint();
+        }
+
+        if (currentTsRangeDeletions != null) {
+          currentTsRangeDeletions.remove(
+              Range.closedOpen(
+                  pendingAdd.getValue().getTimestamp(),
+                  pendingAdd.getValue().getTimestamp().plus(1)));
+        }
+        idsUsed.add(Range.closedOpen(currentId, currentId + 1));
+        output.accept(pendingAdd.getValue(), currentId++);
+      }
+      if (availableIdsForTsRange != null) {
+        availableIdsForTsRange.removeAll(idsUsed);
+      }
+      idsAvailableValue.write(idsAvailable);
+      subRangeDeletionsValue.write(subRangeDeletions);
+    }
+
+    // Remove a timestamp range. Returns ids freed up.
+    void remove(Range<Instant> tsRange) throws ExecutionException, InterruptedException {
+      Map<Range<Instant>, RangeSet<Long>> idsAvailable = readIdsAvailable();
+      Map<Range<Instant>, RangeSet<Instant>> subRangeDeletions = readSubRangeDeletions();
+
+      for (Range<Instant> current = getTrackedRange(tsRange.lowerEndpoint());
+          current.lowerEndpoint().isBefore(tsRange.upperEndpoint());
+          current = getTrackedRange(current.lowerEndpoint().plus(RESOLUTION))) {
+        // TODO(reuvenlax): shouldn't need to iterate over all ranges.
+        boolean rangeCleared;
+        if (!tsRange.encloses(current)) {
+          // This can happen if the beginning or the end of tsRange doesn't fall on a RESOLUTION
+          // boundary. Since we
+          // are deleting a portion of a tracked range, track what we are deleting.
+          RangeSet<Instant> rangeDeletions =
+              subRangeDeletions.computeIfAbsent(current, r -> TreeRangeSet.create());
+          rangeDeletions.add(tsRange.intersection(current));
+          // If we ended up deleting the whole range, than we can simply remove it from the tracking
+          // map.
+          rangeCleared = rangeDeletions.encloses(current);
+        } else {
+          rangeCleared = true;
+        }
+        if (rangeCleared) {
+          // Remove the range from both maps.
+          idsAvailable.remove(current);
+          subRangeDeletions.remove(current);
+        }
+      }
+      idsAvailableValue.write(idsAvailable);
+      subRangeDeletionsValue.write(subRangeDeletions);
+    }
+  }
+
+  @AutoValue
+  abstract static class TimestampedValueWithId<T> {
+    private static final Comparator<TimestampedValueWithId<?>> COMPARATOR =
+        Comparator.<TimestampedValueWithId<?>, Instant>comparing(v -> v.getValue().getTimestamp())
+            .thenComparingLong(TimestampedValueWithId::getId);
+
+    abstract TimestampedValue<T> getValue();
+
+    abstract long getId();
+
+    static <T> TimestampedValueWithId<T> of(TimestampedValue<T> value, long id) {
+      return new AutoValue_WindmillStateInternals_TimestampedValueWithId<>(value, id);
+    }
+
+    static <T> TimestampedValueWithId<T> bound(Instant ts) {
+      return of(TimestampedValue.of(null, ts), Long.MIN_VALUE);
+    }
+  }
+
+  static class WindmillOrderedList<T> extends SimpleWindmillState implements OrderedListState<T> {
     private final StateNamespace namespace;
     private final StateTag<OrderedListState<T>> spec;
     private final ByteString stateKey;
     private final String stateFamily;
     private final Coder<T> elemCoder;
+    private boolean complete;
+    private boolean cleared = false;
+    // We need to sort based on timestamp, but we need objects with the same timestamp to be treated
+    // as unique. We can't use a MultiSet as we can't construct a comparator that uniquely
+    // identifies objects,
+    // so we construct a unique in-memory long ids for each element.
+    private SortedSet<TimestampedValueWithId<T>> pendingAdds =
+        Sets.newTreeSet(TimestampedValueWithId.COMPARATOR);
+
+    private RangeSet<Instant> pendingDeletes = TreeRangeSet.create();
+    private IdTracker idTracker;
+
+    // The default proto values for SortedListRange correspond to the minimum and maximum
+    // timestamps.
+    static final long MIN_TS_MICROS = SortedListRange.getDefaultInstance().getStart();
+    static final long MAX_TS_MICROS = SortedListRange.getDefaultInstance().getLimit();
 
     private WindmillOrderedList(
+        StateTable derivedStateTable,
         StateNamespace namespace,
         StateTag<OrderedListState<T>> spec,
         String stateFamily,
@@ -487,64 +850,226 @@
         boolean isNewKey) {
       this.namespace = namespace;
       this.spec = spec;
+
       this.stateKey = encodeKey(namespace, spec);
       this.stateFamily = stateFamily;
       this.elemCoder = elemCoder;
+      this.complete = isNewKey;
+      this.idTracker = new IdTracker(derivedStateTable, namespace, spec, stateFamily, complete);
     }
 
     @Override
     public Iterable<TimestampedValue<T>> read() {
-      throw new UnsupportedOperationException(
-          String.format("%s is not supported", OrderedListState.class.getSimpleName()));
+      return readRange(null, null);
+    }
+
+    private SortedSet<TimestampedValueWithId<T>> getPendingAddRange(
+        @Nullable Instant minTimestamp, @Nullable Instant limitTimestamp) {
+      SortedSet<TimestampedValueWithId<T>> pendingInRange = pendingAdds;
+      if (minTimestamp != null && limitTimestamp != null) {
+        pendingInRange =
+            pendingInRange.subSet(
+                TimestampedValueWithId.bound(minTimestamp),
+                TimestampedValueWithId.bound(limitTimestamp));
+      } else if (minTimestamp == null && limitTimestamp != null) {
+        pendingInRange = pendingInRange.headSet(TimestampedValueWithId.bound(limitTimestamp));
+      } else if (limitTimestamp == null && minTimestamp != null) {
+        pendingInRange = pendingInRange.tailSet(TimestampedValueWithId.bound(minTimestamp));
+      }
+      return pendingInRange;
     }
 
     @Override
-    public Iterable<TimestampedValue<T>> readRange(Instant minTimestamp, Instant limitTimestamp) {
-      throw new UnsupportedOperationException(
-          String.format("%s is not supported", OrderedListState.class.getSimpleName()));
+    public Iterable<TimestampedValue<T>> readRange(
+        @Nullable Instant minTimestamp, @Nullable Instant limitTimestamp) {
+      idTracker.readLater();
+
+      final Future<Iterable<TimestampedValue<T>>> future = getFuture(minTimestamp, limitTimestamp);
+      try (Closeable scope = scopedReadState()) {
+        SortedSet<TimestampedValueWithId<T>> pendingInRange =
+            getPendingAddRange(minTimestamp, limitTimestamp);
+
+        // Transform the return iterator so it has the same type as pendingAdds. We need to ensure
+        // that the ids don't overlap with any in pendingAdds, so begin with pendingAdds.size().
+        Iterable<TimestampedValueWithId<T>> data =
+            new Iterable<TimestampedValueWithId<T>>() {
+              private Iterable<TimestampedValue<T>> iterable = future.get();
+
+              @Override
+              public Iterator<TimestampedValueWithId<T>> iterator() {
+                return new Iterator<TimestampedValueWithId<T>>() {
+                  private Iterator<TimestampedValue<T>> iter = iterable.iterator();
+                  private long currentId = pendingAdds.size();
+
+                  @Override
+                  public boolean hasNext() {
+                    return iter.hasNext();
+                  }
+
+                  @Override
+                  public TimestampedValueWithId<T> next() {
+                    return TimestampedValueWithId.of(iter.next(), currentId++);
+                  }
+                };
+              }
+            };
+
+        Iterable<TimestampedValueWithId<T>> includingAdds =
+            Iterables.mergeSorted(
+                ImmutableList.of(data, pendingInRange), TimestampedValueWithId.COMPARATOR);
+        Iterable<TimestampedValue<T>> fullIterable =
+            Iterables.filter(
+                Iterables.transform(includingAdds, TimestampedValueWithId::getValue),
+                tv -> !pendingDeletes.contains(tv.getTimestamp()));
+        // TODO(reuvenlax): If we have a known bounded amount of data, cache known ranges.
+        return fullIterable;
+      } catch (InterruptedException | ExecutionException | IOException e) {
+        if (e instanceof InterruptedException) {
+          Thread.currentThread().interrupt();
+        }
+        throw new RuntimeException("Unable to read state", e);
+      }
     }
 
     @Override
     public void clear() {
-      throw new UnsupportedOperationException(
-          String.format("%s is not supported", OrderedListState.class.getSimpleName()));
+      cleared = true;
+      complete = true;
+      pendingAdds.clear();
+      pendingDeletes.clear();
+      try {
+        idTracker.clear();
+      } catch (ExecutionException | InterruptedException e) {
+        throw new RuntimeException(e);
+      }
     }
 
     @Override
     public void clearRange(Instant minTimestamp, Instant limitTimestamp) {
-      throw new UnsupportedOperationException(
-          String.format("%s is not supported", OrderedListState.class.getSimpleName()));
+      getPendingAddRange(minTimestamp, limitTimestamp).clear();
+      pendingDeletes.add(Range.closedOpen(minTimestamp, limitTimestamp));
     }
 
     @Override
     public void add(TimestampedValue<T> value) {
-      throw new UnsupportedOperationException(
-          String.format("%s is not supported", OrderedListState.class.getSimpleName()));
+      // We use the current size of the container as the in-memory id. This works because
+      // pendingAdds is completely
+      // cleared when it is processed (otherwise we could end up with duplicate elements in the same
+      // container). These
+      // are not the ids that will be sent to windmill.
+      pendingAdds.add(TimestampedValueWithId.of(value, pendingAdds.size()));
+      // Leave pendingDeletes alone. Since we can have multiple values with the same timestamp, we
+      // may still need
+      // overlapping deletes to remove previous entries at this timestamp.
     }
 
     @Override
     public ReadableState<Boolean> isEmpty() {
-      throw new UnsupportedOperationException(
-          String.format("%s is not supported", OrderedListState.class.getSimpleName()));
+      return new ReadableState<Boolean>() {
+        @Override
+        public ReadableState<Boolean> readLater() {
+          WindmillOrderedList.this.readLater();
+          return this;
+        }
+
+        @Override
+        public Boolean read() {
+          return Iterables.isEmpty(WindmillOrderedList.this.read());
+        }
+      };
     }
 
     @Override
     public OrderedListState<T> readLater() {
-      throw new UnsupportedOperationException(
-          String.format("%s is not supported", OrderedListState.class.getSimpleName()));
+      return readRangeLater(null, null);
     }
 
     @Override
-    public OrderedListState<T> readRangeLater(Instant minTimestamp, Instant limitTimestamp) {
-      throw new UnsupportedOperationException(
-          String.format("%s is not supported", OrderedListState.class.getSimpleName()));
+    @SuppressWarnings("FutureReturnValueIgnored")
+    public OrderedListState<T> readRangeLater(
+        @Nullable Instant minTimestamp, @Nullable Instant limitTimestamp) {
+      idTracker.readLater();
+      getFuture(minTimestamp, limitTimestamp);
+      return this;
     }
 
     @Override
     public WorkItemCommitRequest persistDirectly(ForKey cache) throws IOException {
       WorkItemCommitRequest.Builder commitBuilder = WorkItemCommitRequest.newBuilder();
+      TagSortedListUpdateRequest.Builder updatesBuilder =
+          commitBuilder.addSortedListUpdatesBuilder().setStateFamily(stateFamily).setTag(stateKey);
+      try {
+        if (cleared) {
+          // Default range.
+          updatesBuilder.addDeletesBuilder().build();
+          cleared = false;
+        }
+
+        if (!pendingAdds.isEmpty()) {
+          // TODO(reuvenlax): Once we start caching data, we should remove this line. We have it
+          // here now
+          // because once we persist
+          // added data we forget about it from the cache, so the object is no longer complete.
+          complete = false;
+
+          TagSortedListInsertRequest.Builder insertBuilder = updatesBuilder.addInsertsBuilder();
+          idTracker.add(
+              pendingAdds,
+              (elem, id) -> {
+                try {
+                  ByteString.Output elementStream = ByteString.newOutput();
+                  elemCoder.encode(elem.getValue(), elementStream, Context.OUTER);
+                  insertBuilder.addEntries(
+                      SortedListEntry.newBuilder()
+                          .setValue(elementStream.toByteString())
+                          .setSortKey(
+                              WindmillTimeUtils.harnessToWindmillTimestamp(elem.getTimestamp()))
+                          .setId(id));
+                } catch (IOException e) {
+                  throw new RuntimeException(e);
+                }
+              });
+          pendingAdds.clear();
+          insertBuilder.build();
+        }
+
+        if (!pendingDeletes.isEmpty()) {
+          for (Range<Instant> range : pendingDeletes.asRanges()) {
+            TagSortedListDeleteRequest.Builder deletesBuilder = updatesBuilder.addDeletesBuilder();
+            deletesBuilder.setRange(
+                SortedListRange.newBuilder()
+                    .setStart(WindmillTimeUtils.harnessToWindmillTimestamp(range.lowerEndpoint()))
+                    .setLimit(WindmillTimeUtils.harnessToWindmillTimestamp(range.upperEndpoint())));
+            deletesBuilder.build();
+            idTracker.remove(range);
+          }
+          pendingDeletes.clear();
+        }
+      } catch (ExecutionException | InterruptedException e) {
+        throw new RuntimeException(e);
+      }
       return commitBuilder.buildPartial();
     }
+
+    private Future<Iterable<TimestampedValue<T>>> getFuture(
+        @Nullable Instant minTimestamp, @Nullable Instant limitTimestamp) {
+      long startSortKey =
+          minTimestamp != null
+              ? WindmillTimeUtils.harnessToWindmillTimestamp(minTimestamp)
+              : MIN_TS_MICROS;
+      long limitSortKey =
+          limitTimestamp != null
+              ? WindmillTimeUtils.harnessToWindmillTimestamp(limitTimestamp)
+              : MAX_TS_MICROS;
+
+      if (complete) {
+        // Right now we don't cache any data, so complete means an empty list.
+        // TODO(reuvenlax): change this once we start caching data.
+        return Futures.immediateFuture(Collections.emptyList());
+      }
+      return reader.orderedListFuture(
+          Range.closedOpen(startSortKey, limitSortKey), stateKey, stateFamily, elemCoder);
+    }
   }
 
   private static class WindmillBag<T> extends SimpleWindmillState implements BagState<T> {
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillStateReader.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillStateReader.java
index 3c131c6..dd93dd2 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillStateReader.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillStateReader.java
@@ -17,11 +17,15 @@
  */
 package org.apache.beam.runners.dataflow.worker;
 
+import com.google.api.client.util.Lists;
+import com.google.auto.value.AutoValue;
+import com.google.common.collect.Iterables;
 import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
 import java.io.IOException;
 import java.io.InputStream;
 import java.util.ArrayList;
 import java.util.Collections;
+import java.util.Comparator;
 import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
@@ -33,23 +37,28 @@
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
 import org.apache.beam.runners.dataflow.worker.windmill.Windmill;
+import org.apache.beam.runners.dataflow.worker.windmill.Windmill.SortedListEntry;
+import org.apache.beam.runners.dataflow.worker.windmill.Windmill.SortedListRange;
 import org.apache.beam.runners.dataflow.worker.windmill.Windmill.TagBag;
+import org.apache.beam.runners.dataflow.worker.windmill.Windmill.TagSortedListFetchRequest;
 import org.apache.beam.runners.dataflow.worker.windmill.Windmill.TagValue;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
 import org.apache.beam.sdk.util.Weighted;
+import org.apache.beam.sdk.values.TimestampedValue;
 import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
 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.Function;
-import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Objects;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.AbstractIterator;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ForwardingList;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Range;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Sets;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.util.concurrent.ForwardingFuture;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.util.concurrent.Futures;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.util.concurrent.SettableFuture;
-import org.checkerframework.checker.nullness.qual.Nullable;
 import org.joda.time.Instant;
 
 /**
@@ -60,6 +69,7 @@
  * <p>CAUTION Watch out for escaping references to the reader ending up inside {@link
  * WindmillStateCache}.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class WindmillStateReader {
   /**
    * Ideal maximum bytes in a TagBag response. However, Windmill will always return at least one
@@ -68,6 +78,12 @@
   public static final long MAX_BAG_BYTES = 8L << 20; // 8MB
 
   /**
+   * Ideal maximum bytes in a TagSortedList response. However, Windmill will always return at least
+   * one value if possible irrespective of this limit.
+   */
+  public static final long MAX_ORDERED_LIST_BYTES = 8L << 20; // 8MB
+
+  /**
    * Ideal maximum bytes in a KeyedGetDataResponse. However, Windmill will always return at least
    * one value if possible irrespective of this limit.
    */
@@ -77,70 +93,66 @@
    * When combined with a key and computationId, represents the unique address for state managed by
    * Windmill.
    */
-  private static class StateTag {
-    private enum Kind {
+  @AutoValue
+  abstract static class StateTag<RequestPositionT> {
+    enum Kind {
       VALUE,
       BAG,
-      WATERMARK;
+      WATERMARK,
+      ORDERED_LIST
     }
 
-    private final Kind kind;
-    private final ByteString tag;
-    private final String stateFamily;
+    abstract Kind getKind();
+
+    abstract ByteString getTag();
+
+    abstract String getStateFamily();
 
     /**
-     * For {@link Kind#BAG} kinds: A previous 'continuation_position' returned by Windmill to signal
-     * the resulting bag was incomplete. Sending that position will request the next page of values.
-     * Null for first request.
+     * For {@link Kind#BAG, Kind#ORDERED_LIST} kinds: A previous 'continuation_position' returned by
+     * Windmill to signal the resulting bag was incomplete. Sending that position will request the
+     * next page of values. Null for first request.
      *
      * <p>Null for other kinds.
      */
-    private final @Nullable Long requestPosition;
+    @Nullable
+    abstract RequestPositionT getRequestPosition();
 
-    private StateTag(
-        Kind kind, ByteString tag, String stateFamily, @Nullable Long requestPosition) {
-      this.kind = kind;
-      this.tag = tag;
-      this.stateFamily = Preconditions.checkNotNull(stateFamily);
-      this.requestPosition = requestPosition;
+    /** For {@link Kind#ORDERED_LIST} kinds: the range to fetch or delete. */
+    @Nullable
+    abstract Range<Long> getSortedListRange();
+
+    static <RequestPositionT> StateTag<RequestPositionT> of(
+        Kind kind, ByteString tag, String stateFamily, @Nullable RequestPositionT requestPosition) {
+      return new AutoValue_WindmillStateReader_StateTag.Builder<RequestPositionT>()
+          .setKind(kind)
+          .setTag(tag)
+          .setStateFamily(stateFamily)
+          .setRequestPosition(requestPosition)
+          .build();
     }
 
-    private StateTag(Kind kind, ByteString tag, String stateFamily) {
-      this(kind, tag, stateFamily, null);
+    static <RequestPositionT> StateTag<RequestPositionT> of(
+        Kind kind, ByteString tag, String stateFamily) {
+      return of(kind, tag, stateFamily, null);
     }
 
-    @Override
-    public boolean equals(@Nullable Object obj) {
-      if (this == obj) {
-        return true;
-      }
+    abstract Builder<RequestPositionT> toBuilder();
 
-      if (!(obj instanceof StateTag)) {
-        return false;
-      }
+    @AutoValue.Builder
+    abstract static class Builder<RequestPositionT> {
+      abstract Builder<RequestPositionT> setKind(Kind kind);
 
-      StateTag that = (StateTag) obj;
-      return Objects.equal(this.kind, that.kind)
-          && Objects.equal(this.tag, that.tag)
-          && Objects.equal(this.stateFamily, that.stateFamily)
-          && Objects.equal(this.requestPosition, that.requestPosition);
-    }
+      abstract Builder<RequestPositionT> setTag(ByteString tag);
 
-    @Override
-    public int hashCode() {
-      return Objects.hashCode(kind, tag, stateFamily, requestPosition);
-    }
+      abstract Builder<RequestPositionT> setStateFamily(String stateFamily);
 
-    @Override
-    public String toString() {
-      return "Tag("
-          + kind
-          + ","
-          + tag.toStringUtf8()
-          + ","
-          + stateFamily
-          + (requestPosition == null ? "" : ("," + requestPosition.toString()))
-          + ")";
+      abstract Builder<RequestPositionT> setRequestPosition(
+          @Nullable RequestPositionT requestPosition);
+
+      abstract Builder<RequestPositionT> setSortedListRange(@Nullable Range<Long> sortedListRange);
+
+      abstract StateTag<RequestPositionT> build();
     }
   }
 
@@ -148,13 +160,13 @@
    * An in-memory collection of deserialized values and an optional continuation position to pass to
    * Windmill when fetching the next page of values.
    */
-  private static class ValuesAndContPosition<T> {
+  private static class ValuesAndContPosition<T, ContinuationT> {
     private final List<T> values;
 
     /** Position to pass to next request for next page of values. Null if done. */
-    private final @Nullable Long continuationPosition;
+    private final @Nullable ContinuationT continuationPosition;
 
-    public ValuesAndContPosition(List<T> values, @Nullable Long continuationPosition) {
+    public ValuesAndContPosition(List<T> values, @Nullable ContinuationT continuationPosition) {
       this.values = values;
       this.continuationPosition = continuationPosition;
     }
@@ -218,13 +230,15 @@
     }
   }
 
-  @VisibleForTesting ConcurrentLinkedQueue<StateTag> pendingLookups = new ConcurrentLinkedQueue<>();
-  private ConcurrentHashMap<StateTag, CoderAndFuture<?, ?>> waiting = new ConcurrentHashMap<>();
+  @VisibleForTesting
+  ConcurrentLinkedQueue<StateTag<?>> pendingLookups = new ConcurrentLinkedQueue<>();
+
+  private ConcurrentHashMap<StateTag<?>, CoderAndFuture<?, ?>> waiting = new ConcurrentHashMap<>();
 
   private <ElemT, FutureT> Future<FutureT> stateFuture(
-      StateTag stateTag, @Nullable Coder<ElemT> coder) {
+      StateTag<?> stateTag, @Nullable Coder<ElemT> coder) {
     CoderAndFuture<ElemT, FutureT> coderAndFuture =
-        new CoderAndFuture<>(coder, SettableFuture.<FutureT>create());
+        new CoderAndFuture<>(coder, SettableFuture.create());
     CoderAndFuture<?, ?> existingCoderAndFutureWildcard =
         waiting.putIfAbsent(stateTag, coderAndFuture);
     if (existingCoderAndFutureWildcard == null) {
@@ -242,7 +256,7 @@
   }
 
   private <ElemT, FutureT> CoderAndFuture<ElemT, FutureT> getWaiting(
-      StateTag stateTag, boolean shouldRemove) {
+      StateTag<?> stateTag, boolean shouldRemove) {
     CoderAndFuture<?, ?> coderAndFutureWildcard;
     if (shouldRemove) {
       coderAndFutureWildcard = waiting.remove(stateTag);
@@ -259,29 +273,41 @@
   }
 
   public Future<Instant> watermarkFuture(ByteString encodedTag, String stateFamily) {
-    return stateFuture(new StateTag(StateTag.Kind.WATERMARK, encodedTag, stateFamily), null);
+    return stateFuture(StateTag.of(StateTag.Kind.WATERMARK, encodedTag, stateFamily), null);
   }
 
   public <T> Future<T> valueFuture(ByteString encodedTag, String stateFamily, Coder<T> coder) {
-    return stateFuture(new StateTag(StateTag.Kind.VALUE, encodedTag, stateFamily), coder);
+    return stateFuture(StateTag.of(StateTag.Kind.VALUE, encodedTag, stateFamily), coder);
   }
 
   public <T> Future<Iterable<T>> bagFuture(
       ByteString encodedTag, String stateFamily, Coder<T> elemCoder) {
     // First request has no continuation position.
-    StateTag stateTag = new StateTag(StateTag.Kind.BAG, encodedTag, stateFamily);
+    StateTag<Long> stateTag = StateTag.of(StateTag.Kind.BAG, encodedTag, stateFamily);
     // Convert the ValuesAndContPosition<T> to Iterable<T>.
-    return valuesToPagingIterableFuture(
-        stateTag, elemCoder, this.<T, ValuesAndContPosition<T>>stateFuture(stateTag, elemCoder));
+    return valuesToPagingIterableFuture(stateTag, elemCoder, this.stateFuture(stateTag, elemCoder));
+  }
+
+  public <T> Future<Iterable<TimestampedValue<T>>> orderedListFuture(
+      Range<Long> range, ByteString encodedTag, String stateFamily, Coder<T> elemCoder) {
+    // First request has no continuation position.
+    StateTag<ByteString> stateTag =
+        StateTag.<ByteString>of(StateTag.Kind.ORDERED_LIST, encodedTag, stateFamily)
+            .toBuilder()
+            .setSortedListRange(Preconditions.checkNotNull(range))
+            .build();
+    return Preconditions.checkNotNull(
+        valuesToPagingIterableFuture(stateTag, elemCoder, this.stateFuture(stateTag, elemCoder)));
   }
 
   /**
-   * Internal request to fetch the next 'page' of values in a TagBag. Return null if no continuation
-   * position is in {@code contStateTag}, which signals there are no more pages.
+   * Internal request to fetch the next 'page' of values. Return null if no continuation position is
+   * in {@code contStateTag}, which signals there are no more pages.
    */
-  private @Nullable <T> Future<ValuesAndContPosition<T>> continuationBagFuture(
-      StateTag contStateTag, Coder<T> elemCoder) {
-    if (contStateTag.requestPosition == null) {
+  private @Nullable <ElemT, ContinuationT, ResultT>
+      Future<ValuesAndContPosition<ResultT, ContinuationT>> continuationFuture(
+          StateTag<ContinuationT> contStateTag, Coder<ElemT> elemCoder) {
+    if (contStateTag.getRequestPosition() == null) {
       // We're done.
       return null;
     }
@@ -338,18 +364,19 @@
   }
 
   /** Function to extract an {@link Iterable} from the continuation-supporting page read future. */
-  private static class ToIterableFunction<T>
-      implements Function<ValuesAndContPosition<T>, Iterable<T>> {
+  private static class ToIterableFunction<ElemT, ContinuationT, ResultT>
+      implements Function<ValuesAndContPosition<ResultT, ContinuationT>, Iterable<ResultT>> {
     /**
      * Reader to request continuation pages from, or {@literal null} if no continuation pages
      * required.
      */
     private @Nullable WindmillStateReader reader;
 
-    private final StateTag stateTag;
-    private final Coder<T> elemCoder;
+    private final StateTag<ContinuationT> stateTag;
+    private final Coder<ElemT> elemCoder;
 
-    public ToIterableFunction(WindmillStateReader reader, StateTag stateTag, Coder<T> elemCoder) {
+    public ToIterableFunction(
+        WindmillStateReader reader, StateTag<ContinuationT> stateTag, Coder<ElemT> elemCoder) {
       this.reader = reader;
       this.stateTag = stateTag;
       this.elemCoder = elemCoder;
@@ -359,7 +386,8 @@
         value = "NP_METHOD_PARAMETER_TIGHTENS_ANNOTATION",
         justification = "https://github.com/google/guava/issues/920")
     @Override
-    public Iterable<T> apply(@Nonnull ValuesAndContPosition<T> valuesAndContPosition) {
+    public Iterable<ResultT> apply(
+        @Nonnull ValuesAndContPosition<ResultT, ContinuationT> valuesAndContPosition) {
       if (valuesAndContPosition.continuationPosition == null) {
         // Number of values is small enough Windmill sent us the entire bag in one response.
         reader = null;
@@ -367,12 +395,16 @@
       } else {
         // Return an iterable which knows how to come back for more.
         StateTag contStateTag =
-            new StateTag(
-                stateTag.kind,
-                stateTag.tag,
-                stateTag.stateFamily,
+            StateTag.of(
+                stateTag.getKind(),
+                stateTag.getTag(),
+                stateTag.getStateFamily(),
                 valuesAndContPosition.continuationPosition);
-        return new BagPagingIterable<>(
+        if (stateTag.getSortedListRange() != null) {
+          contStateTag =
+              contStateTag.toBuilder().setSortedListRange(stateTag.getSortedListRange()).build();
+        }
+        return new PagingIterable<ElemT, ContinuationT, ResultT>(
             reader, valuesAndContPosition.values, contStateTag, elemCoder);
       }
     }
@@ -382,18 +414,20 @@
    * Return future which transforms a {@code ValuesAndContPosition<T>} result into the initial
    * Iterable<T> result expected from the external caller.
    */
-  private <T> Future<Iterable<T>> valuesToPagingIterableFuture(
-      final StateTag stateTag,
-      final Coder<T> elemCoder,
-      final Future<ValuesAndContPosition<T>> future) {
-    return Futures.lazyTransform(future, new ToIterableFunction<T>(this, stateTag, elemCoder));
+  private <ElemT, ResultT, ContinuationT> Future<Iterable<ResultT>> valuesToPagingIterableFuture(
+      final StateTag<ContinuationT> stateTag,
+      final Coder<ElemT> elemCoder,
+      final Future<ValuesAndContPosition<ResultT, ContinuationT>> future) {
+    Function<ValuesAndContPosition<ResultT, ContinuationT>, Iterable<ResultT>> toIterable =
+        new ToIterableFunction<>(this, stateTag, elemCoder);
+    return Futures.lazyTransform(future, toIterable);
   }
 
   public void startBatchAndBlock() {
     // First, drain work out of the pending lookups into a set. These will be the items we fetch.
-    HashSet<StateTag> toFetch = new HashSet<>();
+    HashSet<StateTag<?>> toFetch = Sets.newHashSet();
     while (!pendingLookups.isEmpty()) {
-      StateTag stateTag = pendingLookups.poll();
+      StateTag<?> stateTag = pendingLookups.poll();
       if (stateTag == null) {
         break;
       }
@@ -411,7 +445,6 @@
 
     Windmill.KeyedGetDataRequest request = createRequest(toFetch);
     Windmill.KeyedGetDataResponse response = server.getStateData(computation, request);
-
     if (response == null) {
       throw new RuntimeException("Windmill unexpectedly returned null for request " + request);
     }
@@ -423,47 +456,72 @@
     return bytesRead;
   }
 
-  private Windmill.KeyedGetDataRequest createRequest(Iterable<StateTag> toFetch) {
+  private Windmill.KeyedGetDataRequest createRequest(Iterable<StateTag<?>> toFetch) {
     Windmill.KeyedGetDataRequest.Builder keyedDataBuilder =
         Windmill.KeyedGetDataRequest.newBuilder()
             .setKey(key)
             .setShardingKey(shardingKey)
             .setWorkToken(workToken);
 
-    for (StateTag stateTag : toFetch) {
-      switch (stateTag.kind) {
+    List<StateTag<?>> orderedListsToFetch = Lists.newArrayList();
+    for (StateTag<?> stateTag : toFetch) {
+      switch (stateTag.getKind()) {
         case BAG:
           TagBag.Builder bag =
               keyedDataBuilder
                   .addBagsToFetchBuilder()
-                  .setTag(stateTag.tag)
-                  .setStateFamily(stateTag.stateFamily)
+                  .setTag(stateTag.getTag())
+                  .setStateFamily(stateTag.getStateFamily())
                   .setFetchMaxBytes(MAX_BAG_BYTES);
-          if (stateTag.requestPosition != null) {
+          if (stateTag.getRequestPosition() != null) {
             // We're asking for the next page.
-            bag.setRequestPosition(stateTag.requestPosition);
+            bag.setRequestPosition((Long) stateTag.getRequestPosition());
           }
           break;
 
+        case ORDERED_LIST:
+          orderedListsToFetch.add(stateTag);
+          break;
+
         case WATERMARK:
           keyedDataBuilder
               .addWatermarkHoldsToFetchBuilder()
-              .setTag(stateTag.tag)
-              .setStateFamily(stateTag.stateFamily);
+              .setTag(stateTag.getTag())
+              .setStateFamily(stateTag.getStateFamily());
           break;
 
         case VALUE:
           keyedDataBuilder
               .addValuesToFetchBuilder()
-              .setTag(stateTag.tag)
-              .setStateFamily(stateTag.stateFamily);
+              .setTag(stateTag.getTag())
+              .setStateFamily(stateTag.getStateFamily());
           break;
 
         default:
-          throw new RuntimeException("Unknown kind of tag requested: " + stateTag.kind);
+          throw new RuntimeException("Unknown kind of tag requested: " + stateTag.getKind());
       }
     }
-
+    orderedListsToFetch.sort(
+        Comparator.<StateTag<?>>comparingLong(s -> s.getSortedListRange().lowerEndpoint())
+            .thenComparingLong(s -> s.getSortedListRange().upperEndpoint()));
+    for (StateTag<?> stateTag : orderedListsToFetch) {
+      Range<Long> range = Preconditions.checkNotNull(stateTag.getSortedListRange());
+      TagSortedListFetchRequest.Builder sorted_list =
+          keyedDataBuilder
+              .addSortedListsToFetchBuilder()
+              .setTag(stateTag.getTag())
+              .setStateFamily(stateTag.getStateFamily())
+              .setFetchMaxBytes(MAX_ORDERED_LIST_BYTES);
+      sorted_list.addFetchRanges(
+          SortedListRange.newBuilder()
+              .setStart(range.lowerEndpoint())
+              .setLimit(range.upperEndpoint())
+              .build());
+      if (stateTag.getRequestPosition() != null) {
+        // We're asking for the next page.
+        sorted_list.setRequestPosition((ByteString) stateTag.getRequestPosition());
+      }
+    }
     keyedDataBuilder.setMaxBytes(MAX_KEY_BYTES);
 
     return keyedDataBuilder.build();
@@ -472,14 +530,14 @@
   private void consumeResponse(
       Windmill.KeyedGetDataRequest request,
       Windmill.KeyedGetDataResponse response,
-      Set<StateTag> toFetch) {
+      Set<StateTag<?>> toFetch) {
     bytesRead += response.getSerializedSize();
 
     if (response.getFailed()) {
       // Set up all the futures for this key to throw an exception:
       KeyTokenInvalidException keyTokenInvalidException =
           new KeyTokenInvalidException(key.toStringUtf8());
-      for (StateTag stateTag : toFetch) {
+      for (StateTag<?> stateTag : toFetch) {
         waiting.get(stateTag).future.setException(keyTokenInvalidException);
       }
       return;
@@ -490,8 +548,8 @@
     }
 
     for (Windmill.TagBag bag : response.getBagsList()) {
-      StateTag stateTag =
-          new StateTag(
+      StateTag<Long> stateTag =
+          StateTag.of(
               StateTag.Kind.BAG,
               bag.getTag(),
               bag.getStateFamily(),
@@ -504,8 +562,8 @@
     }
 
     for (Windmill.WatermarkHold hold : response.getWatermarkHoldsList()) {
-      StateTag stateTag =
-          new StateTag(StateTag.Kind.WATERMARK, hold.getTag(), hold.getStateFamily());
+      StateTag<Long> stateTag =
+          StateTag.of(StateTag.Kind.WATERMARK, hold.getTag(), hold.getStateFamily());
       if (!toFetch.remove(stateTag)) {
         throw new IllegalStateException(
             "Received response for unrequested tag " + stateTag + ". Pending tags: " + toFetch);
@@ -514,13 +572,33 @@
     }
 
     for (Windmill.TagValue value : response.getValuesList()) {
-      StateTag stateTag = new StateTag(StateTag.Kind.VALUE, value.getTag(), value.getStateFamily());
+      StateTag<Long> stateTag =
+          StateTag.of(StateTag.Kind.VALUE, value.getTag(), value.getStateFamily());
       if (!toFetch.remove(stateTag)) {
         throw new IllegalStateException(
             "Received response for unrequested tag " + stateTag + ". Pending tags: " + toFetch);
       }
       consumeTagValue(value, stateTag);
     }
+    for (Windmill.TagSortedListFetchResponse sorted_list : response.getTagSortedListsList()) {
+      SortedListRange sortedListRange = Iterables.getOnlyElement(sorted_list.getFetchRangesList());
+      Range<Long> range = Range.closedOpen(sortedListRange.getStart(), sortedListRange.getLimit());
+      StateTag<ByteString> stateTag =
+          StateTag.of(
+                  StateTag.Kind.ORDERED_LIST,
+                  sorted_list.getTag(),
+                  sorted_list.getStateFamily(),
+                  sorted_list.hasRequestPosition() ? sorted_list.getRequestPosition() : null)
+              .toBuilder()
+              .setSortedListRange(range)
+              .build();
+      if (!toFetch.remove(stateTag)) {
+        throw new IllegalStateException(
+            "Received response for unrequested tag " + stateTag + ". Pending tags: " + toFetch);
+      }
+
+      consumeSortedList(sorted_list, stateTag);
+    }
 
     if (!toFetch.isEmpty()) {
       throw new IllegalStateException(
@@ -577,9 +655,31 @@
     return valueList;
   }
 
-  private <T> void consumeBag(TagBag bag, StateTag stateTag) {
+  private <T> List<TimestampedValue<T>> sortedListPageValues(
+      Windmill.TagSortedListFetchResponse sortedListFetchResponse, Coder<T> elemCoder) {
+    if (sortedListFetchResponse.getEntriesCount() == 0) {
+      return new WeightedList<>(Collections.emptyList());
+    }
+
+    WeightedList<TimestampedValue<T>> entryList =
+        new WeightedList<>(new ArrayList<>(sortedListFetchResponse.getEntriesCount()));
+    for (SortedListEntry entry : sortedListFetchResponse.getEntriesList()) {
+      try {
+        T value = elemCoder.decode(entry.getValue().newInput(), Coder.Context.OUTER);
+        entryList.addWeighted(
+            TimestampedValue.of(
+                value, WindmillTimeUtils.windmillToHarnessTimestamp(entry.getSortKey())),
+            entry.getValue().size() + 8);
+      } catch (IOException e) {
+        throw new IllegalStateException("Unable to decode tag sorted list using " + elemCoder, e);
+      }
+    }
+    return entryList;
+  }
+
+  private <T> void consumeBag(TagBag bag, StateTag<Long> stateTag) {
     boolean shouldRemove;
-    if (stateTag.requestPosition == null) {
+    if (stateTag.getRequestPosition() == null) {
       // This is the response for the first page.
       // Leave the future in the cache so subsequent requests for the first page
       // can return immediately.
@@ -590,16 +690,18 @@
       // continuation positions.
       shouldRemove = true;
     }
-    CoderAndFuture<T, ValuesAndContPosition<T>> coderAndFuture = getWaiting(stateTag, shouldRemove);
-    SettableFuture<ValuesAndContPosition<T>> future = coderAndFuture.getNonDoneFuture(stateTag);
+    CoderAndFuture<T, ValuesAndContPosition<T, Long>> coderAndFuture =
+        getWaiting(stateTag, shouldRemove);
+    SettableFuture<ValuesAndContPosition<T, Long>> future =
+        coderAndFuture.getNonDoneFuture(stateTag);
     Coder<T> coder = coderAndFuture.getAndClearCoder();
-    List<T> values = this.<T>bagPageValues(bag, coder);
+    List<T> values = this.bagPageValues(bag, coder);
     future.set(
-        new ValuesAndContPosition<T>(
+        new ValuesAndContPosition<>(
             values, bag.hasContinuationPosition() ? bag.getContinuationPosition() : null));
   }
 
-  private void consumeWatermark(Windmill.WatermarkHold watermarkHold, StateTag stateTag) {
+  private void consumeWatermark(Windmill.WatermarkHold watermarkHold, StateTag<Long> stateTag) {
     CoderAndFuture<Void, Instant> coderAndFuture = getWaiting(stateTag, false);
     SettableFuture<Instant> future = coderAndFuture.getNonDoneFuture(stateTag);
     // No coders for watermarks
@@ -619,7 +721,7 @@
     future.set(hold);
   }
 
-  private <T> void consumeTagValue(TagValue tagValue, StateTag stateTag) {
+  private <T> void consumeTagValue(TagValue tagValue, StateTag<Long> stateTag) {
     CoderAndFuture<T, T> coderAndFuture = getWaiting(stateTag, false);
     SettableFuture<T> future = coderAndFuture.getNonDoneFuture(stateTag);
     Coder<T> coder = coderAndFuture.getAndClearCoder();
@@ -639,6 +741,35 @@
     }
   }
 
+  private <T> void consumeSortedList(
+      Windmill.TagSortedListFetchResponse sortedListFetchResponse, StateTag<ByteString> stateTag) {
+    boolean shouldRemove;
+    if (stateTag.getRequestPosition() == null) {
+      // This is the response for the first page.// Leave the future in the cache so subsequent
+      // requests for the first page
+      // can return immediately.
+      shouldRemove = false;
+    } else {
+      // This is a response for a subsequent page.
+      // Don't cache the future since we may need to make multiple requests with different
+      // continuation positions.
+      shouldRemove = true;
+    }
+
+    CoderAndFuture<T, ValuesAndContPosition<TimestampedValue<T>, ByteString>> coderAndFuture =
+        getWaiting(stateTag, shouldRemove);
+    SettableFuture<ValuesAndContPosition<TimestampedValue<T>, ByteString>> future =
+        coderAndFuture.getNonDoneFuture(stateTag);
+    Coder<T> coder = coderAndFuture.getAndClearCoder();
+    List<TimestampedValue<T>> values = this.sortedListPageValues(sortedListFetchResponse, coder);
+    future.set(
+        new ValuesAndContPosition<>(
+            values,
+            sortedListFetchResponse.hasContinuationPosition()
+                ? sortedListFetchResponse.getContinuationPosition()
+                : null));
+  }
+
   /**
    * An iterable over elements backed by paginated GetData requests to Windmill. The iterable may be
    * iterated over an arbitrary number of times and multiple iterators may be active simultaneously.
@@ -655,7 +786,7 @@
    *       call to iterator.
    * </ol>
    */
-  private static class BagPagingIterable<T> implements Iterable<T> {
+  private static class PagingIterable<ElemT, ContinuationT, ResultT> implements Iterable<ResultT> {
     /**
      * The reader we will use for scheduling continuation pages.
      *
@@ -664,16 +795,19 @@
     private final WindmillStateReader reader;
 
     /** Initial values returned for the first page. Never reclaimed. */
-    private final List<T> firstPage;
+    private final List<ResultT> firstPage;
 
     /** State tag with continuation position set for second page. */
-    private final StateTag secondPagePos;
+    private final StateTag<ContinuationT> secondPagePos;
 
     /** Coder for elements. */
-    private final Coder<T> elemCoder;
+    private final Coder<ElemT> elemCoder;
 
-    private BagPagingIterable(
-        WindmillStateReader reader, List<T> firstPage, StateTag secondPagePos, Coder<T> elemCoder) {
+    private PagingIterable(
+        WindmillStateReader reader,
+        List<ResultT> firstPage,
+        StateTag<ContinuationT> secondPagePos,
+        Coder<ElemT> elemCoder) {
       this.reader = reader;
       this.firstPage = firstPage;
       this.secondPagePos = secondPagePos;
@@ -681,16 +815,16 @@
     }
 
     @Override
-    public Iterator<T> iterator() {
-      return new AbstractIterator<T>() {
-        private Iterator<T> currentPage = firstPage.iterator();
-        private StateTag nextPagePos = secondPagePos;
-        private Future<ValuesAndContPosition<T>> pendingNextPage =
+    public Iterator<ResultT> iterator() {
+      return new AbstractIterator<ResultT>() {
+        private Iterator<ResultT> currentPage = firstPage.iterator();
+        private StateTag<ContinuationT> nextPagePos = secondPagePos;
+        private Future<ValuesAndContPosition<ResultT, ContinuationT>> pendingNextPage =
             // NOTE: The results of continuation page reads are never cached.
-            reader.continuationBagFuture(nextPagePos, elemCoder);
+            reader.continuationFuture(nextPagePos, elemCoder);
 
         @Override
-        protected T computeNext() {
+        protected ResultT computeNext() {
           while (true) {
             if (currentPage.hasNext()) {
               return currentPage.next();
@@ -699,7 +833,7 @@
               return endOfData();
             }
 
-            ValuesAndContPosition<T> valuesAndContPosition;
+            ValuesAndContPosition<ResultT, ContinuationT> valuesAndContPosition;
             try {
               valuesAndContPosition = pendingNextPage.get();
             } catch (InterruptedException | ExecutionException e) {
@@ -710,14 +844,14 @@
             }
             currentPage = valuesAndContPosition.values.iterator();
             nextPagePos =
-                new StateTag(
-                    nextPagePos.kind,
-                    nextPagePos.tag,
-                    nextPagePos.stateFamily,
+                StateTag.of(
+                    nextPagePos.getKind(),
+                    nextPagePos.getTag(),
+                    nextPagePos.getStateFamily(),
                     valuesAndContPosition.continuationPosition);
             pendingNextPage =
                 // NOTE: The results of continuation page reads are never cached.
-                reader.continuationBagFuture(nextPagePos, elemCoder);
+                reader.continuationFuture(nextPagePos, elemCoder);
           }
         }
       };
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillTimerInternals.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillTimerInternals.java
index f46fd49..73c6eb2 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillTimerInternals.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillTimerInternals.java
@@ -46,6 +46,7 @@
  *
  * <p>Includes parsing / assembly of timer tags and some extra methods.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class WindmillTimerInternals implements TimerInternals {
 
   private static final String TIMER_HOLD_PREFIX = "/h";
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindowingWindmillReader.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindowingWindmillReader.java
index 8cfe47e..98edff4 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindowingWindmillReader.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindowingWindmillReader.java
@@ -42,6 +42,7 @@
  * A Reader that receives input data from a Windmill server, and returns a singleton iterable
  * containing the work item.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class WindowingWindmillReader<K, T> extends NativeReader<WindowedValue<KeyedWorkItem<K, T>>> {
 
   private final Coder<K> keyCoder;
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WorkItemStatusClient.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WorkItemStatusClient.java
index 42446d1..cb04a5c 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WorkItemStatusClient.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WorkItemStatusClient.java
@@ -61,6 +61,7 @@
  */
 // Very likely real potential for bugs - https://issues.apache.org/jira/browse/BEAM-6565
 @SuppressFBWarnings("IS2_INCONSISTENT_SYNC")
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class WorkItemStatusClient {
 
   private static final Logger LOG = LoggerFactory.getLogger(WorkItemStatusClient.class);
@@ -188,7 +189,7 @@
     return false;
   }
 
-  private @Nullable synchronized WorkItemServiceState execute(WorkItemStatus status)
+  private synchronized @Nullable WorkItemServiceState execute(WorkItemStatus status)
       throws IOException {
     WorkItemServiceState result = workUnitClient.reportWorkItemStatus(status);
     if (result != null) {
@@ -360,4 +361,8 @@
 
     executionContext.commitMetricUpdates();
   }
+
+  public BatchModeExecutionContext getExecutionContext() {
+    return this.executionContext;
+  }
 }
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSourceOperationExecutor.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSourceOperationExecutor.java
index 4f87291..38307ef 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSourceOperationExecutor.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSourceOperationExecutor.java
@@ -31,6 +31,7 @@
 import org.slf4j.LoggerFactory;
 
 /** An executor for a source operation, defined by a {@code SourceOperationRequest}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class WorkerCustomSourceOperationExecutor implements SourceOperationExecutor {
   private static final Logger LOG =
       LoggerFactory.getLogger(WorkerCustomSourceOperationExecutor.class);
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSources.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSources.java
index c092a63..cfd1500 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSources.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSources.java
@@ -77,6 +77,7 @@
  * <p>Provides a bridge between the high-level {@code Source} API and the low-level {@code
  * CloudSource} class.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class WorkerCustomSources {
   private static final String SERIALIZED_SOURCE = "serialized_source";
   @VisibleForTesting static final String SERIALIZED_SOURCE_SPLITS = "serialized_source_splits";
@@ -632,9 +633,8 @@
       reader.close();
     }
 
-    @Nullable
     @VisibleForTesting
-    static ReportedParallelism longToParallelism(long value) {
+    static @Nullable ReportedParallelism longToParallelism(long value) {
       if (value >= 0) {
         return new ReportedParallelism().setValue(Double.valueOf(value));
       } else {
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WorkerPipelineOptionsFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WorkerPipelineOptionsFactory.java
index 46aa248..4ef8559 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WorkerPipelineOptionsFactory.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WorkerPipelineOptionsFactory.java
@@ -32,6 +32,7 @@
  * A factory used to create {@link DataflowWorkerHarnessOptions} used during the bootstrap process
  * to initialize a Dataflow worker harness.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class WorkerPipelineOptionsFactory {
   private static final Logger LOG = LoggerFactory.getLogger(WorkerPipelineOptionsFactory.class);
 
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/counters/CounterFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/counters/CounterFactory.java
index a5099da..7b62523 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/counters/CounterFactory.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/counters/CounterFactory.java
@@ -35,6 +35,7 @@
 import org.checkerframework.checker.nullness.qual.Nullable;
 
 /** Factory interface for creating counters. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class CounterFactory {
 
   protected <InputT, AccumT> Counter<InputT, AccumT> createCounter(
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/counters/CounterName.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/counters/CounterName.java
index 85d22f3..cc791e0 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/counters/CounterName.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/counters/CounterName.java
@@ -32,6 +32,7 @@
  * during the migration.
  */
 @AutoValue
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public abstract class CounterName {
 
   /** Returns a {@link CounterName} with the given name. */
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/counters/CounterSet.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/counters/CounterSet.java
index 793560d..0cef4d0 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/counters/CounterSet.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/counters/CounterSet.java
@@ -35,6 +35,7 @@
  *
  * <p>Thread-safe.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class CounterSet extends CounterFactory {
   private static final Logger LOG = LoggerFactory.getLogger(CounterSet.class);
 
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/counters/DataflowCounterUpdateExtractor.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/counters/DataflowCounterUpdateExtractor.java
index b4cc8b4..dccd96c 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/counters/DataflowCounterUpdateExtractor.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/counters/DataflowCounterUpdateExtractor.java
@@ -32,6 +32,7 @@
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
 
 /** Factory methods for extracting {@link CounterUpdate} updates from counters. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DataflowCounterUpdateExtractor implements CounterUpdateExtractor<CounterUpdate> {
 
   public static final DataflowCounterUpdateExtractor INSTANCE =
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/counters/DistributionCounterUpdateAggregator.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/counters/DistributionCounterUpdateAggregator.java
index b850864..d183fec 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/counters/DistributionCounterUpdateAggregator.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/counters/DistributionCounterUpdateAggregator.java
@@ -24,6 +24,7 @@
 import com.google.api.services.dataflow.model.DistributionUpdate;
 import java.util.List;
 
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DistributionCounterUpdateAggregator implements CounterUpdateAggregator {
 
   @Override
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/counters/MeanCounterUpdateAggregator.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/counters/MeanCounterUpdateAggregator.java
index 4cc2a46..9c22c42 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/counters/MeanCounterUpdateAggregator.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/counters/MeanCounterUpdateAggregator.java
@@ -24,6 +24,7 @@
 import com.google.api.services.dataflow.model.IntegerMean;
 import java.util.List;
 
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class MeanCounterUpdateAggregator implements CounterUpdateAggregator {
 
   @Override
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/counters/SumCounterUpdateAggregator.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/counters/SumCounterUpdateAggregator.java
index bff2489..219bd07 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/counters/SumCounterUpdateAggregator.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/counters/SumCounterUpdateAggregator.java
@@ -23,6 +23,7 @@
 import com.google.api.services.dataflow.model.CounterUpdate;
 import java.util.List;
 
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SumCounterUpdateAggregator implements CounterUpdateAggregator {
 
   @Override
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/fn/control/BeamFnMapTaskExecutor.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/fn/control/BeamFnMapTaskExecutor.java
index eb684b4..9fc11a1 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/fn/control/BeamFnMapTaskExecutor.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/fn/control/BeamFnMapTaskExecutor.java
@@ -73,6 +73,7 @@
  * <p>Note that this executor is meant to be used with the Fn API. Several of the methods to request
  * splitting, checkpointing, work progress are unimplemented.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BeamFnMapTaskExecutor extends DataflowMapTaskExecutor {
   private static final Logger LOG = LoggerFactory.getLogger(BeamFnMapTaskExecutor.class);
 
@@ -214,9 +215,8 @@
   }
 
   private static class NullProgressTracker implements ProgressTracker {
-    @Nullable
     @Override
-    public Progress getWorkerProgress() {
+    public @Nullable Progress getWorkerProgress() {
       return null;
     }
 
@@ -230,15 +230,13 @@
       return Collections.emptyList();
     }
 
-    @Nullable
     @Override
-    public DynamicSplitResult requestCheckpoint() {
+    public @Nullable DynamicSplitResult requestCheckpoint() {
       return null;
     }
 
-    @Nullable
     @Override
-    public DynamicSplitResult requestDynamicSplit(DynamicSplitRequest splitRequest) {
+    public @Nullable DynamicSplitResult requestDynamicSplit(DynamicSplitRequest splitRequest) {
       return null;
     }
   }
@@ -250,9 +248,8 @@
       this.readOperation = readOperation;
     }
 
-    @Nullable
     @Override
-    public Progress getWorkerProgress() throws Exception {
+    public @Nullable Progress getWorkerProgress() throws Exception {
       return readOperation.getProgress();
     }
 
@@ -266,15 +263,13 @@
       return Collections.emptyList();
     }
 
-    @Nullable
     @Override
-    public DynamicSplitResult requestCheckpoint() throws Exception {
+    public @Nullable DynamicSplitResult requestCheckpoint() throws Exception {
       return readOperation.requestCheckpoint();
     }
 
-    @Nullable
     @Override
-    public DynamicSplitResult requestDynamicSplit(DynamicSplitRequest splitRequest)
+    public @Nullable DynamicSplitResult requestDynamicSplit(DynamicSplitRequest splitRequest)
         throws Exception {
       return readOperation.requestDynamicSplit(splitRequest);
     }
@@ -410,9 +405,8 @@
               .collect(Collectors.toList());
     }
 
-    @Nullable
     @Override
-    public Progress getWorkerProgress() throws Exception {
+    public @Nullable Progress getWorkerProgress() throws Exception {
       return latestProgress.get();
     }
 
@@ -433,16 +427,14 @@
           snapshotGaugeUpdates.values());
     }
 
-    @Nullable
     @Override
-    public DynamicSplitResult requestCheckpoint() throws Exception {
+    public @Nullable DynamicSplitResult requestCheckpoint() throws Exception {
       // TODO: Implement checkpointing
       return null;
     }
 
-    @Nullable
     @Override
-    public DynamicSplitResult requestDynamicSplit(DynamicSplitRequest splitRequest)
+    public @Nullable DynamicSplitResult requestDynamicSplit(DynamicSplitRequest splitRequest)
         throws Exception {
       return readOperation.requestDynamicSplit(splitRequest);
     }
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/fn/control/DataflowSideInputHandlerFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/fn/control/DataflowSideInputHandlerFactory.java
index d0d79f3..a507e93 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/fn/control/DataflowSideInputHandlerFactory.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/fn/control/DataflowSideInputHandlerFactory.java
@@ -35,6 +35,7 @@
 import org.slf4j.LoggerFactory;
 
 /** This handles sideinput in Dataflow. The caller should be based on ExecutableStage framework. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DataflowSideInputHandlerFactory
     implements StateRequestHandlers.SideInputHandlerFactory {
   private static final Logger LOG = LoggerFactory.getLogger(DataflowSideInputHandlerFactory.class);
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/fn/control/ElementCountMonitoringInfoToCounterUpdateTransformer.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/fn/control/ElementCountMonitoringInfoToCounterUpdateTransformer.java
index c10cac5..328fa68 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/fn/control/ElementCountMonitoringInfoToCounterUpdateTransformer.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/fn/control/ElementCountMonitoringInfoToCounterUpdateTransformer.java
@@ -36,6 +36,7 @@
 import org.slf4j.LoggerFactory;
 
 /** MonitoringInfo to CounterUpdate transformer capable to transform ElementCount counters. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ElementCountMonitoringInfoToCounterUpdateTransformer
     implements MonitoringInfoToCounterUpdateTransformer {
 
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/fn/control/ExecutionTimeMonitoringInfoToCounterUpdateTransformer.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/fn/control/ExecutionTimeMonitoringInfoToCounterUpdateTransformer.java
index 05cb9f7..033877e 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/fn/control/ExecutionTimeMonitoringInfoToCounterUpdateTransformer.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/fn/control/ExecutionTimeMonitoringInfoToCounterUpdateTransformer.java
@@ -43,6 +43,7 @@
  *
  * <p>Use getSupportedUrns to get all urns this class supports.
  */
+@SuppressWarnings({"keyfor", "nullness"}) // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ExecutionTimeMonitoringInfoToCounterUpdateTransformer
     implements MonitoringInfoToCounterUpdateTransformer {
 
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/fn/control/MeanByteCountMonitoringInfoToCounterUpdateTransformer.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/fn/control/MeanByteCountMonitoringInfoToCounterUpdateTransformer.java
index ecd2482..945d3e9 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/fn/control/MeanByteCountMonitoringInfoToCounterUpdateTransformer.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/fn/control/MeanByteCountMonitoringInfoToCounterUpdateTransformer.java
@@ -38,6 +38,7 @@
 import org.slf4j.LoggerFactory;
 
 /** MonitoringInfo to CounterUpdate transformer capable to transform MeanByteCount counter. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class MeanByteCountMonitoringInfoToCounterUpdateTransformer
     implements MonitoringInfoToCounterUpdateTransformer {
 
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/fn/control/ProcessRemoteBundleOperation.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/fn/control/ProcessRemoteBundleOperation.java
index 089af0e..842b9b2 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/fn/control/ProcessRemoteBundleOperation.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/fn/control/ProcessRemoteBundleOperation.java
@@ -48,6 +48,7 @@
  * requests a {@link org.apache.beam.runners.fnexecution.control.RemoteBundle}, sends elements to
  * SDK and receive processed results from SDK, passing these elements downstream.
  */
+@SuppressWarnings({"keyfor", "nullness"}) // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ProcessRemoteBundleOperation<InputT> extends ReceivingOperation {
   private static final Logger LOG = LoggerFactory.getLogger(ProcessRemoteBundleOperation.class);
   private final StageBundleFactory stageBundleFactory;
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/fn/control/RegisterAndProcessBundleOperation.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/fn/control/RegisterAndProcessBundleOperation.java
index d0ac83e..993ff07 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/fn/control/RegisterAndProcessBundleOperation.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/fn/control/RegisterAndProcessBundleOperation.java
@@ -93,6 +93,7 @@
  *
  * <p>This operation supports restart.
  */
+@SuppressWarnings({"keyfor", "nullness"}) // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class RegisterAndProcessBundleOperation extends Operation {
   private static final Logger LOG =
       LoggerFactory.getLogger(RegisterAndProcessBundleOperation.class);
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/fn/control/UserDistributionMonitoringInfoToCounterUpdateTransformer.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/fn/control/UserDistributionMonitoringInfoToCounterUpdateTransformer.java
index 652a771..d792345 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/fn/control/UserDistributionMonitoringInfoToCounterUpdateTransformer.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/fn/control/UserDistributionMonitoringInfoToCounterUpdateTransformer.java
@@ -44,6 +44,7 @@
  * Class for transforming MonitoringInfo's containing User counter values, to relevant CounterUpdate
  * proto.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class UserDistributionMonitoringInfoToCounterUpdateTransformer
     implements MonitoringInfoToCounterUpdateTransformer {
 
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/fn/control/UserMonitoringInfoToCounterUpdateTransformer.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/fn/control/UserMonitoringInfoToCounterUpdateTransformer.java
index a02a1df..c12dad0 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/fn/control/UserMonitoringInfoToCounterUpdateTransformer.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/fn/control/UserMonitoringInfoToCounterUpdateTransformer.java
@@ -42,6 +42,7 @@
  * Class for transforming MonitoringInfo's containing User counter values, to relevant CounterUpdate
  * proto.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class UserMonitoringInfoToCounterUpdateTransformer
     implements MonitoringInfoToCounterUpdateTransformer {
 
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/fn/data/RemoteGrpcPortReadOperation.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/fn/data/RemoteGrpcPortReadOperation.java
index bc7145c..df02484 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/fn/data/RemoteGrpcPortReadOperation.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/fn/data/RemoteGrpcPortReadOperation.java
@@ -40,6 +40,7 @@
  *
  * <p>This {@link Operation} supports restart.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class RemoteGrpcPortReadOperation<T> extends Operation {
   private static final Logger LOG = LoggerFactory.getLogger(RemoteGrpcPortReadOperation.class);
   private final Coder<WindowedValue<T>> coder;
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/fn/data/RemoteGrpcPortWriteOperation.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/fn/data/RemoteGrpcPortWriteOperation.java
index 301fab2..01ba6a6 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/fn/data/RemoteGrpcPortWriteOperation.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/fn/data/RemoteGrpcPortWriteOperation.java
@@ -44,6 +44,7 @@
  *
  * <p>This {@link Operation} supports restart.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class RemoteGrpcPortWriteOperation<T> extends ReceivingOperation {
   private static final Logger LOG = LoggerFactory.getLogger(RemoteGrpcPortWriteOperation.class);
 
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/fn/logging/BeamFnLoggingService.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/fn/logging/BeamFnLoggingService.java
index e705dec..e9c4889 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/fn/logging/BeamFnLoggingService.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/fn/logging/BeamFnLoggingService.java
@@ -41,6 +41,7 @@
  * them to the provided {@link org.apache.beam.model.fnexecution.v1.BeamFnApi.LogEntry} {@link
  * Consumer}.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BeamFnLoggingService extends BeamFnLoggingGrpc.BeamFnLoggingImplBase
     implements BeamFnService {
   private static final Logger LOG = LoggerFactory.getLogger(BeamFnLoggingService.class);
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/fn/stream/ServerStreamObserverFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/fn/stream/ServerStreamObserverFactory.java
index e51c0ee..768b70e 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/fn/stream/ServerStreamObserverFactory.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/fn/stream/ServerStreamObserverFactory.java
@@ -40,6 +40,7 @@
  *
  * <p>The specific implementation returned is dependent on {@link PipelineOptions} experiments.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public abstract class ServerStreamObserverFactory {
   public static ServerStreamObserverFactory fromOptions(PipelineOptions options) {
     DataflowPipelineDebugOptions dataflowOptions = options.as(DataflowPipelineDebugOptions.class);
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/graph/CreateExecutableStageNodeFunction.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/graph/CreateExecutableStageNodeFunction.java
index 7b8cbe4..38491db 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/graph/CreateExecutableStageNodeFunction.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/graph/CreateExecutableStageNodeFunction.java
@@ -98,6 +98,7 @@
  * {@link Node} containing an {@link
  * org.apache.beam.runners.core.construction.graph.ExecutableStage}.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class CreateExecutableStageNodeFunction
     implements Function<MutableNetwork<Node, Edge>, Node> {
   private static final String DATA_INPUT_URN = "beam:runner:source:v1";
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/graph/DeduceFlattenLocationsFunction.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/graph/DeduceFlattenLocationsFunction.java
index 84173a1..0a42ff8 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/graph/DeduceFlattenLocationsFunction.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/graph/DeduceFlattenLocationsFunction.java
@@ -58,6 +58,7 @@
  * <p>The ambiguous result means that executing the flatten in either the SDK or Runner is equally
  * inefficient, and thus it can execute in either one.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DeduceFlattenLocationsFunction
     implements Function<MutableNetwork<Node, Edge>, MutableNetwork<Node, Edge>> {
 
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/graph/Edges.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/graph/Edges.java
index e88aa33..7c982d7 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/graph/Edges.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/graph/Edges.java
@@ -23,6 +23,7 @@
 import com.google.auto.value.AutoValue;
 
 /** Container class for different types of graph edges. All edges only have reference equality. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class Edges {
   /** Base class for graph edges. All edges only have reference equality. */
   public abstract static class Edge implements Cloneable {
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/graph/InsertFetchAndFilterStreamingSideInputNodes.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/graph/InsertFetchAndFilterStreamingSideInputNodes.java
index 6fdd0d6..04a772f 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/graph/InsertFetchAndFilterStreamingSideInputNodes.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/graph/InsertFetchAndFilterStreamingSideInputNodes.java
@@ -48,6 +48,7 @@
  * Inserts a {@link ParDoFn} that handles filtering blocked side inputs and fetching ready side
  * inputs for streaming pipelines before user {@link ParDo ParDos} containing side inputs.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class InsertFetchAndFilterStreamingSideInputNodes {
   public static InsertFetchAndFilterStreamingSideInputNodes with(
       RunnerApi.@Nullable Pipeline pipeline) {
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/graph/MapTaskToNetworkFunction.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/graph/MapTaskToNetworkFunction.java
index fd98bac..cae1162 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/graph/MapTaskToNetworkFunction.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/graph/MapTaskToNetworkFunction.java
@@ -55,6 +55,7 @@
  * <p>The outgoing edges of a {@link ParallelInstructionNode} with a {@link ParDoInstruction} are
  * {@link MultiOutputInfoEdge}s. All other edges are {@link DefaultEdge}s.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class MapTaskToNetworkFunction implements Function<MapTask, MutableNetwork<Node, Edge>> {
   private static ParallelInstruction clone(JsonFactory factory, ParallelInstruction instruction) {
     try {
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/graph/Networks.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/graph/Networks.java
index 03d365b..c240139 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/graph/Networks.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/graph/Networks.java
@@ -40,6 +40,7 @@
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.graph.Network;
 
 /** Static utility methods for {@link Network} instances that are directed. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class Networks {
   /**
    * An abstract class that can be extended to apply a function in a type safe manner.
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/graph/Nodes.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/graph/Nodes.java
index 6d80563..e7297ac 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/graph/Nodes.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/graph/Nodes.java
@@ -45,6 +45,7 @@
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.MoreObjects;
 
 /** Container class for different types of network nodes. All nodes only have reference equality. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class Nodes {
   /** Base class for network nodes. All nodes only have reference equality. */
   public abstract static class Node {
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/graph/RegisterNodeFunction.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/graph/RegisterNodeFunction.java
index 1f9574c..82140df 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/graph/RegisterNodeFunction.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/graph/RegisterNodeFunction.java
@@ -92,6 +92,7 @@
  *
  * <p>Testing of all the layers of translation are performed via local service runner tests.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class RegisterNodeFunction implements Function<MutableNetwork<Node, Edge>, Node> {
   /** Must match declared fields within {@code ProcessBundleHandler}. */
   private static final String DATA_INPUT_URN = "beam:runner:source:v1";
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/graph/RemoveFlattenInstructionsFunction.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/graph/RemoveFlattenInstructionsFunction.java
index 7b51fef..738b685 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/graph/RemoveFlattenInstructionsFunction.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/graph/RemoveFlattenInstructionsFunction.java
@@ -33,6 +33,7 @@
  * MapTask}. Remove a Flatten instruction and its PCollection by directly connecting the predecessor
  * PCollections of the Flatten with the successor instructions.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class RemoveFlattenInstructionsFunction
     implements Function<MutableNetwork<Node, Edge>, MutableNetwork<Node, Edge>> {
 
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/graph/ReplacePgbkWithPrecombineFunction.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/graph/ReplacePgbkWithPrecombineFunction.java
index 8f506ee..61623bf 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/graph/ReplacePgbkWithPrecombineFunction.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/graph/ReplacePgbkWithPrecombineFunction.java
@@ -36,6 +36,7 @@
  * A function that replaces PartialGroupByKey nodes with ParDo nodes that can be translated by the
  * {@link RegisterNodeFunction} into a Pre-combine executed by the SDK harness.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ReplacePgbkWithPrecombineFunction
     implements Function<MutableNetwork<Node, Edge>, MutableNetwork<Node, Edge>> {
 
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/logging/DataflowWorkerLoggingHandler.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/logging/DataflowWorkerLoggingHandler.java
index 1af00f1..b9bd677 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/logging/DataflowWorkerLoggingHandler.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/logging/DataflowWorkerLoggingHandler.java
@@ -52,6 +52,7 @@
  * Formats {@link LogRecord} into JSON format for Cloud Logging. Any exception is represented using
  * {@link Throwable#printStackTrace()}.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DataflowWorkerLoggingHandler extends Handler {
   private static final EnumMap<BeamFnApi.LogEntry.Severity.Enum, String>
       BEAM_LOG_LEVEL_TO_CLOUD_LOG_LEVEL;
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/logging/DataflowWorkerLoggingInitializer.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/logging/DataflowWorkerLoggingInitializer.java
index ebef8ea..e386309 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/logging/DataflowWorkerLoggingInitializer.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/logging/DataflowWorkerLoggingInitializer.java
@@ -48,6 +48,7 @@
  * level is INFO, the default location is a file named dataflow-json.log within the system temporary
  * directory and the default file size is 1 GB.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DataflowWorkerLoggingInitializer {
   private static final String ROOT_LOGGER_NAME = "";
 
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/logging/DataflowWorkerLoggingMDC.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/logging/DataflowWorkerLoggingMDC.java
index c489d10..c104ed1 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/logging/DataflowWorkerLoggingMDC.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/logging/DataflowWorkerLoggingMDC.java
@@ -18,6 +18,7 @@
 package org.apache.beam.runners.dataflow.worker.logging;
 
 /** Mapped diagnostic context for the Dataflow worker. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DataflowWorkerLoggingMDC {
   private static final InheritableThreadLocal<String> jobId = new InheritableThreadLocal<>();
   private static final InheritableThreadLocal<String> stageName = new InheritableThreadLocal<>();
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/logging/JulHandlerPrintStreamAdapterFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/logging/JulHandlerPrintStreamAdapterFactory.java
index 79ef587..34a0ae9 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/logging/JulHandlerPrintStreamAdapterFactory.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/logging/JulHandlerPrintStreamAdapterFactory.java
@@ -42,6 +42,7 @@
  * A {@link PrintStream} factory that creates {@link PrintStream}s which output to the specified JUL
  * {@link Handler} at the specified {@link Level}.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class JulHandlerPrintStreamAdapterFactory {
   private static final AtomicBoolean outputWarning = new AtomicBoolean(false);
 
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/options/StreamingDataflowWorkerOptions.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/options/StreamingDataflowWorkerOptions.java
index e638d99..af3df03 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/options/StreamingDataflowWorkerOptions.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/options/StreamingDataflowWorkerOptions.java
@@ -32,6 +32,7 @@
 /** [Internal] Options for configuring StreamingDataflowWorker. */
 @Description("[Internal] Options for configuring StreamingDataflowWorker.")
 @Hidden
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public interface StreamingDataflowWorkerOptions extends DataflowWorkerHarnessOptions {
   @Description("Stub for communicating with Windmill.")
   @Default.InstanceFactory(WindmillServerStubFactory.class)
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/profiler/ScopedProfiler.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/profiler/ScopedProfiler.java
index 784f6aa..0c3545c 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/profiler/ScopedProfiler.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/profiler/ScopedProfiler.java
@@ -22,6 +22,7 @@
 import org.slf4j.LoggerFactory;
 
 /** A wrapper around {@link Profiler} to support more idiomatic usage within Java. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ScopedProfiler {
 
   /** A thin wrapper around {@link Profiler} to allow mocking in tests. */
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/status/DebugCapture.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/status/DebugCapture.java
index fccbd07..8456a0a 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/status/DebugCapture.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/status/DebugCapture.java
@@ -45,6 +45,7 @@
  * DebugCapture encapsulates a simple periodic sender for HTML pages to the debug capture service.
  * It is dynamically configured by the service.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DebugCapture {
   private static final Logger LOG = LoggerFactory.getLogger(DebugCapture.class);
   private static final JsonFactory JSON_FACTORY = new JacksonFactory();
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/status/WorkerStatusPages.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/status/WorkerStatusPages.java
index 764a607..7634b84 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/status/WorkerStatusPages.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/status/WorkerStatusPages.java
@@ -35,6 +35,7 @@
 import org.slf4j.LoggerFactory;
 
 /** Manages the server providing the worker status pages. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class WorkerStatusPages {
 
   private static final Logger LOG = LoggerFactory.getLogger(WorkerStatusPages.class);
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/BoundedQueueExecutor.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/BoundedQueueExecutor.java
index 841eeeb..f4723ae 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/BoundedQueueExecutor.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/BoundedQueueExecutor.java
@@ -24,6 +24,7 @@
 import java.util.concurrent.TimeUnit;
 
 /** Executor that blocks on execute() if its queue is full. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BoundedQueueExecutor extends ThreadPoolExecutor {
   private static class ReducableSemaphore extends Semaphore {
     ReducableSemaphore(int permits) {
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/CloudSourceUtils.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/CloudSourceUtils.java
index 79e90a90..9f3dba3 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/CloudSourceUtils.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/CloudSourceUtils.java
@@ -23,6 +23,7 @@
 import org.apache.beam.runners.dataflow.worker.util.common.worker.NativeReader;
 
 /** Utilities for working with Source Dataflow API definitions and {@link NativeReader} objects. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class CloudSourceUtils {
   /**
    * Returns a copy of the source with {@code baseSpecs} flattened into {@code spec}. On conflict
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/MemoryMonitor.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/MemoryMonitor.java
index 63237bf..c58068b 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/MemoryMonitor.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/MemoryMonitor.java
@@ -77,6 +77,7 @@
  *       count is met. A heap dump is made before shutdown.
  * </ul>
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class MemoryMonitor implements Runnable, StatusDataProvider {
   private static final Logger LOG = LoggerFactory.getLogger(MemoryMonitor.class);
 
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/TimerOrElement.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/TimerOrElement.java
index 6065b68..c661438 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/TimerOrElement.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/TimerOrElement.java
@@ -41,6 +41,7 @@
  * com.google.cloud.dataflow.sdk.util.TimerOrElement$TimerOrElementCoder} and we'd like to be able
  * to rename/move that without breaking things.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class TimerOrElement {
 
   // TimerOrElement should never be created.
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/TaggedReiteratorList.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/TaggedReiteratorList.java
index 94b0dc2..9b81e67 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/TaggedReiteratorList.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/TaggedReiteratorList.java
@@ -30,6 +30,7 @@
  *
  * <p>This class, and the returned iterators, are not threadsafe.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class TaggedReiteratorList extends AbstractList<Reiterator<Object>> {
 
   /** Interface for extracting the tag and value from an opaque element. */
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/BatchingShuffleEntryReader.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/BatchingShuffleEntryReader.java
index 2abfffe..6cc0693 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/BatchingShuffleEntryReader.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/BatchingShuffleEntryReader.java
@@ -28,6 +28,7 @@
 
 /** BatchingShuffleEntryReader provides a mechanism for reading entries from a shuffle dataset. */
 @NotThreadSafe
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public final class BatchingShuffleEntryReader implements ShuffleEntryReader {
   private final ShuffleBatchReader batchReader;
 
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ByteArrayShufflePosition.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ByteArrayShufflePosition.java
index 8201db2..16db915 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ByteArrayShufflePosition.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ByteArrayShufflePosition.java
@@ -31,6 +31,7 @@
  * a way such that lexicographic ordering of the bytes is consistent with the inherent ordering of
  * {@code GroupingShuffleReader} positions.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ByteArrayShufflePosition implements Comparable<ShufflePosition>, ShufflePosition {
   private final byte[] position;
 
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/CachingShuffleBatchReader.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/CachingShuffleBatchReader.java
index 6301f4d..b54be7f 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/CachingShuffleBatchReader.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/CachingShuffleBatchReader.java
@@ -29,6 +29,7 @@
 import org.checkerframework.checker.nullness.qual.Nullable;
 
 /** A {@link ShuffleBatchReader} that caches batches as they're read. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class CachingShuffleBatchReader implements ShuffleBatchReader {
   private final ShuffleBatchReader reader;
   @VisibleForTesting final LoadingCache<BatchRange, Batch> cache;
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/GroupingShuffleEntryIterator.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/GroupingShuffleEntryIterator.java
index 5737868..0de20bd 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/GroupingShuffleEntryIterator.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/GroupingShuffleEntryIterator.java
@@ -36,6 +36,7 @@
  * <p>Much like a {@link NativeReader.NativeReaderIterator}, but without {@code start()}, and not
  * used via the interface of that class, hence doesn't inherit it.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public abstract class GroupingShuffleEntryIterator {
   private static final Logger LOG = LoggerFactory.getLogger(GroupingShuffleEntryIterator.class);
 
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/GroupingShuffleRangeTracker.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/GroupingShuffleRangeTracker.java
index 9d6e6df..1622dd1 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/GroupingShuffleRangeTracker.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/GroupingShuffleRangeTracker.java
@@ -37,6 +37,7 @@
  */
 // Likely real bugs - https://issues.apache.org/jira/browse/BEAM-6563
 @SuppressFBWarnings("IS2_INCONSISTENT_SYNC")
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class GroupingShuffleRangeTracker implements RangeTracker<ShufflePosition> {
   private static final Logger LOG = LoggerFactory.getLogger(GroupingShuffleRangeTracker.class);
 
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/GroupingTables.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/GroupingTables.java
index ddccbf7..31e038c 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/GroupingTables.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/GroupingTables.java
@@ -26,6 +26,7 @@
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
 
 /** Static utility methods that provide {@link GroupingTable} implementations. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class GroupingTables {
   /** Returns a {@link GroupingTable} that groups inputs into a {@link List}. */
   public static <K, V> GroupingTable<K, V, List<V>> buffering(
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/MapTaskExecutor.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/MapTaskExecutor.java
index a23b6be..343b55a 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/MapTaskExecutor.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/MapTaskExecutor.java
@@ -29,6 +29,7 @@
 import org.slf4j.LoggerFactory;
 
 /** An executor for a map task, defined by a list of Operations. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class MapTaskExecutor implements WorkExecutor {
   private static final Logger LOG = LoggerFactory.getLogger(MapTaskExecutor.class);
 
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/OutputObjectAndByteCounter.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/OutputObjectAndByteCounter.java
index fa66871..1678b87 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/OutputObjectAndByteCounter.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/OutputObjectAndByteCounter.java
@@ -28,6 +28,7 @@
 import org.checkerframework.checker.nullness.qual.Nullable;
 
 /** An {@link ElementCounter} that counts output objects, bytes, and mean bytes. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class OutputObjectAndByteCounter implements ElementCounter {
   // Might be null, e.g., undeclared outputs will not have an
   // elementByteSizeObservable.
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ReadOperation.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ReadOperation.java
index e1b87da..3d92e22 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ReadOperation.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ReadOperation.java
@@ -42,6 +42,7 @@
  *
  * <p>Its start() method iterates through all elements of the source and emits them on its output.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ReadOperation extends Operation {
   private static final Logger LOG = LoggerFactory.getLogger(ReadOperation.class);
 
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ShuffleEntry.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ShuffleEntry.java
index 76312b1..a718292 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ShuffleEntry.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ShuffleEntry.java
@@ -21,6 +21,7 @@
 import org.checkerframework.checker.nullness.qual.Nullable;
 
 /** Entry written to/read from a shuffle dataset. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ShuffleEntry {
   final ShufflePosition position;
   final byte[] key;
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ShuffleReadCounter.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ShuffleReadCounter.java
index 306e7b8..e60d8d8 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ShuffleReadCounter.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ShuffleReadCounter.java
@@ -26,6 +26,7 @@
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
 
 /** Counts the Bytes and MSECS spent within a shuffle read. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ShuffleReadCounter {
   private final String originalShuffleStepName;
   private final boolean experimentEnabled;
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/WorkExecutor.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/WorkExecutor.java
index b64434b..b7ade53 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/WorkExecutor.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/WorkExecutor.java
@@ -23,6 +23,7 @@
 import org.checkerframework.checker.nullness.qual.Nullable;
 
 /** Abstract executor for WorkItem tasks. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public interface WorkExecutor extends AutoCloseable {
 
   /** Returns the set of output counters for this task. */
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/WorkProgressUpdater.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/WorkProgressUpdater.java
index c52c1c9..05e933e 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/WorkProgressUpdater.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/WorkProgressUpdater.java
@@ -42,6 +42,7 @@
 // Very likely real potential for bugs - https://issues.apache.org/jira/browse/BEAM-6561
 @SuppressFBWarnings("JLM_JSR166_UTILCONCURRENT_MONITORENTER")
 @NotThreadSafe
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public abstract class WorkProgressUpdater {
   private static final Logger LOG = LoggerFactory.getLogger(WorkProgressUpdater.class);
 
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/WriteOperation.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/WriteOperation.java
index a22b243..a2fda53 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/WriteOperation.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/common/worker/WriteOperation.java
@@ -23,6 +23,7 @@
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.MoreObjects;
 
 /** A write operation. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class WriteOperation extends ReceivingOperation {
   /** The Sink this operation writes to. */
   public final Sink<?> sink;
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/GrpcWindmillServer.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/GrpcWindmillServer.java
index c042312..2ae23ed 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/GrpcWindmillServer.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/GrpcWindmillServer.java
@@ -112,6 +112,7 @@
 // Very likely real potential for bugs - https://issues.apache.org/jira/browse/BEAM-6562
 // Very likely real potential for bugs - https://issues.apache.org/jira/browse/BEAM-6564
 @SuppressFBWarnings({"JLM_JSR166_UTILCONCURRENT_MONITORENTER", "IS2_INCONSISTENT_SYNC"})
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class GrpcWindmillServer extends WindmillServerStub {
   private static final Logger LOG = LoggerFactory.getLogger(GrpcWindmillServer.class);
 
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/WindmillServer.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/WindmillServer.java
index 6ac409b..dceefc2 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/WindmillServer.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/WindmillServer.java
@@ -24,6 +24,7 @@
 import java.nio.file.StandardCopyOption;
 
 /** Implementation of a WindmillServerBase. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class WindmillServer extends WindmillServerBase {
   private static final String WINDMILL_SERVER_JNI_LIBRARY_PROPERTY = "windmill.jni_library";
   private static final String DEFAULT_SHUFFLE_CLIENT_LIBRARY = "libwindmill_service_jni.so";
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/WindmillServerBase.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/WindmillServerBase.java
index 86b673e..370c37f 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/WindmillServerBase.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/WindmillServerBase.java
@@ -25,6 +25,7 @@
  * Implementation of a WindmillServerStub which communcates with an actual windmill server at the
  * specified location.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class WindmillServerBase extends WindmillServerStub {
 
   /** Pointer to the underlying native windmill client object. */
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/WindmillServerStub.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/WindmillServerStub.java
index e07de54..2a73bfa 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/WindmillServerStub.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/WindmillServerStub.java
@@ -39,6 +39,7 @@
 import org.joda.time.Instant;
 
 /** Stub for communicating with a Windmill server. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public abstract class WindmillServerStub implements StatusDataProvider {
 
   /**
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/harness/test/TestExecutors.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/harness/test/TestExecutors.java
index 57c53a3..e2f9322 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/harness/test/TestExecutors.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/harness/test/TestExecutors.java
@@ -29,6 +29,7 @@
  * A {@link TestRule} that validates that all submitted tasks finished and were completed. This
  * allows for testing that tasks have exercised the appropriate shutdown logic.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class TestExecutors {
   public static TestExecutorService from(Supplier<ExecutorService> executorServiceSuppler) {
     return new FromSupplier(executorServiceSuppler);
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/harness/test/TestExecutorsTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/harness/test/TestExecutorsTest.java
index cb98335..b851715 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/harness/test/TestExecutorsTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/harness/test/TestExecutorsTest.java
@@ -33,7 +33,10 @@
 
 /** Tests for {@link TestExecutors}. */
 @RunWith(JUnit4.class)
-@SuppressWarnings("FutureReturnValueIgnored")
+@SuppressWarnings({
+  "FutureReturnValueIgnored",
+  "nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
+})
 public class TestExecutorsTest {
   @Test
   public void testSuccessfulTermination() throws Throwable {
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/harness/test/TestStreamsTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/harness/test/TestStreamsTest.java
index 84e1b34..f634284 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/harness/test/TestStreamsTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/harness/test/TestStreamsTest.java
@@ -31,6 +31,7 @@
 
 /** Tests for {@link TestStreams}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class TestStreamsTest {
   @Test
   public void testOnNextIsCalled() {
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ApplianceShuffleCountersTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ApplianceShuffleCountersTest.java
index 41da44a..921536f 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ApplianceShuffleCountersTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ApplianceShuffleCountersTest.java
@@ -36,6 +36,7 @@
 
 /** Tests for ApplianceShuffleCounters. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ApplianceShuffleCountersTest {
   private static final String DATASET_ID = "dataset";
   private CounterSet counterSet = new CounterSet();
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/AvroByteReaderFactoryTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/AvroByteReaderFactoryTest.java
index 9cc1258..20ab2bc 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/AvroByteReaderFactoryTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/AvroByteReaderFactoryTest.java
@@ -38,7 +38,10 @@
 
 /** Tests for {@link AvroByteReaderFactory}. */
 @RunWith(JUnit4.class)
-@SuppressWarnings("rawtypes")
+@SuppressWarnings({
+  "rawtypes",
+  "nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
+})
 public class AvroByteReaderFactoryTest {
   private final String pathToAvroFile = "/path/to/file.avro";
 
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/AvroByteReaderTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/AvroByteReaderTest.java
index bb17d8b..1317505 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/AvroByteReaderTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/AvroByteReaderTest.java
@@ -55,6 +55,7 @@
 
 /** Tests for AvroByteReader. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class AvroByteReaderTest {
   @Rule public TemporaryFolder tmpFolder = new TemporaryFolder();
 
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/BatchDataflowWorkerTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/BatchDataflowWorkerTest.java
index 8bbe47e..ef477b4 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/BatchDataflowWorkerTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/BatchDataflowWorkerTest.java
@@ -59,6 +59,7 @@
 
 /** Unit tests for {@link BatchDataflowWorker}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BatchDataflowWorkerTest {
 
   private static class WorkerException extends Exception {}
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/BatchModeExecutionContextTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/BatchModeExecutionContextTest.java
index 8bf7009..22779b6 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/BatchModeExecutionContextTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/BatchModeExecutionContextTest.java
@@ -51,6 +51,7 @@
 
 /** Tests for {@link BatchModeExecutionContext}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BatchModeExecutionContextTest {
 
   @Test
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ConcatReaderFactoryTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ConcatReaderFactoryTest.java
index 772500f..06b46d8 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ConcatReaderFactoryTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ConcatReaderFactoryTest.java
@@ -42,6 +42,7 @@
 
 /** Test for {@code ConcatReaderFactory}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ConcatReaderFactoryTest {
 
   Source createSourcesWithInMemorySources(List<List<String>> allData) {
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ConcatReaderTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ConcatReaderTest.java
index 0bfbbf7..86b0862 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ConcatReaderTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ConcatReaderTest.java
@@ -51,6 +51,7 @@
 
 /** Tests for {@code ConcatReader}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ConcatReaderTest {
   private static final String READER_OBJECT = "reader_object";
 
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/CreateIsmShardKeyAndSortKeyDoFnFactoryTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/CreateIsmShardKeyAndSortKeyDoFnFactoryTest.java
index 85873ee..a4ca114 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/CreateIsmShardKeyAndSortKeyDoFnFactoryTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/CreateIsmShardKeyAndSortKeyDoFnFactoryTest.java
@@ -39,6 +39,7 @@
 
 /** Tests for {@link CreateIsmShardKeyAndSortKeyDoFnFactory}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class CreateIsmShardKeyAndSortKeyDoFnFactoryTest {
   @Test
   public void testConversionOfRecord() throws Exception {
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DataflowBatchWorkerHarnessTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DataflowBatchWorkerHarnessTest.java
index 3451743..3f4c2ee 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DataflowBatchWorkerHarnessTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DataflowBatchWorkerHarnessTest.java
@@ -46,6 +46,7 @@
 
 /** Unit tests for {@link DataflowBatchWorkerHarness}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DataflowBatchWorkerHarnessTest {
   @Rule public TestRule restoreSystemProperties = new RestoreSystemProperties();
   @Rule public TestRule restoreLogging = new RestoreDataflowLoggingMDC();
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DataflowElementExecutionTrackerTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DataflowElementExecutionTrackerTest.java
index dc18bf6..7aceaf0 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DataflowElementExecutionTrackerTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DataflowElementExecutionTrackerTest.java
@@ -43,6 +43,7 @@
 
 /** Tests for {@link DataflowElementExecutionTracker}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DataflowElementExecutionTrackerTest {
   @Rule public final ExpectedException thrown = ExpectedException.none();
 
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DataflowExecutionStateTrackerTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DataflowExecutionStateTrackerTest.java
index 30ccedf..8719c9b 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DataflowExecutionStateTrackerTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DataflowExecutionStateTrackerTest.java
@@ -44,6 +44,7 @@
 import org.junit.Test;
 
 /** Tests for {@link DataflowExecutionStateTrackerTest}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DataflowExecutionStateTrackerTest {
 
   private PipelineOptions options;
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DataflowOperationContextTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DataflowOperationContextTest.java
index 2c2fa1a..44fee35 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DataflowOperationContextTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DataflowOperationContextTest.java
@@ -68,6 +68,7 @@
 
 /** Tests for {@link DataflowOperationContext}. */
 @RunWith(Enclosed.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DataflowOperationContextTest {
 
   /**
@@ -219,9 +220,8 @@
               null /* metricsContainer */,
               ScopedProfiler.INSTANCE.emptyScope(),
               clock) {
-            @Nullable
             @Override
-            public CounterUpdate extractUpdate(boolean isFinalUpdate) {
+            public @Nullable CounterUpdate extractUpdate(boolean isFinalUpdate) {
               // not being used for extracting updates
               return null;
             }
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DataflowWorkProgressUpdaterTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DataflowWorkProgressUpdaterTest.java
index f970535..005c1ca 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DataflowWorkProgressUpdaterTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DataflowWorkProgressUpdaterTest.java
@@ -37,11 +37,13 @@
 import com.google.api.services.dataflow.model.Status;
 import com.google.api.services.dataflow.model.WorkItem;
 import com.google.api.services.dataflow.model.WorkItemServiceState;
+import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions;
 import org.apache.beam.runners.dataflow.worker.util.common.worker.NativeReader;
 import org.apache.beam.runners.dataflow.worker.util.common.worker.NativeReader.DynamicSplitRequest;
 import org.apache.beam.runners.dataflow.worker.util.common.worker.NativeReader.DynamicSplitResult;
 import org.apache.beam.runners.dataflow.worker.util.common.worker.StubbedExecutor;
 import org.apache.beam.sdk.extensions.gcp.util.Transport;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
 import org.checkerframework.checker.nullness.qual.Nullable;
 import org.joda.time.Duration;
 import org.joda.time.Instant;
@@ -59,6 +61,7 @@
 
 /** Unit tests for {@link DataflowWorkProgressUpdater}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DataflowWorkProgressUpdaterTest {
 
   private static final long LEASE_MS = 2000;
@@ -68,6 +71,7 @@
   private static final Long WORK_ID = 1234567890L;
   private static final String STEP_ID = "TEST_STEP_ID";
   private static final Duration HOT_KEY_AGE = Duration.standardSeconds(1);
+  private static final String HOT_KEY = "key";
 
   @Rule public final ExpectedException thrown = ExpectedException.none();
 
@@ -76,8 +80,10 @@
   private DataflowWorkProgressUpdater progressUpdater;
   private long startTime;
   private FixedClock clock;
+  private DataflowPipelineOptions options;
   @Mock private WorkItemStatusClient workItemStatusClient;
   @Mock private DataflowWorkExecutor worker;
+  @Mock private BatchModeExecutionContext context;
   @Mock private HotKeyLogger hotKeyLogger;
   @Captor private ArgumentCaptor<DynamicSplitResult> splitResultCaptor;
 
@@ -87,6 +93,8 @@
     startTime = 0L;
     clock = new FixedClock(startTime);
     executor = new StubbedExecutor(clock);
+    options = PipelineOptionsFactory.create().as(DataflowPipelineOptions.class);
+    options.setHotKeyLoggingEnabled(true);
 
     WorkItem workItem = new WorkItem();
     workItem.setProjectId(PROJECT_ID);
@@ -96,9 +104,18 @@
     workItem.setReportStatusInterval(toCloudDuration(Duration.millis(300)));
     workItem.setInitialReportIndex(1L);
 
+    when(workItemStatusClient.getExecutionContext()).thenReturn(context);
+    when(context.getKey()).thenReturn(HOT_KEY);
+
     progressUpdater =
         new DataflowWorkProgressUpdater(
-            workItemStatusClient, workItem, worker, executor.getExecutor(), clock, hotKeyLogger) {
+            workItemStatusClient,
+            workItem,
+            worker,
+            executor.getExecutor(),
+            clock,
+            hotKeyLogger,
+            options) {
 
           // Shorten reporting interval boundaries for faster testing.
           @Override
@@ -136,6 +153,19 @@
     progressUpdater.startReportingProgress();
     executor.runNextRunnable();
 
+    verify(hotKeyLogger, atLeastOnce()).logHotKeyDetection(STEP_ID, HOT_KEY_AGE, HOT_KEY);
+
+    progressUpdater.stopReportingProgress();
+  }
+
+  @Test
+  public void workProgressLogsHotKeyDetectionNotEnabled() throws Exception {
+    options.setHotKeyLoggingEnabled(false);
+    when(workItemStatusClient.reportUpdate(isNull(DynamicSplitResult.class), isA(Duration.class)))
+        .thenReturn(generateServiceState(null, 1000));
+    progressUpdater.startReportingProgress();
+    executor.runNextRunnable();
+
     verify(hotKeyLogger, atLeastOnce()).logHotKeyDetection(STEP_ID, HOT_KEY_AGE);
 
     progressUpdater.stopReportingProgress();
@@ -261,6 +291,7 @@
 
     // And nothing happened after that.
     verify(workItemStatusClient, Mockito.atLeastOnce()).uniqueWorkId();
+    verify(workItemStatusClient, Mockito.atLeastOnce()).getExecutionContext();
     verifyNoMoreInteractions(workItemStatusClient);
   }
 
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DataflowWorkUnitClientTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DataflowWorkUnitClientTest.java
index b96ee2f..dac2940 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DataflowWorkUnitClientTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DataflowWorkUnitClientTest.java
@@ -59,6 +59,7 @@
 
 /** Unit tests for {@link DataflowWorkUnitClient}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DataflowWorkUnitClientTest {
   private static final Logger LOG = LoggerFactory.getLogger(DataflowWorkUnitClientTest.class);
   @Rule public TestRule restoreSystemProperties = new RestoreSystemProperties();
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DataflowWorkerHarnessHelperTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DataflowWorkerHarnessHelperTest.java
index 453dbea..83f56e4 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DataflowWorkerHarnessHelperTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DataflowWorkerHarnessHelperTest.java
@@ -42,6 +42,7 @@
 
 /** Unit tests for {@link DataflowWorkerHarnessHelper}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DataflowWorkerHarnessHelperTest {
   @Rule public TemporaryFolder tmpFolder = new TemporaryFolder();
   @Rule public TestRule restoreSystemProperties = new RestoreSystemProperties();
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DefaultParDoFnFactoryTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DefaultParDoFnFactoryTest.java
index a82878d..eb74da1 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DefaultParDoFnFactoryTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DefaultParDoFnFactoryTest.java
@@ -48,6 +48,7 @@
 
 /** Tests for {@link DefaultParDoFnFactory}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DefaultParDoFnFactoryTest {
 
   private static class TestDoFn extends DoFn<Integer, String> {
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DeltaCounterCellTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DeltaCounterCellTest.java
index c4f5377..1ce778c 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DeltaCounterCellTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DeltaCounterCellTest.java
@@ -38,6 +38,7 @@
 
 /** Tests for {@link DeltaCounterCell}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DeltaCounterCellTest {
 
   private DeltaCounterCell cell;
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DoFnInstanceManagersTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DoFnInstanceManagersTest.java
index 904b462..43006aa 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DoFnInstanceManagersTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DoFnInstanceManagersTest.java
@@ -39,6 +39,7 @@
 
 /** Tests for {@link DoFnInstanceManagers}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DoFnInstanceManagersTest {
   @Rule public ExpectedException thrown = ExpectedException.none();
 
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/FakeWindmillServer.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/FakeWindmillServer.java
index 79c7844..68254ce 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/FakeWindmillServer.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/FakeWindmillServer.java
@@ -58,6 +58,7 @@
 import org.slf4j.LoggerFactory;
 
 /** An in-memory Windmill server that offers provided work and data. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class FakeWindmillServer extends WindmillServerStub {
   private static final Logger LOG = LoggerFactory.getLogger(FakeWindmillServer.class);
 
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/FnApiWindowMappingFnTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/FnApiWindowMappingFnTest.java
index cf3de65..b5d60e3 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/FnApiWindowMappingFnTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/FnApiWindowMappingFnTest.java
@@ -60,6 +60,7 @@
 
 /** Tests for {@link FnApiWindowMappingFn}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class FnApiWindowMappingFnTest {
   private static final ApiServiceDescriptor DATA_SERVICE =
       ApiServiceDescriptor.newBuilder().setUrl("test://data").build();
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/GroupingShuffleReaderTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/GroupingShuffleReaderTest.java
index c78073f..f70d9a8 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/GroupingShuffleReaderTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/GroupingShuffleReaderTest.java
@@ -90,6 +90,7 @@
 
 /** Tests for GroupingShuffleReader. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class GroupingShuffleReaderTest {
   private static final byte[] EMPTY_BYTE_ARRAY = new byte[0];
   private static final List<KV<Integer, List<KV<Integer, Integer>>>> NO_KVS =
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/HotKeyLoggerTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/HotKeyLoggerTest.java
index 1049f2b..b5d91e8 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/HotKeyLoggerTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/HotKeyLoggerTest.java
@@ -34,6 +34,7 @@
 
 @RunWith(PowerMockRunner.class)
 @PrepareForTest({HotKeyLoggerTest.class, LoggerFactory.class})
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class HotKeyLoggerTest {
   @Rule public ExpectedLogs expectedLogs = ExpectedLogs.none(HotKeyLogger.class);
 
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/InMemoryReaderFactoryTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/InMemoryReaderFactoryTest.java
index cdb3b2b..3285c4b 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/InMemoryReaderFactoryTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/InMemoryReaderFactoryTest.java
@@ -39,6 +39,7 @@
 
 /** Tests for InMemoryReaderFactory. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class InMemoryReaderFactoryTest {
   static <T> Source createInMemoryCloudSource(
       List<T> elements, Long start, Long end, Coder<T> coder) throws Exception {
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/IntrinsicMapTaskExecutorFactoryTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/IntrinsicMapTaskExecutorFactoryTest.java
index 30a0284..71b7dde 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/IntrinsicMapTaskExecutorFactoryTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/IntrinsicMapTaskExecutorFactoryTest.java
@@ -115,6 +115,7 @@
 
 /** Tests for {@link IntrinsicMapTaskExecutorFactory}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class IntrinsicMapTaskExecutorFactoryTest {
   private static final String STAGE = "test";
 
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/IntrinsicMapTaskExecutorTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/IntrinsicMapTaskExecutorTest.java
index 5ad42db..27ed394 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/IntrinsicMapTaskExecutorTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/IntrinsicMapTaskExecutorTest.java
@@ -81,6 +81,7 @@
 
 /** Tests for {@link MapTaskExecutor}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class IntrinsicMapTaskExecutorTest {
 
   private static final String COUNTER_PREFIX = "test-";
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/IsmReaderFactoryTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/IsmReaderFactoryTest.java
index 4bf9353..d3fa01e 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/IsmReaderFactoryTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/IsmReaderFactoryTest.java
@@ -53,6 +53,7 @@
 
 /** Tests for {@link IsmReaderFactory}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class IsmReaderFactoryTest {
   private DataflowPipelineOptions options;
   private Cache<Object, Object> logicalReferenceCache;
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/IsmReaderTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/IsmReaderTest.java
index 92e3065..e170bfe 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/IsmReaderTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/IsmReaderTest.java
@@ -91,6 +91,7 @@
 
 /** Tests for {@link IsmReader}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class IsmReaderTest {
   private static final long BLOOM_FILTER_SIZE_LIMIT = 10_000;
   private static final int TEST_BLOCK_SIZE = 1024;
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/IsmSideInputReaderTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/IsmSideInputReaderTest.java
index db981a9..9d90ad4 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/IsmSideInputReaderTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/IsmSideInputReaderTest.java
@@ -141,6 +141,7 @@
  * equivalently to their numeric representation for non-negative values.
  */
 @RunWith(JUnit4.class)
+@SuppressWarnings({"keyfor", "nullness"}) // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class IsmSideInputReaderTest {
   private static final Logger LOG = LoggerFactory.getLogger(IsmSideInputReaderTest.class);
   private static final long BLOOM_FILTER_SIZE_LIMIT = 10_000;
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/LazilyInitializedSideInputReaderTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/LazilyInitializedSideInputReaderTest.java
index 213871f..1c37880 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/LazilyInitializedSideInputReaderTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/LazilyInitializedSideInputReaderTest.java
@@ -40,6 +40,7 @@
 
 /** Tests for {@link LazilyInitializedSideInputReader}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class LazilyInitializedSideInputReaderTest {
   private static final String TEST_TAG = "test_tag";
 
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/LogRecordMatcher.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/LogRecordMatcher.java
index 00b09aa..5d8e6ee 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/LogRecordMatcher.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/LogRecordMatcher.java
@@ -25,6 +25,7 @@
 import org.hamcrest.TypeSafeMatcher;
 
 /** Hamcrest matcher for asserts on {@link LogRecord} instances. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public final class LogRecordMatcher extends TypeSafeMatcher<LogRecord> {
   private final String substring;
   private final Matcher<Level> levelMatcher;
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/LogRecordMatcherTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/LogRecordMatcherTest.java
index 1c1e0d0..f968e04 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/LogRecordMatcherTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/LogRecordMatcherTest.java
@@ -31,7 +31,10 @@
 
 /** Unit tests for {@link LogRecordMatcher}. */
 @RunWith(JUnit4.class)
-@SuppressWarnings("AssertionFailureIgnored")
+@SuppressWarnings({
+  "AssertionFailureIgnored",
+  "nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
+})
 public class LogRecordMatcherTest {
   @Test
   public void testMatchingLogRecord() {
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/NoOpSourceOperationExecutorTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/NoOpSourceOperationExecutorTest.java
index 5ddd960..7fdd6d3 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/NoOpSourceOperationExecutorTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/NoOpSourceOperationExecutorTest.java
@@ -33,6 +33,7 @@
 
 /** Tests for {@link NoOpSourceOperationExecutor} */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class NoOpSourceOperationExecutorTest {
 
   private PipelineOptions options;
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/PairWithConstantKeyDoFnFactoryTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/PairWithConstantKeyDoFnFactoryTest.java
index ebb32fa..ee14c3e 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/PairWithConstantKeyDoFnFactoryTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/PairWithConstantKeyDoFnFactoryTest.java
@@ -38,6 +38,7 @@
 
 /** Tests for {@link PairWithConstantKeyDoFnFactory}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class PairWithConstantKeyDoFnFactoryTest {
   @Test
   public void testConversionOfRecord() throws Exception {
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/PartialGroupByKeyParDoFnsTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/PartialGroupByKeyParDoFnsTest.java
index 966710d..a2d9789 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/PartialGroupByKeyParDoFnsTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/PartialGroupByKeyParDoFnsTest.java
@@ -86,7 +86,11 @@
 
 /** Tests for {@link PartialGroupByKeyParDoFns}. */
 @RunWith(JUnit4.class)
-@SuppressWarnings({"rawtypes", "unchecked"})
+@SuppressWarnings({
+  "rawtypes",
+  "unchecked",
+  "nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
+})
 public class PartialGroupByKeyParDoFnsTest {
   @Mock private StreamingSideInputFetcher<KV<String, Integer>, BoundedWindow> mockSideInputFetcher;
   @Mock private BagState<WindowedValue<KV<String, Integer>>> elemsBag;
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/PartitioningShuffleReaderTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/PartitioningShuffleReaderTest.java
index cf784e1..6a5e285 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/PartitioningShuffleReaderTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/PartitioningShuffleReaderTest.java
@@ -45,6 +45,7 @@
 
 /** Tests for PartitioningShuffleReader. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class PartitioningShuffleReaderTest {
   private static final List<WindowedValue<KV<Integer, String>>> NO_KVS = Collections.emptyList();
 
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/PubsubReaderTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/PubsubReaderTest.java
index 9cae865..5b94554 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/PubsubReaderTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/PubsubReaderTest.java
@@ -42,6 +42,7 @@
 
 /** Unit tests for {@link PubsubReader}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class PubsubReaderTest {
   @Mock StreamingModeExecutionContext mockContext;
 
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/PubsubSinkTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/PubsubSinkTest.java
index a01356e..e83f769 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/PubsubSinkTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/PubsubSinkTest.java
@@ -40,6 +40,7 @@
 
 /** Unit tests for {@link PubsubSink}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class PubsubSinkTest {
   @Mock StreamingModeExecutionContext mockContext;
 
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ReaderCacheTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ReaderCacheTest.java
index 76dd70e..a8f31c9 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ReaderCacheTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ReaderCacheTest.java
@@ -38,6 +38,7 @@
 
 /** Tests for {@link ReaderCache}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ReaderCacheTest {
 
   private static final String C_ID = "computationId";
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ReaderFactoryTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ReaderFactoryTest.java
index 21273b4..bee4bab 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ReaderFactoryTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ReaderFactoryTest.java
@@ -39,6 +39,7 @@
 
 /** Tests for ReaderFactory. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ReaderFactoryTest {
 
   static class TestReaderFactory implements ReaderFactory {
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ReaderTestUtils.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ReaderTestUtils.java
index 343661a..8676a72 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ReaderTestUtils.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ReaderTestUtils.java
@@ -38,6 +38,7 @@
  * Helpers for testing {@link NativeReader} and related classes, especially {@link
  * NativeReaderIterator#getProgress} and {@link NativeReaderIterator#requestDynamicSplit}.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ReaderTestUtils {
   public static Position positionAtIndex(@Nullable Long index) {
     return new Position().setRecordIndex(index);
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ReifyTimestampAndWindowsParDoFnFactoryTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ReifyTimestampAndWindowsParDoFnFactoryTest.java
index cf8621d..91a90d3 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ReifyTimestampAndWindowsParDoFnFactoryTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ReifyTimestampAndWindowsParDoFnFactoryTest.java
@@ -33,6 +33,7 @@
 import org.junit.Test;
 
 /** Tests for {@link ReifyTimestampAndWindowsParDoFnFactory} */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ReifyTimestampAndWindowsParDoFnFactoryTest {
 
   private <K, V> void verifyReifiedIsInTheSameWindows(WindowedValue<KV<K, V>> elem)
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/SdkHarnessRegistryTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/SdkHarnessRegistryTest.java
index b2910e4..c10d792 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/SdkHarnessRegistryTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/SdkHarnessRegistryTest.java
@@ -38,6 +38,7 @@
 
 /** Unit tests for {@link SdkHarnessRegistry}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SdkHarnessRegistryTest {
 
   @Test
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ShuffleReaderFactoryTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ShuffleReaderFactoryTest.java
index 95ece65..e70b549 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ShuffleReaderFactoryTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ShuffleReaderFactoryTest.java
@@ -47,7 +47,11 @@
  * PartitioningShuffleReaderFactory.
  */
 @RunWith(JUnit4.class)
-@SuppressWarnings({"rawtypes", "unchecked"})
+@SuppressWarnings({
+  "rawtypes",
+  "unchecked",
+  "nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
+})
 public class ShuffleReaderFactoryTest {
   <T extends NativeReader> T runTestCreateShuffleReader(
       byte[] shuffleReaderConfig,
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ShuffleSinkTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ShuffleSinkTest.java
index 1a87b3c..2b6c376 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ShuffleSinkTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ShuffleSinkTest.java
@@ -48,6 +48,7 @@
 
 /** Tests for {@link ShuffleSink}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ShuffleSinkTest {
   private static final List<KV<Integer, String>> NO_KVS = Collections.emptyList();
 
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/SimpleParDoFnTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/SimpleParDoFnTest.java
index 2933bf5..8350698 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/SimpleParDoFnTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/SimpleParDoFnTest.java
@@ -71,6 +71,7 @@
 
 /** Tests for {@link SimpleParDoFn}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SimpleParDoFnTest {
   @Rule public ExpectedException thrown = ExpectedException.none();
 
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/SourceOperationExecutorFactoryTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/SourceOperationExecutorFactoryTest.java
index 16d35a3..ce479f3 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/SourceOperationExecutorFactoryTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/SourceOperationExecutorFactoryTest.java
@@ -35,6 +35,7 @@
 
 /** Tests for {@link SourceOperationExecutorFactory}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SourceOperationExecutorFactoryTest {
 
   @Mock public DataflowExecutionContext<?> executionContext;
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StateFetcherTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StateFetcherTest.java
index 8144d8e..1b21d32 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StateFetcherTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StateFetcherTest.java
@@ -57,6 +57,7 @@
 
 /** Unit tests for {@link StateFetcher}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class StateFetcherTest {
   private static final String STATE_FAMILY = "state";
 
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorkerTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorkerTest.java
index 4031d02..e061b31 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorkerTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorkerTest.java
@@ -30,6 +30,7 @@
 import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
+import static org.mockito.ArgumentMatchers.eq;
 import static org.mockito.ArgumentMatchers.nullable;
 import static org.mockito.Matchers.any;
 import static org.mockito.Mockito.atLeast;
@@ -166,6 +167,7 @@
 
 /** Unit tests for {@link StreamingDataflowWorker}. */
 @RunWith(Parameterized.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class StreamingDataflowWorkerTest {
 
   private final boolean streamingEngine;
@@ -763,6 +765,76 @@
     verify(hotKeyLogger, atLeastOnce()).logHotKeyDetection(nullable(String.class), any());
   }
 
+  @Test
+  public void testHotKeyLogging() throws Exception {
+    // This is to test that the worker can correctly log the key from a hot key.
+    List<ParallelInstruction> instructions =
+        Arrays.asList(
+            makeSourceInstruction(KvCoder.of(StringUtf8Coder.of(), StringUtf8Coder.of())),
+            makeSinkInstruction(KvCoder.of(StringUtf8Coder.of(), StringUtf8Coder.of()), 0));
+
+    FakeWindmillServer server = new FakeWindmillServer(errorCollector);
+    server.setIsReady(false);
+
+    StreamingConfigTask streamingConfig = new StreamingConfigTask();
+    streamingConfig.setStreamingComputationConfigs(
+        ImmutableList.of(makeDefaultStreamingComputationConfig(instructions)));
+    streamingConfig.setWindmillServiceEndpoint("foo");
+    WorkItem workItem = new WorkItem();
+    workItem.setStreamingConfigTask(streamingConfig);
+    when(mockWorkUnitClient.getGlobalStreamingConfigWorkItem()).thenReturn(Optional.of(workItem));
+
+    StreamingDataflowWorkerOptions options =
+        createTestingPipelineOptions(server, "--hotKeyLoggingEnabled=true");
+    StreamingDataflowWorker worker = makeWorker(instructions, options, true /* publishCounters */);
+    worker.start();
+
+    final int numIters = 2000;
+    for (int i = 0; i < numIters; ++i) {
+      server.addWorkToOffer(makeInput(i, 0, "key", DEFAULT_SHARDING_KEY));
+    }
+
+    Map<Long, Windmill.WorkItemCommitRequest> result = server.waitForAndGetCommits(numIters);
+    worker.stop();
+
+    verify(hotKeyLogger, atLeastOnce())
+        .logHotKeyDetection(nullable(String.class), any(), eq("key"));
+  }
+
+  @Test
+  public void testHotKeyLoggingNotEnabled() throws Exception {
+    // This is to test that the worker can correctly log the key from a hot key.
+    List<ParallelInstruction> instructions =
+        Arrays.asList(
+            makeSourceInstruction(KvCoder.of(StringUtf8Coder.of(), StringUtf8Coder.of())),
+            makeSinkInstruction(KvCoder.of(StringUtf8Coder.of(), StringUtf8Coder.of()), 0));
+
+    FakeWindmillServer server = new FakeWindmillServer(errorCollector);
+    server.setIsReady(false);
+
+    StreamingConfigTask streamingConfig = new StreamingConfigTask();
+    streamingConfig.setStreamingComputationConfigs(
+        ImmutableList.of(makeDefaultStreamingComputationConfig(instructions)));
+    streamingConfig.setWindmillServiceEndpoint("foo");
+    WorkItem workItem = new WorkItem();
+    workItem.setStreamingConfigTask(streamingConfig);
+    when(mockWorkUnitClient.getGlobalStreamingConfigWorkItem()).thenReturn(Optional.of(workItem));
+
+    StreamingDataflowWorkerOptions options = createTestingPipelineOptions(server);
+    StreamingDataflowWorker worker = makeWorker(instructions, options, true /* publishCounters */);
+    worker.start();
+
+    final int numIters = 2000;
+    for (int i = 0; i < numIters; ++i) {
+      server.addWorkToOffer(makeInput(i, 0, "key", DEFAULT_SHARDING_KEY));
+    }
+
+    Map<Long, Windmill.WorkItemCommitRequest> result = server.waitForAndGetCommits(numIters);
+    worker.stop();
+
+    verify(hotKeyLogger, atLeastOnce()).logHotKeyDetection(nullable(String.class), any());
+  }
+
   static class BlockingFn extends DoFn<String, String> implements TestRule {
 
     public static CountDownLatch blocker = new CountDownLatch(1);
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingGroupAlsoByWindowFnsTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingGroupAlsoByWindowFnsTest.java
index 0d5a883..a6d327b 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingGroupAlsoByWindowFnsTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingGroupAlsoByWindowFnsTest.java
@@ -87,6 +87,7 @@
 
 /** Unit tests for {@link StreamingGroupAlsoByWindowsDoFns}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class StreamingGroupAlsoByWindowFnsTest {
   private static final String KEY = "k";
   private static final String STATE_FAMILY = "stateFamily";
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingGroupAlsoByWindowsReshuffleDoFnTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingGroupAlsoByWindowsReshuffleDoFnTest.java
index 81a2bf5..8d8a5ef 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingGroupAlsoByWindowsReshuffleDoFnTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingGroupAlsoByWindowsReshuffleDoFnTest.java
@@ -59,6 +59,7 @@
 
 /** Unit tests for {@link StreamingGroupAlsoByWindowReshuffleFn}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class StreamingGroupAlsoByWindowsReshuffleDoFnTest {
   private static final String KEY = "k";
   private static final long WORK_TOKEN = 1000L;
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingKeyedWorkItemSideInputDoFnRunnerTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingKeyedWorkItemSideInputDoFnRunnerTest.java
index 5d6f1e7..77bf910 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingKeyedWorkItemSideInputDoFnRunnerTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingKeyedWorkItemSideInputDoFnRunnerTest.java
@@ -68,6 +68,7 @@
 
 /** Unit tests for {@link StreamingKeyedWorkItemSideInputDoFnRunner}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class StreamingKeyedWorkItemSideInputDoFnRunnerTest {
   private static final FixedWindows WINDOW_FN = FixedWindows.of(Duration.millis(10));
   private static TupleTag<KV<String, Integer>> mainOutputTag = new TupleTag<>();
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContextTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContextTest.java
index d5071d7..795b993 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContextTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContextTest.java
@@ -80,6 +80,7 @@
 
 /** Tests for {@link StreamingModeExecutionContext}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class StreamingModeExecutionContextTest {
 
   @Mock private StateFetcher stateFetcher;
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingPCollectionViewWriterDoFnFactoryTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingPCollectionViewWriterDoFnFactoryTest.java
index a776cdb..e6b5b56 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingPCollectionViewWriterDoFnFactoryTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingPCollectionViewWriterDoFnFactoryTest.java
@@ -38,6 +38,7 @@
 
 /** Tests for {@link StreamingPCollectionViewWriterDoFnFactory}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class StreamingPCollectionViewWriterDoFnFactoryTest {
   @Test
   public void testConstruction() throws Exception {
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputDoFnRunnerTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputDoFnRunnerTest.java
index 8d87a10..308400c 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputDoFnRunnerTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputDoFnRunnerTest.java
@@ -78,6 +78,7 @@
 
 /** Unit tests for {@link StreamingSideInputDoFnRunner}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class StreamingSideInputDoFnRunnerTest {
 
   private static final FixedWindows WINDOW_FN = FixedWindows.of(Duration.millis(10));
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputFetcherTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputFetcherTest.java
index c9a4227..f117715 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputFetcherTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputFetcherTest.java
@@ -64,6 +64,7 @@
 
 /** Tests for {@link StreamingSideInputFetcher}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class StreamingSideInputFetcherTest {
 
   private static final FixedWindows WINDOW_FN = FixedWindows.of(Duration.millis(10));
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/TestOperationContext.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/TestOperationContext.java
index 9a895fb..6aabf1d 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/TestOperationContext.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/TestOperationContext.java
@@ -30,6 +30,7 @@
 import org.checkerframework.checker.nullness.qual.Nullable;
 
 /** {@link OperationContext} for testing purposes. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class TestOperationContext extends DataflowOperationContext {
 
   /** ExecutionState for testing. */
@@ -69,9 +70,8 @@
     @Override
     public void reportLull(Thread trackedThread, long millis) {}
 
-    @Nullable
     @Override
-    public CounterUpdate extractUpdate(boolean isFinalUpdate) {
+    public @Nullable CounterUpdate extractUpdate(boolean isFinalUpdate) {
       return null;
     }
   }
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/TestShuffleReadCounterFactory.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/TestShuffleReadCounterFactory.java
index c802eff..3c7dedf 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/TestShuffleReadCounterFactory.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/TestShuffleReadCounterFactory.java
@@ -28,6 +28,7 @@
  * shuffleReadCounters created for each shuffle step. Note: There is one ShuffleReadCounter for each
  * GroupingShuffleReader associated with a unique GBK/shuffle.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class TestShuffleReadCounterFactory extends ShuffleReadCounterFactory {
 
   private TreeMap<String, ShuffleReadCounter> originalShuffleStepToCounter;
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/TestShuffleReaderTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/TestShuffleReaderTest.java
index ace0eb0..482f2db 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/TestShuffleReaderTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/TestShuffleReaderTest.java
@@ -35,6 +35,7 @@
 
 /** Tests of TestShuffleReader. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class TestShuffleReaderTest {
   static final String START_KEY = "ddd";
   static final String END_KEY = "mmm";
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ToIsmRecordForMultimapDoFnFactoryTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ToIsmRecordForMultimapDoFnFactoryTest.java
index df53228..b5410b1 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ToIsmRecordForMultimapDoFnFactoryTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ToIsmRecordForMultimapDoFnFactoryTest.java
@@ -38,6 +38,7 @@
 
 /** Tests for {@link ToIsmRecordForMultimapDoFnFactory}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ToIsmRecordForMultimapDoFnFactoryTest {
   @Test
   public void testConversionOfRecord() throws Exception {
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/UngroupedShuffleReaderTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/UngroupedShuffleReaderTest.java
index 31e3c45..50a8f53 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/UngroupedShuffleReaderTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/UngroupedShuffleReaderTest.java
@@ -40,6 +40,7 @@
 
 /** Tests for UngroupedShuffleReader. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class UngroupedShuffleReaderTest {
   private static final Instant timestamp = new Instant(123000);
   private static final IntervalWindow window = new IntervalWindow(timestamp, timestamp.plus(1000));
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/UserParDoFnFactoryTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/UserParDoFnFactoryTest.java
index 9cc9cb0..f6530c5 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/UserParDoFnFactoryTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/UserParDoFnFactoryTest.java
@@ -77,6 +77,7 @@
 
 /** Tests for {@link UserParDoFnFactory}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class UserParDoFnFactoryTest {
   static class TestDoFn extends DoFn<Integer, String> {
     enum State {
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ValuesDoFnFactoryTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ValuesDoFnFactoryTest.java
index f1078dd..f8707d8 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ValuesDoFnFactoryTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ValuesDoFnFactoryTest.java
@@ -34,6 +34,7 @@
 
 /** Tests for {@link ValuesDoFnFactory}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ValuesDoFnFactoryTest {
 
   @Test
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WindmillKeyedWorkItemTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WindmillKeyedWorkItemTest.java
index 40d4c74..a73ece7 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WindmillKeyedWorkItemTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WindmillKeyedWorkItemTest.java
@@ -52,6 +52,7 @@
 
 /** Tests for {@link WindmillKeyedWorkItem}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class WindmillKeyedWorkItemTest {
 
   private static final String STATE_FAMILY = "state";
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WindmillStateCacheTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WindmillStateCacheTest.java
index 8042140..c3ab48a 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WindmillStateCacheTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WindmillStateCacheTest.java
@@ -40,6 +40,7 @@
 
 /** Tests for {@link WindmillStateCache}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class WindmillStateCacheTest {
 
   private static final String COMPUTATION = "computation";
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WindmillStateInternalsTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WindmillStateInternalsTest.java
index 52bfcd4..4b962f3 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WindmillStateInternalsTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WindmillStateInternalsTest.java
@@ -19,6 +19,7 @@
 
 import static org.apache.beam.runners.dataflow.worker.DataflowMatchers.ByteStringMatcher.byteStringEq;
 import static org.apache.beam.sdk.testing.SystemNanoTimeSleeper.sleepMillis;
+import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
@@ -27,17 +28,23 @@
 import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.when;
 
+import com.google.common.collect.Iterables;
 import java.io.Closeable;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collections;
+import java.util.Map;
 import java.util.concurrent.TimeUnit;
 import org.apache.beam.runners.core.StateNamespace;
 import org.apache.beam.runners.core.StateNamespaceForTest;
 import org.apache.beam.runners.core.StateTag;
 import org.apache.beam.runners.core.StateTags;
 import org.apache.beam.runners.dataflow.options.DataflowWorkerHarnessOptions;
+import org.apache.beam.runners.dataflow.worker.WindmillStateInternals.IdTracker;
+import org.apache.beam.runners.dataflow.worker.WindmillStateInternals.WindmillOrderedList;
 import org.apache.beam.runners.dataflow.worker.windmill.Windmill;
 import org.apache.beam.runners.dataflow.worker.windmill.Windmill.TagBag;
+import org.apache.beam.runners.dataflow.worker.windmill.Windmill.TagSortedListUpdateRequest;
 import org.apache.beam.runners.dataflow.worker.windmill.Windmill.TagValue;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.coders.StringUtf8Coder;
@@ -46,15 +53,19 @@
 import org.apache.beam.sdk.state.BagState;
 import org.apache.beam.sdk.state.CombiningState;
 import org.apache.beam.sdk.state.GroupingState;
+import org.apache.beam.sdk.state.OrderedListState;
 import org.apache.beam.sdk.state.ReadableState;
 import org.apache.beam.sdk.state.ValueState;
 import org.apache.beam.sdk.state.WatermarkHoldState;
 import org.apache.beam.sdk.transforms.Sum;
 import org.apache.beam.sdk.transforms.windowing.TimestampCombiner;
 import org.apache.beam.sdk.util.CoderUtils;
+import org.apache.beam.sdk.values.TimestampedValue;
 import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Supplier;
 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.Range;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.RangeSet;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.util.concurrent.Futures;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.util.concurrent.SettableFuture;
 import org.hamcrest.Matchers;
@@ -71,6 +82,7 @@
 
 /** Tests for {@link WindmillStateInternals}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class WindmillStateInternalsTest {
 
   private static final StateNamespace NAMESPACE = new StateNamespaceForTest("ns");
@@ -170,6 +182,341 @@
     return result;
   }
 
+  public static final Range<Long> FULL_ORDERED_LIST_RANGE =
+      Range.closedOpen(WindmillOrderedList.MIN_TS_MICROS, WindmillOrderedList.MAX_TS_MICROS);
+
+  @Test
+  public void testOrderedListAddBeforeRead() throws Exception {
+    StateTag<OrderedListState<String>> addr =
+        StateTags.orderedList("orderedList", StringUtf8Coder.of());
+    OrderedListState<String> orderedList = underTest.state(NAMESPACE, addr);
+
+    SettableFuture<Iterable<TimestampedValue<String>>> future = SettableFuture.create();
+    when(mockReader.orderedListFuture(
+            FULL_ORDERED_LIST_RANGE,
+            key(NAMESPACE, "orderedList"),
+            STATE_FAMILY,
+            StringUtf8Coder.of()))
+        .thenReturn(future);
+
+    orderedList.readLater();
+
+    final TimestampedValue<String> helloValue =
+        TimestampedValue.of("hello", Instant.ofEpochMilli(100));
+    final TimestampedValue<String> worldValue =
+        TimestampedValue.of("world", Instant.ofEpochMilli(75));
+    final TimestampedValue<String> goodbyeValue =
+        TimestampedValue.of("goodbye", Instant.ofEpochMilli(50));
+
+    orderedList.add(helloValue);
+    waitAndSet(future, Arrays.asList(worldValue), 200);
+    assertThat(orderedList.read(), Matchers.contains(worldValue, helloValue));
+
+    orderedList.add(goodbyeValue);
+    assertThat(orderedList.read(), Matchers.contains(goodbyeValue, worldValue, helloValue));
+  }
+
+  @Test
+  public void testOrderedListClearBeforeRead() throws Exception {
+    StateTag<OrderedListState<String>> addr =
+        StateTags.orderedList("orderedList", StringUtf8Coder.of());
+    OrderedListState<String> orderedListState = underTest.state(NAMESPACE, addr);
+
+    final TimestampedValue<String> helloElement = TimestampedValue.of("hello", Instant.EPOCH);
+    orderedListState.clear();
+    orderedListState.add(helloElement);
+    assertThat(orderedListState.read(), Matchers.containsInAnyOrder(helloElement));
+
+    // Shouldn't need to read from windmill for this.
+    Mockito.verifyZeroInteractions(mockReader);
+  }
+
+  @Test
+  public void testOrderedListIsEmptyFalse() throws Exception {
+    StateTag<OrderedListState<String>> addr =
+        StateTags.orderedList("orderedList", StringUtf8Coder.of());
+    OrderedListState<String> orderedList = underTest.state(NAMESPACE, addr);
+
+    SettableFuture<Iterable<TimestampedValue<String>>> future = SettableFuture.create();
+    when(mockReader.orderedListFuture(
+            FULL_ORDERED_LIST_RANGE,
+            key(NAMESPACE, "orderedList"),
+            STATE_FAMILY,
+            StringUtf8Coder.of()))
+        .thenReturn(future);
+    ReadableState<Boolean> result = orderedList.isEmpty().readLater();
+    Mockito.verify(mockReader)
+        .orderedListFuture(
+            FULL_ORDERED_LIST_RANGE,
+            key(NAMESPACE, "orderedList"),
+            STATE_FAMILY,
+            StringUtf8Coder.of());
+
+    waitAndSet(future, Arrays.asList(TimestampedValue.of("world", Instant.EPOCH)), 200);
+    assertThat(result.read(), Matchers.is(false));
+  }
+
+  @Test
+  public void testOrderedListIsEmptyTrue() throws Exception {
+    StateTag<OrderedListState<String>> addr =
+        StateTags.orderedList("orderedList", StringUtf8Coder.of());
+    OrderedListState<String> orderedList = underTest.state(NAMESPACE, addr);
+
+    SettableFuture<Iterable<TimestampedValue<String>>> future = SettableFuture.create();
+    when(mockReader.orderedListFuture(
+            FULL_ORDERED_LIST_RANGE,
+            key(NAMESPACE, "orderedList"),
+            STATE_FAMILY,
+            StringUtf8Coder.of()))
+        .thenReturn(future);
+    ReadableState<Boolean> result = orderedList.isEmpty().readLater();
+    Mockito.verify(mockReader)
+        .orderedListFuture(
+            FULL_ORDERED_LIST_RANGE,
+            key(NAMESPACE, "orderedList"),
+            STATE_FAMILY,
+            StringUtf8Coder.of());
+
+    waitAndSet(future, Collections.emptyList(), 200);
+    assertThat(result.read(), Matchers.is(true));
+  }
+
+  @Test
+  public void testOrderedListIsEmptyAfterClear() throws Exception {
+    StateTag<OrderedListState<String>> addr =
+        StateTags.orderedList("orderedList", StringUtf8Coder.of());
+    OrderedListState<String> orderedList = underTest.state(NAMESPACE, addr);
+
+    orderedList.clear();
+    ReadableState<Boolean> result = orderedList.isEmpty();
+    Mockito.verify(mockReader, never())
+        .orderedListFuture(
+            FULL_ORDERED_LIST_RANGE,
+            key(NAMESPACE, "orderedList"),
+            STATE_FAMILY,
+            StringUtf8Coder.of());
+    assertThat(result.read(), Matchers.is(true));
+
+    orderedList.add(TimestampedValue.of("hello", Instant.EPOCH));
+    assertThat(result.read(), Matchers.is(false));
+  }
+
+  @Test
+  public void testOrderedListAddPersist() throws Exception {
+    StateTag<OrderedListState<String>> addr =
+        StateTags.orderedList("orderedList", StringUtf8Coder.of());
+    OrderedListState<String> orderedList = underTest.state(NAMESPACE, addr);
+
+    SettableFuture<Map<Range<Instant>, RangeSet<Long>>> orderedListFuture = SettableFuture.create();
+    orderedListFuture.set(null);
+    SettableFuture<Map<Range<Instant>, RangeSet<Instant>>> deletionsFuture =
+        SettableFuture.create();
+    deletionsFuture.set(null);
+    when(mockReader.valueFuture(
+            systemKey(NAMESPACE, "orderedList" + IdTracker.IDS_AVAILABLE_STR),
+            STATE_FAMILY,
+            IdTracker.IDS_AVAILABLE_CODER))
+        .thenReturn(orderedListFuture);
+    when(mockReader.valueFuture(
+            systemKey(NAMESPACE, "orderedList" + IdTracker.DELETIONS_STR),
+            STATE_FAMILY,
+            IdTracker.SUBRANGE_DELETIONS_CODER))
+        .thenReturn(deletionsFuture);
+
+    orderedList.add(TimestampedValue.of("hello", Instant.ofEpochMilli(1)));
+
+    Windmill.WorkItemCommitRequest.Builder commitBuilder =
+        Windmill.WorkItemCommitRequest.newBuilder();
+    underTest.persist(commitBuilder);
+
+    assertEquals(1, commitBuilder.getSortedListUpdatesCount());
+    TagSortedListUpdateRequest updates = commitBuilder.getSortedListUpdates(0);
+    assertEquals(key(NAMESPACE, "orderedList"), updates.getTag());
+    assertEquals(1, updates.getInsertsCount());
+    assertEquals(1, updates.getInserts(0).getEntriesCount());
+
+    assertEquals("hello", updates.getInserts(0).getEntries(0).getValue().toStringUtf8());
+    assertEquals(1000, updates.getInserts(0).getEntries(0).getSortKey());
+    assertEquals(IdTracker.MIN_ID, updates.getInserts(0).getEntries(0).getId());
+  }
+
+  @Test
+  public void testOrderedListClearPersist() throws Exception {
+    StateTag<OrderedListState<String>> addr =
+        StateTags.orderedList("orderedList", StringUtf8Coder.of());
+    OrderedListState<String> orderedListState = underTest.state(NAMESPACE, addr);
+
+    orderedListState.add(TimestampedValue.of("hello", Instant.ofEpochMilli(1)));
+    orderedListState.clear();
+    orderedListState.add(TimestampedValue.of("world", Instant.ofEpochMilli(2)));
+    orderedListState.add(TimestampedValue.of("world", Instant.ofEpochMilli(2)));
+
+    Windmill.WorkItemCommitRequest.Builder commitBuilder =
+        Windmill.WorkItemCommitRequest.newBuilder();
+    underTest.persist(commitBuilder);
+
+    assertEquals(1, commitBuilder.getSortedListUpdatesCount());
+    TagSortedListUpdateRequest updates = commitBuilder.getSortedListUpdates(0);
+    assertEquals(STATE_FAMILY, updates.getStateFamily());
+    assertEquals(key(NAMESPACE, "orderedList"), updates.getTag());
+    assertEquals(1, updates.getInsertsCount());
+    assertEquals(2, updates.getInserts(0).getEntriesCount());
+
+    assertEquals("world", updates.getInserts(0).getEntries(0).getValue().toStringUtf8());
+    assertEquals("world", updates.getInserts(0).getEntries(1).getValue().toStringUtf8());
+    assertEquals(2000, updates.getInserts(0).getEntries(0).getSortKey());
+    assertEquals(2000, updates.getInserts(0).getEntries(1).getSortKey());
+    assertEquals(IdTracker.MIN_ID, updates.getInserts(0).getEntries(0).getId());
+    assertEquals(IdTracker.MIN_ID + 1, updates.getInserts(0).getEntries(1).getId());
+    Mockito.verifyNoMoreInteractions(mockReader);
+  }
+
+  @Test
+  public void testOrderedListDeleteRangePersist() {
+    SettableFuture<Map<Range<Instant>, RangeSet<Long>>> orderedListFuture = SettableFuture.create();
+    orderedListFuture.set(null);
+    SettableFuture<Map<Range<Instant>, RangeSet<Instant>>> deletionsFuture =
+        SettableFuture.create();
+    deletionsFuture.set(null);
+    when(mockReader.valueFuture(
+            systemKey(NAMESPACE, "orderedList" + IdTracker.IDS_AVAILABLE_STR),
+            STATE_FAMILY,
+            IdTracker.IDS_AVAILABLE_CODER))
+        .thenReturn(orderedListFuture);
+    when(mockReader.valueFuture(
+            systemKey(NAMESPACE, "orderedList" + IdTracker.DELETIONS_STR),
+            STATE_FAMILY,
+            IdTracker.SUBRANGE_DELETIONS_CODER))
+        .thenReturn(deletionsFuture);
+
+    StateTag<OrderedListState<String>> addr =
+        StateTags.orderedList("orderedList", StringUtf8Coder.of());
+    OrderedListState<String> orderedListState = underTest.state(NAMESPACE, addr);
+
+    orderedListState.add(TimestampedValue.of("hello", Instant.ofEpochMilli(1)));
+    orderedListState.add(TimestampedValue.of("hello", Instant.ofEpochMilli(2)));
+    orderedListState.add(TimestampedValue.of("hello", Instant.ofEpochMilli(2)));
+    orderedListState.add(TimestampedValue.of("world", Instant.ofEpochMilli(3)));
+    orderedListState.add(TimestampedValue.of("world", Instant.ofEpochMilli(4)));
+    orderedListState.clearRange(Instant.ofEpochMilli(2), Instant.ofEpochMilli(4));
+    Windmill.WorkItemCommitRequest.Builder commitBuilder =
+        Windmill.WorkItemCommitRequest.newBuilder();
+    underTest.persist(commitBuilder);
+
+    assertEquals(1, commitBuilder.getSortedListUpdatesCount());
+    TagSortedListUpdateRequest updates = commitBuilder.getSortedListUpdates(0);
+    assertEquals(STATE_FAMILY, updates.getStateFamily());
+    assertEquals(key(NAMESPACE, "orderedList"), updates.getTag());
+    assertEquals(1, updates.getInsertsCount());
+    assertEquals(2, updates.getInserts(0).getEntriesCount());
+
+    assertEquals("hello", updates.getInserts(0).getEntries(0).getValue().toStringUtf8());
+    assertEquals("world", updates.getInserts(0).getEntries(1).getValue().toStringUtf8());
+    assertEquals(1000, updates.getInserts(0).getEntries(0).getSortKey());
+    assertEquals(4000, updates.getInserts(0).getEntries(1).getSortKey());
+    assertEquals(IdTracker.MIN_ID, updates.getInserts(0).getEntries(0).getId());
+    assertEquals(IdTracker.MIN_ID + 1, updates.getInserts(0).getEntries(1).getId());
+  }
+
+  @Test
+  public void testOrderedListMergePendingAdds() {
+    SettableFuture<Map<Range<Instant>, RangeSet<Long>>> orderedListFuture = SettableFuture.create();
+    orderedListFuture.set(null);
+    SettableFuture<Map<Range<Instant>, RangeSet<Instant>>> deletionsFuture =
+        SettableFuture.create();
+    deletionsFuture.set(null);
+    when(mockReader.valueFuture(
+            systemKey(NAMESPACE, "orderedList" + IdTracker.IDS_AVAILABLE_STR),
+            STATE_FAMILY,
+            IdTracker.IDS_AVAILABLE_CODER))
+        .thenReturn(orderedListFuture);
+    when(mockReader.valueFuture(
+            systemKey(NAMESPACE, "orderedList" + IdTracker.DELETIONS_STR),
+            STATE_FAMILY,
+            IdTracker.SUBRANGE_DELETIONS_CODER))
+        .thenReturn(deletionsFuture);
+
+    SettableFuture<Iterable<TimestampedValue<String>>> fromStorage = SettableFuture.create();
+    when(mockReader.orderedListFuture(
+            FULL_ORDERED_LIST_RANGE,
+            key(NAMESPACE, "orderedList"),
+            STATE_FAMILY,
+            StringUtf8Coder.of()))
+        .thenReturn(fromStorage);
+
+    StateTag<OrderedListState<String>> addr =
+        StateTags.orderedList("orderedList", StringUtf8Coder.of());
+    OrderedListState<String> orderedListState = underTest.state(NAMESPACE, addr);
+
+    orderedListState.add(TimestampedValue.of("second", Instant.ofEpochMilli(1)));
+    orderedListState.add(TimestampedValue.of("third", Instant.ofEpochMilli(2)));
+    orderedListState.add(TimestampedValue.of("fourth", Instant.ofEpochMilli(2)));
+    orderedListState.add(TimestampedValue.of("eighth", Instant.ofEpochMilli(10)));
+    orderedListState.add(TimestampedValue.of("ninth", Instant.ofEpochMilli(15)));
+
+    fromStorage.set(
+        ImmutableList.of(
+            TimestampedValue.of("first", Instant.ofEpochMilli(-1)),
+            TimestampedValue.of("fifth", Instant.ofEpochMilli(5)),
+            TimestampedValue.of("sixth", Instant.ofEpochMilli(5)),
+            TimestampedValue.of("seventh", Instant.ofEpochMilli(5)),
+            TimestampedValue.of("tenth", Instant.ofEpochMilli(20))));
+
+    TimestampedValue[] expected =
+        Iterables.toArray(
+            ImmutableList.of(
+                TimestampedValue.of("first", Instant.ofEpochMilli(-1)),
+                TimestampedValue.of("second", Instant.ofEpochMilli(1)),
+                TimestampedValue.of("third", Instant.ofEpochMilli(2)),
+                TimestampedValue.of("fourth", Instant.ofEpochMilli(2)),
+                TimestampedValue.of("fifth", Instant.ofEpochMilli(5)),
+                TimestampedValue.of("sixth", Instant.ofEpochMilli(5)),
+                TimestampedValue.of("seventh", Instant.ofEpochMilli(5)),
+                TimestampedValue.of("eighth", Instant.ofEpochMilli(10)),
+                TimestampedValue.of("ninth", Instant.ofEpochMilli(15)),
+                TimestampedValue.of("tenth", Instant.ofEpochMilli(20))),
+            TimestampedValue.class);
+
+    TimestampedValue[] read = Iterables.toArray(orderedListState.read(), TimestampedValue.class);
+    assertArrayEquals(expected, read);
+  }
+
+  @Test
+  public void testOrderedListPersistEmpty() throws Exception {
+    StateTag<OrderedListState<String>> addr =
+        StateTags.orderedList("orderedList", StringUtf8Coder.of());
+    OrderedListState<String> orderedListState = underTest.state(NAMESPACE, addr);
+
+    orderedListState.clear();
+
+    Windmill.WorkItemCommitRequest.Builder commitBuilder =
+        Windmill.WorkItemCommitRequest.newBuilder();
+    underTest.persist(commitBuilder);
+
+    // 1 bag update = the clear
+    assertEquals(1, commitBuilder.getSortedListUpdatesCount());
+    TagSortedListUpdateRequest updates = commitBuilder.getSortedListUpdates(0);
+    assertEquals(1, updates.getDeletesCount());
+    assertEquals(WindmillOrderedList.MIN_TS_MICROS, updates.getDeletes(0).getRange().getStart());
+    assertEquals(WindmillOrderedList.MAX_TS_MICROS, updates.getDeletes(0).getRange().getLimit());
+  }
+
+  @Test
+  public void testNewOrderedListNoFetch() throws Exception {
+    StateTag<OrderedListState<String>> addr =
+        StateTags.orderedList("orderedList", StringUtf8Coder.of());
+    OrderedListState<String> orderedList = underTestNewKey.state(NAMESPACE, addr);
+
+    assertThat(orderedList.read(), Matchers.emptyIterable());
+
+    // Shouldn't need to read from windmill for this.
+    Mockito.verifyZeroInteractions(mockReader);
+  }
+
+  // test ordered list cleared before read
+  // test fetch + add + read
+  // test ids
+
   @Test
   public void testBagAddBeforeRead() throws Exception {
     StateTag<BagState<String>> addr = StateTags.bag("bag", StringUtf8Coder.of());
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WindmillStateReaderTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WindmillStateReaderTest.java
index f2628ff..1cc1b2e 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WindmillStateReaderTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WindmillStateReaderTest.java
@@ -26,11 +26,16 @@
 import java.util.concurrent.Future;
 import org.apache.beam.runners.dataflow.worker.windmill.Windmill;
 import org.apache.beam.runners.dataflow.worker.windmill.Windmill.KeyedGetDataRequest;
+import org.apache.beam.runners.dataflow.worker.windmill.Windmill.SortedListEntry;
+import org.apache.beam.runners.dataflow.worker.windmill.Windmill.SortedListRange;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.coders.VarIntCoder;
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.values.TimestampedValue;
 import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
 import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString.Output;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Charsets;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Range;
 import org.hamcrest.Matchers;
 import org.joda.time.Instant;
 import org.junit.Before;
@@ -44,7 +49,10 @@
 
 /** Tests for {@link WindmillStateReader}. */
 @RunWith(JUnit4.class)
-@SuppressWarnings("FutureReturnValueIgnored")
+@SuppressWarnings({
+  "FutureReturnValueIgnored",
+  "nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
+})
 public class WindmillStateReaderTest {
   private static final VarIntCoder INT_CODER = VarIntCoder.of();
 
@@ -199,6 +207,263 @@
   }
 
   @Test
+  public void testReadSortedList() throws Exception {
+    long beginning = SortedListRange.getDefaultInstance().getStart();
+    long end = SortedListRange.getDefaultInstance().getLimit();
+    Future<Iterable<TimestampedValue<Integer>>> future =
+        underTest.orderedListFuture(
+            Range.closedOpen(beginning, end), STATE_KEY_1, STATE_FAMILY, INT_CODER);
+    Mockito.verifyNoMoreInteractions(mockWindmill);
+
+    // Fetch the entire list.
+    Windmill.KeyedGetDataRequest.Builder expectedRequest =
+        Windmill.KeyedGetDataRequest.newBuilder()
+            .setKey(DATA_KEY)
+            .setShardingKey(SHARDING_KEY)
+            .setWorkToken(WORK_TOKEN)
+            .setMaxBytes(WindmillStateReader.MAX_KEY_BYTES)
+            .addSortedListsToFetch(
+                Windmill.TagSortedListFetchRequest.newBuilder()
+                    .setTag(STATE_KEY_1)
+                    .setStateFamily(STATE_FAMILY)
+                    .addFetchRanges(SortedListRange.newBuilder().setStart(beginning).setLimit(end))
+                    .setFetchMaxBytes(WindmillStateReader.MAX_BAG_BYTES));
+
+    Windmill.KeyedGetDataResponse.Builder response =
+        Windmill.KeyedGetDataResponse.newBuilder()
+            .setKey(DATA_KEY)
+            .addTagSortedLists(
+                Windmill.TagSortedListFetchResponse.newBuilder()
+                    .setTag(STATE_KEY_1)
+                    .setStateFamily(STATE_FAMILY)
+                    .addEntries(
+                        SortedListEntry.newBuilder().setValue(intData(5)).setSortKey(5000).setId(5))
+                    .addEntries(
+                        SortedListEntry.newBuilder().setValue(intData(6)).setSortKey(6000).setId(5))
+                    .addEntries(
+                        SortedListEntry.newBuilder().setValue(intData(7)).setSortKey(7000).setId(7))
+                    .addEntries(
+                        SortedListEntry.newBuilder().setValue(intData(8)).setSortKey(8000).setId(8))
+                    .addFetchRanges(
+                        SortedListRange.newBuilder().setStart(beginning).setLimit(end)));
+
+    Mockito.when(mockWindmill.getStateData(COMPUTATION, expectedRequest.build()))
+        .thenReturn(response.build());
+
+    Iterable<TimestampedValue<Integer>> results = future.get();
+    Mockito.verify(mockWindmill).getStateData(COMPUTATION, expectedRequest.build());
+    for (TimestampedValue<Integer> unused : results) {
+      // Iterate over the results to force loading all the pages.
+    }
+    Mockito.verifyNoMoreInteractions(mockWindmill);
+
+    assertThat(
+        results,
+        Matchers.contains(
+            TimestampedValue.of(5, Instant.ofEpochMilli(5)),
+            TimestampedValue.of(6, Instant.ofEpochMilli(6)),
+            TimestampedValue.of(7, Instant.ofEpochMilli(7)),
+            TimestampedValue.of(8, Instant.ofEpochMilli(8))));
+    assertNoReader(future);
+  }
+
+  @Test
+  public void testReadSortedListRanges() throws Exception {
+    Future<Iterable<TimestampedValue<Integer>>> future1 =
+        underTest.orderedListFuture(Range.closedOpen(0L, 5L), STATE_KEY_1, STATE_FAMILY, INT_CODER);
+    Future<Iterable<TimestampedValue<Integer>>> future2 =
+        underTest.orderedListFuture(Range.closedOpen(5L, 6L), STATE_KEY_1, STATE_FAMILY, INT_CODER);
+    Future<Iterable<TimestampedValue<Integer>>> future3 =
+        underTest.orderedListFuture(
+            Range.closedOpen(6L, 10L), STATE_KEY_1, STATE_FAMILY, INT_CODER);
+    Mockito.verifyNoMoreInteractions(mockWindmill);
+
+    // Fetch the entire list.
+    Windmill.KeyedGetDataRequest.Builder expectedRequest =
+        Windmill.KeyedGetDataRequest.newBuilder()
+            .setKey(DATA_KEY)
+            .setShardingKey(SHARDING_KEY)
+            .setWorkToken(WORK_TOKEN)
+            .setMaxBytes(WindmillStateReader.MAX_KEY_BYTES)
+            .addSortedListsToFetch(
+                Windmill.TagSortedListFetchRequest.newBuilder()
+                    .setTag(STATE_KEY_1)
+                    .setStateFamily(STATE_FAMILY)
+                    .addFetchRanges(SortedListRange.newBuilder().setStart(0).setLimit(5))
+                    .setFetchMaxBytes(WindmillStateReader.MAX_BAG_BYTES))
+            .addSortedListsToFetch(
+                Windmill.TagSortedListFetchRequest.newBuilder()
+                    .setTag(STATE_KEY_1)
+                    .setStateFamily(STATE_FAMILY)
+                    .addFetchRanges(SortedListRange.newBuilder().setStart(5).setLimit(6))
+                    .setFetchMaxBytes(WindmillStateReader.MAX_BAG_BYTES))
+            .addSortedListsToFetch(
+                Windmill.TagSortedListFetchRequest.newBuilder()
+                    .setTag(STATE_KEY_1)
+                    .setStateFamily(STATE_FAMILY)
+                    .addFetchRanges(SortedListRange.newBuilder().setStart(6).setLimit(10))
+                    .setFetchMaxBytes(WindmillStateReader.MAX_BAG_BYTES));
+
+    Windmill.KeyedGetDataResponse.Builder response =
+        Windmill.KeyedGetDataResponse.newBuilder()
+            .setKey(DATA_KEY)
+            .addTagSortedLists(
+                Windmill.TagSortedListFetchResponse.newBuilder()
+                    .setTag(STATE_KEY_1)
+                    .setStateFamily(STATE_FAMILY)
+                    .addEntries(
+                        SortedListEntry.newBuilder().setValue(intData(5)).setSortKey(5000).setId(5))
+                    .addFetchRanges(SortedListRange.newBuilder().setStart(0).setLimit(5)))
+            .addTagSortedLists(
+                Windmill.TagSortedListFetchResponse.newBuilder()
+                    .setTag(STATE_KEY_1)
+                    .setStateFamily(STATE_FAMILY)
+                    .addEntries(
+                        SortedListEntry.newBuilder().setValue(intData(6)).setSortKey(6000).setId(5))
+                    .addEntries(
+                        SortedListEntry.newBuilder().setValue(intData(7)).setSortKey(7000).setId(7))
+                    .addFetchRanges(SortedListRange.newBuilder().setStart(5).setLimit(6)))
+            .addTagSortedLists(
+                Windmill.TagSortedListFetchResponse.newBuilder()
+                    .setTag(STATE_KEY_1)
+                    .setStateFamily(STATE_FAMILY)
+                    .addEntries(
+                        SortedListEntry.newBuilder().setValue(intData(8)).setSortKey(8000).setId(8))
+                    .addFetchRanges(SortedListRange.newBuilder().setStart(6).setLimit(10)));
+
+    Mockito.when(mockWindmill.getStateData(COMPUTATION, expectedRequest.build()))
+        .thenReturn(response.build());
+
+    {
+      Iterable<TimestampedValue<Integer>> results = future1.get();
+      Mockito.verify(mockWindmill).getStateData(COMPUTATION, expectedRequest.build());
+      for (TimestampedValue<Integer> unused : results) {
+        // Iterate over the results to force loading all the pages.
+      }
+      Mockito.verifyNoMoreInteractions(mockWindmill);
+      assertThat(results, Matchers.contains(TimestampedValue.of(5, Instant.ofEpochMilli(5))));
+      assertNoReader(future1);
+    }
+
+    {
+      Iterable<TimestampedValue<Integer>> results = future2.get();
+      Mockito.verify(mockWindmill).getStateData(COMPUTATION, expectedRequest.build());
+      for (TimestampedValue<Integer> unused : results) {
+        // Iterate over the results to force loading all the pages.
+      }
+      Mockito.verifyNoMoreInteractions(mockWindmill);
+      assertThat(
+          results,
+          Matchers.contains(
+              TimestampedValue.of(6, Instant.ofEpochMilli(6)),
+              TimestampedValue.of(7, Instant.ofEpochMilli(7))));
+      assertNoReader(future2);
+    }
+
+    {
+      Iterable<TimestampedValue<Integer>> results = future3.get();
+      Mockito.verify(mockWindmill).getStateData(COMPUTATION, expectedRequest.build());
+      for (TimestampedValue<Integer> unused : results) {
+        // Iterate over the results to force loading all the pages.
+      }
+      Mockito.verifyNoMoreInteractions(mockWindmill);
+      assertThat(results, Matchers.contains(TimestampedValue.of(8, Instant.ofEpochMilli(8))));
+      assertNoReader(future3);
+    }
+  }
+
+  @Test
+  public void testReadSortedListWithContinuations() throws Exception {
+    long beginning = SortedListRange.getDefaultInstance().getStart();
+    long end = SortedListRange.getDefaultInstance().getLimit();
+
+    Future<Iterable<TimestampedValue<Integer>>> future =
+        underTest.orderedListFuture(
+            Range.closedOpen(beginning, end), STATE_KEY_1, STATE_FAMILY, INT_CODER);
+
+    Mockito.verifyNoMoreInteractions(mockWindmill);
+
+    Windmill.KeyedGetDataRequest.Builder expectedRequest1 =
+        Windmill.KeyedGetDataRequest.newBuilder()
+            .setKey(DATA_KEY)
+            .setShardingKey(SHARDING_KEY)
+            .setWorkToken(WORK_TOKEN)
+            .setMaxBytes(WindmillStateReader.MAX_KEY_BYTES)
+            .addSortedListsToFetch(
+                Windmill.TagSortedListFetchRequest.newBuilder()
+                    .setTag(STATE_KEY_1)
+                    .setStateFamily(STATE_FAMILY)
+                    .addFetchRanges(SortedListRange.newBuilder().setStart(beginning).setLimit(end))
+                    .setFetchMaxBytes(WindmillStateReader.MAX_BAG_BYTES));
+
+    final ByteString CONT = ByteString.copyFrom("CONTINUATION", Charsets.UTF_8);
+    Windmill.KeyedGetDataResponse.Builder response1 =
+        Windmill.KeyedGetDataResponse.newBuilder()
+            .setKey(DATA_KEY)
+            .addTagSortedLists(
+                Windmill.TagSortedListFetchResponse.newBuilder()
+                    .setTag(STATE_KEY_1)
+                    .setStateFamily(STATE_FAMILY)
+                    .addEntries(
+                        SortedListEntry.newBuilder().setValue(intData(5)).setSortKey(5000).setId(5))
+                    .setContinuationPosition(CONT)
+                    .addFetchRanges(
+                        SortedListRange.newBuilder().setStart(beginning).setLimit(end)));
+
+    Windmill.KeyedGetDataRequest.Builder expectedRequest2 =
+        Windmill.KeyedGetDataRequest.newBuilder()
+            .setKey(DATA_KEY)
+            .setShardingKey(SHARDING_KEY)
+            .setWorkToken(WORK_TOKEN)
+            .setMaxBytes(WindmillStateReader.MAX_KEY_BYTES)
+            .addSortedListsToFetch(
+                Windmill.TagSortedListFetchRequest.newBuilder()
+                    .setTag(STATE_KEY_1)
+                    .setStateFamily(STATE_FAMILY)
+                    .addFetchRanges(SortedListRange.newBuilder().setStart(beginning).setLimit(end))
+                    .setRequestPosition(CONT)
+                    .setFetchMaxBytes(WindmillStateReader.MAX_BAG_BYTES));
+
+    Windmill.KeyedGetDataResponse.Builder response2 =
+        Windmill.KeyedGetDataResponse.newBuilder()
+            .setKey(DATA_KEY)
+            .addTagSortedLists(
+                Windmill.TagSortedListFetchResponse.newBuilder()
+                    .setTag(STATE_KEY_1)
+                    .setStateFamily(STATE_FAMILY)
+                    .addEntries(
+                        SortedListEntry.newBuilder().setValue(intData(6)).setSortKey(6000).setId(5))
+                    .addEntries(
+                        SortedListEntry.newBuilder().setValue(intData(7)).setSortKey(7000).setId(7))
+                    .addEntries(
+                        SortedListEntry.newBuilder().setValue(intData(8)).setSortKey(8000).setId(8))
+                    .addFetchRanges(SortedListRange.newBuilder().setStart(beginning).setLimit(end))
+                    .setRequestPosition(CONT));
+
+    Mockito.when(mockWindmill.getStateData(COMPUTATION, expectedRequest1.build()))
+        .thenReturn(response1.build());
+    Mockito.when(mockWindmill.getStateData(COMPUTATION, expectedRequest2.build()))
+        .thenReturn(response2.build());
+
+    Iterable<TimestampedValue<Integer>> results = future.get();
+    Mockito.verify(mockWindmill).getStateData(COMPUTATION, expectedRequest1.build());
+    for (TimestampedValue<Integer> unused : results) {
+      // Iterate over the results to force loading all the pages.
+    }
+    Mockito.verify(mockWindmill).getStateData(COMPUTATION, expectedRequest2.build());
+    Mockito.verifyNoMoreInteractions(mockWindmill);
+
+    assertThat(
+        results,
+        Matchers.contains(
+            TimestampedValue.of(5, Instant.ofEpochMilli(5)),
+            TimestampedValue.of(6, Instant.ofEpochMilli(6)),
+            TimestampedValue.of(7, Instant.ofEpochMilli(7)),
+            TimestampedValue.of(8, Instant.ofEpochMilli(8))));
+    // NOTE: The future will still contain a reference to the underlying reader.
+  }
+
+  @Test
   public void testReadValue() throws Exception {
     Future<Integer> future = underTest.valueFuture(STATE_KEY_1, STATE_FAMILY, INT_CODER);
     Mockito.verifyNoMoreInteractions(mockWindmill);
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WindmillStateTestUtils.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WindmillStateTestUtils.java
index 17da531..8b7b630 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WindmillStateTestUtils.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WindmillStateTestUtils.java
@@ -25,6 +25,7 @@
 import java.util.HashSet;
 
 /** Static helpers for testing Windmill state. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class WindmillStateTestUtils {
   /**
    * Assert that no field (including compiler-generated fields) within {@code obj} point back to a
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WindmillTimeUtilsTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WindmillTimeUtilsTest.java
index 5f910c3..79a690c 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WindmillTimeUtilsTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WindmillTimeUtilsTest.java
@@ -30,6 +30,7 @@
 
 /** Unit tests for {@link WindmillTimeUtils}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class WindmillTimeUtilsTest {
   @Test
   public void testWindmillToHarnessWatermark() {
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WorkItemStatusClientTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WorkItemStatusClientTest.java
index ca3644b..a857faf 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WorkItemStatusClientTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WorkItemStatusClientTest.java
@@ -83,6 +83,7 @@
 
 /** Tests for {@link WorkItemStatusClient}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class WorkItemStatusClientTest {
 
   private static final String PROJECT_ID = "ProjectId";
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSourcesTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSourcesTest.java
index 593899c..14510db 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSourcesTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSourcesTest.java
@@ -124,6 +124,7 @@
 
 /** Tests for {@link WorkerCustomSources}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class WorkerCustomSourcesTest {
   @Rule public ExpectedException expectedException = ExpectedException.none();
   @Rule public ExpectedLogs logged = ExpectedLogs.none(WorkerCustomSources.class);
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/apiary/ApiaryTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/apiary/ApiaryTest.java
index 42ac390..c6ee204 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/apiary/ApiaryTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/apiary/ApiaryTest.java
@@ -29,6 +29,7 @@
 
 /** Tests for {@link Apiary}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings({"keyfor", "nullness"}) // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ApiaryTest {
   @Test
   public void testNullSafeList() {
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/counters/DistributionCounterUpdateAggregatorTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/counters/DistributionCounterUpdateAggregatorTest.java
index 8dac8a7..3ba9e3f 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/counters/DistributionCounterUpdateAggregatorTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/counters/DistributionCounterUpdateAggregatorTest.java
@@ -31,6 +31,7 @@
 import org.junit.Before;
 import org.junit.Test;
 
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DistributionCounterUpdateAggregatorTest {
 
   private List<CounterUpdate> counterUpdates;
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/counters/MeanCounterUpdateAggregatorTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/counters/MeanCounterUpdateAggregatorTest.java
index 9ea7a31..64e8e87 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/counters/MeanCounterUpdateAggregatorTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/counters/MeanCounterUpdateAggregatorTest.java
@@ -31,6 +31,7 @@
 import org.junit.Before;
 import org.junit.Test;
 
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class MeanCounterUpdateAggregatorTest {
 
   private List<CounterUpdate> counterUpdates;
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/counters/SumCounterUpdateAggregatorTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/counters/SumCounterUpdateAggregatorTest.java
index e30354f..bc4f252 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/counters/SumCounterUpdateAggregatorTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/counters/SumCounterUpdateAggregatorTest.java
@@ -30,6 +30,7 @@
 import org.junit.Before;
 import org.junit.Test;
 
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SumCounterUpdateAggregatorTest {
   private List<CounterUpdate> counterUpdates;
   private SumCounterUpdateAggregator aggregator;
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/fn/BeamFnControlServiceTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/fn/BeamFnControlServiceTest.java
index 76fb6d0..715305e 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/fn/BeamFnControlServiceTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/fn/BeamFnControlServiceTest.java
@@ -50,6 +50,7 @@
 
 /** Tests for {@link BeamFnControlService}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BeamFnControlServiceTest {
   @Rule
   public GrpcCleanupRule grpcCleanupRule = new GrpcCleanupRule().setTimeout(10, TimeUnit.SECONDS);
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/fn/control/BeamFnMapTaskExecutorTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/fn/control/BeamFnMapTaskExecutorTest.java
index 8726478..ecf8f58 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/fn/control/BeamFnMapTaskExecutorTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/fn/control/BeamFnMapTaskExecutorTest.java
@@ -72,6 +72,7 @@
 
 /** Tests for {@link BeamFnMapTaskExecutor}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BeamFnMapTaskExecutorTest {
 
   @Mock private OperationContext mockContext;
@@ -218,35 +219,31 @@
   private DataflowStepContext generateDataflowStepContext(String valuesPrefix) {
     NameContext nc =
         new NameContext() {
-          @Nullable
           @Override
-          public String stageName() {
+          public @Nullable String stageName() {
             return valuesPrefix + "Stage";
           }
 
-          @Nullable
           @Override
-          public String originalName() {
+          public @Nullable String originalName() {
             return valuesPrefix + "OriginalName";
           }
 
-          @Nullable
           @Override
-          public String systemName() {
+          public @Nullable String systemName() {
             return valuesPrefix + "SystemName";
           }
 
-          @Nullable
           @Override
-          public String userName() {
+          public @Nullable String userName() {
             return valuesPrefix + "UserName";
           }
         };
     DataflowStepContext dsc =
         new DataflowStepContext(nc) {
-          @Nullable
           @Override
-          public <W extends BoundedWindow> TimerData getNextFiredTimer(Coder<W> windowCoder) {
+          public <W extends BoundedWindow> @Nullable TimerData getNextFiredTimer(
+              Coder<W> windowCoder) {
             return null;
           }
 
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/fn/control/DataflowSideInputHandlerFactoryTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/fn/control/DataflowSideInputHandlerFactoryTest.java
index 9bd796e..f555644 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/fn/control/DataflowSideInputHandlerFactoryTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/fn/control/DataflowSideInputHandlerFactoryTest.java
@@ -50,6 +50,7 @@
 
 /** Test for {@link DataflowSideInputHandlerFactory} */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public final class DataflowSideInputHandlerFactoryTest {
 
   private static final String TRANSFORM_ID = "transformId";
@@ -73,9 +74,8 @@
   public void setUp() {
     fakeSideInputReader =
         new SideInputReader() {
-          @Nullable
           @Override
-          public <T> T get(PCollectionView<T> view, BoundedWindow window) {
+          public <T> @Nullable T get(PCollectionView<T> view, BoundedWindow window) {
             assertEquals(GlobalWindow.INSTANCE, window);
             assertEquals(SIDE_INPUT_NAME, view.getTagInternal().getId());
 
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/fn/control/ElementCountMonitoringInfoToCounterUpdateTransformerTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/fn/control/ElementCountMonitoringInfoToCounterUpdateTransformerTest.java
index c9f09b4..a160f98 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/fn/control/ElementCountMonitoringInfoToCounterUpdateTransformerTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/fn/control/ElementCountMonitoringInfoToCounterUpdateTransformerTest.java
@@ -41,6 +41,7 @@
 import org.mockito.Mock;
 import org.mockito.MockitoAnnotations;
 
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ElementCountMonitoringInfoToCounterUpdateTransformerTest {
 
   @Rule public final ExpectedException exception = ExpectedException.none();
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/fn/control/ExecutionTimeMonitoringInfoToCounterUpdateTransformerTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/fn/control/ExecutionTimeMonitoringInfoToCounterUpdateTransformerTest.java
index f99cf5a..f413f19 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/fn/control/ExecutionTimeMonitoringInfoToCounterUpdateTransformerTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/fn/control/ExecutionTimeMonitoringInfoToCounterUpdateTransformerTest.java
@@ -44,6 +44,7 @@
 import org.mockito.Mock;
 import org.mockito.MockitoAnnotations;
 
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ExecutionTimeMonitoringInfoToCounterUpdateTransformerTest {
 
   @Rule public final ExpectedException exception = ExpectedException.none();
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/fn/control/FnApiMonitoringInfoToCounterUpdateTransformerTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/fn/control/FnApiMonitoringInfoToCounterUpdateTransformerTest.java
index 70ba274..46c6578 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/fn/control/FnApiMonitoringInfoToCounterUpdateTransformerTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/fn/control/FnApiMonitoringInfoToCounterUpdateTransformerTest.java
@@ -33,6 +33,7 @@
 import org.mockito.Mock;
 import org.mockito.MockitoAnnotations;
 
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class FnApiMonitoringInfoToCounterUpdateTransformerTest {
 
   @Mock private UserMonitoringInfoToCounterUpdateTransformer mockTransformer2;
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/fn/control/MeanByteCountMonitoringInfoToCounterUpdateTransformerTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/fn/control/MeanByteCountMonitoringInfoToCounterUpdateTransformerTest.java
index 796e513..649548e 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/fn/control/MeanByteCountMonitoringInfoToCounterUpdateTransformerTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/fn/control/MeanByteCountMonitoringInfoToCounterUpdateTransformerTest.java
@@ -42,6 +42,7 @@
 import org.mockito.Mock;
 import org.mockito.MockitoAnnotations;
 
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class MeanByteCountMonitoringInfoToCounterUpdateTransformerTest {
 
   @Rule public final ExpectedException exception = ExpectedException.none();
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/fn/control/RegisterAndProcessBundleOperationTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/fn/control/RegisterAndProcessBundleOperationTest.java
index fdcb3af..96a1284 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/fn/control/RegisterAndProcessBundleOperationTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/fn/control/RegisterAndProcessBundleOperationTest.java
@@ -96,7 +96,10 @@
 
 /** Tests for {@link RegisterAndProcessBundleOperation}. */
 @RunWith(JUnit4.class)
-@SuppressWarnings("FutureReturnValueIgnored")
+@SuppressWarnings({
+  "FutureReturnValueIgnored",
+  "nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
+})
 public class RegisterAndProcessBundleOperationTest {
   private static final BeamFnApi.RegisterRequest REGISTER_REQUEST =
       BeamFnApi.RegisterRequest.newBuilder()
@@ -470,9 +473,8 @@
 
     SideInputReader fakeSideInputReader =
         new SideInputReader() {
-          @Nullable
           @Override
-          public <T> T get(PCollectionView<T> view, BoundedWindow window) {
+          public <T> @Nullable T get(PCollectionView<T> view, BoundedWindow window) {
             assertEquals(GlobalWindow.INSTANCE, window);
             assertEquals("testSideInputId", view.getTagInternal().getId());
             return (T)
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/fn/control/SingularProcessBundleProgressTrackerTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/fn/control/SingularProcessBundleProgressTrackerTest.java
index 2af59c6..3a61797 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/fn/control/SingularProcessBundleProgressTrackerTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/fn/control/SingularProcessBundleProgressTrackerTest.java
@@ -37,6 +37,7 @@
 
 /** Tests for {@link BeamFnMapTaskExecutor.SingularProcessBundleProgressTracker}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SingularProcessBundleProgressTrackerTest {
 
   private static class TestProgress implements Progress {
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/fn/control/UserDistributionMonitoringInfoToCounterUpdateTransformerTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/fn/control/UserDistributionMonitoringInfoToCounterUpdateTransformerTest.java
index 4197aab..ea367c8 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/fn/control/UserDistributionMonitoringInfoToCounterUpdateTransformerTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/fn/control/UserDistributionMonitoringInfoToCounterUpdateTransformerTest.java
@@ -44,6 +44,7 @@
 import org.mockito.Mock;
 import org.mockito.MockitoAnnotations;
 
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class UserDistributionMonitoringInfoToCounterUpdateTransformerTest {
 
   @Rule public final ExpectedException exception = ExpectedException.none();
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/fn/control/UserMonitoringInfoToCounterUpdateTransformerTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/fn/control/UserMonitoringInfoToCounterUpdateTransformerTest.java
index b76bfae..659a47b 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/fn/control/UserMonitoringInfoToCounterUpdateTransformerTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/fn/control/UserMonitoringInfoToCounterUpdateTransformerTest.java
@@ -43,6 +43,7 @@
 import org.mockito.Mock;
 import org.mockito.MockitoAnnotations;
 
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class UserMonitoringInfoToCounterUpdateTransformerTest {
 
   @Rule public final ExpectedException exception = ExpectedException.none();
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/fn/data/BeamFnDataGrpcServiceTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/fn/data/BeamFnDataGrpcServiceTest.java
index 1d680c8..fde88aa 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/fn/data/BeamFnDataGrpcServiceTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/fn/data/BeamFnDataGrpcServiceTest.java
@@ -75,7 +75,10 @@
 
 /** Tests for {@link BeamFnDataGrpcService}. */
 @RunWith(JUnit4.class)
-@SuppressWarnings("FutureReturnValueIgnored")
+@SuppressWarnings({
+  "FutureReturnValueIgnored",
+  "nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
+})
 public class BeamFnDataGrpcServiceTest {
   private static final String TRANSFORM_ID = "888";
   private static final Coder<WindowedValue<String>> CODER =
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/fn/data/RemoteGrpcPortReadOperationTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/fn/data/RemoteGrpcPortReadOperationTest.java
index cd40fd3..55100ed 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/fn/data/RemoteGrpcPortReadOperationTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/fn/data/RemoteGrpcPortReadOperationTest.java
@@ -56,6 +56,7 @@
 
 /** Tests for {@link RemoteGrpcPortReadOperation}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class RemoteGrpcPortReadOperationTest {
   private static final Coder<WindowedValue<String>> CODER =
       WindowedValue.getFullCoder(StringUtf8Coder.of(), GlobalWindow.Coder.INSTANCE);
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/fn/data/RemoteGrpcPortWriteOperationTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/fn/data/RemoteGrpcPortWriteOperationTest.java
index f2c3e7e..4c89930 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/fn/data/RemoteGrpcPortWriteOperationTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/fn/data/RemoteGrpcPortWriteOperationTest.java
@@ -50,6 +50,7 @@
 
 /** Tests for {@link RemoteGrpcPortWriteOperation}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class RemoteGrpcPortWriteOperationTest {
   private static final Coder<WindowedValue<String>> CODER =
       WindowedValue.getFullCoder(StringUtf8Coder.of(), GlobalWindow.Coder.INSTANCE);
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/fn/logging/BeamFnLoggingServiceTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/fn/logging/BeamFnLoggingServiceTest.java
index 114ded8..9fd8519 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/fn/logging/BeamFnLoggingServiceTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/fn/logging/BeamFnLoggingServiceTest.java
@@ -52,6 +52,7 @@
 
 /** Tests for {@link BeamFnLoggingService}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BeamFnLoggingServiceTest {
   private Server server;
 
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/fn/stream/ServerStreamObserverFactoryTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/fn/stream/ServerStreamObserverFactoryTest.java
index e41fd69..172690f 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/fn/stream/ServerStreamObserverFactoryTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/fn/stream/ServerStreamObserverFactoryTest.java
@@ -35,6 +35,7 @@
 
 /** Tests for {@link ServerStreamObserverFactory}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ServerStreamObserverFactoryTest {
   @Mock private CallStreamObserver<String> mockResponseObserver;
 
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/graph/CloneAmbiguousFlattensFunctionTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/graph/CloneAmbiguousFlattensFunctionTest.java
index 9175cd2..405d7d5 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/graph/CloneAmbiguousFlattensFunctionTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/graph/CloneAmbiguousFlattensFunctionTest.java
@@ -43,6 +43,7 @@
 
 /** Tests for {@link CloneAmbiguousFlattensFunction}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public final class CloneAmbiguousFlattensFunctionTest {
 
   /** A node that stores nothing. Used for testing with nodes that have no ExecutionLocation. */
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/graph/CreateRegisterFnOperationFunctionTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/graph/CreateRegisterFnOperationFunctionTest.java
index 9da1af1..8d56cec 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/graph/CreateRegisterFnOperationFunctionTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/graph/CreateRegisterFnOperationFunctionTest.java
@@ -58,6 +58,7 @@
 
 /** Tests for {@link CreateRegisterFnOperationFunction}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class CreateRegisterFnOperationFunctionTest {
 
   @Mock private Supplier<Node> portSupplier;
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/graph/LengthPrefixUnknownCodersTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/graph/LengthPrefixUnknownCodersTest.java
index 575e754..c39f169 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/graph/LengthPrefixUnknownCodersTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/graph/LengthPrefixUnknownCodersTest.java
@@ -73,6 +73,7 @@
 
 /** Tests for {@link LengthPrefixUnknownCoders}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class LengthPrefixUnknownCodersTest {
   private static final Coder<WindowedValue<KV<String, Integer>>> windowedValueCoder =
       WindowedValue.getFullCoder(
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/graph/NetworksTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/graph/NetworksTest.java
index e04d434..3310c50 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/graph/NetworksTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/graph/NetworksTest.java
@@ -46,6 +46,7 @@
 
 /** Tests for {@link Networks}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class NetworksTest {
   @Test
   public void testTopologicalSortWithEmptyNetwork() {
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/graph/NodesTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/graph/NodesTest.java
index d87ec98..869d489 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/graph/NodesTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/graph/NodesTest.java
@@ -55,6 +55,7 @@
 
 /** Tests for {@link Nodes}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class NodesTest {
   private static final String PCOLLECTION_ID = "fakeId";
 
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/graph/ReplacePgbkWithPrecombineFunctionTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/graph/ReplacePgbkWithPrecombineFunctionTest.java
index e20082c..dd3145a 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/graph/ReplacePgbkWithPrecombineFunctionTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/graph/ReplacePgbkWithPrecombineFunctionTest.java
@@ -46,6 +46,7 @@
 
 /** Tests for {@link ReplacePgbkWithPrecombineFunction}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public final class ReplacePgbkWithPrecombineFunctionTest {
 
   @Test
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/logging/DataflowWorkerLoggingHandlerTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/logging/DataflowWorkerLoggingHandlerTest.java
index 84adfcd..aa8e09f 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/logging/DataflowWorkerLoggingHandlerTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/logging/DataflowWorkerLoggingHandlerTest.java
@@ -47,6 +47,7 @@
 
 /** Unit tests for {@link DataflowWorkerLoggingHandler}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DataflowWorkerLoggingHandlerTest {
   @Rule public TestRule restoreMDC = new RestoreDataflowLoggingMDC();
 
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/logging/DataflowWorkerLoggingInitializerTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/logging/DataflowWorkerLoggingInitializerTest.java
index 4d04277..2192bbe 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/logging/DataflowWorkerLoggingInitializerTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/logging/DataflowWorkerLoggingInitializerTest.java
@@ -65,6 +65,7 @@
  * not safe to assert on log counts or whether the retrieved log collection is empty.
  */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DataflowWorkerLoggingInitializerTest {
   @Rule public TemporaryFolder logFolder = new TemporaryFolder();
 
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/logging/JulHandlerPrintStreamAdapterFactoryTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/logging/JulHandlerPrintStreamAdapterFactoryTest.java
index eff5f34..782d1a3 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/logging/JulHandlerPrintStreamAdapterFactoryTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/logging/JulHandlerPrintStreamAdapterFactoryTest.java
@@ -37,6 +37,7 @@
 
 /** Tests for {@link JulHandlerPrintStreamAdapterFactory}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class JulHandlerPrintStreamAdapterFactoryTest {
   private static final String LOGGER_NAME = "test";
 
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/profiler/ScopedProfilerTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/profiler/ScopedProfilerTest.java
index c2aefc7..deb97d0 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/profiler/ScopedProfilerTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/profiler/ScopedProfilerTest.java
@@ -31,6 +31,7 @@
 
 /** Tests for {@link ScopedProfiler}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ScopedProfilerTest {
 
   @Test
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/status/DebugCaptureTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/status/DebugCaptureTest.java
index 767cda6..6c5d065 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/status/DebugCaptureTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/status/DebugCaptureTest.java
@@ -42,6 +42,7 @@
 
 /** Tests for {@link DebugCapture}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DebugCaptureTest {
   private static final String PROJECT_ID = "some-project";
   private static final String REGION = "some-region";
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/status/WorkerStatusPagesTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/status/WorkerStatusPagesTest.java
index 8373bea..0ff3e24 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/status/WorkerStatusPagesTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/status/WorkerStatusPagesTest.java
@@ -34,6 +34,7 @@
 
 /** Tests for {@link WorkerStatusPages}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class WorkerStatusPagesTest {
 
   private final Server server = new Server();
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/testing/GenericJsonMatcherTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/testing/GenericJsonMatcherTest.java
index 146fb17..966a967 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/testing/GenericJsonMatcherTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/testing/GenericJsonMatcherTest.java
@@ -30,6 +30,7 @@
 
 /** Tests for {@link GenericJsonMatcher}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class GenericJsonMatcherTest {
 
   @Test
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/testing/RestoreDataflowLoggingMDC.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/testing/RestoreDataflowLoggingMDC.java
index eed8ca9..c5561d5 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/testing/RestoreDataflowLoggingMDC.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/testing/RestoreDataflowLoggingMDC.java
@@ -21,6 +21,7 @@
 import org.junit.rules.ExternalResource;
 
 /** Saves, clears and restores the current thread-local logging parameters for tests. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class RestoreDataflowLoggingMDC extends ExternalResource {
   private String previousJobId;
   private String previousStageName;
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/testing/TestCountingSource.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/testing/TestCountingSource.java
index 6771e9d..5b5b0c3 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/testing/TestCountingSource.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/testing/TestCountingSource.java
@@ -43,6 +43,7 @@
  * The reader will occasionally return false from {@code advance}, in order to simulate a source
  * where not all the data is available immediately.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class TestCountingSource
     extends UnboundedSource<KV<Integer, Integer>, TestCountingSource.CounterMark> {
   private static final Logger LOG = LoggerFactory.getLogger(TestCountingSource.class);
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/CounterHamcrestMatchers.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/CounterHamcrestMatchers.java
index 2cb0833..429b20f 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/CounterHamcrestMatchers.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/CounterHamcrestMatchers.java
@@ -36,6 +36,7 @@
 import org.hamcrest.TypeSafeMatcher;
 
 /** Matchers for {@link Counter} and {@link CounterUpdate}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public final class CounterHamcrestMatchers {
   private CounterHamcrestMatchers() {}
 
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/GroupAlsoByWindowProperties.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/GroupAlsoByWindowProperties.java
index 9badb6d..d6dc0ba 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/GroupAlsoByWindowProperties.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/GroupAlsoByWindowProperties.java
@@ -63,6 +63,7 @@
  * which the implementation is applicable. For example, some {@code GroupAlsoByWindows} may not
  * support merging windows.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class GroupAlsoByWindowProperties {
 
   /**
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/MemoryMonitorTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/MemoryMonitorTest.java
index a7f3d71..4d3c787 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/MemoryMonitorTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/MemoryMonitorTest.java
@@ -39,6 +39,7 @@
  * Test the memory monitor will block threads when the server is in a (faked) GC thrashing state.
  */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class MemoryMonitorTest {
 
   @Rule public TemporaryFolder tempFolder = new TemporaryFolder();
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/TaggedReiteratorListTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/TaggedReiteratorListTest.java
index 04f3344..aa5c3c4 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/TaggedReiteratorListTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/TaggedReiteratorListTest.java
@@ -29,6 +29,7 @@
 
 /** Tests for {@link TaggedReiteratorList}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings({"keyfor", "nullness"}) // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class TaggedReiteratorListTest {
 
   @Test
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/BatchingShuffleEntryReaderTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/BatchingShuffleEntryReaderTest.java
index 9ee670b..7b3ddcf 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/BatchingShuffleEntryReaderTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/BatchingShuffleEntryReaderTest.java
@@ -39,6 +39,7 @@
 
 /** Unit tests for {@link BatchingShuffleEntryReader}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public final class BatchingShuffleEntryReaderTest {
   private static final byte[] KEY = {0xA};
   private static final byte[] SKEY = {0xB};
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ExecutorTestUtils.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ExecutorTestUtils.java
index bc9a8b8..9f26575 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ExecutorTestUtils.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ExecutorTestUtils.java
@@ -32,7 +32,11 @@
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
 
 /** Utilities for tests. */
-@SuppressWarnings({"rawtypes", "unchecked"})
+@SuppressWarnings({
+  "rawtypes",
+  "unchecked",
+  "nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
+})
 public class ExecutorTestUtils {
   // Do not instantiate.
   private ExecutorTestUtils() {}
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/GroupingShuffleEntryIteratorTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/GroupingShuffleEntryIteratorTest.java
index 22e02db..993c5f2 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/GroupingShuffleEntryIteratorTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/GroupingShuffleEntryIteratorTest.java
@@ -56,6 +56,7 @@
 
 /** Tests for {@link GroupingShuffleEntryIterator}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class GroupingShuffleEntryIteratorTest {
   private static final ByteArrayShufflePosition START_POSITION =
       ByteArrayShufflePosition.of("aaa".getBytes(StandardCharsets.UTF_8));
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/GroupingTablesTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/GroupingTablesTest.java
index 9de05e8..5f606f7 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/GroupingTablesTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/GroupingTablesTest.java
@@ -48,6 +48,7 @@
 
 /** Unit tests for {@link GroupingTables}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class GroupingTablesTest {
 
   @Test
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/MapTaskExecutorTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/MapTaskExecutorTest.java
index e428bee..15ca5bc 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/MapTaskExecutorTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/MapTaskExecutorTest.java
@@ -75,6 +75,7 @@
 
 /** Tests for {@link MapTaskExecutor}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class MapTaskExecutorTest {
 
   private static final String COUNTER_PREFIX = "test-";
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/OutputObjectAndByteCounterTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/OutputObjectAndByteCounterTest.java
index 67370bd..ec85e6e 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/OutputObjectAndByteCounterTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/OutputObjectAndByteCounterTest.java
@@ -44,6 +44,7 @@
 
 /** Tests for {@link OutputObjectAndByteCounter}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class OutputObjectAndByteCounterTest {
 
   private final CounterSet counterSet = new CounterSet();
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ReadOperationTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ReadOperationTest.java
index db360d3..2453394 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ReadOperationTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ReadOperationTest.java
@@ -72,6 +72,7 @@
 
 /** Tests for ReadOperation. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ReadOperationTest {
 
   private static final String COUNTER_PREFIX = "test-";
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ShuffleEntryTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ShuffleEntryTest.java
index 0a949c9..0c3277a 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ShuffleEntryTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/ShuffleEntryTest.java
@@ -29,6 +29,7 @@
 
 /** Unit tests for {@link ShuffleEntry}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ShuffleEntryTest {
   private static final byte[] KEY = {0xA};
   private static final byte[] SKEY = {0xB};
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/StubbedExecutor.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/StubbedExecutor.java
index 7555f2b..a28a6a1 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/StubbedExecutor.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/StubbedExecutor.java
@@ -41,6 +41,7 @@
  * get the executor and use it to schedule the initial task, then use {@link runNextRunnable()} to
  * run the initial task, then use {@link runNextRunnable()} to run the second task, etc.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class StubbedExecutor {
   private static final Logger LOG = LoggerFactory.getLogger(StubbedExecutor.class);
 
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/TestOutputReceiver.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/TestOutputReceiver.java
index afd86aa..9d9ffa1 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/TestOutputReceiver.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/TestOutputReceiver.java
@@ -30,6 +30,7 @@
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
 
 /** An OutputReceiver that allows the output elements to be retrieved. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class TestOutputReceiver extends OutputReceiver {
   private static final String OBJECT_COUNTER_NAME = "-ObjectCount";
   private static final String MEAN_BYTE_COUNTER_NAME = "-MeanByteCount";
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/WorkProgressUpdaterTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/WorkProgressUpdaterTest.java
index 3b060c8..0010804 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/WorkProgressUpdaterTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/common/worker/WorkProgressUpdaterTest.java
@@ -35,6 +35,7 @@
 
 /** Unit tests for {@link WorkProgressUpdater}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class WorkProgressUpdaterTest {
   /**
    * WorkProgressUpdater relies on subclasses to implement some of its functionality, particularly
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/GrpcWindmillServerTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/GrpcWindmillServerTest.java
index f0d9f82..94d160c 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/GrpcWindmillServerTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/GrpcWindmillServerTest.java
@@ -80,6 +80,7 @@
 
 /** Unit tests for {@link GrpcWindmillServer}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class GrpcWindmillServerTest {
   private static final Logger LOG = LoggerFactory.getLogger(GrpcWindmillServerTest.class);
 
diff --git a/runners/google-cloud-dataflow-java/worker/windmill/src/main/proto/windmill.proto b/runners/google-cloud-dataflow-java/worker/windmill/src/main/proto/windmill.proto
index b90eaa7..b0e8bda 100644
--- a/runners/google-cloud-dataflow-java/worker/windmill/src/main/proto/windmill.proto
+++ b/runners/google-cloud-dataflow-java/worker/windmill/src/main/proto/windmill.proto
@@ -131,7 +131,7 @@
 message TagSortedListFetchRequest {
   optional bytes tag = 1;
   optional string state_family = 2;
-  optional SortedListRange fetch_range = 3;
+  repeated SortedListRange fetch_ranges = 3;
 
   // Sets a limit on the maximum response value bytes
   optional int64 fetch_max_bytes = 5 [default = 0x7fffffffffffffff];
@@ -146,7 +146,11 @@
   optional string state_family = 2;
   repeated SortedListEntry entries = 3;
   optional bytes continuation_position = 4;
-}
+  // Fetch ranges copied from request.
+  repeated SortedListRange fetch_ranges = 5;
+  // Request position copied from request.
+  optional bytes request_position = 6;
+  }
 
 message TagSortedListUpdateRequest {
   optional bytes tag = 1;
@@ -253,6 +257,7 @@
   optional fixed64 sharding_key = 6;
   repeated TagValue values_to_fetch = 3;
   repeated TagBag bags_to_fetch = 8;
+  // Must be at most one sorted_list_to_fetch for a given state family and tag.
   repeated TagSortedListFetchRequest sorted_lists_to_fetch = 9;
   repeated WatermarkHold watermark_holds_to_fetch = 5;
 
@@ -282,6 +287,7 @@
   optional bool failed = 2;
   repeated TagValue values = 3;
   repeated TagBag bags = 6;
+  // There is one TagSortedListFetchResponse per state-family, tag pair.
   repeated TagSortedListFetchResponse tag_sorted_lists = 8;
   repeated WatermarkHold watermark_holds = 5;
 
diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/GrpcContextHeaderAccessorProvider.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/GrpcContextHeaderAccessorProvider.java
index 4c7899c..c80bb5a 100644
--- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/GrpcContextHeaderAccessorProvider.java
+++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/GrpcContextHeaderAccessorProvider.java
@@ -30,6 +30,7 @@
  * A HeaderAccessorProvider which intercept the header in a GRPC request and expose the relevant
  * fields.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class GrpcContextHeaderAccessorProvider {
 
   private static final Key<String> WORKER_ID_KEY =
diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/GrpcFnServer.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/GrpcFnServer.java
index 8d6aff1..a56c648 100644
--- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/GrpcFnServer.java
+++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/GrpcFnServer.java
@@ -31,6 +31,7 @@
  * A {@link Server gRPC Server} which manages a single {@link FnService}. The lifetime of the
  * service is bound to the {@link GrpcFnServer}.
  */
+@SuppressWarnings({"nullness", "keyfor"}) // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class GrpcFnServer<ServiceT extends FnService> implements AutoCloseable {
   /**
    * Create a {@link GrpcFnServer} for the provided {@link FnService} running on an arbitrary port.
diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/artifact/ArtifactRetrievalService.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/artifact/ArtifactRetrievalService.java
index 4ba4cea..bcd6c53 100644
--- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/artifact/ArtifactRetrievalService.java
+++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/artifact/ArtifactRetrievalService.java
@@ -36,6 +36,7 @@
 import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.stub.StreamObserver;
 
 /** An {@link ArtifactRetrievalService} that uses {@link FileSystems} as its backing storage. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ArtifactRetrievalService
     extends ArtifactRetrievalServiceGrpc.ArtifactRetrievalServiceImplBase implements FnService {
 
diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/artifact/ArtifactStagingService.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/artifact/ArtifactStagingService.java
index 4655738..8066c4c 100644
--- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/artifact/ArtifactStagingService.java
+++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/artifact/ArtifactStagingService.java
@@ -64,6 +64,7 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ArtifactStagingService
     extends ArtifactStagingServiceGrpc.ArtifactStagingServiceImplBase implements FnService {
 
diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/DefaultJobBundleFactory.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/DefaultJobBundleFactory.java
index 2663146..adce5d4 100644
--- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/DefaultJobBundleFactory.java
+++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/DefaultJobBundleFactory.java
@@ -91,6 +91,7 @@
  * is called for a stage.
  */
 @ThreadSafe
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DefaultJobBundleFactory implements JobBundleFactory {
   private static final Logger LOG = LoggerFactory.getLogger(DefaultJobBundleFactory.class);
   private static final IdGenerator factoryIdGenerator = IdGenerators.incrementingLongs();
diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/FnApiControlClient.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/FnApiControlClient.java
index 1694cb3..451cc2b 100644
--- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/FnApiControlClient.java
+++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/FnApiControlClient.java
@@ -46,6 +46,7 @@
  *
  * <p>This low-level client is responsible only for correlating requests with responses.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class FnApiControlClient implements Closeable, InstructionRequestHandler {
   private static final Logger LOG = LoggerFactory.getLogger(FnApiControlClient.class);
 
diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/ProcessBundleDescriptors.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/ProcessBundleDescriptors.java
index e76c130..c4a5fdb 100644
--- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/ProcessBundleDescriptors.java
+++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/ProcessBundleDescriptors.java
@@ -37,7 +37,6 @@
 import org.apache.beam.model.pipeline.v1.RunnerApi.ExecutableStagePayload.WireCoderSetting;
 import org.apache.beam.model.pipeline.v1.RunnerApi.PCollection;
 import org.apache.beam.model.pipeline.v1.RunnerApi.PTransform;
-import org.apache.beam.runners.core.construction.ModelCoders;
 import org.apache.beam.runners.core.construction.RehydratedComponents;
 import org.apache.beam.runners.core.construction.Timer;
 import org.apache.beam.runners.core.construction.graph.ExecutableStage;
@@ -59,7 +58,6 @@
 import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder;
 import org.apache.beam.sdk.values.KV;
 import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.InvalidProtocolBufferException;
-import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableTable;
@@ -68,6 +66,7 @@
 
 /** Utility methods for creating {@link ProcessBundleDescriptor} instances. */
 // TODO: Rename to ExecutableStages?
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ProcessBundleDescriptors {
 
   /**
@@ -141,9 +140,7 @@
 
     Map<String, Map<String, TimerSpec>> timerSpecs = forTimerSpecs(stage, components);
 
-    if (bagUserStateSpecs.size() > 0 || timerSpecs.size() > 0) {
-      lengthPrefixKeyCoder(stage.getInputPCollection().getId(), components);
-    }
+    lengthPrefixAnyInputCoder(stage.getInputPCollection().getId(), components);
 
     // Copy data from components to ProcessBundleDescriptor.
     ProcessBundleDescriptor.Builder bundleDescriptorBuilder =
@@ -174,26 +171,18 @@
   }
 
   /**
-   * Patches the input coder of a stateful transform to ensure that the byte representation of a key
-   * used to partition the input element at the Runner, matches the key byte representation received
-   * for state requests and timers from the SDK Harness. Stateful transforms always have a KvCoder
-   * as input.
+   * Patches the input coder of the transform to ensure that the byte representation of input used
+   * at the Runner, matches the byte representation received from the SDK Harness.
    */
-  private static void lengthPrefixKeyCoder(
-      String inputColId, Components.Builder componentsBuilder) {
-    RunnerApi.PCollection pcollection = componentsBuilder.getPcollectionsOrThrow(inputColId);
-    RunnerApi.Coder kvCoder = componentsBuilder.getCodersOrThrow(pcollection.getCoderId());
-    Preconditions.checkState(
-        ModelCoders.KV_CODER_URN.equals(kvCoder.getSpec().getUrn()),
-        "Stateful executable stages must use a KV coder, but is: %s",
-        kvCoder.getSpec().getUrn());
-    String keyCoderId = ModelCoders.getKvCoderComponents(kvCoder).keyCoderId();
-    // Retain the original coder, but wrap in LengthPrefixCoder
-    String newKeyCoderId =
-        LengthPrefixUnknownCoders.addLengthPrefixedCoder(keyCoderId, componentsBuilder, false);
-    // Replace old key coder with LengthPrefixCoder<old_key_coder>
-    kvCoder = kvCoder.toBuilder().setComponentCoderIds(0, newKeyCoderId).build();
-    componentsBuilder.putCoders(pcollection.getCoderId(), kvCoder);
+  private static void lengthPrefixAnyInputCoder(
+      String inputPCollectionId, Components.Builder componentsBuilder) {
+    RunnerApi.PCollection pcollection =
+        componentsBuilder.getPcollectionsOrThrow(inputPCollectionId);
+    String newInputCoderId =
+        LengthPrefixUnknownCoders.addLengthPrefixedCoder(
+            pcollection.getCoderId(), componentsBuilder, false);
+    componentsBuilder.putPcollections(
+        inputPCollectionId, pcollection.toBuilder().setCoderId(newInputCoderId).build());
   }
 
   private static Map<String, Coder<WindowedValue<?>>> addStageOutputs(
diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/ReferenceCountingExecutableStageContextFactory.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/ReferenceCountingExecutableStageContextFactory.java
index e39cced..10b0698 100644
--- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/ReferenceCountingExecutableStageContextFactory.java
+++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/ReferenceCountingExecutableStageContextFactory.java
@@ -42,6 +42,7 @@
  * {@link ExecutableStageContext.Factory} which counts ExecutableStageContext reference for book
  * keeping.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ReferenceCountingExecutableStageContextFactory
     implements ExecutableStageContext.Factory {
   private static final Logger LOG =
diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/SdkHarnessClient.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/SdkHarnessClient.java
index 47c694f..592189e 100644
--- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/SdkHarnessClient.java
+++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/SdkHarnessClient.java
@@ -64,6 +64,7 @@
  * <p>This provides a Java-friendly wrapper around {@link InstructionRequestHandler} and {@link
  * CloseableFnDataReceiver}, which handle lower-level gRPC message wrangling.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SdkHarnessClient implements AutoCloseable {
   private static final Logger LOG = LoggerFactory.getLogger(SdkHarnessClient.class);
 
diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/StageBundleFactory.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/StageBundleFactory.java
index 0a31fc8..c3c72a2 100644
--- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/StageBundleFactory.java
+++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/StageBundleFactory.java
@@ -27,6 +27,7 @@
  * <p>Closing a StageBundleFactory signals that the stage has completed and any resources bound to
  * its lifetime can be cleaned up.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public interface StageBundleFactory extends AutoCloseable {
   /** Get a new {@link RemoteBundle bundle} for processing the data in an executable stage. */
   default RemoteBundle getBundle(
diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/TimerReceiverFactory.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/TimerReceiverFactory.java
index 258afdb..aea4bf9 100644
--- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/TimerReceiverFactory.java
+++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/TimerReceiverFactory.java
@@ -45,6 +45,7 @@
  * <p>If the incoming timer is being cleared, the {@link TimerData} sets the fire and hold
  * timestamps to {@link BoundedWindow#TIMESTAMP_MAX_VALUE}.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class TimerReceiverFactory {
   private static final Logger LOG = LoggerFactory.getLogger(TimerReceiverFactory.class);
 
diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/data/GrpcDataService.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/data/GrpcDataService.java
index fc85077..dda7ea5 100644
--- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/data/GrpcDataService.java
+++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/data/GrpcDataService.java
@@ -49,6 +49,7 @@
  * <p>This service transmits all outgoing {@link BeamFnApi.Elements} messages to the first client
  * that connects.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class GrpcDataService extends BeamFnDataGrpc.BeamFnDataImplBase
     implements FnService, FnDataService {
   private static final Logger LOG = LoggerFactory.getLogger(GrpcDataService.class);
diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/environment/DockerCommand.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/environment/DockerCommand.java
index fd4b66d..9c15347 100644
--- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/environment/DockerCommand.java
+++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/environment/DockerCommand.java
@@ -42,6 +42,7 @@
 @SuppressFBWarnings(
     value = "OS_OPEN_STREAM",
     justification = "BufferedReader wraps stream we don't own and should not close")
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class DockerCommand {
   private static final Logger LOG = LoggerFactory.getLogger(DockerCommand.class);
 
diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/environment/EmbeddedEnvironmentFactory.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/environment/EmbeddedEnvironmentFactory.java
index 7757129..72a4713 100644
--- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/environment/EmbeddedEnvironmentFactory.java
+++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/environment/EmbeddedEnvironmentFactory.java
@@ -48,6 +48,7 @@
  * An {@link EnvironmentFactory} that communicates to a {@link FnHarness} which is executing in the
  * same process.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class EmbeddedEnvironmentFactory implements EnvironmentFactory {
   private static final Logger LOG = LoggerFactory.getLogger(EmbeddedEnvironmentFactory.class);
 
diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/environment/ProcessManager.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/environment/ProcessManager.java
index 0d31f3b..d30bb1e 100644
--- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/environment/ProcessManager.java
+++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/environment/ProcessManager.java
@@ -35,6 +35,7 @@
 
 /** A simple process manager which forks processes and kills them if necessary. */
 @ThreadSafe
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ProcessManager {
   private static final Logger LOG = LoggerFactory.getLogger(ProcessManager.class);
 
diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/logging/GrpcLoggingService.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/logging/GrpcLoggingService.java
index ce0c94e..6ff8b99 100644
--- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/logging/GrpcLoggingService.java
+++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/logging/GrpcLoggingService.java
@@ -30,6 +30,7 @@
 import org.slf4j.LoggerFactory;
 
 /** An implementation of the Beam Fn Logging Service over gRPC. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class GrpcLoggingService extends BeamFnLoggingGrpc.BeamFnLoggingImplBase
     implements FnService {
   private static final Logger LOG = LoggerFactory.getLogger(GrpcLoggingService.class);
diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/provisioning/StaticGrpcProvisionService.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/provisioning/StaticGrpcProvisionService.java
index f1b883e..e20e25d 100644
--- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/provisioning/StaticGrpcProvisionService.java
+++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/provisioning/StaticGrpcProvisionService.java
@@ -33,6 +33,7 @@
 /**
  * A {@link ProvisionServiceImplBase provision service} that returns a static response to all calls.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class StaticGrpcProvisionService extends ProvisionServiceGrpc.ProvisionServiceImplBase
     implements FnService {
   public static StaticGrpcProvisionService create(
diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/state/InMemoryBagUserStateFactory.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/state/InMemoryBagUserStateFactory.java
index 52d249a..b259290 100644
--- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/state/InMemoryBagUserStateFactory.java
+++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/state/InMemoryBagUserStateFactory.java
@@ -34,6 +34,7 @@
  * Holds user state in memory. Only one key is active at a time due to the GroupReduceFunction being
  * called once per key. Needs to be reset via {@code resetForNewKey()} before processing a new key.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class InMemoryBagUserStateFactory<K, V, W extends BoundedWindow>
     implements StateRequestHandlers.BagUserStateHandlerFactory<K, V, W> {
 
diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/state/StateRequestHandlers.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/state/StateRequestHandlers.java
index bfafa7d..cab8c3a 100644
--- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/state/StateRequestHandlers.java
+++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/state/StateRequestHandlers.java
@@ -62,6 +62,7 @@
  *
  * <p>TODO: Add a variant which works on {@link ByteString}s to remove encoding/decoding overhead.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class StateRequestHandlers {
 
   /**
diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/translation/BatchSideInputHandlerFactory.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/translation/BatchSideInputHandlerFactory.java
index 810e459..2f90413 100644
--- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/translation/BatchSideInputHandlerFactory.java
+++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/translation/BatchSideInputHandlerFactory.java
@@ -43,6 +43,7 @@
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables;
 
 /** {@link StateRequestHandler} that uses a {@link SideInputGetter} to access side inputs. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BatchSideInputHandlerFactory implements SideInputHandlerFactory {
 
   // Map from side input id to global PCollection id.
diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/translation/PipelineTranslatorUtils.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/translation/PipelineTranslatorUtils.java
index 4705290..6c018ca 100644
--- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/translation/PipelineTranslatorUtils.java
+++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/translation/PipelineTranslatorUtils.java
@@ -52,6 +52,7 @@
 import org.joda.time.Instant;
 
 /** Utilities for pipeline translation. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public final class PipelineTranslatorUtils {
   private PipelineTranslatorUtils() {}
 
diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/EmbeddedSdkHarness.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/EmbeddedSdkHarness.java
index bf38250..4d5eba1 100644
--- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/EmbeddedSdkHarness.java
+++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/EmbeddedSdkHarness.java
@@ -42,6 +42,7 @@
  * {@link FnHarness} to properly execute, and provides access to the associated client and harness
  * during test execution.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class EmbeddedSdkHarness extends ExternalResource implements TestRule {
 
   public static EmbeddedSdkHarness create() {
diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/artifact/ArtifactRetrievalServiceTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/artifact/ArtifactRetrievalServiceTest.java
index 5c0fe2d..929b2b6 100644
--- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/artifact/ArtifactRetrievalServiceTest.java
+++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/artifact/ArtifactRetrievalServiceTest.java
@@ -45,6 +45,7 @@
 import org.junit.runners.JUnit4;
 
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ArtifactRetrievalServiceTest {
   private static final int TEST_BUFFER_SIZE = 1 << 10;
   private GrpcFnServer<ArtifactRetrievalService> retrievalServer;
diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/artifact/ArtifactStagingServiceTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/artifact/ArtifactStagingServiceTest.java
index b52106e..3d7764b 100644
--- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/artifact/ArtifactStagingServiceTest.java
+++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/artifact/ArtifactStagingServiceTest.java
@@ -47,6 +47,7 @@
 import org.junit.runners.JUnit4;
 
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ArtifactStagingServiceTest {
   private static final int TEST_BUFFER_SIZE = 1 << 10;
   private ArtifactStagingService stagingService;
diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/DefaultJobBundleFactoryTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/DefaultJobBundleFactoryTest.java
index dd8eee6..d310292 100644
--- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/DefaultJobBundleFactoryTest.java
+++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/DefaultJobBundleFactoryTest.java
@@ -82,6 +82,7 @@
 
 /** Tests for {@link DefaultJobBundleFactory}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DefaultJobBundleFactoryTest {
   @Rule public ExpectedException thrown = ExpectedException.none();
   @Mock private EnvironmentFactory envFactory;
diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/FnApiControlClientPoolServiceTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/FnApiControlClientPoolServiceTest.java
index b49008a..d580aee 100644
--- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/FnApiControlClientPoolServiceTest.java
+++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/FnApiControlClientPoolServiceTest.java
@@ -49,6 +49,7 @@
 
 /** Unit tests for {@link FnApiControlClientPoolService}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class FnApiControlClientPoolServiceTest {
 
   private final ControlClientPool pool = MapControlClientPool.create();
diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/FnApiControlClientTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/FnApiControlClientTest.java
index 976d153..cb36d36 100644
--- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/FnApiControlClientTest.java
+++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/FnApiControlClientTest.java
@@ -46,6 +46,7 @@
 
 /** Unit tests for {@link FnApiControlClient}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class FnApiControlClientTest {
 
   @Rule public ExpectedException thrown = ExpectedException.none();
diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/ProcessBundleDescriptorsTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/ProcessBundleDescriptorsTest.java
index 98fe899..2377736 100644
--- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/ProcessBundleDescriptorsTest.java
+++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/ProcessBundleDescriptorsTest.java
@@ -29,11 +29,14 @@
 import org.apache.beam.runners.core.construction.CoderTranslation;
 import org.apache.beam.runners.core.construction.ModelCoderRegistrar;
 import org.apache.beam.runners.core.construction.ModelCoders;
+import org.apache.beam.runners.core.construction.PTransformTranslation;
 import org.apache.beam.runners.core.construction.PipelineTranslation;
 import org.apache.beam.runners.core.construction.graph.ExecutableStage;
 import org.apache.beam.runners.core.construction.graph.FusedPipeline;
 import org.apache.beam.runners.core.construction.graph.GreedyPipelineFuser;
 import org.apache.beam.runners.core.construction.graph.PipelineNode;
+import org.apache.beam.runners.core.construction.graph.ProtoOverrides;
+import org.apache.beam.runners.core.construction.graph.SplittableParDoExpander;
 import org.apache.beam.runners.core.construction.graph.TimerReference;
 import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.coders.Coder;
@@ -48,15 +51,19 @@
 import org.apache.beam.sdk.state.TimerSpec;
 import org.apache.beam.sdk.state.TimerSpecs;
 import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.DoFn.ProcessContext;
+import org.apache.beam.sdk.transforms.DoFn.ProcessElement;
 import org.apache.beam.sdk.transforms.GroupByKey;
 import org.apache.beam.sdk.transforms.Impulse;
 import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker;
 import org.apache.beam.sdk.values.KV;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Optional;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables;
 import org.junit.Test;
 
 /** Tests for {@link ProcessBundleDescriptors}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ProcessBundleDescriptorsTest implements Serializable {
 
   /**
@@ -151,6 +158,99 @@
     ensureLengthPrefixed(timerKeyCoder, originalKeyCoder, pbsCoderMap);
   }
 
+  @Test
+  public void testLengthPrefixingOfInputCoderExecutableStage() throws Exception {
+    Pipeline p = Pipeline.create();
+    Coder<Void> voidCoder = VoidCoder.of();
+    assertThat(ModelCoderRegistrar.isKnownCoder(voidCoder), is(false));
+    p.apply("impulse", Impulse.create())
+        .apply(
+            ParDo.of(
+                new DoFn<byte[], Void>() {
+                  @ProcessElement
+                  public void process(ProcessContext ctxt) {}
+                }))
+        .setCoder(voidCoder)
+        .apply(
+            ParDo.of(
+                new DoFn<Void, Void>() {
+                  @ProcessElement
+                  public void processElement(
+                      ProcessContext context, RestrictionTracker<Void, Void> tracker) {}
+
+                  @GetInitialRestriction
+                  public Void getInitialRestriction() {
+                    return null;
+                  }
+
+                  @NewTracker
+                  public SomeTracker newTracker(@Restriction Void restriction) {
+                    return null;
+                  }
+                }))
+        .setCoder(voidCoder);
+    RunnerApi.Pipeline pipelineProto = PipelineTranslation.toProto(p);
+    RunnerApi.Pipeline pipelineWithSdfExpanded =
+        ProtoOverrides.updateTransform(
+            PTransformTranslation.PAR_DO_TRANSFORM_URN,
+            pipelineProto,
+            SplittableParDoExpander.createSizedReplacement());
+    FusedPipeline fused = GreedyPipelineFuser.fuse(pipelineWithSdfExpanded);
+    Optional<ExecutableStage> optionalStage =
+        Iterables.tryFind(
+            fused.getFusedStages(),
+            (ExecutableStage stage) ->
+                stage.getTransforms().stream()
+                    .anyMatch(
+                        transform ->
+                            transform
+                                .getTransform()
+                                .getSpec()
+                                .getUrn()
+                                .equals(
+                                    PTransformTranslation
+                                        .SPLITTABLE_PROCESS_SIZED_ELEMENTS_AND_RESTRICTIONS_URN)));
+    checkState(
+        optionalStage.isPresent(),
+        "Expected a stage with SPLITTABLE_PROCESS_SIZED_ELEMENTS_AND_RESTRICTIONS_URN.");
+
+    ExecutableStage stage = optionalStage.get();
+    PipelineNode.PCollectionNode inputPCollection = stage.getInputPCollection();
+    Map<String, RunnerApi.Coder> stageCoderMap = stage.getComponents().getCodersMap();
+    RunnerApi.Coder originalMainInputCoder =
+        stageCoderMap.get(inputPCollection.getPCollection().getCoderId());
+
+    BeamFnApi.ProcessBundleDescriptor pbd =
+        ProcessBundleDescriptors.fromExecutableStage(
+                "test_stage", stage, Endpoints.ApiServiceDescriptor.getDefaultInstance())
+            .getProcessBundleDescriptor();
+    Map<String, RunnerApi.Coder> pbsCoderMap = pbd.getCodersMap();
+
+    RunnerApi.Coder pbsMainInputCoder =
+        pbsCoderMap.get(pbd.getPcollectionsOrThrow(inputPCollection.getId()).getCoderId());
+
+    RunnerApi.Coder kvCoder =
+        pbsCoderMap.get(ModelCoders.getKvCoderComponents(pbsMainInputCoder).keyCoderId());
+    RunnerApi.Coder keyCoder =
+        pbsCoderMap.get(ModelCoders.getKvCoderComponents(kvCoder).keyCoderId());
+    RunnerApi.Coder valueKvCoder =
+        pbsCoderMap.get(ModelCoders.getKvCoderComponents(kvCoder).valueCoderId());
+    RunnerApi.Coder valueCoder =
+        pbsCoderMap.get(ModelCoders.getKvCoderComponents(valueKvCoder).keyCoderId());
+
+    RunnerApi.Coder originalKvCoder =
+        stageCoderMap.get(ModelCoders.getKvCoderComponents(originalMainInputCoder).keyCoderId());
+    RunnerApi.Coder originalKeyCoder =
+        stageCoderMap.get(ModelCoders.getKvCoderComponents(originalKvCoder).keyCoderId());
+    RunnerApi.Coder originalvalueKvCoder =
+        stageCoderMap.get(ModelCoders.getKvCoderComponents(originalKvCoder).valueCoderId());
+    RunnerApi.Coder originalvalueCoder =
+        stageCoderMap.get(ModelCoders.getKvCoderComponents(originalvalueKvCoder).keyCoderId());
+
+    ensureLengthPrefixed(keyCoder, originalKeyCoder, pbsCoderMap);
+    ensureLengthPrefixed(valueCoder, originalvalueCoder, pbsCoderMap);
+  }
+
   private static void ensureLengthPrefixed(
       RunnerApi.Coder coder,
       RunnerApi.Coder originalCoder,
@@ -160,4 +260,6 @@
     String lengthPrefixedWrappedCoderId = coder.getComponentCoderIds(0);
     assertThat(pbsCoderMap.get(lengthPrefixedWrappedCoderId), is(originalCoder));
   }
+
+  private abstract static class SomeTracker extends RestrictionTracker<Void, Void> {}
 }
diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/RemoteExecutionTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/RemoteExecutionTest.java
index 95968ad..ce03412 100644
--- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/RemoteExecutionTest.java
+++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/RemoteExecutionTest.java
@@ -157,6 +157,7 @@
  * going through pipeline fusion.
  */
 @RunWith(JUnit4.class)
+@SuppressWarnings({"keyfor", "nullness"}) // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class RemoteExecutionTest implements Serializable {
   @Rule public transient ResetDateTimeProvider resetDateTimeProvider = new ResetDateTimeProvider();
 
diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/SdkHarnessClientTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/SdkHarnessClientTest.java
index e78f84c..473ddc3 100644
--- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/SdkHarnessClientTest.java
+++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/SdkHarnessClientTest.java
@@ -100,6 +100,7 @@
 
 /** Unit tests for {@link SdkHarnessClient}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SdkHarnessClientTest {
 
   @Mock public FnApiControlClient fnApiControlClient;
diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/SingleEnvironmentInstanceJobBundleFactoryTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/SingleEnvironmentInstanceJobBundleFactoryTest.java
index 79a15ba..0784b76 100644
--- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/SingleEnvironmentInstanceJobBundleFactoryTest.java
+++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/SingleEnvironmentInstanceJobBundleFactoryTest.java
@@ -59,6 +59,7 @@
 
 /** Tests for {@link SingleEnvironmentInstanceJobBundleFactory}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SingleEnvironmentInstanceJobBundleFactoryTest {
   @Mock private EnvironmentFactory environmentFactory;
   @Mock private InstructionRequestHandler instructionRequestHandler;
diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/environment/DockerEnvironmentFactoryTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/environment/DockerEnvironmentFactoryTest.java
index 2068c8a..f347741 100644
--- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/environment/DockerEnvironmentFactoryTest.java
+++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/environment/DockerEnvironmentFactoryTest.java
@@ -59,6 +59,7 @@
 import org.mockito.MockitoAnnotations;
 
 /** Tests for {@link DockerEnvironmentFactory}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DockerEnvironmentFactoryTest {
 
   private static final ApiServiceDescriptor SERVICE_DESCRIPTOR =
diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/environment/ProcessEnvironmentFactoryTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/environment/ProcessEnvironmentFactoryTest.java
index 3b5b689..23ec47b 100644
--- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/environment/ProcessEnvironmentFactoryTest.java
+++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/environment/ProcessEnvironmentFactoryTest.java
@@ -49,6 +49,7 @@
 
 /** Tests for {@link ProcessEnvironmentFactory}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ProcessEnvironmentFactoryTest {
 
   private static final ApiServiceDescriptor SERVICE_DESCRIPTOR =
diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/environment/ProcessManagerTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/environment/ProcessManagerTest.java
index a1377f3..9a0705a 100644
--- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/environment/ProcessManagerTest.java
+++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/environment/ProcessManagerTest.java
@@ -40,6 +40,7 @@
 
 /** Tests for {@link ProcessManager}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ProcessManagerTest {
 
   @Test
diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/state/GrpcStateServiceTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/state/GrpcStateServiceTest.java
index aa986c9..efb8f06 100644
--- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/state/GrpcStateServiceTest.java
+++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/state/GrpcStateServiceTest.java
@@ -43,6 +43,7 @@
 
 /** Tests for {@link org.apache.beam.runners.fnexecution.state.GrpcStateService}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class GrpcStateServiceTest {
   private static final long TIMEOUT_MS = 30 * 1000;
 
diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/status/BeamWorkerStatusGrpcServiceTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/status/BeamWorkerStatusGrpcServiceTest.java
index 9b04914..c9945ab 100644
--- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/status/BeamWorkerStatusGrpcServiceTest.java
+++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/status/BeamWorkerStatusGrpcServiceTest.java
@@ -54,6 +54,7 @@
 import org.mockito.MockitoAnnotations;
 
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BeamWorkerStatusGrpcServiceTest {
 
   @Rule public final GrpcCleanupRule grpcCleanup = new GrpcCleanupRule();
diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/status/WorkerStatusClientTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/status/WorkerStatusClientTest.java
index 5fa143e..cc9c3d2 100644
--- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/status/WorkerStatusClientTest.java
+++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/status/WorkerStatusClientTest.java
@@ -36,6 +36,7 @@
 import org.mockito.MockitoAnnotations;
 
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class WorkerStatusClientTest {
 
   @Mock public StreamObserver<BeamFnApi.WorkerStatusRequest> mockObserver;
diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/translation/BatchSideInputHandlerFactoryTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/translation/BatchSideInputHandlerFactoryTest.java
index 1f45945..da14e08 100644
--- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/translation/BatchSideInputHandlerFactoryTest.java
+++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/translation/BatchSideInputHandlerFactoryTest.java
@@ -62,6 +62,7 @@
 
 /** Tests for {@link BatchSideInputHandlerFactory}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BatchSideInputHandlerFactoryTest {
 
   private static final String TRANSFORM_ID = "transform-id";
diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/wire/LengthPrefixUnknownCodersTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/wire/LengthPrefixUnknownCodersTest.java
index a423b36..c360e9d 100644
--- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/wire/LengthPrefixUnknownCodersTest.java
+++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/wire/LengthPrefixUnknownCodersTest.java
@@ -45,6 +45,7 @@
 
 /** Tests for {@link LengthPrefixUnknownCoders}. */
 @RunWith(Parameterized.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class LengthPrefixUnknownCodersTest {
 
   private static class UnknownCoder extends CustomCoder<String> {
diff --git a/runners/java-job-service/src/main/java/org/apache/beam/runners/jobsubmission/InMemoryJobService.java b/runners/java-job-service/src/main/java/org/apache/beam/runners/jobsubmission/InMemoryJobService.java
index e27d56d..0a73e75 100644
--- a/runners/java-job-service/src/main/java/org/apache/beam/runners/jobsubmission/InMemoryJobService.java
+++ b/runners/java-job-service/src/main/java/org/apache/beam/runners/jobsubmission/InMemoryJobService.java
@@ -75,6 +75,7 @@
  *
  * <p>TODO: replace in-memory job management state with persistent solution.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class InMemoryJobService extends JobServiceGrpc.JobServiceImplBase implements FnService {
   private static final Logger LOG = LoggerFactory.getLogger(InMemoryJobService.class);
 
diff --git a/runners/java-job-service/src/main/java/org/apache/beam/runners/jobsubmission/JobInvocation.java b/runners/java-job-service/src/main/java/org/apache/beam/runners/jobsubmission/JobInvocation.java
index e0a6d97..847cfabe 100644
--- a/runners/java-job-service/src/main/java/org/apache/beam/runners/jobsubmission/JobInvocation.java
+++ b/runners/java-job-service/src/main/java/org/apache/beam/runners/jobsubmission/JobInvocation.java
@@ -45,6 +45,7 @@
 import org.slf4j.LoggerFactory;
 
 /** Internal representation of a Job which has been invoked (prepared and run) by a client. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class JobInvocation {
 
   private static final Logger LOG = LoggerFactory.getLogger(JobInvocation.class);
diff --git a/runners/java-job-service/src/main/java/org/apache/beam/runners/jobsubmission/JobInvoker.java b/runners/java-job-service/src/main/java/org/apache/beam/runners/jobsubmission/JobInvoker.java
index 145dfae..14fc9f6 100644
--- a/runners/java-job-service/src/main/java/org/apache/beam/runners/jobsubmission/JobInvoker.java
+++ b/runners/java-job-service/src/main/java/org/apache/beam/runners/jobsubmission/JobInvoker.java
@@ -28,6 +28,7 @@
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.util.concurrent.ThreadFactoryBuilder;
 
 /** Factory to create {@link JobInvocation} instances. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public abstract class JobInvoker {
 
   private final ListeningExecutorService executorService;
diff --git a/runners/java-job-service/src/main/java/org/apache/beam/runners/jobsubmission/JobServerDriver.java b/runners/java-job-service/src/main/java/org/apache/beam/runners/jobsubmission/JobServerDriver.java
index a93716a..cd2227f 100644
--- a/runners/java-job-service/src/main/java/org/apache/beam/runners/jobsubmission/JobServerDriver.java
+++ b/runners/java-job-service/src/main/java/org/apache/beam/runners/jobsubmission/JobServerDriver.java
@@ -32,6 +32,7 @@
 import org.slf4j.LoggerFactory;
 
 /** Shared code for starting and serving an {@link InMemoryJobService}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public abstract class JobServerDriver implements Runnable {
 
   private static final Logger LOG = LoggerFactory.getLogger(JobServerDriver.class);
diff --git a/runners/java-job-service/src/main/java/org/apache/beam/runners/jobsubmission/PortablePipelineJarCreator.java b/runners/java-job-service/src/main/java/org/apache/beam/runners/jobsubmission/PortablePipelineJarCreator.java
index a0c5983..3f694e5 100644
--- a/runners/java-job-service/src/main/java/org/apache/beam/runners/jobsubmission/PortablePipelineJarCreator.java
+++ b/runners/java-job-service/src/main/java/org/apache/beam/runners/jobsubmission/PortablePipelineJarCreator.java
@@ -63,6 +63,7 @@
  * <p>Each {@link PortablePipelineJarCreator} instance is not threadsafe; a new instance is expected
  * to be constructed and {@link #run} once per job.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class PortablePipelineJarCreator implements PortablePipelineRunner {
   private static final Logger LOG = LoggerFactory.getLogger(PortablePipelineJarCreator.class);
 
diff --git a/runners/java-job-service/src/main/java/org/apache/beam/runners/jobsubmission/PortablePipelineJarUtils.java b/runners/java-job-service/src/main/java/org/apache/beam/runners/jobsubmission/PortablePipelineJarUtils.java
index 838afe5..da1adff 100644
--- a/runners/java-job-service/src/main/java/org/apache/beam/runners/jobsubmission/PortablePipelineJarUtils.java
+++ b/runners/java-job-service/src/main/java/org/apache/beam/runners/jobsubmission/PortablePipelineJarUtils.java
@@ -64,6 +64,7 @@
  *   <li>...Java classes...
  * </ul>
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public abstract class PortablePipelineJarUtils {
   private static final String ARTIFACT_FOLDER = "artifacts";
   private static final String PIPELINE_FOLDER = "BEAM-PIPELINE";
diff --git a/runners/java-job-service/src/test/java/org/apache/beam/runners/jobsubmission/InMemoryJobServiceTest.java b/runners/java-job-service/src/test/java/org/apache/beam/runners/jobsubmission/InMemoryJobServiceTest.java
index 1e51721..fd7e79f 100644
--- a/runners/java-job-service/src/test/java/org/apache/beam/runners/jobsubmission/InMemoryJobServiceTest.java
+++ b/runners/java-job-service/src/test/java/org/apache/beam/runners/jobsubmission/InMemoryJobServiceTest.java
@@ -51,6 +51,7 @@
 
 /** Tests for {@link InMemoryJobService}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class InMemoryJobServiceTest {
 
   private static final String TEST_JOB_NAME = "test-job";
diff --git a/runners/java-job-service/src/test/java/org/apache/beam/runners/jobsubmission/JobInvocationTest.java b/runners/java-job-service/src/test/java/org/apache/beam/runners/jobsubmission/JobInvocationTest.java
index ae0a247..2e46c05 100644
--- a/runners/java-job-service/src/test/java/org/apache/beam/runners/jobsubmission/JobInvocationTest.java
+++ b/runners/java-job-service/src/test/java/org/apache/beam/runners/jobsubmission/JobInvocationTest.java
@@ -42,6 +42,7 @@
 import org.junit.Test;
 
 /** Tests for {@link JobInvocation}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class JobInvocationTest {
 
   private static ExecutorService executorService;
diff --git a/runners/java-job-service/src/test/java/org/apache/beam/runners/jobsubmission/PortablePipelineJarCreatorTest.java b/runners/java-job-service/src/test/java/org/apache/beam/runners/jobsubmission/PortablePipelineJarCreatorTest.java
index 8ab11b8..70e2c56 100644
--- a/runners/java-job-service/src/test/java/org/apache/beam/runners/jobsubmission/PortablePipelineJarCreatorTest.java
+++ b/runners/java-job-service/src/test/java/org/apache/beam/runners/jobsubmission/PortablePipelineJarCreatorTest.java
@@ -51,6 +51,7 @@
 
 /** Unit tests for {@link PortablePipelineJarCreator}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class PortablePipelineJarCreatorTest implements Serializable {
 
   @Mock private JarFile inputJar;
diff --git a/runners/jet/src/main/java/org/apache/beam/runners/jet/DAGBuilder.java b/runners/jet/src/main/java/org/apache/beam/runners/jet/DAGBuilder.java
index 57451e5..81e54de 100644
--- a/runners/jet/src/main/java/org/apache/beam/runners/jet/DAGBuilder.java
+++ b/runners/jet/src/main/java/org/apache/beam/runners/jet/DAGBuilder.java
@@ -41,6 +41,7 @@
 import org.apache.beam.sdk.values.PCollectionView;
 
 /** Utility class for wiring up Jet DAGs based on Beam pipelines. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DAGBuilder {
 
   private final DAG dag = new DAG();
diff --git a/runners/jet/src/main/java/org/apache/beam/runners/jet/JetGraphVisitor.java b/runners/jet/src/main/java/org/apache/beam/runners/jet/JetGraphVisitor.java
index 59e1db4..da0e581 100644
--- a/runners/jet/src/main/java/org/apache/beam/runners/jet/JetGraphVisitor.java
+++ b/runners/jet/src/main/java/org/apache/beam/runners/jet/JetGraphVisitor.java
@@ -27,6 +27,7 @@
 import org.apache.beam.sdk.values.PValue;
 
 /** Logic that specifies how to apply translations when traversing the nodes of a Beam pipeline. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class JetGraphVisitor extends Pipeline.PipelineVisitor.Defaults {
 
   private final JetTranslationContext translationContext;
diff --git a/runners/jet/src/main/java/org/apache/beam/runners/jet/JetPipelineResult.java b/runners/jet/src/main/java/org/apache/beam/runners/jet/JetPipelineResult.java
index 09fd3a3..03ffcc3 100644
--- a/runners/jet/src/main/java/org/apache/beam/runners/jet/JetPipelineResult.java
+++ b/runners/jet/src/main/java/org/apache/beam/runners/jet/JetPipelineResult.java
@@ -38,6 +38,7 @@
 import org.slf4j.LoggerFactory;
 
 /** Jet specific implementation of {@link PipelineResult}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class JetPipelineResult implements PipelineResult {
 
   private static final Logger LOG = LoggerFactory.getLogger(JetPipelineResult.class);
diff --git a/runners/jet/src/main/java/org/apache/beam/runners/jet/JetTransformTranslators.java b/runners/jet/src/main/java/org/apache/beam/runners/jet/JetTransformTranslators.java
index 644de89..2d216ba 100644
--- a/runners/jet/src/main/java/org/apache/beam/runners/jet/JetTransformTranslators.java
+++ b/runners/jet/src/main/java/org/apache/beam/runners/jet/JetTransformTranslators.java
@@ -61,7 +61,10 @@
 import org.apache.beam.sdk.values.TupleTag;
 import org.apache.beam.sdk.values.WindowingStrategy;
 
-@SuppressWarnings("unchecked")
+@SuppressWarnings({
+  "unchecked",
+  "nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
+})
 class JetTransformTranslators {
 
   /** A map from a Transform URN to the translator. */
@@ -91,9 +94,8 @@
         AppliedPTransform<?, ?, ?> appliedTransform,
         Node node,
         JetTranslationContext context) {
-      Map.Entry<TupleTag<?>, PValue> output = Utils.getOutput(appliedTransform);
-      Coder outputCoder =
-          Utils.getCoder((PCollection) Utils.getOutput(appliedTransform).getValue());
+      Map.Entry<TupleTag<?>, PCollection<?>> output = Utils.getOutput(appliedTransform);
+      Coder outputCoder = Utils.getCoder(output.getValue());
 
       String transformName = appliedTransform.getFullName();
       DAGBuilder dagBuilder = context.getDagBuilder();
@@ -152,7 +154,7 @@
       boolean usesStateOrTimers = Utils.usesStateOrTimers(appliedTransform);
       DoFn<?, ?> doFn = Utils.getDoFn(appliedTransform);
 
-      Map<TupleTag<?>, PValue> outputs = Utils.getOutputs(appliedTransform);
+      Map<TupleTag<?>, PCollection<?>> outputs = Utils.getOutputs(appliedTransform);
 
       TupleTag<?> mainOutputTag;
       try {
@@ -241,7 +243,7 @@
         }
       }
 
-      for (Map.Entry<TupleTag<?>, PValue> entry : outputs.entrySet()) {
+      for (Map.Entry<TupleTag<?>, PCollection<?>> entry : outputs.entrySet()) {
         TupleTag<?> pCollId = entry.getKey();
         String edgeId = Utils.getTupleTagId(entry.getValue());
         dagBuilder.registerCollectionOfEdge(edgeId, pCollId.getId());
@@ -268,7 +270,7 @@
           (PCollection<KV<K, InputT>>) Utils.getInput(appliedTransform);
       WindowedValue.WindowedValueCoder<KV<K, InputT>> inputCoder =
           Utils.getWindowedValueCoder(input);
-      Map.Entry<TupleTag<?>, PValue> output = Utils.getOutput(appliedTransform);
+      Map.Entry<TupleTag<?>, PCollection<?>> output = Utils.getOutput(appliedTransform);
       Coder outputCoder = Utils.getCoder((PCollection) output.getValue());
 
       WindowingStrategy<?, ?> windowingStrategy = input.getWindowingStrategy();
@@ -315,8 +317,8 @@
       String vertexId = dagBuilder.newVertexId(transformName);
       PCollection<T> input = (PCollection<T>) Utils.getInput(appliedTransform);
       Coder inputCoder = Utils.getCoder(input);
-      Map.Entry<TupleTag<?>, PValue> output = Utils.getOutput(appliedTransform);
-      Coder outputCoder = Utils.getCoder((PCollection) output.getValue());
+      Map.Entry<TupleTag<?>, PCollection<?>> output = Utils.getOutput(appliedTransform);
+      Coder outputCoder = Utils.getCoder(output.getValue());
 
       Vertex vertex =
           dagBuilder.addVertex(
@@ -350,8 +352,8 @@
       Map<String, Coder> inputCoders =
           Utils.getCoders(
               Utils.getInputs(appliedTransform), e -> Utils.getTupleTagId(e.getValue()));
-      Map.Entry<TupleTag<?>, PValue> output = Utils.getOutput(appliedTransform);
-      Coder outputCoder = Utils.getCoder((PCollection) output.getValue());
+      Map.Entry<TupleTag<?>, PCollection<?>> output = Utils.getOutput(appliedTransform);
+      Coder outputCoder = Utils.getCoder(output.getValue());
 
       DAGBuilder dagBuilder = context.getDagBuilder();
       String vertexId = dagBuilder.newVertexId(appliedTransform.getFullName());
@@ -387,7 +389,7 @@
       PCollection<WindowedValue> input =
           (PCollection<WindowedValue>) Utils.getInput(appliedTransform);
       Coder inputCoder = Utils.getCoder(input);
-      Map.Entry<TupleTag<?>, PValue> output = Utils.getOutput(appliedTransform);
+      Map.Entry<TupleTag<?>, PCollection<?>> output = Utils.getOutput(appliedTransform);
       Coder outputCoder =
           Utils.getCoder((PCollection) Utils.getOutput(appliedTransform).getValue());
 
@@ -420,7 +422,7 @@
       DAGBuilder dagBuilder = context.getDagBuilder();
       String vertexId = dagBuilder.newVertexId(transformName);
 
-      Map.Entry<TupleTag<?>, PValue> output = Utils.getOutput(appliedTransform);
+      Map.Entry<TupleTag<?>, PCollection<?>> output = Utils.getOutput(appliedTransform);
       Coder outputCoder =
           Utils.getCoder((PCollection) Utils.getOutput(appliedTransform).getValue());
       Vertex vertex = dagBuilder.addVertex(vertexId, ImpulseP.supplier(outputCoder, vertexId));
diff --git a/runners/jet/src/main/java/org/apache/beam/runners/jet/Utils.java b/runners/jet/src/main/java/org/apache/beam/runners/jet/Utils.java
index 2bd1b5d..0fb175c 100644
--- a/runners/jet/src/main/java/org/apache/beam/runners/jet/Utils.java
+++ b/runners/jet/src/main/java/org/apache/beam/runners/jet/Utils.java
@@ -19,7 +19,6 @@
 
 import static com.hazelcast.jet.impl.util.ExceptionUtil.rethrow;
 import static java.util.stream.Collectors.toList;
-import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkState;
 
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
@@ -58,6 +57,7 @@
 import org.checkerframework.checker.nullness.qual.Nullable;
 
 /** Various common methods used by the Jet based runner. */
+@SuppressWarnings({"nullness", "keyfor"}) // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class Utils {
 
   public static String getTupleTagId(PValue value) {
@@ -77,7 +77,7 @@
     return TransformInputs.nonAdditionalInputs(node.toAppliedPTransform(pipeline));
   }
 
-  static Map<TupleTag<?>, PValue> getInputs(AppliedPTransform<?, ?, ?> appliedTransform) {
+  static Map<TupleTag<?>, PCollection<?>> getInputs(AppliedPTransform<?, ?, ?> appliedTransform) {
     return appliedTransform.getInputs();
   }
 
@@ -93,14 +93,15 @@
     return Iterables.getOnlyElement(TransformInputs.nonAdditionalInputs(appliedTransform));
   }
 
-  static Map<TupleTag<?>, PValue> getOutputs(AppliedPTransform<?, ?, ?> appliedTransform) {
+  static Map<TupleTag<?>, PCollection<?>> getOutputs(AppliedPTransform<?, ?, ?> appliedTransform) {
     if (appliedTransform.getTransform() == null) {
       return null;
     }
     return appliedTransform.getOutputs();
   }
 
-  static Map.Entry<TupleTag<?>, PValue> getOutput(AppliedPTransform<?, ?, ?> appliedTransform) {
+  static Map.Entry<TupleTag<?>, PCollection<?>> getOutput(
+      AppliedPTransform<?, ?, ?> appliedTransform) {
     return Iterables.getOnlyElement(getOutputs(appliedTransform).entrySet());
   }
 
@@ -133,12 +134,10 @@
   }
 
   static <T> Map<T, Coder> getCoders(
-      Map<TupleTag<?>, PValue> pCollections,
-      Function<Map.Entry<TupleTag<?>, PValue>, T> tupleTagExtractor) {
+      Map<TupleTag<?>, PCollection<?>> pCollections,
+      Function<Map.Entry<TupleTag<?>, PCollection<?>>, T> tupleTagExtractor) {
     return pCollections.entrySet().stream()
-        .collect(
-            Collectors.toMap(
-                tupleTagExtractor, e -> getCoder((PCollection) e.getValue()), (v1, v2) -> v1));
+        .collect(Collectors.toMap(tupleTagExtractor, e -> getCoder(e.getValue()), (v1, v2) -> v1));
   }
 
   static Map<TupleTag<?>, Coder<?>> getOutputValueCoders(
@@ -191,19 +190,13 @@
   static WindowingStrategy<?, ?> getWindowingStrategy(AppliedPTransform<?, ?, ?> appliedTransform) {
     // assume that the windowing strategy is the same for all outputs
 
-    Map<TupleTag<?>, PValue> outputs = getOutputs(appliedTransform);
+    Map<TupleTag<?>, PCollection<?>> outputs = getOutputs(appliedTransform);
 
     if (outputs == null || outputs.isEmpty()) {
       throw new IllegalStateException("No outputs defined.");
     }
 
-    PValue taggedValue = outputs.values().iterator().next();
-    checkState(
-        taggedValue instanceof PCollection,
-        "Within ParDo, got a non-PCollection output %s of type %s",
-        taggedValue,
-        taggedValue.getClass().getSimpleName());
-    PCollection<?> coll = (PCollection<?>) taggedValue;
+    PCollection<?> coll = outputs.values().iterator().next();
     return coll.getWindowingStrategy();
   }
 
diff --git a/runners/jet/src/main/java/org/apache/beam/runners/jet/metrics/JetMetricResults.java b/runners/jet/src/main/java/org/apache/beam/runners/jet/metrics/JetMetricResults.java
index 391c068..a5ceead 100644
--- a/runners/jet/src/main/java/org/apache/beam/runners/jet/metrics/JetMetricResults.java
+++ b/runners/jet/src/main/java/org/apache/beam/runners/jet/metrics/JetMetricResults.java
@@ -38,6 +38,7 @@
 import org.checkerframework.checker.nullness.qual.Nullable;
 
 /** Jet specific {@link MetricResults}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class JetMetricResults extends MetricResults {
 
   @GuardedBy("this")
diff --git a/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/AbstractParDoP.java b/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/AbstractParDoP.java
index 2a7ac14..a39ed76 100644
--- a/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/AbstractParDoP.java
+++ b/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/AbstractParDoP.java
@@ -65,6 +65,7 @@
 import org.apache.beam.sdk.values.WindowingStrategy;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Lists;
 
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 abstract class AbstractParDoP<InputT, OutputT> implements Processor {
 
   private final SerializablePipelineOptions pipelineOptions;
diff --git a/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/AssignWindowP.java b/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/AssignWindowP.java
index 138f2bb..d79c2dc 100644
--- a/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/AssignWindowP.java
+++ b/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/AssignWindowP.java
@@ -37,6 +37,7 @@
  *
  * @param <T> type of element being windowed
  */
+@SuppressWarnings({"nullness", "keyfor"}) // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class AssignWindowP<T> extends AbstractProcessor {
 
   @SuppressWarnings({"FieldCanBeLocal", "unused"})
diff --git a/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/BoundedSourceP.java b/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/BoundedSourceP.java
index 42a2b20..f1b9ced 100644
--- a/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/BoundedSourceP.java
+++ b/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/BoundedSourceP.java
@@ -42,6 +42,7 @@
  * Jet {@link com.hazelcast.jet.core.Processor} implementation for reading from a bounded Beam
  * source.
  */
+@SuppressWarnings({"nullness", "keyfor"}) // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BoundedSourceP<T> extends AbstractProcessor implements Traverser {
 
   private final Traverser<BoundedSource<T>> shardsTraverser;
diff --git a/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/FlattenP.java b/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/FlattenP.java
index a3debed..f0a659f 100644
--- a/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/FlattenP.java
+++ b/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/FlattenP.java
@@ -30,6 +30,7 @@
 import org.apache.beam.sdk.util.WindowedValue;
 
 /** Jet {@link com.hazelcast.jet.core.Processor} implementation for Beam's Flatten primitive. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class FlattenP extends AbstractProcessor {
 
   private final Map<Integer, Coder> inputOrdinalCoders;
diff --git a/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/ImpulseP.java b/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/ImpulseP.java
index 2ab67db..ec95c5d 100644
--- a/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/ImpulseP.java
+++ b/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/ImpulseP.java
@@ -34,6 +34,7 @@
 /**
  * /** * Jet {@link com.hazelcast.jet.core.Processor} implementation for Beam's Impulse primitive.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ImpulseP extends AbstractProcessor {
 
   private final boolean active;
diff --git a/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/StatefulParDoP.java b/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/StatefulParDoP.java
index 22dbd21..ebc9893 100644
--- a/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/StatefulParDoP.java
+++ b/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/StatefulParDoP.java
@@ -51,6 +51,7 @@
 /**
  * Jet {@link com.hazelcast.jet.core.Processor} implementation for Beam's stateful ParDo primitive.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class StatefulParDoP<OutputT>
     extends AbstractParDoP<KV<?, ?>, OutputT> { // todo: unify with ParDoP?
 
diff --git a/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/UnboundedSourceP.java b/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/UnboundedSourceP.java
index 37008b8..1811032 100644
--- a/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/UnboundedSourceP.java
+++ b/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/UnboundedSourceP.java
@@ -42,6 +42,7 @@
  * Jet {@link com.hazelcast.jet.core.Processor} implementation for reading from an unbounded Beam
  * source.
  */
+@SuppressWarnings({"nullness", "keyfor"}) // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class UnboundedSourceP<T, CmT extends UnboundedSource.CheckpointMark>
     extends AbstractProcessor {
 
diff --git a/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/ViewP.java b/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/ViewP.java
index 7121422..0e4e2dc 100644
--- a/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/ViewP.java
+++ b/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/ViewP.java
@@ -42,6 +42,7 @@
  * primitives. Collects all input {@link WindowedValue}s, groups them by windows and keys and when
  * input is complete emits them.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ViewP extends AbstractProcessor {
 
   private final TimestampCombiner timestampCombiner;
diff --git a/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/WindowGroupP.java b/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/WindowGroupP.java
index c96dd99..e06af0d 100644
--- a/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/WindowGroupP.java
+++ b/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/WindowGroupP.java
@@ -67,6 +67,7 @@
  * @param <K> key type of {@link KV} values from the output of this primitive
  * @param <V> type of elements being windowed
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class WindowGroupP<K, V> extends AbstractProcessor {
 
   private static final int PROCESSING_TIME_MIN_INCREMENT = 100;
diff --git a/runners/jet/src/test/java/org/apache/beam/runners/jet/TestJetRunner.java b/runners/jet/src/test/java/org/apache/beam/runners/jet/TestJetRunner.java
index a476b50..b735538 100644
--- a/runners/jet/src/test/java/org/apache/beam/runners/jet/TestJetRunner.java
+++ b/runners/jet/src/test/java/org/apache/beam/runners/jet/TestJetRunner.java
@@ -42,10 +42,10 @@
 import org.apache.beam.sdk.util.CoderUtils;
 import org.apache.beam.sdk.values.PBegin;
 import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PValue;
 import org.apache.beam.sdk.values.TupleTag;
 
 /** Slightly altered version of the Jet based runner, used in unit-tests. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class TestJetRunner extends PipelineRunner<PipelineResult> {
 
   /** A map from a Transform URN to the translator. */
@@ -126,7 +126,7 @@
 
       // events in the transform are not serializable, we have to translate them. We'll also flatten
       // the collection.
-      Map.Entry<TupleTag<?>, PValue> output = Utils.getOutput(appliedTransform);
+      Map.Entry<TupleTag<?>, PCollection<?>> output = Utils.getOutput(appliedTransform);
       Coder outputCoder = Utils.getCoder((PCollection) output.getValue());
       TestStream.TestStreamCoder<T> payloadCoder =
           TestStream.TestStreamCoder.of(testStream.getValueCoder());
diff --git a/runners/jet/src/test/java/org/apache/beam/runners/jet/TestStreamP.java b/runners/jet/src/test/java/org/apache/beam/runners/jet/TestStreamP.java
index 84174f8..e83358b 100644
--- a/runners/jet/src/test/java/org/apache/beam/runners/jet/TestStreamP.java
+++ b/runners/jet/src/test/java/org/apache/beam/runners/jet/TestStreamP.java
@@ -40,6 +40,7 @@
  * Jet {@link com.hazelcast.jet.core.Processor} implementation for Beam's {@link TestStream}
  * transform.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class TestStreamP extends AbstractProcessor {
 
   private final Traverser traverser;
diff --git a/runners/portability/java/src/main/java/org/apache/beam/runners/portability/CloseableResource.java b/runners/portability/java/src/main/java/org/apache/beam/runners/portability/CloseableResource.java
index 5067ac1..197c509 100644
--- a/runners/portability/java/src/main/java/org/apache/beam/runners/portability/CloseableResource.java
+++ b/runners/portability/java/src/main/java/org/apache/beam/runners/portability/CloseableResource.java
@@ -37,6 +37,7 @@
  *
  * <p>Not thread-safe.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class CloseableResource<T> implements AutoCloseable {
 
   private final T resource;
diff --git a/runners/portability/java/src/main/java/org/apache/beam/runners/portability/JobServicePipelineResult.java b/runners/portability/java/src/main/java/org/apache/beam/runners/portability/JobServicePipelineResult.java
index cab1e4d..35988ac 100644
--- a/runners/portability/java/src/main/java/org/apache/beam/runners/portability/JobServicePipelineResult.java
+++ b/runners/portability/java/src/main/java/org/apache/beam/runners/portability/JobServicePipelineResult.java
@@ -39,6 +39,7 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+@SuppressWarnings({"keyfor", "nullness"}) // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class JobServicePipelineResult implements PipelineResult, AutoCloseable {
 
   private static final long POLL_INTERVAL_MS = 3_000;
diff --git a/runners/portability/java/src/main/java/org/apache/beam/runners/portability/PortableMetrics.java b/runners/portability/java/src/main/java/org/apache/beam/runners/portability/PortableMetrics.java
index 270e852..fb2403c 100644
--- a/runners/portability/java/src/main/java/org/apache/beam/runners/portability/PortableMetrics.java
+++ b/runners/portability/java/src/main/java/org/apache/beam/runners/portability/PortableMetrics.java
@@ -43,6 +43,7 @@
 import org.apache.beam.sdk.metrics.MetricsFilter;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables;
 
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class PortableMetrics extends MetricResults {
   private static final String NAMESPACE_LABEL = "NAMESPACE";
   private static final String METRIC_NAME_LABEL = "NAME";
diff --git a/runners/portability/java/src/main/java/org/apache/beam/runners/portability/PortableRunner.java b/runners/portability/java/src/main/java/org/apache/beam/runners/portability/PortableRunner.java
index dbc9902..00523cf 100644
--- a/runners/portability/java/src/main/java/org/apache/beam/runners/portability/PortableRunner.java
+++ b/runners/portability/java/src/main/java/org/apache/beam/runners/portability/PortableRunner.java
@@ -56,6 +56,7 @@
 import org.slf4j.LoggerFactory;
 
 /** A {@link PipelineRunner} a {@link Pipeline} against a {@code JobService}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class PortableRunner extends PipelineRunner<PipelineResult> {
 
   private static final Logger LOG = LoggerFactory.getLogger(PortableRunner.class);
diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaExecutionContext.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaExecutionContext.java
index d86b5cf..301cde8 100644
--- a/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaExecutionContext.java
+++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaExecutionContext.java
@@ -48,6 +48,7 @@
 import org.slf4j.LoggerFactory;
 
 /** Runtime context for the Samza runner. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SamzaExecutionContext implements ApplicationContainerContext {
   private static final Logger LOG = LoggerFactory.getLogger(SamzaExecutionContext.class);
   private static final String SAMZA_WORKER_ID = "samza_py_worker_id";
diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaJobServerDriver.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaJobServerDriver.java
index 2b748aa..1b247fd 100644
--- a/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaJobServerDriver.java
+++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaJobServerDriver.java
@@ -38,6 +38,7 @@
 
 /** Driver program that starts a job server. */
 // TODO extend JobServerDriver
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SamzaJobServerDriver {
   private static final Logger LOG = LoggerFactory.getLogger(SamzaJobServerDriver.class);
 
diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaPipelineResult.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaPipelineResult.java
index 068503d..de77f65 100644
--- a/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaPipelineResult.java
+++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaPipelineResult.java
@@ -34,6 +34,7 @@
 import org.slf4j.LoggerFactory;
 
 /** The result from executing a Samza Pipeline. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SamzaPipelineResult implements PipelineResult {
   private static final Logger LOG = LoggerFactory.getLogger(SamzaPipelineResult.class);
   private static final long DEFAULT_SHUTDOWN_MS = 5000L;
diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaRunner.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaRunner.java
index 82c0b83..1f7fd60 100644
--- a/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaRunner.java
+++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaRunner.java
@@ -54,6 +54,7 @@
  * A {@link PipelineRunner} that executes the operations in the {@link Pipeline} into an equivalent
  * Samza plan.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SamzaRunner extends PipelineRunner<SamzaPipelineResult> {
   private static final Logger LOG = LoggerFactory.getLogger(SamzaRunner.class);
   private static final String BEAM_DOT_GRAPH = "beamDotGraph";
diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaRunnerOverrideConfigs.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaRunnerOverrideConfigs.java
index 498f63f..678276b 100644
--- a/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaRunnerOverrideConfigs.java
+++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/SamzaRunnerOverrideConfigs.java
@@ -20,6 +20,7 @@
 import java.time.Duration;
 
 /** A helper class for holding all the beam runner specific samza configs. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SamzaRunnerOverrideConfigs {
   public static final String BEAM_RUNNER_CONFIG_PREFIX = "beam.override.";
   // whether the job is in portable mode
diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/adapter/BoundedSourceSystem.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/adapter/BoundedSourceSystem.java
index 219147a..c6e388f 100644
--- a/runners/samza/src/main/java/org/apache/beam/runners/samza/adapter/BoundedSourceSystem.java
+++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/adapter/BoundedSourceSystem.java
@@ -67,6 +67,7 @@
  * the job is restarted the bounded source will be consumed from the beginning.
  */
 // TODO: instrumentation for the consumer
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BoundedSourceSystem {
   private static final Logger LOG = LoggerFactory.getLogger(BoundedSourceSystem.class);
 
diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/adapter/UnboundedSourceSystem.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/adapter/UnboundedSourceSystem.java
index e9a37e5..675f557 100644
--- a/runners/samza/src/main/java/org/apache/beam/runners/samza/adapter/UnboundedSourceSystem.java
+++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/adapter/UnboundedSourceSystem.java
@@ -70,6 +70,7 @@
  * A Samza system that supports reading from a Beam {@link UnboundedSource}. The source is split
  * into partitions. Samza creates the job model by assigning partitions to Samza tasks.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class UnboundedSourceSystem {
   private static final Logger LOG = LoggerFactory.getLogger(UnboundedSourceSystem.class);
 
diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/container/BeamContainerRunner.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/container/BeamContainerRunner.java
index d49c23f..62b1cb3 100644
--- a/runners/samza/src/main/java/org/apache/beam/runners/samza/container/BeamContainerRunner.java
+++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/container/BeamContainerRunner.java
@@ -33,6 +33,7 @@
 import org.slf4j.LoggerFactory;
 
 /** Runs the beam Yarn container, using the static global job model. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BeamContainerRunner implements ApplicationRunner {
   private static final Logger LOG = LoggerFactory.getLogger(BeamContainerRunner.class);
 
diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/container/ContainerCfgFactory.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/container/ContainerCfgFactory.java
index 1f4d999..304c549 100644
--- a/runners/samza/src/main/java/org/apache/beam/runners/samza/container/ContainerCfgFactory.java
+++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/container/ContainerCfgFactory.java
@@ -31,6 +31,7 @@
 import org.slf4j.LoggerFactory;
 
 /** Factory for the Beam yarn container to load job model. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ContainerCfgFactory implements ConfigFactory {
   private static final Logger LOG = LoggerFactory.getLogger(ContainerCfgFactory.class);
 
diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/metrics/SamzaMetricsContainer.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/metrics/SamzaMetricsContainer.java
index ca01e34..28b6bd8 100644
--- a/runners/samza/src/main/java/org/apache/beam/runners/samza/metrics/SamzaMetricsContainer.java
+++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/metrics/SamzaMetricsContainer.java
@@ -37,6 +37,7 @@
  * This class holds the {@link MetricsContainer}s for BEAM metrics, and update the results to Samza
  * metrics.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SamzaMetricsContainer {
   private static final String BEAM_METRICS_GROUP = "BeamMetrics";
   private static final String DELIMITER = "-";
diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/DoFnOp.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/DoFnOp.java
index d99c0c6..3513070 100644
--- a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/DoFnOp.java
+++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/DoFnOp.java
@@ -66,6 +66,7 @@
 import org.slf4j.LoggerFactory;
 
 /** Samza operator for {@link DoFn}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DoFnOp<InT, FnOutT, OutT> implements Op<InT, OutT, Void> {
   private static final Logger LOG = LoggerFactory.getLogger(DoFnOp.class);
   private static final long MIN_BUNDLE_CHECK_TIME_MS = 10L;
diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/DoFnRunnerWithKeyedInternals.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/DoFnRunnerWithKeyedInternals.java
index fce6de0..1c1cae0 100644
--- a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/DoFnRunnerWithKeyedInternals.java
+++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/DoFnRunnerWithKeyedInternals.java
@@ -30,6 +30,7 @@
 import org.joda.time.Instant;
 
 /** This class wraps a DoFnRunner with keyed StateInternals and TimerInternals access. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DoFnRunnerWithKeyedInternals<InputT, OutputT> implements DoFnRunner<InputT, OutputT> {
   private final DoFnRunner<InputT, OutputT> underlying;
   private final KeyedInternals keyedInternals;
diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/GroupByKeyOp.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/GroupByKeyOp.java
index 5262e33..59207cd 100644
--- a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/GroupByKeyOp.java
+++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/GroupByKeyOp.java
@@ -55,6 +55,7 @@
 import org.slf4j.LoggerFactory;
 
 /** Samza operator for {@link org.apache.beam.sdk.transforms.GroupByKey}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class GroupByKeyOp<K, InputT, OutputT>
     implements Op<KeyedWorkItem<K, InputT>, KV<K, OutputT>, K> {
   private static final Logger LOG = LoggerFactory.getLogger(GroupByKeyOp.class);
diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/KeyedInternals.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/KeyedInternals.java
index 430a2d8..9206c80 100644
--- a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/KeyedInternals.java
+++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/KeyedInternals.java
@@ -36,6 +36,7 @@
 
 /** Provides access to the keyed StateInternals and TimerInternals. */
 @ThreadSafe
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class KeyedInternals<K> {
 
   private static class KeyedStates<K> {
@@ -153,9 +154,8 @@
       return getInternals().currentProcessingTime();
     }
 
-    @Nullable
     @Override
-    public Instant currentSynchronizedProcessingTime() {
+    public @Nullable Instant currentSynchronizedProcessingTime() {
       return getInternals().currentSynchronizedProcessingTime();
     }
 
@@ -164,9 +164,8 @@
       return getInternals().currentInputWatermarkTime();
     }
 
-    @Nullable
     @Override
-    public Instant currentOutputWatermarkTime() {
+    public @Nullable Instant currentOutputWatermarkTime() {
       return getInternals().currentOutputWatermarkTime();
     }
   }
diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/KeyedTimerData.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/KeyedTimerData.java
index ef925f0..8a4e4d9 100644
--- a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/KeyedTimerData.java
+++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/KeyedTimerData.java
@@ -41,6 +41,7 @@
  * {@link TimerInternals.TimerData} with key, used by {@link SamzaTimerInternalsFactory}. Implements
  * {@link Comparable} by first comparing the wrapped TimerData then the key.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class KeyedTimerData<K> implements Comparable<KeyedTimerData<K>> {
   private final byte[] keyBytes;
   private final K key;
diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/OpAdapter.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/OpAdapter.java
index e663a04..b80aba4 100644
--- a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/OpAdapter.java
+++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/OpAdapter.java
@@ -34,6 +34,7 @@
 import org.slf4j.LoggerFactory;
 
 /** Adaptor class that runs a Samza {@link Op} for BEAM in the Samza {@link FlatMapFunction}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class OpAdapter<InT, OutT, K>
     implements FlatMapFunction<OpMessage<InT>, OpMessage<OutT>>,
         WatermarkFunction<OpMessage<OutT>>,
diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/OpMessage.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/OpMessage.java
index ae33322..b97051e 100644
--- a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/OpMessage.java
+++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/OpMessage.java
@@ -26,6 +26,7 @@
  * contains either an element of main inputs or the collection results from a view (used as side
  * input).
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class OpMessage<T> {
   /**
    * Type of the element(s) in the message.
diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SamzaAssignContext.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SamzaAssignContext.java
index c0b8456..f6bb25f 100644
--- a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SamzaAssignContext.java
+++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SamzaAssignContext.java
@@ -23,6 +23,7 @@
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables;
 import org.joda.time.Instant;
 
+@SuppressWarnings({"keyfor", "nullness"}) // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class SamzaAssignContext<InT, W extends BoundedWindow> extends WindowFn<InT, W>.AssignContext {
   private final WindowedValue<InT> value;
 
diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SamzaDoFnRunners.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SamzaDoFnRunners.java
index 881f287..188507c 100644
--- a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SamzaDoFnRunners.java
+++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SamzaDoFnRunners.java
@@ -55,6 +55,7 @@
 import org.joda.time.Instant;
 
 /** A factory for Samza runner translator to create underlying DoFnRunner used in {@link DoFnOp}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SamzaDoFnRunners {
 
   /** Create DoFnRunner for java runner. */
diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SamzaStoreStateInternals.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SamzaStoreStateInternals.java
index 2d03709..05f1467 100644
--- a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SamzaStoreStateInternals.java
+++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SamzaStoreStateInternals.java
@@ -77,6 +77,7 @@
 import org.joda.time.Instant;
 
 /** {@link StateInternals} that uses Samza local {@link KeyValueStore} to manage state. */
+@SuppressWarnings({"keyfor", "nullness"}) // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SamzaStoreStateInternals<K> implements StateInternals {
   static final String BEAM_STORE = "beamStore";
 
diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SamzaTimerInternalsFactory.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SamzaTimerInternalsFactory.java
index 9cb230e..f5b8389 100644
--- a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SamzaTimerInternalsFactory.java
+++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SamzaTimerInternalsFactory.java
@@ -58,6 +58,7 @@
  * of the {@link org.apache.beam.runners.core.TimerInternals.TimerData} added to the sorted timer
  * set, and removes the ready timers when the watermark is advanced.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SamzaTimerInternalsFactory<K> implements TimerInternalsFactory<K> {
   private static final Logger LOG = LoggerFactory.getLogger(SamzaTimerInternalsFactory.class);
 
diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SplittableParDoProcessKeyedElementsOp.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SplittableParDoProcessKeyedElementsOp.java
index ae5406c..72e72d8 100644
--- a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SplittableParDoProcessKeyedElementsOp.java
+++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SplittableParDoProcessKeyedElementsOp.java
@@ -58,6 +58,7 @@
 import org.slf4j.LoggerFactory;
 
 /** Samza operator for {@link org.apache.beam.sdk.transforms.GroupByKey}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SplittableParDoProcessKeyedElementsOp<
         InputT, OutputT, RestrictionT, PositionT, WatermarkEstimatorStateT>
     implements Op<KeyedWorkItem<byte[], KV<InputT, RestrictionT>>, RawUnionValue, byte[]> {
diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/ConfigBuilder.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/ConfigBuilder.java
index b3ee221..f4ca586 100644
--- a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/ConfigBuilder.java
+++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/ConfigBuilder.java
@@ -55,6 +55,7 @@
 import org.slf4j.LoggerFactory;
 
 /** Builder class to generate configs for BEAM samza runner during runtime. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ConfigBuilder {
   private static final Logger LOG = LoggerFactory.getLogger(ConfigBuilder.class);
 
diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/ConfigContext.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/ConfigContext.java
index ee9d9ea..adc9125 100644
--- a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/ConfigContext.java
+++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/ConfigContext.java
@@ -26,6 +26,7 @@
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables;
 
 /** Helper that provides context data such as output for config generation. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ConfigContext {
   private final Map<PValue, String> idMap;
   private AppliedPTransform<?, ?, ?> currentTransform;
diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/FlattenPCollectionsTranslator.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/FlattenPCollectionsTranslator.java
index 9e21614..50e62a2 100644
--- a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/FlattenPCollectionsTranslator.java
+++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/FlattenPCollectionsTranslator.java
@@ -32,7 +32,6 @@
 import org.apache.beam.sdk.runners.TransformHierarchy;
 import org.apache.beam.sdk.transforms.Flatten;
 import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PValue;
 import org.apache.beam.sdk.values.TupleTag;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables;
 import org.apache.samza.operators.MessageStream;
@@ -52,16 +51,7 @@
     final PCollection<T> output = ctx.getOutput(transform);
 
     final List<MessageStream<OpMessage<T>>> inputStreams = new ArrayList<>();
-    for (Map.Entry<TupleTag<?>, PValue> taggedPValue : node.getInputs().entrySet()) {
-      if (!(taggedPValue.getValue() instanceof PCollection)) {
-        throw new IllegalArgumentException(
-            String.format(
-                "Got non-PCollection input for flatten. Tag: %s. Input: %s. Type: %s",
-                taggedPValue.getKey(),
-                taggedPValue.getValue(),
-                taggedPValue.getValue().getClass()));
-      }
-
+    for (Map.Entry<TupleTag<?>, PCollection<?>> taggedPValue : node.getInputs().entrySet()) {
       @SuppressWarnings("unchecked")
       final PCollection<T> input = (PCollection<T>) taggedPValue.getValue();
       inputStreams.add(ctx.getMessageStream(input));
diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/GroupByKeyTranslator.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/GroupByKeyTranslator.java
index a14ac4c..6d2b2b6 100644
--- a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/GroupByKeyTranslator.java
+++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/GroupByKeyTranslator.java
@@ -53,6 +53,7 @@
 import org.apache.samza.serializers.KVSerde;
 
 /** Translates {@link GroupByKey} to Samza {@link GroupByKeyOp}. */
+@SuppressWarnings({"keyfor", "nullness"}) // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class GroupByKeyTranslator<K, InputT, OutputT>
     implements TransformTranslator<
         PTransform<PCollection<KV<K, InputT>>, PCollection<KV<K, OutputT>>>> {
diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/PViewToIdMapper.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/PViewToIdMapper.java
index 08bcc71..735cbc8 100644
--- a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/PViewToIdMapper.java
+++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/PViewToIdMapper.java
@@ -29,6 +29,7 @@
  * This class generates an ID for each {@link PValue} during a topological traversal of the BEAM
  * {@link Pipeline}.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class PViewToIdMapper extends Pipeline.PipelineVisitor.Defaults {
   private final Map<PValue, String> idMap = new HashMap<>();
   private int nextId;
diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/ParDoBoundMultiTranslator.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/ParDoBoundMultiTranslator.java
index 78b0f9d..4ef913a 100644
--- a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/ParDoBoundMultiTranslator.java
+++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/ParDoBoundMultiTranslator.java
@@ -52,7 +52,6 @@
 import org.apache.beam.sdk.util.WindowedValue;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PCollectionView;
-import org.apache.beam.sdk.values.PValue;
 import org.apache.beam.sdk.values.TupleTag;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterators;
 import org.apache.samza.operators.MessageStream;
@@ -64,6 +63,7 @@
  * Translates {@link org.apache.beam.sdk.transforms.ParDo.MultiOutput} or ExecutableStage in
  * portable api to Samza {@link DoFnOp}.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class ParDoBoundMultiTranslator<InT, OutT>
     implements TransformTranslator<ParDo.MultiOutput<InT, OutT>>,
         TransformConfigGenerator<ParDo.MultiOutput<InT, OutT>> {
@@ -112,21 +112,21 @@
         transform.getSideInputs().values().stream()
             .map(ctx::<InT>getViewStream)
             .collect(Collectors.toList());
-    final ArrayList<Map.Entry<TupleTag<?>, PValue>> outputs =
+    final ArrayList<Map.Entry<TupleTag<?>, PCollection<?>>> outputs =
         new ArrayList<>(node.getOutputs().entrySet());
 
     final Map<TupleTag<?>, Integer> tagToIndexMap = new HashMap<>();
     final Map<Integer, PCollection<?>> indexToPCollectionMap = new HashMap<>();
 
     for (int index = 0; index < outputs.size(); ++index) {
-      final Map.Entry<TupleTag<?>, PValue> taggedOutput = outputs.get(index);
+      final Map.Entry<TupleTag<?>, PCollection<?>> taggedOutput = outputs.get(index);
       tagToIndexMap.put(taggedOutput.getKey(), index);
 
       if (!(taggedOutput.getValue() instanceof PCollection)) {
         throw new IllegalArgumentException(
             "Expected side output to be PCollection, but was: " + taggedOutput.getValue());
       }
-      final PCollection<?> sideOutputCollection = (PCollection<?>) taggedOutput.getValue();
+      final PCollection<?> sideOutputCollection = taggedOutput.getValue();
       indexToPCollectionMap.put(index, sideOutputCollection);
     }
 
diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/PortableTranslationContext.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/PortableTranslationContext.java
index c40913d..ef6524e 100644
--- a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/PortableTranslationContext.java
+++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/PortableTranslationContext.java
@@ -52,6 +52,7 @@
  * provides other context data such as input and output of a {@link
  * org.apache.beam.model.pipeline.v1.RunnerApi.PTransform}.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class PortableTranslationContext {
   private final Map<String, MessageStream<?>> messsageStreams = new HashMap<>();
   private final StreamApplicationDescriptor appDescriptor;
diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SamzaImpulseSystemFactory.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SamzaImpulseSystemFactory.java
index cb7413d..45743c3 100644
--- a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SamzaImpulseSystemFactory.java
+++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SamzaImpulseSystemFactory.java
@@ -43,6 +43,7 @@
  * This is a trivial system for generating impulse event in Samza when translating IMPULSE transform
  * in portable api.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SamzaImpulseSystemFactory implements SystemFactory {
   @Override
   public SystemConsumer getConsumer(
diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SamzaPipelineTranslator.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SamzaPipelineTranslator.java
index c4424d6..a9302f3 100644
--- a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SamzaPipelineTranslator.java
+++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SamzaPipelineTranslator.java
@@ -38,6 +38,7 @@
 import org.slf4j.LoggerFactory;
 
 /** This class knows all the translators from a primitive BEAM transform to a Samza operator. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SamzaPipelineTranslator {
   private static final Logger LOG = LoggerFactory.getLogger(SamzaPipelineTranslator.class);
 
diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SamzaPortablePipelineTranslator.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SamzaPortablePipelineTranslator.java
index 372b362..987bda0 100644
--- a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SamzaPortablePipelineTranslator.java
+++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SamzaPortablePipelineTranslator.java
@@ -32,6 +32,7 @@
  * Portable specific samza pipeline translator. This is the entry point for translating a portable
  * pipeline
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SamzaPortablePipelineTranslator {
   private static final Logger LOG = LoggerFactory.getLogger(SamzaPortablePipelineTranslator.class);
 
diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SplittableParDoTranslators.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SplittableParDoTranslators.java
index 87f2e8f..b7b70e7 100644
--- a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SplittableParDoTranslators.java
+++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SplittableParDoTranslators.java
@@ -38,13 +38,13 @@
 import org.apache.beam.sdk.util.WindowedValue;
 import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.sdk.values.PValue;
 import org.apache.beam.sdk.values.TupleTag;
 import org.apache.beam.sdk.values.WindowingStrategy;
 import org.apache.samza.operators.MessageStream;
 import org.apache.samza.serializers.KVSerde;
 
 /** A set of translators for {@link SplittableParDo}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SplittableParDoTranslators {
 
   /**
@@ -65,14 +65,14 @@
         TranslationContext ctx) {
       final PCollection<KV<byte[], KV<InputT, RestrictionT>>> input = ctx.getInput(transform);
 
-      final ArrayList<Map.Entry<TupleTag<?>, PValue>> outputs =
+      final ArrayList<Map.Entry<TupleTag<?>, PCollection<?>>> outputs =
           new ArrayList<>(node.getOutputs().entrySet());
 
       final Map<TupleTag<?>, Integer> tagToIndexMap = new HashMap<>();
       final Map<Integer, PCollection<?>> indexToPCollectionMap = new HashMap<>();
 
       for (int index = 0; index < outputs.size(); ++index) {
-        final Map.Entry<TupleTag<?>, PValue> taggedOutput = outputs.get(index);
+        final Map.Entry<TupleTag<?>, PCollection<?>> taggedOutput = outputs.get(index);
         tagToIndexMap.put(taggedOutput.getKey(), index);
 
         if (!(taggedOutput.getValue() instanceof PCollection)) {
diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/TranslationContext.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/TranslationContext.java
index fc0d4a7..6d983b2 100644
--- a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/TranslationContext.java
+++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/TranslationContext.java
@@ -65,6 +65,7 @@
  * MessageStream}. It also provides other context data such as input and output of a {@link
  * PTransform}.
  */
+@SuppressWarnings({"keyfor", "nullness"}) // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class TranslationContext {
   private static final Logger LOG = LoggerFactory.getLogger(TranslationContext.class);
   private final StreamApplicationDescriptor appDescriptor;
diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/util/SamzaCoders.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/util/SamzaCoders.java
index eb9e92b..c97388c 100644
--- a/runners/samza/src/main/java/org/apache/beam/runners/samza/util/SamzaCoders.java
+++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/util/SamzaCoders.java
@@ -27,6 +27,7 @@
 import org.apache.samza.serializers.Serde;
 
 /** Utils for Coders creation/conversion in Samza. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SamzaCoders {
 
   private SamzaCoders() {}
diff --git a/runners/samza/src/test/java/org/apache/beam/runners/samza/adapter/BoundedSourceSystemTest.java b/runners/samza/src/test/java/org/apache/beam/runners/samza/adapter/BoundedSourceSystemTest.java
index ca31f86..3bd3820 100644
--- a/runners/samza/src/test/java/org/apache/beam/runners/samza/adapter/BoundedSourceSystemTest.java
+++ b/runners/samza/src/test/java/org/apache/beam/runners/samza/adapter/BoundedSourceSystemTest.java
@@ -49,6 +49,7 @@
 import org.junit.Test;
 
 /** Tests for {@link BoundedSourceSystem}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BoundedSourceSystemTest {
   private static final SystemStreamPartition DEFAULT_SSP =
       new SystemStreamPartition("default-system", "default-system", new Partition(0));
diff --git a/runners/samza/src/test/java/org/apache/beam/runners/samza/adapter/TestSourceHelpers.java b/runners/samza/src/test/java/org/apache/beam/runners/samza/adapter/TestSourceHelpers.java
index b020a9a..8daa553 100644
--- a/runners/samza/src/test/java/org/apache/beam/runners/samza/adapter/TestSourceHelpers.java
+++ b/runners/samza/src/test/java/org/apache/beam/runners/samza/adapter/TestSourceHelpers.java
@@ -35,6 +35,7 @@
 import org.joda.time.Instant;
 
 /** Helper classes and functions to build source for testing. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class TestSourceHelpers {
 
   private TestSourceHelpers() {}
diff --git a/runners/samza/src/test/java/org/apache/beam/runners/samza/adapter/TestUnboundedSource.java b/runners/samza/src/test/java/org/apache/beam/runners/samza/adapter/TestUnboundedSource.java
index 0439354..d494400 100644
--- a/runners/samza/src/test/java/org/apache/beam/runners/samza/adapter/TestUnboundedSource.java
+++ b/runners/samza/src/test/java/org/apache/beam/runners/samza/adapter/TestUnboundedSource.java
@@ -43,6 +43,7 @@
  *
  * @param <T> element type
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class TestUnboundedSource<T> extends UnboundedSource<T, TestCheckpointMark> {
   // each list of events is a split
   private final List<List<Event<T>>> events;
diff --git a/runners/samza/src/test/java/org/apache/beam/runners/samza/adapter/UnboundedSourceSystemTest.java b/runners/samza/src/test/java/org/apache/beam/runners/samza/adapter/UnboundedSourceSystemTest.java
index 8cc2ba7..42bcf7c 100644
--- a/runners/samza/src/test/java/org/apache/beam/runners/samza/adapter/UnboundedSourceSystemTest.java
+++ b/runners/samza/src/test/java/org/apache/beam/runners/samza/adapter/UnboundedSourceSystemTest.java
@@ -52,6 +52,7 @@
 import org.junit.Test;
 
 /** Tests for {@link UnboundedSourceSystem}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class UnboundedSourceSystemTest {
 
   // A reasonable time to wait to get all messages from the source assuming no blocking.
diff --git a/runners/samza/src/test/java/org/apache/beam/runners/samza/runtime/SamzaStoreStateInternalsTest.java b/runners/samza/src/test/java/org/apache/beam/runners/samza/runtime/SamzaStoreStateInternalsTest.java
index 5f88e4d..f8ec385 100644
--- a/runners/samza/src/test/java/org/apache/beam/runners/samza/runtime/SamzaStoreStateInternalsTest.java
+++ b/runners/samza/src/test/java/org/apache/beam/runners/samza/runtime/SamzaStoreStateInternalsTest.java
@@ -70,6 +70,7 @@
 import org.junit.Test;
 
 /** Tests for SamzaStoreStateInternals. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SamzaStoreStateInternalsTest implements Serializable {
   @Rule
   public final transient TestPipeline pipeline =
diff --git a/runners/samza/src/test/java/org/apache/beam/runners/samza/runtime/SamzaTimerInternalsFactoryTest.java b/runners/samza/src/test/java/org/apache/beam/runners/samza/runtime/SamzaTimerInternalsFactoryTest.java
index 02b1cad..8c1a521 100644
--- a/runners/samza/src/test/java/org/apache/beam/runners/samza/runtime/SamzaTimerInternalsFactoryTest.java
+++ b/runners/samza/src/test/java/org/apache/beam/runners/samza/runtime/SamzaTimerInternalsFactoryTest.java
@@ -67,6 +67,7 @@
  * Tests for {@link SamzaTimerInternalsFactory}. Covers both event-time timers and processing-timer
  * timers.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SamzaTimerInternalsFactoryTest {
   @Rule public transient TemporaryFolder temporaryFolder = new TemporaryFolder();
 
diff --git a/runners/samza/src/test/java/org/apache/beam/runners/samza/translation/ConfigGeneratorTest.java b/runners/samza/src/test/java/org/apache/beam/runners/samza/translation/ConfigGeneratorTest.java
index c4b910a..fb1be41 100644
--- a/runners/samza/src/test/java/org/apache/beam/runners/samza/translation/ConfigGeneratorTest.java
+++ b/runners/samza/src/test/java/org/apache/beam/runners/samza/translation/ConfigGeneratorTest.java
@@ -50,6 +50,7 @@
 import org.junit.Test;
 
 /** Test config generations for {@link org.apache.beam.runners.samza.SamzaRunner}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ConfigGeneratorTest {
   private static final String APP_RUNNER_CLASS = "app.runner.class";
   private static final String JOB_FACTORY_CLASS = "job.factory.class";
diff --git a/runners/samza/src/test/java/org/apache/beam/runners/samza/translation/SamzaImpulseSystemTest.java b/runners/samza/src/test/java/org/apache/beam/runners/samza/translation/SamzaImpulseSystemTest.java
index 0cfddf5..bdd2f46 100644
--- a/runners/samza/src/test/java/org/apache/beam/runners/samza/translation/SamzaImpulseSystemTest.java
+++ b/runners/samza/src/test/java/org/apache/beam/runners/samza/translation/SamzaImpulseSystemTest.java
@@ -34,6 +34,7 @@
  * Tests for {@link
  * org.apache.beam.runners.samza.translation.SamzaImpulseSystemFactory.SamzaImpulseSystemConsumer}.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SamzaImpulseSystemTest {
   @Test
   public void testSamzaImpulseSystemConsumer() throws Exception {
diff --git a/runners/samza/src/test/java/org/apache/beam/runners/samza/util/TestHashIdGenerator.java b/runners/samza/src/test/java/org/apache/beam/runners/samza/util/TestHashIdGenerator.java
index 881ce7f..1f0ff18 100644
--- a/runners/samza/src/test/java/org/apache/beam/runners/samza/util/TestHashIdGenerator.java
+++ b/runners/samza/src/test/java/org/apache/beam/runners/samza/util/TestHashIdGenerator.java
@@ -31,6 +31,7 @@
 import org.junit.Test;
 
 /** Test class for {@link HashIdGenerator}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class TestHashIdGenerator {
 
   @Test
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkJobInvoker.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkJobInvoker.java
index eed0f6b..8e90293 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkJobInvoker.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkJobInvoker.java
@@ -34,6 +34,7 @@
 import org.slf4j.LoggerFactory;
 
 /** Creates a job invocation to manage the Spark runner's execution of a portable pipeline. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SparkJobInvoker extends JobInvoker {
 
   private static final Logger LOG = LoggerFactory.getLogger(SparkJobInvoker.class);
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkNativePipelineVisitor.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkNativePipelineVisitor.java
index 077203c..043ed25 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkNativePipelineVisitor.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkNativePipelineVisitor.java
@@ -38,6 +38,7 @@
  * Pipeline visitor for translating a Beam pipeline into equivalent Spark operations. Used for
  * debugging purposes using {@link SparkRunnerDebugger}.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SparkNativePipelineVisitor extends SparkRunner.Evaluator {
   private final List<NativeTransform> transforms;
   private final List<String> knownCompositesPackages =
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineResult.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineResult.java
index 5c0656d..16a2595 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineResult.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineResult.java
@@ -39,6 +39,7 @@
 import org.joda.time.Duration;
 
 /** Represents a Spark pipeline execution result. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public abstract class SparkPipelineResult implements PipelineResult {
 
   final Future pipelineExecution;
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineRunner.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineRunner.java
index 9ba700f..bb220c4 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineRunner.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkPipelineRunner.java
@@ -66,6 +66,7 @@
 import org.slf4j.LoggerFactory;
 
 /** Runs a portable pipeline on Apache Spark. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SparkPipelineRunner implements PortablePipelineRunner {
 
   private static final Logger LOG = LoggerFactory.getLogger(SparkPipelineRunner.class);
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunner.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunner.java
index 35e85e0..283b0c6 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunner.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunner.java
@@ -89,6 +89,7 @@
  * SparkPipelineOptionsFactory.create(); options.setSparkMaster("spark://host:port");
  * SparkPipelineResult result = (SparkPipelineResult) p.run(); }
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public final class SparkRunner extends PipelineRunner<SparkPipelineResult> {
 
   private static final Logger LOG = LoggerFactory.getLogger(SparkRunner.class);
@@ -453,7 +454,7 @@
       // --- determine if node is bounded/unbounded.
       // usually, the input determines if the PCollection to apply the next transformation to
       // is BOUNDED or UNBOUNDED, meaning RDD/DStream.
-      Map<TupleTag<?>, PValue> pValues;
+      Map<TupleTag<?>, PCollection<?>> pValues;
       if (node.getInputs().isEmpty()) {
         // in case of a PBegin, it's the output.
         pValues = node.getOutputs();
@@ -468,18 +469,14 @@
           : translator.translateUnbounded(transform);
     }
 
-    protected PCollection.IsBounded isBoundedCollection(Collection<PValue> pValues) {
+    protected PCollection.IsBounded isBoundedCollection(Collection<PCollection<?>> pValues) {
       // anything that is not a PCollection, is BOUNDED.
       // For PCollections:
       // BOUNDED behaves as the Identity Element, BOUNDED + BOUNDED = BOUNDED
       // while BOUNDED + UNBOUNDED = UNBOUNDED.
       PCollection.IsBounded isBounded = PCollection.IsBounded.BOUNDED;
-      for (PValue pValue : pValues) {
-        if (pValue instanceof PCollection) {
-          isBounded = isBounded.and(((PCollection) pValue).isBounded());
-        } else {
-          isBounded = isBounded.and(PCollection.IsBounded.BOUNDED);
-        }
+      for (PCollection<?> pValue : pValues) {
+        isBounded = isBounded.and(pValue.isBounded());
       }
       return isBounded;
     }
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunnerDebugger.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunnerDebugger.java
index 5ff3aa4..9a549c9 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunnerDebugger.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunnerDebugger.java
@@ -50,6 +50,7 @@
  * String sparkPipeline = result.getDebugString();
  * }</pre>
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public final class SparkRunnerDebugger extends PipelineRunner<SparkPipelineResult> {
 
   private static final Logger LOG = LoggerFactory.getLogger(SparkRunnerDebugger.class);
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/TestSparkRunner.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/TestSparkRunner.java
index 1907fab..7ec19ac 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/TestSparkRunner.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/TestSparkRunner.java
@@ -57,6 +57,7 @@
  * SparkPipelineOptionsFactory.create(); options.setSparkMaster("spark://host:port");
  * SparkPipelineResult result = (SparkPipelineResult) p.run(); }
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public final class TestSparkRunner extends PipelineRunner<SparkPipelineResult> {
 
   private static final Logger LOG = LoggerFactory.getLogger(TestSparkRunner.class);
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/AggregatorsAccumulator.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/AggregatorsAccumulator.java
index f6019da..f9cc770 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/AggregatorsAccumulator.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/AggregatorsAccumulator.java
@@ -38,6 +38,7 @@
  * @see <a
  *     href="https://spark.apache.org/docs/latest/api/java/org/apache/spark/util/AccumulatorV2.html">accumulatorsV2</a>
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class AggregatorsAccumulator {
   private static final Logger LOG = LoggerFactory.getLogger(AggregatorsAccumulator.class);
 
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/NamedAggregators.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/NamedAggregators.java
index 450788a..31834a7 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/NamedAggregators.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/NamedAggregators.java
@@ -30,6 +30,7 @@
  * create a map of named aggregators and instantiate in the the spark context before the job is
  * launched. We can then add aggregators on the fly in Spark.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class NamedAggregators implements Serializable {
   /** Map from aggregator name to current state. */
   private final Map<String, State<?, ?, ?>> mNamedAggregators = new TreeMap<>();
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/CoderHelpers.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/CoderHelpers.java
index 45b5d1b..64407eb 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/CoderHelpers.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/CoderHelpers.java
@@ -38,6 +38,7 @@
 import scala.Tuple2;
 
 /** Serialization utility class. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public final class CoderHelpers {
   private CoderHelpers() {}
 
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/StatelessJavaSerializer.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/StatelessJavaSerializer.java
index d546156..19ce29d 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/StatelessJavaSerializer.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/StatelessJavaSerializer.java
@@ -43,6 +43,7 @@
  * https://issues.apache.org/jira/browse/GROOVY-1627
  * https://github.com/spring-projects/spring-loaded/issues/107
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class StatelessJavaSerializer extends Serializer {
 
   // Since Kryo uses reflection to sequentially look for constructor signatures, starting
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/CreateStream.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/CreateStream.java
index 708bf89..328bc48 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/CreateStream.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/CreateStream.java
@@ -86,6 +86,7 @@
  * @param <T> The type of the element in this stream.
  */
 // TODO: write a proper Builder enforcing all those rules mentioned.
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public final class CreateStream<T> extends PTransform<PBegin, PCollection<T>> {
   public static final String TRANSFORM_URN = "beam:transform:spark:createstream:v1";
 
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/MicrobatchSource.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/MicrobatchSource.java
index b0c778e..723470c 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/MicrobatchSource.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/MicrobatchSource.java
@@ -46,6 +46,7 @@
  * A {@link Source} that accommodates Spark's micro-batch oriented nature and wraps an {@link
  * UnboundedSource}.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class MicrobatchSource<T, CheckpointMarkT extends UnboundedSource.CheckpointMark>
     extends Source<T> {
 
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/SourceDStream.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/SourceDStream.java
index 8c5de3c..e9e462a 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/SourceDStream.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/SourceDStream.java
@@ -50,6 +50,7 @@
  * SparkPipelineOptions#getMinReadTimeMillis()}. Records bound is controlled by the {@link
  * RateController} mechanism.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class SourceDStream<T, CheckpointMarkT extends UnboundedSource.CheckpointMark>
     extends InputDStream<Tuple2<Source<T>, CheckpointMarkT>> {
   private static final Logger LOG = LoggerFactory.getLogger(SourceDStream.class);
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/SourceRDD.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/SourceRDD.java
index 3267696..0c9d34c 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/SourceRDD.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/SourceRDD.java
@@ -52,6 +52,7 @@
 import scala.collection.JavaConversions;
 
 /** Classes implementing Beam {@link Source} {@link RDD}s. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SourceRDD {
 
   /**
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/SparkUnboundedSource.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/SparkUnboundedSource.java
index 9103b2d..c7d93e1 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/SparkUnboundedSource.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/SparkUnboundedSource.java
@@ -77,6 +77,7 @@
  *       while reporting the properties of the read (such as number of records) to the tracker.
  * </ul>
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SparkUnboundedSource {
 
   public static <T, CheckpointMarkT extends CheckpointMark> UnboundedDataset<T> read(
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/metrics/MetricsAccumulator.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/metrics/MetricsAccumulator.java
index 922838d..d5e209e 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/metrics/MetricsAccumulator.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/metrics/MetricsAccumulator.java
@@ -39,6 +39,7 @@
  * @see <a
  *     href="https://spark.apache.org/docs/2.4.4/streaming-programming-guide.html#accumulators-broadcast-variables-and-checkpoints">accumulatorsV2</a>
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class MetricsAccumulator {
   private static final Logger LOG = LoggerFactory.getLogger(MetricsAccumulator.class);
 
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/metrics/WithMetricsSupport.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/metrics/WithMetricsSupport.java
index 9450986..337647f 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/metrics/WithMetricsSupport.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/metrics/WithMetricsSupport.java
@@ -47,6 +47,7 @@
  * <p>{@link MetricRegistry} is not an interface, so this is not a by-the-book decorator. That said,
  * it delegates all metric related getters to the "decorated" instance.
  */
+@SuppressWarnings({"keyfor", "nullness"}) // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class WithMetricsSupport extends MetricRegistry {
 
   private static final Logger LOG = LoggerFactory.getLogger(WithMetricsSupport.class);
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/stateful/SparkGroupAlsoByWindowViaWindowSet.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/stateful/SparkGroupAlsoByWindowViaWindowSet.java
index b741050..131d3c8 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/stateful/SparkGroupAlsoByWindowViaWindowSet.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/stateful/SparkGroupAlsoByWindowViaWindowSet.java
@@ -93,6 +93,7 @@
  * bounds the types of state and output to be the same, a (state, output) tuple is used, filtering
  * the state (and output if no firing) in the following steps.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SparkGroupAlsoByWindowViaWindowSet implements Serializable {
   private static final Logger LOG =
       LoggerFactory.getLogger(SparkGroupAlsoByWindowViaWindowSet.class);
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/stateful/SparkStateInternals.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/stateful/SparkStateInternals.java
index d2673a3..bd2541e 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/stateful/SparkStateInternals.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/stateful/SparkStateInternals.java
@@ -48,6 +48,7 @@
 import org.joda.time.Instant;
 
 /** An implementation of {@link StateInternals} for the SparkRunner. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class SparkStateInternals<K> implements StateInternals {
 
   private final K key;
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/stateful/SparkTimerInternals.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/stateful/SparkTimerInternals.java
index a0c62a5..ab9b35d 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/stateful/SparkTimerInternals.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/stateful/SparkTimerInternals.java
@@ -37,6 +37,7 @@
 import org.joda.time.Instant;
 
 /** An implementation of {@link TimerInternals} for the SparkRunner. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SparkTimerInternals implements TimerInternals {
   private final Instant highWatermark;
   private final Instant synchronizedProcessingTime;
@@ -131,9 +132,8 @@
     return Instant.now();
   }
 
-  @Nullable
   @Override
-  public Instant currentSynchronizedProcessingTime() {
+  public @Nullable Instant currentSynchronizedProcessingTime() {
     return synchronizedProcessingTime;
   }
 
@@ -147,9 +147,8 @@
     inputWatermark = highWatermark;
   }
 
-  @Nullable
   @Override
-  public Instant currentOutputWatermarkTime() {
+  public @Nullable Instant currentOutputWatermarkTime() {
     return null;
   }
 
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/stateful/StateSpecFunctions.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/stateful/StateSpecFunctions.java
index 1398f8e..544dbd5 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/stateful/StateSpecFunctions.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/stateful/StateSpecFunctions.java
@@ -52,6 +52,7 @@
 import scala.runtime.AbstractFunction3;
 
 /** A class containing {@link org.apache.spark.streaming.StateSpec} mappingFunctions. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class StateSpecFunctions {
   private static final Logger LOG = LoggerFactory.getLogger(StateSpecFunctions.class);
 
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/SparkStructuredStreamingPipelineResult.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/SparkStructuredStreamingPipelineResult.java
index 62369fb..a5dd211 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/SparkStructuredStreamingPipelineResult.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/SparkStructuredStreamingPipelineResult.java
@@ -36,6 +36,7 @@
 import org.joda.time.Duration;
 
 /** Represents a Spark pipeline execution result. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SparkStructuredStreamingPipelineResult implements PipelineResult {
 
   final Future pipelineExecution;
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/SparkStructuredStreamingRunner.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/SparkStructuredStreamingRunner.java
index 773c5f9..1e29a67 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/SparkStructuredStreamingRunner.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/SparkStructuredStreamingRunner.java
@@ -67,6 +67,7 @@
  * SparkStructuredStreamingPipelineResult result = (SparkStructuredStreamingPipelineResult) p.run();
  * }
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public final class SparkStructuredStreamingRunner
     extends PipelineRunner<SparkStructuredStreamingPipelineResult> {
 
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/aggregators/AggregatorsAccumulator.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/aggregators/AggregatorsAccumulator.java
index 51667d8..7ef3b67 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/aggregators/AggregatorsAccumulator.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/aggregators/AggregatorsAccumulator.java
@@ -29,6 +29,7 @@
  * @see <a
  *     href="https://spark.apache.org/docs/latest/api/java/org/apache/spark/util/AccumulatorV2.html">accumulatorsV2</a>
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class AggregatorsAccumulator {
   private static final Logger LOG = LoggerFactory.getLogger(AggregatorsAccumulator.class);
 
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/aggregators/NamedAggregators.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/aggregators/NamedAggregators.java
index 49b50b7..e052d7c 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/aggregators/NamedAggregators.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/aggregators/NamedAggregators.java
@@ -30,6 +30,7 @@
  * create a map of named aggregators and instantiate in the the spark context before the job is
  * launched. We can then add aggregators on the fly in Spark.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class NamedAggregators implements Serializable {
   /** Map from aggregator name to current state. */
   private final Map<String, State<?, ?, ?>> mNamedAggregators = new TreeMap<>();
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/metrics/MetricsAccumulator.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/metrics/MetricsAccumulator.java
index abd35a5..9e6563b 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/metrics/MetricsAccumulator.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/metrics/MetricsAccumulator.java
@@ -30,6 +30,7 @@
  * @see <a
  *     href="https://spark.apache.org/docs/2.4.4/streaming-programming-guide.html#accumulators-broadcast-variables-and-checkpoints">accumulatorsV2</a>
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class MetricsAccumulator {
   private static final Logger LOG = LoggerFactory.getLogger(MetricsAccumulator.class);
 
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/metrics/WithMetricsSupport.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/metrics/WithMetricsSupport.java
index d567be9..13541b3 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/metrics/WithMetricsSupport.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/metrics/WithMetricsSupport.java
@@ -47,6 +47,7 @@
  * <p>{@link MetricRegistry} is not an interface, so this is not a by-the-book decorator. That said,
  * it delegates all metric related getters to the "decorated" instance.
  */
+@SuppressWarnings({"keyfor", "nullness"}) // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class WithMetricsSupport extends MetricRegistry {
 
   private static final Logger LOG = LoggerFactory.getLogger(WithMetricsSupport.class);
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/PipelineTranslator.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/PipelineTranslator.java
index 8e7277d..cb8f84c 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/PipelineTranslator.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/PipelineTranslator.java
@@ -38,6 +38,7 @@
  * PipelineTranslatorStreaming}. If we have a batch job, it is instantiated as a {@link
  * PipelineTranslatorBatch}.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public abstract class PipelineTranslator extends Pipeline.PipelineVisitor.Defaults {
   private int depth = 0;
   private static final Logger LOG = LoggerFactory.getLogger(PipelineTranslator.class);
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/TranslationContext.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/TranslationContext.java
index dc48388..92f1d48 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/TranslationContext.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/TranslationContext.java
@@ -52,6 +52,7 @@
  * Base class that gives a context for {@link PTransform} translation: keeping track of the
  * datasets, the {@link SparkSession}, the current transform being translated.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class TranslationContext {
 
   private static final Logger LOG = LoggerFactory.getLogger(TranslationContext.class);
@@ -173,7 +174,7 @@
     return Iterables.getOnlyElement(TransformInputs.nonAdditionalInputs(currentTransform));
   }
 
-  public Map<TupleTag<?>, PValue> getInputs() {
+  public Map<TupleTag<?>, PCollection<?>> getInputs() {
     return currentTransform.getInputs();
   }
 
@@ -181,7 +182,7 @@
     return Iterables.getOnlyElement(currentTransform.getOutputs().values());
   }
 
-  public Map<TupleTag<?>, PValue> getOutputs() {
+  public Map<TupleTag<?>, PCollection<?>> getOutputs() {
     return currentTransform.getOutputs();
   }
 
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/DoFnFunction.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/DoFnFunction.java
index d26a358..690b8fa 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/DoFnFunction.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/DoFnFunction.java
@@ -52,6 +52,7 @@
  * all outputs with the output number. Afterwards a filter will filter out those elements that are
  * not to be in a specific output.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DoFnFunction<InputT, OutputT>
     implements MapPartitionsFunction<WindowedValue<InputT>, Tuple2<TupleTag<?>, WindowedValue<?>>> {
 
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/FlattenTranslatorBatch.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/FlattenTranslatorBatch.java
index bd9a4a7..b712540 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/FlattenTranslatorBatch.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/FlattenTranslatorBatch.java
@@ -29,13 +29,14 @@
 import org.apache.beam.sdk.values.PValue;
 import org.apache.spark.sql.Dataset;
 
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class FlattenTranslatorBatch<T>
     implements TransformTranslator<PTransform<PCollectionList<T>, PCollection<T>>> {
 
   @Override
   public void translateTransform(
       PTransform<PCollectionList<T>, PCollection<T>> transform, TranslationContext context) {
-    Collection<PValue> pcollectionList = context.getInputs().values();
+    Collection<PCollection<?>> pcollectionList = context.getInputs().values();
     Dataset<WindowedValue<T>> result = null;
     if (pcollectionList.isEmpty()) {
       result = context.emptyDataset();
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/ParDoTranslatorBatch.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/ParDoTranslatorBatch.java
index e165f94..6e968f8 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/ParDoTranslatorBatch.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/ParDoTranslatorBatch.java
@@ -59,6 +59,7 @@
  * @param <InputT>
  * @param <OutputT>
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class ParDoTranslatorBatch<InputT, OutputT>
     implements TransformTranslator<PTransform<PCollection<InputT>, PCollectionTuple>> {
 
@@ -89,7 +90,7 @@
     // Init main variables
     PValue input = context.getInput();
     Dataset<WindowedValue<InputT>> inputDataSet = context.getDataset(input);
-    Map<TupleTag<?>, PValue> outputs = context.getOutputs();
+    Map<TupleTag<?>, PCollection<?>> outputs = context.getOutputs();
     TupleTag<?> mainOutputTag = getTupleTag(context);
     List<TupleTag<?>> outputTags = new ArrayList<>(outputs.keySet());
     WindowingStrategy<?, ?> windowingStrategy =
@@ -142,7 +143,7 @@
         inputDataSet.mapPartitions(doFnWrapper, EncoderHelpers.fromBeamCoder(multipleOutputCoder));
     if (outputs.entrySet().size() > 1) {
       allOutputs.persist();
-      for (Map.Entry<TupleTag<?>, PValue> output : outputs.entrySet()) {
+      for (Map.Entry<TupleTag<?>, PCollection<?>> output : outputs.entrySet()) {
         pruneOutputFilteredByTag(context, allOutputs, output, windowCoder);
       }
     } else {
@@ -219,7 +220,7 @@
   private void pruneOutputFilteredByTag(
       TranslationContext context,
       Dataset<Tuple2<TupleTag<?>, WindowedValue<?>>> allOutputs,
-      Map.Entry<TupleTag<?>, PValue> output,
+      Map.Entry<TupleTag<?>, PCollection<?>> output,
       Coder<? extends BoundedWindow> windowCoder) {
     Dataset<Tuple2<TupleTag<?>, WindowedValue<?>>> filteredDataset =
         allOutputs.filter(new DoFnFilterFunction(output.getKey()));
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/PipelineTranslatorBatch.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/PipelineTranslatorBatch.java
index 9ef2cb2..dee6478 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/PipelineTranslatorBatch.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/PipelineTranslatorBatch.java
@@ -41,6 +41,7 @@
  * only the components specific to batch: registry of batch {@link TransformTranslator} and registry
  * lookup code.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class PipelineTranslatorBatch extends PipelineTranslator {
 
   // --------------------------------------------------------------------------------------------
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/WindowAssignTranslatorBatch.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/WindowAssignTranslatorBatch.java
index 4ac8a3f..9ade3cb 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/WindowAssignTranslatorBatch.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/WindowAssignTranslatorBatch.java
@@ -28,6 +28,7 @@
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.spark.sql.Dataset;
 
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class WindowAssignTranslatorBatch<T>
     implements TransformTranslator<PTransform<PCollection<T>, PCollection<T>>> {
 
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/functions/SparkSideInputReader.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/functions/SparkSideInputReader.java
index 2ffd369..e21676f 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/functions/SparkSideInputReader.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/functions/SparkSideInputReader.java
@@ -46,6 +46,7 @@
 import org.checkerframework.checker.nullness.qual.Nullable;
 
 /** A {@link SideInputReader} for the Spark Batch Runner. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SparkSideInputReader implements SideInputReader {
   private static final Set<String> SUPPORTED_MATERIALIZATIONS =
       ImmutableSet.of(
@@ -74,9 +75,8 @@
     this.broadcastStateData = broadcastStateData;
   }
 
-  @Nullable
   @Override
-  public <T> T get(PCollectionView<T> view, BoundedWindow window) {
+  public <T> @Nullable T get(PCollectionView<T> view, BoundedWindow window) {
     checkNotNull(view, "View passed to sideInput cannot be null");
     TupleTag<?> tag = view.getTagInternal();
     checkNotNull(sideInputs.get(tag), "Side input for " + view + " not available.");
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/helpers/EncoderHelpers.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/helpers/EncoderHelpers.java
index e1dabf7..1068e9f 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/helpers/EncoderHelpers.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/helpers/EncoderHelpers.java
@@ -48,6 +48,7 @@
 import scala.reflect.ClassTag$;
 
 /** {@link Encoders} utility class. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class EncoderHelpers {
   /**
    * Wrap a Beam coder into a Spark Encoder using Catalyst Expression Encoders (which uses java code
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/helpers/MultiOuputCoder.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/helpers/MultiOuputCoder.java
index 82f0e4f..06919b2 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/helpers/MultiOuputCoder.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/helpers/MultiOuputCoder.java
@@ -35,6 +35,7 @@
  *
  * @param <T> type of the elements in the collection
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class MultiOuputCoder<T> extends CustomCoder<Tuple2<TupleTag<T>, WindowedValue<T>>> {
   Coder<TupleTag> tupleTagCoder;
   Map<TupleTag<?>, Coder<?>> coderMap;
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/helpers/SideInputBroadcast.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/helpers/SideInputBroadcast.java
index d019ead..0de9494 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/helpers/SideInputBroadcast.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/helpers/SideInputBroadcast.java
@@ -24,6 +24,7 @@
 import org.apache.spark.broadcast.Broadcast;
 
 /** Broadcast helper for side inputs. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SideInputBroadcast implements Serializable {
 
   private final Map<String, Broadcast<?>> bcast = new HashMap<>();
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/streaming/PipelineTranslatorStreaming.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/streaming/PipelineTranslatorStreaming.java
index 42a5fdc..1f0cde7 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/streaming/PipelineTranslatorStreaming.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/streaming/PipelineTranslatorStreaming.java
@@ -34,6 +34,7 @@
  * contains only the components specific to streaming: registry of streaming {@link
  * TransformTranslator} and registry lookup code.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class PipelineTranslatorStreaming extends PipelineTranslator {
   // --------------------------------------------------------------------------------------------
   //  Transform Translator Registry
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/utils/CachedSideInputReader.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/utils/CachedSideInputReader.java
index eb6aa43..a23ae7a 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/utils/CachedSideInputReader.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/utils/CachedSideInputReader.java
@@ -30,6 +30,7 @@
 import org.slf4j.LoggerFactory;
 
 /** {@link SideInputReader} that caches materialized views. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class CachedSideInputReader implements SideInputReader {
 
   private static final Logger LOG = LoggerFactory.getLogger(CachedSideInputReader.class);
@@ -51,9 +52,8 @@
     this.delegate = delegate;
   }
 
-  @Nullable
   @Override
-  public <T> T get(PCollectionView<T> view, BoundedWindow window) {
+  public <T> @Nullable T get(PCollectionView<T> view, BoundedWindow window) {
     @SuppressWarnings("unchecked")
     final Cache<Key<T>, Value<T>> materializedCasted =
         (Cache) SideInputStorage.getMaterializedSideInputs();
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/utils/SideInputStorage.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/utils/SideInputStorage.java
index 3d5a6c7..b7cac05 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/utils/SideInputStorage.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/utils/SideInputStorage.java
@@ -29,6 +29,7 @@
  * Cache deserialized side inputs for executor so every task doesn't need to deserialize them again.
  * Side inputs are stored in {@link Cache} with 5 minutes expireAfterAccess.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class SideInputStorage {
 
   /** JVM deserialized side input cache. */
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/BoundedDataset.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/BoundedDataset.java
index c4e96c1..2fefe7a 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/BoundedDataset.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/BoundedDataset.java
@@ -39,6 +39,7 @@
  * objects; in which case they do not need to be converted to bytes since they are not transferred
  * across the network until they are broadcast.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BoundedDataset<T> implements Dataset {
   // only set if creating an RDD from a static collection
   private transient @Nullable JavaSparkContext jsc;
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/EvaluationContext.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/EvaluationContext.java
index 57d1da6..d75f72e 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/EvaluationContext.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/EvaluationContext.java
@@ -50,6 +50,7 @@
  * The EvaluationContext allows us to define pipeline instructions and translate between {@code
  * PObject<T>}s or {@code PCollection<T>}s and Ts or DStreams/RDDs of Ts.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class EvaluationContext {
   private final JavaSparkContext jsc;
   private JavaStreamingContext jssc;
@@ -112,7 +113,7 @@
     return input;
   }
 
-  public <T> Map<TupleTag<?>, PValue> getInputs(PTransform<?, ?> transform) {
+  public <T> Map<TupleTag<?>, PCollection<?>> getInputs(PTransform<?, ?> transform) {
     checkArgument(currentTransform != null, "can only be called with non-null currentTransform");
     checkArgument(
         currentTransform.getTransform() == transform, "can only be called with current transform");
@@ -125,7 +126,7 @@
     return output;
   }
 
-  public Map<TupleTag<?>, PValue> getOutputs(PTransform<?, ?> transform) {
+  public Map<TupleTag<?>, PCollection<?>> getOutputs(PTransform<?, ?> transform) {
     checkArgument(currentTransform != null, "can only be called with non-null currentTransform");
     checkArgument(
         currentTransform.getTransform() == transform, "can only be called with current transform");
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/GroupNonMergingWindowsFunctions.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/GroupNonMergingWindowsFunctions.java
index 42cd72c..0c0bc95 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/GroupNonMergingWindowsFunctions.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/GroupNonMergingWindowsFunctions.java
@@ -46,6 +46,7 @@
 import scala.Tuple2;
 
 /** Functions for GroupByKey with Non-Merging windows translations to Spark. */
+@SuppressWarnings({"keyfor", "nullness"}) // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class GroupNonMergingWindowsFunctions {
 
   private static final Logger LOG = LoggerFactory.getLogger(GroupNonMergingWindowsFunctions.class);
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/MultiDoFnFunction.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/MultiDoFnFunction.java
index 77ddff2..cdc0ed1 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/MultiDoFnFunction.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/MultiDoFnFunction.java
@@ -60,6 +60,7 @@
  * @param <InputT> Input type for DoFunction.
  * @param <OutputT> Output type for DoFunction.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class MultiDoFnFunction<InputT, OutputT>
     implements PairFlatMapFunction<Iterator<WindowedValue<InputT>>, TupleTag<?>, WindowedValue<?>> {
 
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkBatchPortablePipelineTranslator.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkBatchPortablePipelineTranslator.java
index 3de05e3..0853df9 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkBatchPortablePipelineTranslator.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkBatchPortablePipelineTranslator.java
@@ -72,6 +72,7 @@
 import scala.Tuple2;
 
 /** Translates a bounded portable pipeline into a Spark job. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SparkBatchPortablePipelineTranslator
     implements SparkPortablePipelineTranslator<SparkTranslationContext> {
 
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkContextFactory.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkContextFactory.java
index aa12192..db6afee 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkContextFactory.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkContextFactory.java
@@ -26,6 +26,7 @@
 import org.slf4j.LoggerFactory;
 
 /** The Spark context factory. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public final class SparkContextFactory {
   private static final Logger LOG = LoggerFactory.getLogger(SparkContextFactory.class);
 
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkExecutableStageExtractionFunction.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkExecutableStageExtractionFunction.java
index a5d1bbd..88baa0d 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkExecutableStageExtractionFunction.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkExecutableStageExtractionFunction.java
@@ -23,6 +23,7 @@
 import org.apache.beam.sdk.util.WindowedValue;
 import org.apache.spark.api.java.function.FlatMapFunction;
 
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class SparkExecutableStageExtractionFunction<OutputT>
     implements FlatMapFunction<RawUnionValue, WindowedValue<OutputT>> {
   private final int unionTag;
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkExecutableStageFunction.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkExecutableStageFunction.java
index d28b9bd..66ba6bb 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkExecutableStageFunction.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkExecutableStageFunction.java
@@ -77,6 +77,7 @@
  * The resulting data set should be further processed by a {@link
  * SparkExecutableStageExtractionFunction}.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class SparkExecutableStageFunction<InputT, SideInputT>
     implements FlatMapFunction<Iterator<WindowedValue<InputT>>, RawUnionValue> {
 
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkPCollectionView.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkPCollectionView.java
index 15739c6..73a8c3d 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkPCollectionView.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkPCollectionView.java
@@ -31,6 +31,7 @@
 import scala.Tuple2;
 
 /** SparkPCollectionView is used to pass serialized views to lambdas. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SparkPCollectionView implements Serializable {
 
   private static final Logger LOG = LoggerFactory.getLogger(SparkPCollectionView.class);
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkStreamingPortablePipelineTranslator.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkStreamingPortablePipelineTranslator.java
index 48312ad..5ee3bef 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkStreamingPortablePipelineTranslator.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkStreamingPortablePipelineTranslator.java
@@ -71,6 +71,7 @@
 import scala.Tuple2;
 
 /** Translates an unbounded portable pipeline into a Spark job. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SparkStreamingPortablePipelineTranslator
     implements SparkPortablePipelineTranslator<SparkStreamingTranslationContext> {
 
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkTranslationContext.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkTranslationContext.java
index a96a784..9360c0c 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkTranslationContext.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkTranslationContext.java
@@ -34,6 +34,7 @@
  * Translation context used to lazily store Spark data sets during portable pipeline translation and
  * compute them after translation.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SparkTranslationContext {
   private final JavaSparkContext jsc;
   final JobInfo jobInfo;
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformTranslator.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformTranslator.java
index 6bcc432..59ddaef 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformTranslator.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformTranslator.java
@@ -18,7 +18,6 @@
 package org.apache.beam.runners.spark.translation;
 
 import static org.apache.beam.runners.spark.translation.TranslationUtils.canAvoidRddSerialization;
-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.checkState;
 
 import java.util.Arrays;
@@ -65,7 +64,6 @@
 import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PCollectionView;
-import org.apache.beam.sdk.values.PValue;
 import org.apache.beam.sdk.values.TupleTag;
 import org.apache.beam.sdk.values.WindowingStrategy;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
@@ -84,6 +82,7 @@
 import scala.Tuple2;
 
 /** Supports translation between a Beam transform, and Spark's operations on RDDs. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public final class TransformTranslator {
 
   private TransformTranslator() {}
@@ -93,19 +92,14 @@
       @SuppressWarnings("unchecked")
       @Override
       public void evaluate(Flatten.PCollections<T> transform, EvaluationContext context) {
-        Collection<PValue> pcs = context.getInputs(transform).values();
+        Collection<PCollection<?>> pcs = context.getInputs(transform).values();
         JavaRDD<WindowedValue<T>> unionRDD;
         if (pcs.isEmpty()) {
           unionRDD = context.getSparkContext().emptyRDD();
         } else {
           JavaRDD<WindowedValue<T>>[] rdds = new JavaRDD[pcs.size()];
           int index = 0;
-          for (PValue pc : pcs) {
-            checkArgument(
-                pc instanceof PCollection,
-                "Flatten had non-PCollection value in input: %s of type %s",
-                pc,
-                pc.getClass().getSimpleName());
+          for (PCollection<?> pc : pcs) {
             rdds[index] = ((BoundedDataset<T>) context.borrowDataset(pc)).getRDD();
             index++;
           }
@@ -412,7 +406,7 @@
           all = inRDD.mapPartitionsToPair(multiDoFnFunction);
         }
 
-        Map<TupleTag<?>, PValue> outputs = context.getOutputs(transform);
+        Map<TupleTag<?>, PCollection<?>> outputs = context.getOutputs(transform);
         if (outputs.size() > 1) {
           StorageLevel level = StorageLevel.fromString(context.storageLevel());
           if (canAvoidRddSerialization(level)) {
@@ -429,7 +423,7 @@
                     .mapToPair(TranslationUtils.getTupleTagDecodeFunction(coderMap));
           }
         }
-        for (Map.Entry<TupleTag<?>, PValue> output : outputs.entrySet()) {
+        for (Map.Entry<TupleTag<?>, PCollection<?>> output : outputs.entrySet()) {
           JavaPairRDD<TupleTag<?>, WindowedValue<?>> filtered =
               all.filter(new TranslationUtils.TupleTagFilter(output.getKey()));
           // Object is the best we can do since different outputs can have different tags
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TranslationUtils.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TranslationUtils.java
index 2b15639..f05465e 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TranslationUtils.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TranslationUtils.java
@@ -42,7 +42,6 @@
 import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PCollectionView;
-import org.apache.beam.sdk.values.PValue;
 import org.apache.beam.sdk.values.TupleTag;
 import org.apache.beam.sdk.values.WindowingStrategy;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
@@ -60,6 +59,7 @@
 import scala.Tuple2;
 
 /** A set of utilities to help translating Beam transformations into Spark transformations. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public final class TranslationUtils {
 
   private TranslationUtils() {}
@@ -350,12 +350,12 @@
    * @return mapping between TupleTag and a coder
    */
   public static Map<TupleTag<?>, Coder<WindowedValue<?>>> getTupleTagCoders(
-      Map<TupleTag<?>, PValue> outputs) {
+      Map<TupleTag<?>, PCollection<?>> outputs) {
     Map<TupleTag<?>, Coder<WindowedValue<?>>> coderMap = new HashMap<>(outputs.size());
 
-    for (Map.Entry<TupleTag<?>, PValue> output : outputs.entrySet()) {
+    for (Map.Entry<TupleTag<?>, PCollection<?>> output : outputs.entrySet()) {
       // we get the first PValue as all of them are fro the same type.
-      PCollection<?> pCollection = (PCollection<?>) output.getValue();
+      PCollection<?> pCollection = output.getValue();
       Coder<?> coder = pCollection.getCoder();
       Coder<? extends BoundedWindow> wCoder =
           pCollection.getWindowingStrategy().getWindowFn().windowCoder();
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/ValueAndCoderLazySerializable.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/ValueAndCoderLazySerializable.java
index 5b67c93..09d5e2e 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/ValueAndCoderLazySerializable.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/ValueAndCoderLazySerializable.java
@@ -48,6 +48,7 @@
  *
  * @param <T> element type
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public final class ValueAndCoderLazySerializable<T> implements Serializable {
   private T value;
   // Re-use a field to save space in-memory. This is either a byte[] or a Coder, depending on
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/Checkpoint.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/Checkpoint.java
index 2783dca..f07aa46 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/Checkpoint.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/Checkpoint.java
@@ -32,6 +32,7 @@
 import org.slf4j.LoggerFactory;
 
 /** Checkpoint data to make it available in future pipeline runs. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class Checkpoint {
   private static final String TEMP_FILE_SUFFIX = ".tmp";
   private static final String BACKUP_FILE_SUFFIX = ".bak";
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingTransformTranslator.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingTransformTranslator.java
index f1681e6..d65010c 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingTransformTranslator.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingTransformTranslator.java
@@ -98,6 +98,7 @@
 import scala.reflect.ClassTag$;
 
 /** Supports translation between a Beam transform, and Spark's operations on DStreams. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public final class StreamingTransformTranslator {
 
   private StreamingTransformTranslator() {}
@@ -245,7 +246,7 @@
       @SuppressWarnings("unchecked")
       @Override
       public void evaluate(Flatten.PCollections<T> transform, EvaluationContext context) {
-        Map<TupleTag<?>, PValue> pcs = context.getInputs(transform);
+        Map<TupleTag<?>, PCollection<?>> pcs = context.getInputs(transform);
         // since this is a streaming pipeline, at least one of the PCollections to "flatten" are
         // unbounded, meaning it represents a DStream.
         // So we could end up with an unbounded unified DStream.
@@ -460,7 +461,7 @@
                           sideInputMapping));
                 });
 
-        Map<TupleTag<?>, PValue> outputs = context.getOutputs(transform);
+        Map<TupleTag<?>, PCollection<?>> outputs = context.getOutputs(transform);
         if (outputs.size() > 1) {
           // Caching can cause Serialization, we need to code to bytes
           // more details in https://issues.apache.org/jira/browse/BEAM-2669
@@ -472,7 +473,7 @@
                   .mapToPair(TranslationUtils.getTupleTagDecodeFunction(coderMap));
         }
 
-        for (Map.Entry<TupleTag<?>, PValue> output : outputs.entrySet()) {
+        for (Map.Entry<TupleTag<?>, PCollection<?>> output : outputs.entrySet()) {
           @SuppressWarnings("unchecked")
           JavaPairDStream<TupleTag<?>, WindowedValue<?>> filtered =
               all.filter(new TranslationUtils.TupleTagFilter(output.getKey()));
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/WatermarkSyncedDStream.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/WatermarkSyncedDStream.java
index 85dac37..62370fc 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/WatermarkSyncedDStream.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/WatermarkSyncedDStream.java
@@ -56,6 +56,7 @@
  * <p>See also <a href="https://issues.apache.org/jira/browse/BEAM-2671">BEAM-2671</a>, <a
  * href="https://issues.apache.org/jira/browse/BEAM-2789">BEAM-2789</a>.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class WatermarkSyncedDStream<T> extends InputDStream<WindowedValue<T>> {
 
   private static final Logger LOG =
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/util/CachedSideInputReader.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/util/CachedSideInputReader.java
index 6a663e2..035b0f2 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/util/CachedSideInputReader.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/util/CachedSideInputReader.java
@@ -30,6 +30,7 @@
 import org.slf4j.LoggerFactory;
 
 /** {@link SideInputReader} that caches materialized views. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class CachedSideInputReader implements SideInputReader {
 
   private static final Logger LOG = LoggerFactory.getLogger(CachedSideInputReader.class);
@@ -51,9 +52,8 @@
     this.delegate = delegate;
   }
 
-  @Nullable
   @Override
-  public <T> T get(PCollectionView<T> view, BoundedWindow window) {
+  public <T> @Nullable T get(PCollectionView<T> view, BoundedWindow window) {
     @SuppressWarnings("unchecked")
     final Cache<Key<T>, Value<T>> materializedCasted =
         (Cache) SideInputStorage.getMaterializedSideInputs();
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/util/GlobalWatermarkHolder.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/util/GlobalWatermarkHolder.java
index 5abc432..094f7d5 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/util/GlobalWatermarkHolder.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/util/GlobalWatermarkHolder.java
@@ -55,6 +55,7 @@
  * <p>For each source, holds a queue for the watermarks of each micro-batch that was read, and
  * advances the watermarks according to the queue (first-in-first-out).
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class GlobalWatermarkHolder {
 
   private static final Logger LOG = LoggerFactory.getLogger(GlobalWatermarkHolder.class);
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/util/SideInputBroadcast.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/util/SideInputBroadcast.java
index 6cd3b97..79310a2 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/util/SideInputBroadcast.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/util/SideInputBroadcast.java
@@ -31,6 +31,7 @@
  * Broadcast helper for side inputs. Helps to do the transformation from bytes transform to
  * broadcast transform to value by coder
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SideInputBroadcast<T> implements Serializable {
 
   private static final Logger LOG = LoggerFactory.getLogger(SideInputBroadcast.class);
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/util/SideInputStorage.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/util/SideInputStorage.java
index c60dfee..5571c3f 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/util/SideInputStorage.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/util/SideInputStorage.java
@@ -29,6 +29,7 @@
  * Cache deserialized side inputs for executor so every task doesn't need to deserialize them again.
  * Side inputs are stored in {@link Cache} with 5 minutes expireAfterAccess.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class SideInputStorage {
 
   /** JVM deserialized side input cache. */
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/util/SparkCompat.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/util/SparkCompat.java
index 0491220..2282695 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/util/SparkCompat.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/util/SparkCompat.java
@@ -31,6 +31,7 @@
 import org.apache.spark.streaming.api.java.JavaStreamingContext;
 
 /** A set of functions to provide API compatibility between Spark 2 and Spark 3. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SparkCompat {
   /**
    * Union of dStreams in the given StreamingContext.
diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/util/SparkSideInputReader.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/util/SparkSideInputReader.java
index f05d2d9..5d47727 100644
--- a/runners/spark/src/main/java/org/apache/beam/runners/spark/util/SparkSideInputReader.java
+++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/util/SparkSideInputReader.java
@@ -40,6 +40,7 @@
 import org.checkerframework.checker.nullness.qual.Nullable;
 
 /** A {@link SideInputReader} for the SparkRunner. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SparkSideInputReader implements SideInputReader {
   private final Map<TupleTag<?>, KV<WindowingStrategy<?, ?>, SideInputBroadcast<?>>> sideInputs;
 
@@ -48,9 +49,8 @@
     this.sideInputs = sideInputs;
   }
 
-  @Nullable
   @Override
-  public <T> T get(PCollectionView<T> view, BoundedWindow window) {
+  public <T> @Nullable T get(PCollectionView<T> view, BoundedWindow window) {
     // --- validate sideInput.
     checkNotNull(view, "The PCollectionView passed to sideInput cannot be null ");
     KV<WindowingStrategy<?, ?>, SideInputBroadcast<?>> windowedBroadcastHelper =
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/CacheTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/CacheTest.java
index eb9dcd9..6f48978 100644
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/CacheTest.java
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/CacheTest.java
@@ -43,6 +43,7 @@
 import org.junit.Test;
 
 /** Tests of {@link Dataset#cache(String, Coder)}} scenarios. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class CacheTest {
 
   /**
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/GlobalWatermarkHolderTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/GlobalWatermarkHolderTest.java
index 1abd639..ce4d2a0 100644
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/GlobalWatermarkHolderTest.java
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/GlobalWatermarkHolderTest.java
@@ -33,6 +33,7 @@
 import org.junit.rules.ExpectedException;
 
 /** A test suite for the propagation of watermarks in the Spark runner. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class GlobalWatermarkHolderTest {
 
   @Rule public ClearWatermarksRule clearWatermarksRule = new ClearWatermarksRule();
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/ProvidedSparkContextTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/ProvidedSparkContextTest.java
index 4a57ade..c614f8d 100644
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/ProvidedSparkContextTest.java
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/ProvidedSparkContextTest.java
@@ -36,6 +36,7 @@
 import org.junit.Test;
 
 /** Provided Spark Context tests. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ProvidedSparkContextTest {
   private static final String[] WORDS_ARRAY = {
     "hi there", "hi", "hi sue bob",
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/SparkPipelineStateTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/SparkPipelineStateTest.java
index 385fd45..cb594ec 100644
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/SparkPipelineStateTest.java
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/SparkPipelineStateTest.java
@@ -42,6 +42,7 @@
 import org.junit.rules.TestName;
 
 /** This suite tests that various scenarios result in proper states of the pipeline. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SparkPipelineStateTest implements Serializable {
 
   private static class MyCustomException extends RuntimeException {
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/aggregators/metrics/sink/InMemoryMetrics.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/aggregators/metrics/sink/InMemoryMetrics.java
index a4b3e54..31e62ae 100644
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/aggregators/metrics/sink/InMemoryMetrics.java
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/aggregators/metrics/sink/InMemoryMetrics.java
@@ -25,6 +25,7 @@
 import org.apache.spark.metrics.sink.Sink;
 
 /** An in-memory {@link Sink} implementation for tests. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class InMemoryMetrics implements Sink {
 
   private static WithMetricsSupport extendedMetricsRegistry;
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/io/AvroPipelineTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/io/AvroPipelineTest.java
index fa49a9f..99b9023 100644
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/io/AvroPipelineTest.java
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/io/AvroPipelineTest.java
@@ -41,6 +41,7 @@
 import org.junit.rules.TemporaryFolder;
 
 /** Avro pipeline test. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class AvroPipelineTest {
 
   private File inputFile;
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/io/NumShardsTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/io/NumShardsTest.java
index 777d1d4..d2a7a28 100644
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/io/NumShardsTest.java
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/io/NumShardsTest.java
@@ -41,6 +41,7 @@
 import org.junit.rules.TemporaryFolder;
 
 /** Number of shards test. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class NumShardsTest {
 
   private static final String[] WORDS_ARRAY = {
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/io/ReaderToIteratorAdapterTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/io/ReaderToIteratorAdapterTest.java
index 019cbe6..7705659 100644
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/io/ReaderToIteratorAdapterTest.java
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/io/ReaderToIteratorAdapterTest.java
@@ -31,6 +31,7 @@
 import org.junit.rules.ExpectedException;
 
 /** Test for {@link SourceRDD.Bounded.ReaderToIteratorAdapter}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ReaderToIteratorAdapterTest {
 
   @Rule public ExpectedException exception = ExpectedException.none();
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/structuredstreaming/StructuredStreamingPipelineStateTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/structuredstreaming/StructuredStreamingPipelineStateTest.java
index b44df7b..ea49257 100644
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/structuredstreaming/StructuredStreamingPipelineStateTest.java
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/structuredstreaming/StructuredStreamingPipelineStateTest.java
@@ -46,6 +46,7 @@
 
 /** This suite tests that various scenarios result in proper states of the pipeline. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class StructuredStreamingPipelineStateTest implements Serializable {
 
   private static class MyCustomException extends RuntimeException {
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/structuredstreaming/aggregators/metrics/sink/InMemoryMetrics.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/structuredstreaming/aggregators/metrics/sink/InMemoryMetrics.java
index 8649e91..a254def 100644
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/structuredstreaming/aggregators/metrics/sink/InMemoryMetrics.java
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/structuredstreaming/aggregators/metrics/sink/InMemoryMetrics.java
@@ -28,6 +28,7 @@
 
 /** An in-memory {@link Sink} implementation for tests. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class InMemoryMetrics implements Sink {
 
   private static WithMetricsSupport extendedMetricsRegistry;
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/structuredstreaming/aggregators/metrics/sink/SparkMetricsSinkTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/structuredstreaming/aggregators/metrics/sink/SparkMetricsSinkTest.java
index a7523aa..87dca56 100644
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/structuredstreaming/aggregators/metrics/sink/SparkMetricsSinkTest.java
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/structuredstreaming/aggregators/metrics/sink/SparkMetricsSinkTest.java
@@ -49,6 +49,7 @@
  */
 @Ignore("Has been failing since at least c350188ef7a8704c7336f3c20a1ab2144abbcd4a")
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SparkMetricsSinkTest {
   @Rule public ExternalResource inMemoryMetricsSink = new InMemoryMetricsSinkRule();
 
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/CombineTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/CombineTest.java
index 52e60a3..d91dac8 100644
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/CombineTest.java
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/CombineTest.java
@@ -45,6 +45,7 @@
 
 /** Test class for beam to spark {@link org.apache.beam.sdk.transforms.Combine} translation. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class CombineTest implements Serializable {
   private static Pipeline pipeline;
 
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/ComplexSourceTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/ComplexSourceTest.java
index 0175d03..8920e6c 100644
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/ComplexSourceTest.java
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/ComplexSourceTest.java
@@ -41,6 +41,7 @@
 
 /** Test class for beam to spark source translation. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ComplexSourceTest implements Serializable {
   @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder();
   private static File file;
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/FlattenTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/FlattenTest.java
index e126d06..654215d 100644
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/FlattenTest.java
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/FlattenTest.java
@@ -34,6 +34,7 @@
 
 /** Test class for beam to spark flatten translation. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class FlattenTest implements Serializable {
   private static Pipeline pipeline;
 
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/GroupByKeyTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/GroupByKeyTest.java
index 16992a0..3c8df74 100644
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/GroupByKeyTest.java
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/GroupByKeyTest.java
@@ -47,6 +47,7 @@
 
 /** Test class for beam to spark {@link ParDo} translation. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class GroupByKeyTest implements Serializable {
   private static Pipeline pipeline;
 
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/ParDoTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/ParDoTest.java
index 16d9a8b..9591828 100644
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/ParDoTest.java
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/ParDoTest.java
@@ -39,6 +39,7 @@
 
 /** Test class for beam to spark {@link ParDo} translation. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ParDoTest implements Serializable {
   private static Pipeline pipeline;
 
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/SimpleSourceTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/SimpleSourceTest.java
index 98f9f8d..a43815e 100644
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/SimpleSourceTest.java
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/SimpleSourceTest.java
@@ -46,6 +46,7 @@
 
 /** Test class for beam to spark source translation. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SimpleSourceTest implements Serializable {
   private static Pipeline pipeline;
   @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder();
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/WindowAssignTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/WindowAssignTest.java
index b8b4101..9ce1762 100644
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/WindowAssignTest.java
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/WindowAssignTest.java
@@ -38,6 +38,7 @@
 
 /** Test class for beam to spark window assign translation. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class WindowAssignTest implements Serializable {
   private static Pipeline pipeline;
 
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/streaming/SimpleSourceTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/streaming/SimpleSourceTest.java
index a06d2ce..e84723c 100644
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/streaming/SimpleSourceTest.java
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/streaming/SimpleSourceTest.java
@@ -33,6 +33,7 @@
 
 /** Test class for beam to spark source translation. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SimpleSourceTest implements Serializable {
   private static Pipeline pipeline;
   @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder();
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/structuredstreaming/utils/SerializationDebugger.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/structuredstreaming/utils/SerializationDebugger.java
index b384b9b..cf272a8 100644
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/structuredstreaming/utils/SerializationDebugger.java
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/structuredstreaming/utils/SerializationDebugger.java
@@ -27,6 +27,7 @@
 import java.util.List;
 
 /** A {@code SerializationDebugger} for Spark Runner. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SerializationDebugger {
 
   public static void testSerialization(Object object, File to) throws IOException {
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SparkExecutableStageFunctionTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SparkExecutableStageFunctionTest.java
index ca23d3d..3f7cc11 100644
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SparkExecutableStageFunctionTest.java
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SparkExecutableStageFunctionTest.java
@@ -63,6 +63,7 @@
 import org.mockito.MockitoAnnotations;
 
 /** Unit tests for {@link SparkExecutableStageFunction}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SparkExecutableStageFunctionTest {
   @Mock private SparkExecutableStageContextFactory contextFactory;
   @Mock private ExecutableStageContext stageContext;
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/ResumeFromCheckpointStreamingTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/ResumeFromCheckpointStreamingTest.java
index 16a19bc..3038b3d 100644
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/ResumeFromCheckpointStreamingTest.java
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/ResumeFromCheckpointStreamingTest.java
@@ -100,6 +100,7 @@
  * side-input that is expected to recover as well.
  */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ResumeFromCheckpointStreamingTest implements Serializable {
   private static final EmbeddedKafkaCluster.EmbeddedZookeeper EMBEDDED_ZOOKEEPER =
       new EmbeddedKafkaCluster.EmbeddedZookeeper();
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/TrackStreamingSourcesTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/TrackStreamingSourcesTest.java
index 07d2a2a..c3ebb46 100644
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/TrackStreamingSourcesTest.java
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/TrackStreamingSourcesTest.java
@@ -53,6 +53,7 @@
  * A test suite that tests tracking of the streaming sources created an {@link
  * org.apache.beam.runners.spark.translation.streaming.UnboundedDataset}.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class TrackStreamingSourcesTest {
 
   @Rule public ReuseSparkContextRule reuseContext = ReuseSparkContextRule.yes();
diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/utils/EmbeddedKafkaCluster.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/utils/EmbeddedKafkaCluster.java
index 7794d5b..c293184 100644
--- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/utils/EmbeddedKafkaCluster.java
+++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/utils/EmbeddedKafkaCluster.java
@@ -36,6 +36,7 @@
 import org.slf4j.LoggerFactory;
 
 /** Embedded Kafka cluster. https://gist.github.com/fjavieralba/7930018 */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class EmbeddedKafkaCluster {
 
   private static final Logger LOG = LoggerFactory.getLogger(EmbeddedKafkaCluster.class);
diff --git a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/BeamBatchTSetEnvironment.java b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/BeamBatchTSetEnvironment.java
index e034765..9395fdc 100644
--- a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/BeamBatchTSetEnvironment.java
+++ b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/BeamBatchTSetEnvironment.java
@@ -26,6 +26,7 @@
  * expects. This will be used to develop the TSet graph corresponding to the Beam Pipeline which
  * will later be injected into the correct TSetEnvironment one submitted.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BeamBatchTSetEnvironment extends BatchTSetEnvironment {
 
   public BeamBatchTSetEnvironment() {
diff --git a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/BeamBatchWorker.java b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/BeamBatchWorker.java
index 840cf56..958b8e3 100644
--- a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/BeamBatchWorker.java
+++ b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/BeamBatchWorker.java
@@ -45,6 +45,7 @@
  * The Twister2 worker that will execute the job logic once the job is submitted from the run
  * method.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BeamBatchWorker implements Serializable, BatchTSetIWorker {
 
   private static final String SIDEINPUTS = "sideInputs";
diff --git a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/Twister2PipelineExecutionEnvironment.java b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/Twister2PipelineExecutionEnvironment.java
index a3d19fd..6dc2d37 100644
--- a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/Twister2PipelineExecutionEnvironment.java
+++ b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/Twister2PipelineExecutionEnvironment.java
@@ -33,6 +33,7 @@
 import org.slf4j.LoggerFactory;
 
 /** Twister2PipelineExecutionEnvironment. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class Twister2PipelineExecutionEnvironment {
   private static final Logger LOG =
       Logger.getLogger(Twister2PipelineExecutionEnvironment.class.getName());
diff --git a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/Twister2PipelineResult.java b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/Twister2PipelineResult.java
index 4ebe705..4e0f0e8 100644
--- a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/Twister2PipelineResult.java
+++ b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/Twister2PipelineResult.java
@@ -25,6 +25,7 @@
 import org.mortbay.log.Log;
 
 /** Represents a Twister2 pipeline execution result. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class Twister2PipelineResult implements PipelineResult {
 
   PipelineResult.State state;
diff --git a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/Twister2Runner.java b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/Twister2Runner.java
index 2273256..2b16c30d 100644
--- a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/Twister2Runner.java
+++ b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/Twister2Runner.java
@@ -65,6 +65,7 @@
  * to a Twister2 Plan and then executing them either locally or on a Twister2 cluster, depending on
  * the configuration.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class Twister2Runner extends PipelineRunner<PipelineResult> {
 
   private static final Logger LOG = Logger.getLogger(Twister2Runner.class.getName());
diff --git a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/Twister2TestRunner.java b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/Twister2TestRunner.java
index fef8781..9d11869 100644
--- a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/Twister2TestRunner.java
+++ b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/Twister2TestRunner.java
@@ -28,6 +28,7 @@
  * to a Twister2 Plan and then executing them either locally or on a Twister2 cluster, depending on
  * the configuration. This is used for testing the Twister2 runner
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class Twister2TestRunner extends PipelineRunner<PipelineResult> {
 
   private Twister2Runner delegate;
diff --git a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/Twister2TranslationContext.java b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/Twister2TranslationContext.java
index b663846..e0a4ccb 100644
--- a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/Twister2TranslationContext.java
+++ b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/Twister2TranslationContext.java
@@ -37,6 +37,7 @@
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables;
 
 /** Twister2TranslationContext. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public abstract class Twister2TranslationContext {
   private final Twister2PipelineOptions options;
   protected final Map<PValue, TSet<?>> dataSets = new LinkedHashMap<>();
@@ -73,7 +74,7 @@
     return tSet;
   }
 
-  public <T> Map<TupleTag<?>, PValue> getInputs() {
+  public <T> Map<TupleTag<?>, PCollection<?>> getInputs() {
     return currentTransform.getInputs();
   }
 
@@ -89,7 +90,7 @@
     return currentTransform;
   }
 
-  public Map<TupleTag<?>, PValue> getOutputs() {
+  public Map<TupleTag<?>, PCollection<?>> getOutputs() {
     return getCurrentTransform().getOutputs();
   }
 
diff --git a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translation/wrappers/Twister2BoundedSource.java b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translation/wrappers/Twister2BoundedSource.java
index 43d41da..7b539b7 100644
--- a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translation/wrappers/Twister2BoundedSource.java
+++ b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translation/wrappers/Twister2BoundedSource.java
@@ -41,6 +41,7 @@
 import org.apache.beam.sdk.util.WindowedValue;
 
 /** Twister2 wrapper for Bounded Source. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class Twister2BoundedSource<T> extends BaseSourceFunc<WindowedValue<T>> {
   private static final Logger LOG = Logger.getLogger(Twister2BoundedSource.class.getName());
 
diff --git a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translation/wrappers/Twister2EmptySource.java b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translation/wrappers/Twister2EmptySource.java
index e6b6faa..bc8b406 100644
--- a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translation/wrappers/Twister2EmptySource.java
+++ b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translation/wrappers/Twister2EmptySource.java
@@ -25,6 +25,7 @@
  *
  * @param <T> type of the source
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class Twister2EmptySource<T> extends BaseSourceFunc<WindowedValue<T>> {
   @Override
   public boolean hasNext() {
diff --git a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/Twister2BatchPipelineTranslator.java b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/Twister2BatchPipelineTranslator.java
index 42dd7e5..69ad3fb 100644
--- a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/Twister2BatchPipelineTranslator.java
+++ b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/Twister2BatchPipelineTranslator.java
@@ -35,6 +35,7 @@
 import org.checkerframework.checker.nullness.qual.Nullable;
 
 /** Twister pipeline translator for batch pipelines. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class Twister2BatchPipelineTranslator extends Twister2PipelineTranslator {
 
   private static final Logger LOG =
diff --git a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/batch/FlattenTranslatorBatch.java b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/batch/FlattenTranslatorBatch.java
index 936c647..f64bb3b 100644
--- a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/batch/FlattenTranslatorBatch.java
+++ b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/batch/FlattenTranslatorBatch.java
@@ -29,6 +29,7 @@
 import org.apache.beam.runners.twister2.translators.BatchTransformTranslator;
 import org.apache.beam.sdk.transforms.Flatten;
 import org.apache.beam.sdk.util.WindowedValue;
+import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PValue;
 
 /** Flatten translator. */
@@ -37,7 +38,7 @@
   @Override
   public void translateNode(
       Flatten.PCollections<T> transform, Twister2BatchTranslationContext context) {
-    Collection<PValue> pcs = context.getInputs().values();
+    Collection<PCollection<?>> pcs = context.getInputs().values();
     List<BatchTSetImpl<WindowedValue<T>>> tSets = new ArrayList<>();
     BatchTSetImpl<WindowedValue<T>> unionTSet;
 
diff --git a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/batch/ParDoMultiOutputTranslatorBatch.java b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/batch/ParDoMultiOutputTranslatorBatch.java
index 26f4487..dd41b3c 100644
--- a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/batch/ParDoMultiOutputTranslatorBatch.java
+++ b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/batch/ParDoMultiOutputTranslatorBatch.java
@@ -39,12 +39,12 @@
 import org.apache.beam.sdk.util.WindowedValue;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PCollectionView;
-import org.apache.beam.sdk.values.PValue;
 import org.apache.beam.sdk.values.TupleTag;
 import org.apache.beam.sdk.values.WindowingStrategy;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Maps;
 
 /** ParDo translator. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ParDoMultiOutputTranslatorBatch<InputT, OutputT>
     implements BatchTransformTranslator<ParDo.MultiOutput<InputT, OutputT>> {
 
@@ -66,7 +66,7 @@
     Coder<InputT> inputCoder = (Coder<InputT>) context.getInput(transform).getCoder();
     Map<String, PCollectionView<?>> sideInputMapping;
 
-    Map<TupleTag<?>, PValue> outputs = context.getOutputs();
+    Map<TupleTag<?>, PCollection<?>> outputs = context.getOutputs();
     Map<TupleTag<?>, Coder<?>> outputCoders = context.getOutputCoders();
 
     // DoFnSignature signature = DoFnSignatures.getSignature(transform.getFn().getClass());
@@ -119,7 +119,7 @@
                     outputMap,
                     sideInputMapping));
 
-    for (Map.Entry<TupleTag<?>, PValue> output : outputs.entrySet()) {
+    for (Map.Entry<TupleTag<?>, PCollection<?>> output : outputs.entrySet()) {
       ComputeTSet<WindowedValue<OutputT>, Iterator<RawUnionValue>> tempTSet =
           outputTSet.direct().compute(new OutputTagFilter(outputMap.get(output.getKey())));
       context.setOutputDataSet((PCollection) output.getValue(), tempTSet);
diff --git a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/AssignWindowsFunction.java b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/AssignWindowsFunction.java
index 7bb0e66..95584f7 100644
--- a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/AssignWindowsFunction.java
+++ b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/AssignWindowsFunction.java
@@ -38,6 +38,7 @@
 import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.InvalidProtocolBufferException;
 
 /** Assign Windows function. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class AssignWindowsFunction<T>
     implements ComputeCollectorFunc<WindowedValue<T>, Iterator<WindowedValue<T>>> {
   private static final Logger LOG = Logger.getLogger(AssignWindowsFunction.class.getName());
diff --git a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/ByteToWindowFunction.java b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/ByteToWindowFunction.java
index 8cf1742..a8f9602 100644
--- a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/ByteToWindowFunction.java
+++ b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/ByteToWindowFunction.java
@@ -37,6 +37,7 @@
 import org.apache.beam.sdk.values.KV;
 
 /** ByteToWindow function. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ByteToWindowFunction<K, V>
     implements MapFunc<KV<K, Iterable<WindowedValue<V>>>, Tuple<byte[], Iterator<byte[]>>> {
   private transient Coder<K> keyCoder;
diff --git a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/ByteToWindowFunctionPrimitive.java b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/ByteToWindowFunctionPrimitive.java
index a15bea7..4ea6130 100644
--- a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/ByteToWindowFunctionPrimitive.java
+++ b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/ByteToWindowFunctionPrimitive.java
@@ -32,6 +32,7 @@
 import org.apache.beam.sdk.values.KV;
 
 /** ByteToWindow function. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ByteToWindowFunctionPrimitive<K, V>
     implements MapFunc<WindowedValue<KV<K, V>>, Tuple<byte[], byte[]>> {
   private transient Coder<K> keyCoder;
diff --git a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/DoFnFunction.java b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/DoFnFunction.java
index 796a847..9ad63f9 100644
--- a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/DoFnFunction.java
+++ b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/DoFnFunction.java
@@ -62,6 +62,7 @@
 import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.InvalidProtocolBufferException;
 
 /** DoFn function. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DoFnFunction<OutputT, InputT>
     implements ComputeCollectorFunc<RawUnionValue, Iterator<WindowedValue<InputT>>> {
   private static final Logger LOG = Logger.getLogger(DoFnFunction.class.getName());
diff --git a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/GroupByWindowFunction.java b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/GroupByWindowFunction.java
index 21a2503..ea87dbc 100644
--- a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/GroupByWindowFunction.java
+++ b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/GroupByWindowFunction.java
@@ -56,6 +56,7 @@
 import org.joda.time.Instant;
 
 /** GroupBy window function. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class GroupByWindowFunction<K, V, W extends BoundedWindow>
     implements FlatMapFunc<WindowedValue<KV<K, Iterable<V>>>, KV<K, Iterable<WindowedValue<V>>>> {
   private static final Logger LOG = Logger.getLogger(GroupByWindowFunction.class.getName());
diff --git a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/MapToTupleFunction.java b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/MapToTupleFunction.java
index 87c5d85..37252a2 100644
--- a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/MapToTupleFunction.java
+++ b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/MapToTupleFunction.java
@@ -30,6 +30,7 @@
 import org.apache.beam.sdk.values.KV;
 
 /** Map to tuple function. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class MapToTupleFunction<K, V>
     implements MapFunc<Tuple<byte[], byte[]>, WindowedValue<KV<K, V>>> {
 
diff --git a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/OutputTagFilter.java b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/OutputTagFilter.java
index 52b751c..5fbbd7f 100644
--- a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/OutputTagFilter.java
+++ b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/OutputTagFilter.java
@@ -25,6 +25,7 @@
 import org.apache.beam.sdk.util.WindowedValue;
 
 /** Output tag filter. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class OutputTagFilter<OutputT, InputT>
     implements ComputeCollectorFunc<WindowedValue<OutputT>, Iterator<RawUnionValue>> {
 
diff --git a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/Twister2SinkFunction.java b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/Twister2SinkFunction.java
index 06bb2cc..6800d86 100644
--- a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/Twister2SinkFunction.java
+++ b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/Twister2SinkFunction.java
@@ -22,6 +22,7 @@
 import edu.iu.dsc.tws.api.tset.fn.SinkFunc;
 
 /** Sink Function that collects results. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class Twister2SinkFunction<T> implements SinkFunc<T> {
 
   public Twister2SinkFunction() {
diff --git a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/internal/SystemReduceFnBuffering.java b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/internal/SystemReduceFnBuffering.java
index 6658abd..a8063bc 100644
--- a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/internal/SystemReduceFnBuffering.java
+++ b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/internal/SystemReduceFnBuffering.java
@@ -30,6 +30,7 @@
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
 import org.apache.beam.sdk.util.SerializableUtils;
 
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SystemReduceFnBuffering<K, T, W extends BoundedWindow>
     extends SystemReduceFn<K, T, Iterable<T>, Iterable<T>, W> {
   private static final String BUFFER_NAME = "buf";
diff --git a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/utils/TranslationUtils.java b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/utils/TranslationUtils.java
index 56564fb..2f9b726 100644
--- a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/utils/TranslationUtils.java
+++ b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/utils/TranslationUtils.java
@@ -25,6 +25,7 @@
 import org.apache.beam.sdk.util.WindowedValue;
 
 /** doc. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public final class TranslationUtils {
   private static final Logger LOG = Logger.getLogger(TranslationUtils.class.getName());
 
diff --git a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/utils/Twister2AssignContext.java b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/utils/Twister2AssignContext.java
index 9a50bfe..7d4eb70 100644
--- a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/utils/Twister2AssignContext.java
+++ b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/utils/Twister2AssignContext.java
@@ -24,6 +24,7 @@
 import org.joda.time.Instant;
 
 /** doc. */
+@SuppressWarnings({"keyfor", "nullness"}) // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class Twister2AssignContext<T, W extends BoundedWindow>
     extends WindowFn<T, W>.AssignContext {
 
diff --git a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/utils/Twister2SideInputReader.java b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/utils/Twister2SideInputReader.java
index 1e700cf..1671184 100644
--- a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/utils/Twister2SideInputReader.java
+++ b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/utils/Twister2SideInputReader.java
@@ -41,6 +41,7 @@
 import org.apache.beam.sdk.values.WindowingStrategy;
 import org.checkerframework.checker.nullness.qual.Nullable;
 
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class Twister2SideInputReader implements SideInputReader {
 
   private final TSetContext runtimeContext;
@@ -52,9 +53,8 @@
     this.runtimeContext = context;
   }
 
-  @Nullable
   @Override
-  public <T> T get(PCollectionView<T> view, BoundedWindow window) {
+  public <T> @Nullable T get(PCollectionView<T> view, BoundedWindow window) {
     checkNotNull(view, "View passed to sideInput cannot be null");
     TupleTag<?> tag = view.getTagInternal();
     checkNotNull(sideInputs.get(tag), "Side input for " + view + " not available.");
diff --git a/sdks/go/pkg/beam/core/runtime/graphx/translate.go b/sdks/go/pkg/beam/core/runtime/graphx/translate.go
index 70d39a3..a52dd43 100644
--- a/sdks/go/pkg/beam/core/runtime/graphx/translate.go
+++ b/sdks/go/pkg/beam/core/runtime/graphx/translate.go
@@ -120,6 +120,10 @@
 
 // Marshal converts a graph to a model pipeline.
 func Marshal(edges []*graph.MultiEdge, opt *Options) (*pipepb.Pipeline, error) {
+	if len(edges) == 0 {
+		return nil, errors.New("empty graph")
+	}
+	
 	tree := NewScopeTree(edges)
 
 	m := newMarshaller(opt)
diff --git a/sdks/go/pkg/beam/core/runtime/harness/harness.go b/sdks/go/pkg/beam/core/runtime/harness/harness.go
index f161340..2eff7fc 100644
--- a/sdks/go/pkg/beam/core/runtime/harness/harness.go
+++ b/sdks/go/pkg/beam/core/runtime/harness/harness.go
@@ -435,12 +435,12 @@
 				},
 			},
 		}
-	case req.GetProcessBundleProgressMetadata() != nil:
-		msg := req.GetProcessBundleProgressMetadata()
+	case req.GetMonitoringInfos() != nil:
+		msg := req.GetMonitoringInfos()
 		return &fnpb.InstructionResponse{
 			InstructionId: string(instID),
-			Response: &fnpb.InstructionResponse_ProcessBundleProgressMetadata{
-				ProcessBundleProgressMetadata: &fnpb.ProcessBundleProgressMetadataResponse{
+			Response: &fnpb.InstructionResponse_MonitoringInfos{
+				MonitoringInfos: &fnpb.MonitoringInfosMetadataResponse{
 					MonitoringInfo: shortIdsToInfos(msg.GetMonitoringInfoId()),
 				},
 			},
diff --git a/sdks/go/pkg/beam/io/filesystem/filesystem.go b/sdks/go/pkg/beam/io/filesystem/filesystem.go
index 018c928..f979a74 100644
--- a/sdks/go/pkg/beam/io/filesystem/filesystem.go
+++ b/sdks/go/pkg/beam/io/filesystem/filesystem.go
@@ -61,6 +61,8 @@
 	// OpenRead opens a file for writing. If the file already exist, it will be
 	// overwritten.
 	OpenWrite(ctx context.Context, filename string) (io.WriteCloser, error)
+	// Size returns the size of a file in bytes.
+	Size(ctx context.Context, filename string) (int64, error)
 }
 
 func getScheme(path string) string {
diff --git a/sdks/go/pkg/beam/io/filesystem/gcs/gcs.go b/sdks/go/pkg/beam/io/filesystem/gcs/gcs.go
index f167532..2cff920 100644
--- a/sdks/go/pkg/beam/io/filesystem/gcs/gcs.go
+++ b/sdks/go/pkg/beam/io/filesystem/gcs/gcs.go
@@ -124,3 +124,18 @@
 
 	return f.client.Bucket(bucket).Object(object).NewWriter(ctx), nil
 }
+
+func (f *fs) Size(ctx context.Context, filename string) (int64, error) {
+	bucket, object, err := gcsx.ParseObject(filename)
+	if err != nil {
+		return -1, err
+	}
+
+	obj := f.client.Bucket(bucket).Object(object)
+	attrs, err := obj.Attrs(ctx)
+	if err != nil {
+		return -1, err
+	}
+
+	return attrs.Size, nil
+}
diff --git a/sdks/go/pkg/beam/io/filesystem/local/local.go b/sdks/go/pkg/beam/io/filesystem/local/local.go
index aed4802..f7cd600 100644
--- a/sdks/go/pkg/beam/io/filesystem/local/local.go
+++ b/sdks/go/pkg/beam/io/filesystem/local/local.go
@@ -32,7 +32,7 @@
 type fs struct{}
 
 // New creates a new local filesystem.
-func New(ctx context.Context) filesystem.Interface {
+func New(_ context.Context) filesystem.Interface {
 	return &fs{}
 }
 
@@ -40,17 +40,29 @@
 	return nil
 }
 
-func (f *fs) List(ctx context.Context, glob string) ([]string, error) {
+func (f *fs) List(_ context.Context, glob string) ([]string, error) {
 	return filepath.Glob(glob)
 }
 
-func (f *fs) OpenRead(ctx context.Context, filename string) (io.ReadCloser, error) {
+func (f *fs) OpenRead(_ context.Context, filename string) (io.ReadCloser, error) {
 	return os.Open(filename)
 }
 
-func (f *fs) OpenWrite(ctx context.Context, filename string) (io.WriteCloser, error) {
+func (f *fs) OpenWrite(_ context.Context, filename string) (io.WriteCloser, error) {
 	if err := os.MkdirAll(filepath.Dir(filename), 0755); err != nil {
 		return nil, err
 	}
 	return os.OpenFile(filename, os.O_CREATE|os.O_WRONLY|os.O_TRUNC, 0644)
 }
+
+func (f *fs) Size(_ context.Context, filename string) (int64, error) {
+	file, err := os.Open(filename)
+	if err != nil {
+		return -1, err
+	}
+	info, err := file.Stat()
+	if err != nil {
+		return -1, err
+	}
+	return info.Size(), nil
+}
diff --git a/sdks/go/pkg/beam/io/filesystem/memfs/memory.go b/sdks/go/pkg/beam/io/filesystem/memfs/memory.go
index b9d19cc..8005815 100644
--- a/sdks/go/pkg/beam/io/filesystem/memfs/memory.go
+++ b/sdks/go/pkg/beam/io/filesystem/memfs/memory.go
@@ -41,7 +41,7 @@
 }
 
 // New returns the global memory filesystem.
-func New(ctx context.Context) filesystem.Interface {
+func New(_ context.Context) filesystem.Interface {
 	return instance
 }
 
@@ -49,7 +49,7 @@
 	return nil
 }
 
-func (f *fs) List(ctx context.Context, glob string) ([]string, error) {
+func (f *fs) List(_ context.Context, _ string) ([]string, error) {
 	f.mu.Lock()
 	defer f.mu.Unlock()
 
@@ -61,7 +61,7 @@
 	return ret, nil
 }
 
-func (f *fs) OpenRead(ctx context.Context, filename string) (io.ReadCloser, error) {
+func (f *fs) OpenRead(_ context.Context, filename string) (io.ReadCloser, error) {
 	f.mu.Lock()
 	defer f.mu.Unlock()
 
@@ -71,10 +71,20 @@
 	return nil, os.ErrNotExist
 }
 
-func (f *fs) OpenWrite(ctx context.Context, filename string) (io.WriteCloser, error) {
+func (f *fs) OpenWrite(_ context.Context, filename string) (io.WriteCloser, error) {
 	return &commitWriter{key: filename}, nil
 }
 
+func (f *fs) Size(_ context.Context, filename string) (int64, error) {
+	f.mu.Lock()
+	defer f.mu.Unlock()
+
+	if v, ok := f.m[normalize(filename)]; ok {
+		return int64(len(v)), nil
+	}
+	return -1, os.ErrNotExist
+}
+
 // Write stores the given key and value in the global store.
 func Write(key string, value []byte) {
 	instance.mu.Lock()
diff --git a/sdks/go/pkg/beam/io/filesystem/memfs/memory_test.go b/sdks/go/pkg/beam/io/filesystem/memfs/memory_test.go
index de3f7f9..08b012d 100644
--- a/sdks/go/pkg/beam/io/filesystem/memfs/memory_test.go
+++ b/sdks/go/pkg/beam/io/filesystem/memfs/memory_test.go
@@ -72,3 +72,23 @@
 		t.Errorf("Read(foo2) = %v, want foo", string(foo))
 	}
 }
+
+func TestSize(t *testing.T) {
+	ctx := context.Background()
+	fs := New(ctx)
+
+	names := []string{"foo", "foobar"}
+	for _, name := range names {
+		file := []byte(name)
+		if err := filesystem.Write(ctx, fs, name, file); err != nil {
+			t.Fatal(err)
+		}
+		size, err := fs.Size(ctx, name)
+		if err != nil {
+			t.Errorf("Size(%v) failed: %v", name, err)
+		}
+		if size != int64(len(name)) {
+			t.Errorf("Size(%v) incorrect: got %v, want %v", name, size, len(name))
+		}
+	}
+}
diff --git a/sdks/go/pkg/beam/io/synthetic/source.go b/sdks/go/pkg/beam/io/synthetic/source.go
index 4e50cce..f023de0 100644
--- a/sdks/go/pkg/beam/io/synthetic/source.go
+++ b/sdks/go/pkg/beam/io/synthetic/source.go
@@ -23,6 +23,8 @@
 package synthetic
 
 import (
+	"bytes"
+	"encoding/json"
 	"fmt"
 	"github.com/apache/beam/sdks/go/pkg/beam/core/sdf"
 	"math/rand"
@@ -125,10 +127,10 @@
 // ProcessElement creates a number of random elements based on the restriction
 // tracker received. Each element is a random byte slice key and value, in the
 // form of KV<[]byte, []byte>.
-func (fn *sourceFn) ProcessElement(rt *sdf.LockRTracker, _ SourceConfig, emit func([]byte, []byte)) error {
+func (fn *sourceFn) ProcessElement(rt *sdf.LockRTracker, config SourceConfig, emit func([]byte, []byte)) error {
 	for i := rt.GetRestriction().(offsetrange.Restriction).Start; rt.TryClaim(i) == true; i++ {
-		key := make([]byte, 8)
-		val := make([]byte, 8)
+		key := make([]byte, config.KeySize)
+		val := make([]byte, config.ValueSize)
 		if _, err := fn.rng.Read(key); err != nil {
 			return err
 		}
@@ -165,6 +167,8 @@
 		cfg: SourceConfig{
 			NumElements:   1, // 0 is invalid (drops elements).
 			InitialSplits: 1, // 0 is invalid (drops elements).
+			KeySize:       8, // 0 is invalid (drops elements).
+			ValueSize:     8, // 0 is invalid (drops elements).
 		},
 	}
 }
@@ -197,6 +201,24 @@
 	return b
 }
 
+// KeySize determines the size of the key of elements for the source to
+// generate.
+//
+// Valid values are in the range of [1, ...] and the default value is 8.
+func (b *SourceConfigBuilder) KeySize(val int) *SourceConfigBuilder {
+	b.cfg.KeySize = val
+	return b
+}
+
+// ValueSize determines the size of the value of elements for the source to
+// generate.
+//
+// Valid values are in the range of [1, ...] and the default value is 8.
+func (b *SourceConfigBuilder) ValueSize(val int) *SourceConfigBuilder {
+	b.cfg.ValueSize = val
+	return b
+}
+
 // Build constructs the SourceConfig initialized by this builder. It also
 // performs error checking on the fields, and panics if any have been set to
 // invalid values.
@@ -207,6 +229,32 @@
 	if b.cfg.NumElements <= 0 {
 		panic(fmt.Sprintf("SourceConfig.NumElements must be >= 1. Got: %v", b.cfg.NumElements))
 	}
+	if b.cfg.KeySize <= 0 {
+		panic(fmt.Sprintf("SourceConfig.KeySize must be >= 1. Got: %v", b.cfg.KeySize))
+	}
+	if b.cfg.ValueSize <= 0 {
+		panic(fmt.Sprintf("SourceConfig.ValueSize must be >= 1. Got: %v", b.cfg.ValueSize))
+	}
+	return b.cfg
+}
+
+// BuildFromJSON constructs the SourceConfig by populating it with the parsed
+// JSON. Panics if there is an error in the syntax of the JSON or if the input
+// contains unknown object keys.
+//
+// An example of valid JSON object:
+// {
+// 	 "num_records": 5,
+// 	 "key_size": 5,
+// 	 "value_size": 5
+// }
+func (b *SourceConfigBuilder) BuildFromJSON(jsonData []byte) SourceConfig {
+	decoder := json.NewDecoder(bytes.NewReader(jsonData))
+	decoder.DisallowUnknownFields()
+
+	if err := decoder.Decode(&b.cfg); err != nil {
+		panic(fmt.Sprintf("Could not unmarshal SourceConfig: %v", err))
+	}
 	return b.cfg
 }
 
@@ -214,6 +262,8 @@
 // synthetic source. It should be created via a SourceConfigBuilder, not by
 // directly initializing it (the fields are public to allow encoding).
 type SourceConfig struct {
-	NumElements   int
-	InitialSplits int
+	NumElements   int `json:"num_records"`
+	InitialSplits int `json:"initial_splits"`
+	KeySize       int `json:"key_size"`
+	ValueSize     int `json:"value_size"`
 }
diff --git a/sdks/go/pkg/beam/io/synthetic/source_test.go b/sdks/go/pkg/beam/io/synthetic/source_test.go
index 0bd5bd0..4ca3393 100644
--- a/sdks/go/pkg/beam/io/synthetic/source_test.go
+++ b/sdks/go/pkg/beam/io/synthetic/source_test.go
@@ -48,6 +48,38 @@
 	}
 }
 
+// TestSourceConfig_KeyValueSize tests that setting the size of the key and the
+// value works correctly.
+func TestSourceConfig_KeyValueSize(t *testing.T) {
+	tests := []struct {
+		size int
+		want int
+	}{
+		{size: 1, want: 1},
+		{size: 42, want: 42},
+	}
+	for _, test := range tests {
+		test := test
+		t.Run(fmt.Sprintf("(size = %v)", test.size), func(t *testing.T) {
+			dfn := sourceFn{}
+			cfg := DefaultSourceConfig().KeySize(test.size).ValueSize(test.size).Build()
+
+			keys, values, err := simulateSourceFn(t, &dfn, cfg)
+			if err != nil {
+				t.Errorf("Failure processing sourceFn: %v", err)
+			}
+			if got := len(keys[0]); got != test.want {
+				t.Errorf("SourceFn emitted keys of wrong size: got: %v, want: %v",
+					got, test.want)
+			}
+			if got := len(values[0]); got != test.want {
+				t.Errorf("SourceFn emitted values of wrong size: got: %v, want: %v",
+					got, test.want)
+			}
+		})
+	}
+}
+
 // TestSourceConfig_InitialSplits tests that the InitialSplits config option
 // works correctly.
 func TestSourceConfig_InitialSplits(t *testing.T) {
@@ -107,6 +139,29 @@
 	})
 }
 
+// TestSourceConfig_BuildFromJSON tests correctness of building the
+// SourceConfig from JSON data.
+func TestSourceConfig_BuildFromJSON(t *testing.T) {
+	tests := []struct {
+		jsonData string
+		want     SourceConfig
+	}{
+		{
+			jsonData: "{\"num_records\": 5, \"key_size\": 2, \"value_size\": 3}",
+			want:     DefaultSourceConfig().NumElements(5).KeySize(2).ValueSize(3).Build(),
+		},
+	}
+	for _, test := range tests {
+		test := test
+		t.Run(fmt.Sprintf("(jsonData = %v)", test.jsonData), func(t *testing.T) {
+			got := DefaultSourceConfig().BuildFromJSON([]byte(test.jsonData))
+			if got != test.want {
+				t.Errorf("Invalid SourceConfig: got: %#v, want: %#v", got, test.want)
+			}
+		})
+	}
+}
+
 // simulateSourceFn calls CreateInitialRestriction, SplitRestriction,
 // CreateTracker, and ProcessElement on the given sourceFn with the given
 // SourceConfig, and outputs the resulting output elements. This method isn't
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 98fbcbc..ef11c79 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
@@ -1,33 +1,61 @@
+//
+// 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.
+
+//
+// Protocol Buffers describing the Fn API and boostrapping.
+//
+// TODO: Usage of plural names in lists looks awkward in Java
+// e.g. getOutputsMap, addCodersBuilder
+//
+// TODO: gRPC / proto field names conflict with generated code
+// e.g. "class" in java, "output" in python
+
 // Code generated by protoc-gen-go. DO NOT EDIT.
+// versions:
+// 	protoc-gen-go v1.25.0-devel
+// 	protoc        v3.13.0
 // source: beam_fn_api.proto
 
+// TODO: Consider consolidating common components in another package
+// and language namespaces for re-use with Runner Api.
+
 package fnexecution_v1
 
 import (
 	context "context"
-	fmt "fmt"
 	pipeline_v1 "github.com/apache/beam/sdks/go/pkg/beam/model/pipeline_v1"
-	proto "github.com/golang/protobuf/proto"
 	_ "github.com/golang/protobuf/protoc-gen-go/descriptor"
 	duration "github.com/golang/protobuf/ptypes/duration"
 	timestamp "github.com/golang/protobuf/ptypes/timestamp"
-	_ "github.com/golang/protobuf/ptypes/wrappers"
 	grpc "google.golang.org/grpc"
 	codes "google.golang.org/grpc/codes"
 	status "google.golang.org/grpc/status"
-	math "math"
+	protoreflect "google.golang.org/protobuf/reflect/protoreflect"
+	protoimpl "google.golang.org/protobuf/runtime/protoimpl"
+	reflect "reflect"
+	sync "sync"
 )
 
-// Reference imports to suppress errors if they are not otherwise used.
-var _ = proto.Marshal
-var _ = fmt.Errorf
-var _ = math.Inf
-
-// This is a compile-time assertion to ensure that this generated file
-// is compatible with the proto package it is being compiled against.
-// A compilation error at this line likely means your copy of the
-// proto package needs to be updated.
-const _ = proto.ProtoPackageIsVersion3 // please upgrade the proto package
+const (
+	// Verify that this generated code is sufficiently up-to-date.
+	_ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion)
+	// Verify that runtime/protoimpl is sufficiently up-to-date.
+	_ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20)
+)
 
 type LogEntry_Severity_Enum int32
 
@@ -51,34 +79,55 @@
 	LogEntry_Severity_CRITICAL LogEntry_Severity_Enum = 7
 )
 
-var LogEntry_Severity_Enum_name = map[int32]string{
-	0: "UNSPECIFIED",
-	1: "TRACE",
-	2: "DEBUG",
-	3: "INFO",
-	4: "NOTICE",
-	5: "WARN",
-	6: "ERROR",
-	7: "CRITICAL",
-}
+// Enum value maps for LogEntry_Severity_Enum.
+var (
+	LogEntry_Severity_Enum_name = map[int32]string{
+		0: "UNSPECIFIED",
+		1: "TRACE",
+		2: "DEBUG",
+		3: "INFO",
+		4: "NOTICE",
+		5: "WARN",
+		6: "ERROR",
+		7: "CRITICAL",
+	}
+	LogEntry_Severity_Enum_value = map[string]int32{
+		"UNSPECIFIED": 0,
+		"TRACE":       1,
+		"DEBUG":       2,
+		"INFO":        3,
+		"NOTICE":      4,
+		"WARN":        5,
+		"ERROR":       6,
+		"CRITICAL":    7,
+	}
+)
 
-var LogEntry_Severity_Enum_value = map[string]int32{
-	"UNSPECIFIED": 0,
-	"TRACE":       1,
-	"DEBUG":       2,
-	"INFO":        3,
-	"NOTICE":      4,
-	"WARN":        5,
-	"ERROR":       6,
-	"CRITICAL":    7,
+func (x LogEntry_Severity_Enum) Enum() *LogEntry_Severity_Enum {
+	p := new(LogEntry_Severity_Enum)
+	*p = x
+	return p
 }
 
 func (x LogEntry_Severity_Enum) String() string {
-	return proto.EnumName(LogEntry_Severity_Enum_name, int32(x))
+	return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x))
 }
 
+func (LogEntry_Severity_Enum) Descriptor() protoreflect.EnumDescriptor {
+	return file_beam_fn_api_proto_enumTypes[0].Descriptor()
+}
+
+func (LogEntry_Severity_Enum) Type() protoreflect.EnumType {
+	return &file_beam_fn_api_proto_enumTypes[0]
+}
+
+func (x LogEntry_Severity_Enum) Number() protoreflect.EnumNumber {
+	return protoreflect.EnumNumber(x)
+}
+
+// Deprecated: Use LogEntry_Severity_Enum.Descriptor instead.
 func (LogEntry_Severity_Enum) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_6d954c03a4758710, []int{29, 1, 0}
+	return file_beam_fn_api_proto_rawDescGZIP(), []int{29, 1, 0}
 }
 
 // A descriptor for connecting to a remote port using the Beam Fn Data API.
@@ -86,92 +135,108 @@
 // runner and the SDK).
 // Stable
 type RemoteGrpcPort struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
 	// (Required) An API descriptor which describes where to
 	// connect to including any authentication that is required.
 	ApiServiceDescriptor *pipeline_v1.ApiServiceDescriptor `protobuf:"bytes,1,opt,name=api_service_descriptor,json=apiServiceDescriptor,proto3" json:"api_service_descriptor,omitempty"`
 	// (Required) The ID of the Coder that will be used to encode and decode data
 	// sent over this port.
-	CoderId              string   `protobuf:"bytes,2,opt,name=coder_id,json=coderId,proto3" json:"coder_id,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
+	CoderId string `protobuf:"bytes,2,opt,name=coder_id,json=coderId,proto3" json:"coder_id,omitempty"`
 }
 
-func (m *RemoteGrpcPort) Reset()         { *m = RemoteGrpcPort{} }
-func (m *RemoteGrpcPort) String() string { return proto.CompactTextString(m) }
-func (*RemoteGrpcPort) ProtoMessage()    {}
+func (x *RemoteGrpcPort) Reset() {
+	*x = RemoteGrpcPort{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_fn_api_proto_msgTypes[0]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *RemoteGrpcPort) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*RemoteGrpcPort) ProtoMessage() {}
+
+func (x *RemoteGrpcPort) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_fn_api_proto_msgTypes[0]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use RemoteGrpcPort.ProtoReflect.Descriptor instead.
 func (*RemoteGrpcPort) Descriptor() ([]byte, []int) {
-	return fileDescriptor_6d954c03a4758710, []int{0}
+	return file_beam_fn_api_proto_rawDescGZIP(), []int{0}
 }
 
-func (m *RemoteGrpcPort) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_RemoteGrpcPort.Unmarshal(m, b)
-}
-func (m *RemoteGrpcPort) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_RemoteGrpcPort.Marshal(b, m, deterministic)
-}
-func (m *RemoteGrpcPort) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_RemoteGrpcPort.Merge(m, src)
-}
-func (m *RemoteGrpcPort) XXX_Size() int {
-	return xxx_messageInfo_RemoteGrpcPort.Size(m)
-}
-func (m *RemoteGrpcPort) XXX_DiscardUnknown() {
-	xxx_messageInfo_RemoteGrpcPort.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_RemoteGrpcPort proto.InternalMessageInfo
-
-func (m *RemoteGrpcPort) GetApiServiceDescriptor() *pipeline_v1.ApiServiceDescriptor {
-	if m != nil {
-		return m.ApiServiceDescriptor
+func (x *RemoteGrpcPort) GetApiServiceDescriptor() *pipeline_v1.ApiServiceDescriptor {
+	if x != nil {
+		return x.ApiServiceDescriptor
 	}
 	return nil
 }
 
-func (m *RemoteGrpcPort) GetCoderId() string {
-	if m != nil {
-		return m.CoderId
+func (x *RemoteGrpcPort) GetCoderId() string {
+	if x != nil {
+		return x.CoderId
 	}
 	return ""
 }
 
 // Requests the ProcessBundleDescriptor with the given id.
 type GetProcessBundleDescriptorRequest struct {
-	ProcessBundleDescriptorId string   `protobuf:"bytes,1,opt,name=process_bundle_descriptor_id,json=processBundleDescriptorId,proto3" json:"process_bundle_descriptor_id,omitempty"`
-	XXX_NoUnkeyedLiteral      struct{} `json:"-"`
-	XXX_unrecognized          []byte   `json:"-"`
-	XXX_sizecache             int32    `json:"-"`
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	ProcessBundleDescriptorId string `protobuf:"bytes,1,opt,name=process_bundle_descriptor_id,json=processBundleDescriptorId,proto3" json:"process_bundle_descriptor_id,omitempty"`
 }
 
-func (m *GetProcessBundleDescriptorRequest) Reset()         { *m = GetProcessBundleDescriptorRequest{} }
-func (m *GetProcessBundleDescriptorRequest) String() string { return proto.CompactTextString(m) }
-func (*GetProcessBundleDescriptorRequest) ProtoMessage()    {}
+func (x *GetProcessBundleDescriptorRequest) Reset() {
+	*x = GetProcessBundleDescriptorRequest{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_fn_api_proto_msgTypes[1]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *GetProcessBundleDescriptorRequest) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*GetProcessBundleDescriptorRequest) ProtoMessage() {}
+
+func (x *GetProcessBundleDescriptorRequest) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_fn_api_proto_msgTypes[1]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use GetProcessBundleDescriptorRequest.ProtoReflect.Descriptor instead.
 func (*GetProcessBundleDescriptorRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_6d954c03a4758710, []int{1}
+	return file_beam_fn_api_proto_rawDescGZIP(), []int{1}
 }
 
-func (m *GetProcessBundleDescriptorRequest) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_GetProcessBundleDescriptorRequest.Unmarshal(m, b)
-}
-func (m *GetProcessBundleDescriptorRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_GetProcessBundleDescriptorRequest.Marshal(b, m, deterministic)
-}
-func (m *GetProcessBundleDescriptorRequest) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_GetProcessBundleDescriptorRequest.Merge(m, src)
-}
-func (m *GetProcessBundleDescriptorRequest) XXX_Size() int {
-	return xxx_messageInfo_GetProcessBundleDescriptorRequest.Size(m)
-}
-func (m *GetProcessBundleDescriptorRequest) XXX_DiscardUnknown() {
-	xxx_messageInfo_GetProcessBundleDescriptorRequest.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_GetProcessBundleDescriptorRequest proto.InternalMessageInfo
-
-func (m *GetProcessBundleDescriptorRequest) GetProcessBundleDescriptorId() string {
-	if m != nil {
-		return m.ProcessBundleDescriptorId
+func (x *GetProcessBundleDescriptorRequest) GetProcessBundleDescriptorId() string {
+	if x != nil {
+		return x.ProcessBundleDescriptorId
 	}
 	return ""
 }
@@ -181,56 +246,113 @@
 // matching instruction id.
 // Stable
 type InstructionRequest struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
 	// (Required) A unique identifier provided by the runner which represents
 	// this requests execution. The InstructionResponse MUST have the matching id.
 	InstructionId string `protobuf:"bytes,1,opt,name=instruction_id,json=instructionId,proto3" json:"instruction_id,omitempty"`
 	// (Required) A request that the SDK Harness needs to interpret.
 	//
-	// Types that are valid to be assigned to Request:
+	// Types that are assignable to Request:
 	//	*InstructionRequest_ProcessBundle
 	//	*InstructionRequest_ProcessBundleProgress
 	//	*InstructionRequest_ProcessBundleSplit
 	//	*InstructionRequest_FinalizeBundle
-	//	*InstructionRequest_ProcessBundleProgressMetadata
+	//	*InstructionRequest_MonitoringInfos
 	//	*InstructionRequest_Register
-	Request              isInstructionRequest_Request `protobuf_oneof:"request"`
-	XXX_NoUnkeyedLiteral struct{}                     `json:"-"`
-	XXX_unrecognized     []byte                       `json:"-"`
-	XXX_sizecache        int32                        `json:"-"`
+	Request isInstructionRequest_Request `protobuf_oneof:"request"`
 }
 
-func (m *InstructionRequest) Reset()         { *m = InstructionRequest{} }
-func (m *InstructionRequest) String() string { return proto.CompactTextString(m) }
-func (*InstructionRequest) ProtoMessage()    {}
+func (x *InstructionRequest) Reset() {
+	*x = InstructionRequest{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_fn_api_proto_msgTypes[2]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *InstructionRequest) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*InstructionRequest) ProtoMessage() {}
+
+func (x *InstructionRequest) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_fn_api_proto_msgTypes[2]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use InstructionRequest.ProtoReflect.Descriptor instead.
 func (*InstructionRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_6d954c03a4758710, []int{2}
+	return file_beam_fn_api_proto_rawDescGZIP(), []int{2}
 }
 
-func (m *InstructionRequest) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_InstructionRequest.Unmarshal(m, b)
-}
-func (m *InstructionRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_InstructionRequest.Marshal(b, m, deterministic)
-}
-func (m *InstructionRequest) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_InstructionRequest.Merge(m, src)
-}
-func (m *InstructionRequest) XXX_Size() int {
-	return xxx_messageInfo_InstructionRequest.Size(m)
-}
-func (m *InstructionRequest) XXX_DiscardUnknown() {
-	xxx_messageInfo_InstructionRequest.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_InstructionRequest proto.InternalMessageInfo
-
-func (m *InstructionRequest) GetInstructionId() string {
-	if m != nil {
-		return m.InstructionId
+func (x *InstructionRequest) GetInstructionId() string {
+	if x != nil {
+		return x.InstructionId
 	}
 	return ""
 }
 
+func (m *InstructionRequest) GetRequest() isInstructionRequest_Request {
+	if m != nil {
+		return m.Request
+	}
+	return nil
+}
+
+func (x *InstructionRequest) GetProcessBundle() *ProcessBundleRequest {
+	if x, ok := x.GetRequest().(*InstructionRequest_ProcessBundle); ok {
+		return x.ProcessBundle
+	}
+	return nil
+}
+
+func (x *InstructionRequest) GetProcessBundleProgress() *ProcessBundleProgressRequest {
+	if x, ok := x.GetRequest().(*InstructionRequest_ProcessBundleProgress); ok {
+		return x.ProcessBundleProgress
+	}
+	return nil
+}
+
+func (x *InstructionRequest) GetProcessBundleSplit() *ProcessBundleSplitRequest {
+	if x, ok := x.GetRequest().(*InstructionRequest_ProcessBundleSplit); ok {
+		return x.ProcessBundleSplit
+	}
+	return nil
+}
+
+func (x *InstructionRequest) GetFinalizeBundle() *FinalizeBundleRequest {
+	if x, ok := x.GetRequest().(*InstructionRequest_FinalizeBundle); ok {
+		return x.FinalizeBundle
+	}
+	return nil
+}
+
+func (x *InstructionRequest) GetMonitoringInfos() *MonitoringInfosMetadataRequest {
+	if x, ok := x.GetRequest().(*InstructionRequest_MonitoringInfos); ok {
+		return x.MonitoringInfos
+	}
+	return nil
+}
+
+func (x *InstructionRequest) GetRegister() *RegisterRequest {
+	if x, ok := x.GetRequest().(*InstructionRequest_Register); ok {
+		return x.Register
+	}
+	return nil
+}
+
 type isInstructionRequest_Request interface {
 	isInstructionRequest_Request()
 }
@@ -251,11 +373,12 @@
 	FinalizeBundle *FinalizeBundleRequest `protobuf:"bytes,1004,opt,name=finalize_bundle,json=finalizeBundle,proto3,oneof"`
 }
 
-type InstructionRequest_ProcessBundleProgressMetadata struct {
-	ProcessBundleProgressMetadata *ProcessBundleProgressMetadataRequest `protobuf:"bytes,1005,opt,name=process_bundle_progress_metadata,json=processBundleProgressMetadata,proto3,oneof"`
+type InstructionRequest_MonitoringInfos struct {
+	MonitoringInfos *MonitoringInfosMetadataRequest `protobuf:"bytes,1005,opt,name=monitoring_infos,json=monitoringInfos,proto3,oneof"`
 }
 
 type InstructionRequest_Register struct {
+	// DEPRECATED
 	Register *RegisterRequest `protobuf:"bytes,1000,opt,name=register,proto3,oneof"`
 }
 
@@ -267,74 +390,17 @@
 
 func (*InstructionRequest_FinalizeBundle) isInstructionRequest_Request() {}
 
-func (*InstructionRequest_ProcessBundleProgressMetadata) isInstructionRequest_Request() {}
+func (*InstructionRequest_MonitoringInfos) isInstructionRequest_Request() {}
 
 func (*InstructionRequest_Register) isInstructionRequest_Request() {}
 
-func (m *InstructionRequest) GetRequest() isInstructionRequest_Request {
-	if m != nil {
-		return m.Request
-	}
-	return nil
-}
-
-func (m *InstructionRequest) GetProcessBundle() *ProcessBundleRequest {
-	if x, ok := m.GetRequest().(*InstructionRequest_ProcessBundle); ok {
-		return x.ProcessBundle
-	}
-	return nil
-}
-
-func (m *InstructionRequest) GetProcessBundleProgress() *ProcessBundleProgressRequest {
-	if x, ok := m.GetRequest().(*InstructionRequest_ProcessBundleProgress); ok {
-		return x.ProcessBundleProgress
-	}
-	return nil
-}
-
-func (m *InstructionRequest) GetProcessBundleSplit() *ProcessBundleSplitRequest {
-	if x, ok := m.GetRequest().(*InstructionRequest_ProcessBundleSplit); ok {
-		return x.ProcessBundleSplit
-	}
-	return nil
-}
-
-func (m *InstructionRequest) GetFinalizeBundle() *FinalizeBundleRequest {
-	if x, ok := m.GetRequest().(*InstructionRequest_FinalizeBundle); ok {
-		return x.FinalizeBundle
-	}
-	return nil
-}
-
-func (m *InstructionRequest) GetProcessBundleProgressMetadata() *ProcessBundleProgressMetadataRequest {
-	if x, ok := m.GetRequest().(*InstructionRequest_ProcessBundleProgressMetadata); ok {
-		return x.ProcessBundleProgressMetadata
-	}
-	return nil
-}
-
-func (m *InstructionRequest) GetRegister() *RegisterRequest {
-	if x, ok := m.GetRequest().(*InstructionRequest_Register); ok {
-		return x.Register
-	}
-	return nil
-}
-
-// XXX_OneofWrappers is for the internal use of the proto package.
-func (*InstructionRequest) XXX_OneofWrappers() []interface{} {
-	return []interface{}{
-		(*InstructionRequest_ProcessBundle)(nil),
-		(*InstructionRequest_ProcessBundleProgress)(nil),
-		(*InstructionRequest_ProcessBundleSplit)(nil),
-		(*InstructionRequest_FinalizeBundle)(nil),
-		(*InstructionRequest_ProcessBundleProgressMetadata)(nil),
-		(*InstructionRequest_Register)(nil),
-	}
-}
-
 // The response for an associated request the SDK had been asked to fulfill.
 // Stable
 type InstructionResponse struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
 	// (Required) A reference provided by the runner which represents a requests
 	// execution. The InstructionResponse MUST have the matching id when
 	// responding to the runner.
@@ -346,58 +412,111 @@
 	// If the instruction did not fail, it is required to return an equivalent
 	// response type depending on the request this matches.
 	//
-	// Types that are valid to be assigned to Response:
+	// Types that are assignable to Response:
 	//	*InstructionResponse_ProcessBundle
 	//	*InstructionResponse_ProcessBundleProgress
 	//	*InstructionResponse_ProcessBundleSplit
 	//	*InstructionResponse_FinalizeBundle
-	//	*InstructionResponse_ProcessBundleProgressMetadata
+	//	*InstructionResponse_MonitoringInfos
 	//	*InstructionResponse_Register
-	Response             isInstructionResponse_Response `protobuf_oneof:"response"`
-	XXX_NoUnkeyedLiteral struct{}                       `json:"-"`
-	XXX_unrecognized     []byte                         `json:"-"`
-	XXX_sizecache        int32                          `json:"-"`
+	Response isInstructionResponse_Response `protobuf_oneof:"response"`
 }
 
-func (m *InstructionResponse) Reset()         { *m = InstructionResponse{} }
-func (m *InstructionResponse) String() string { return proto.CompactTextString(m) }
-func (*InstructionResponse) ProtoMessage()    {}
+func (x *InstructionResponse) Reset() {
+	*x = InstructionResponse{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_fn_api_proto_msgTypes[3]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *InstructionResponse) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*InstructionResponse) ProtoMessage() {}
+
+func (x *InstructionResponse) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_fn_api_proto_msgTypes[3]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use InstructionResponse.ProtoReflect.Descriptor instead.
 func (*InstructionResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_6d954c03a4758710, []int{3}
+	return file_beam_fn_api_proto_rawDescGZIP(), []int{3}
 }
 
-func (m *InstructionResponse) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_InstructionResponse.Unmarshal(m, b)
-}
-func (m *InstructionResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_InstructionResponse.Marshal(b, m, deterministic)
-}
-func (m *InstructionResponse) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_InstructionResponse.Merge(m, src)
-}
-func (m *InstructionResponse) XXX_Size() int {
-	return xxx_messageInfo_InstructionResponse.Size(m)
-}
-func (m *InstructionResponse) XXX_DiscardUnknown() {
-	xxx_messageInfo_InstructionResponse.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_InstructionResponse proto.InternalMessageInfo
-
-func (m *InstructionResponse) GetInstructionId() string {
-	if m != nil {
-		return m.InstructionId
+func (x *InstructionResponse) GetInstructionId() string {
+	if x != nil {
+		return x.InstructionId
 	}
 	return ""
 }
 
-func (m *InstructionResponse) GetError() string {
-	if m != nil {
-		return m.Error
+func (x *InstructionResponse) GetError() string {
+	if x != nil {
+		return x.Error
 	}
 	return ""
 }
 
+func (m *InstructionResponse) GetResponse() isInstructionResponse_Response {
+	if m != nil {
+		return m.Response
+	}
+	return nil
+}
+
+func (x *InstructionResponse) GetProcessBundle() *ProcessBundleResponse {
+	if x, ok := x.GetResponse().(*InstructionResponse_ProcessBundle); ok {
+		return x.ProcessBundle
+	}
+	return nil
+}
+
+func (x *InstructionResponse) GetProcessBundleProgress() *ProcessBundleProgressResponse {
+	if x, ok := x.GetResponse().(*InstructionResponse_ProcessBundleProgress); ok {
+		return x.ProcessBundleProgress
+	}
+	return nil
+}
+
+func (x *InstructionResponse) GetProcessBundleSplit() *ProcessBundleSplitResponse {
+	if x, ok := x.GetResponse().(*InstructionResponse_ProcessBundleSplit); ok {
+		return x.ProcessBundleSplit
+	}
+	return nil
+}
+
+func (x *InstructionResponse) GetFinalizeBundle() *FinalizeBundleResponse {
+	if x, ok := x.GetResponse().(*InstructionResponse_FinalizeBundle); ok {
+		return x.FinalizeBundle
+	}
+	return nil
+}
+
+func (x *InstructionResponse) GetMonitoringInfos() *MonitoringInfosMetadataResponse {
+	if x, ok := x.GetResponse().(*InstructionResponse_MonitoringInfos); ok {
+		return x.MonitoringInfos
+	}
+	return nil
+}
+
+func (x *InstructionResponse) GetRegister() *RegisterResponse {
+	if x, ok := x.GetResponse().(*InstructionResponse_Register); ok {
+		return x.Register
+	}
+	return nil
+}
+
 type isInstructionResponse_Response interface {
 	isInstructionResponse_Response()
 }
@@ -418,11 +537,12 @@
 	FinalizeBundle *FinalizeBundleResponse `protobuf:"bytes,1004,opt,name=finalize_bundle,json=finalizeBundle,proto3,oneof"`
 }
 
-type InstructionResponse_ProcessBundleProgressMetadata struct {
-	ProcessBundleProgressMetadata *ProcessBundleProgressMetadataResponse `protobuf:"bytes,1005,opt,name=process_bundle_progress_metadata,json=processBundleProgressMetadata,proto3,oneof"`
+type InstructionResponse_MonitoringInfos struct {
+	MonitoringInfos *MonitoringInfosMetadataResponse `protobuf:"bytes,1005,opt,name=monitoring_infos,json=monitoringInfos,proto3,oneof"`
 }
 
 type InstructionResponse_Register struct {
+	// DEPRECATED
 	Register *RegisterResponse `protobuf:"bytes,1000,opt,name=register,proto3,oneof"`
 }
 
@@ -434,148 +554,106 @@
 
 func (*InstructionResponse_FinalizeBundle) isInstructionResponse_Response() {}
 
-func (*InstructionResponse_ProcessBundleProgressMetadata) isInstructionResponse_Response() {}
+func (*InstructionResponse_MonitoringInfos) isInstructionResponse_Response() {}
 
 func (*InstructionResponse_Register) isInstructionResponse_Response() {}
 
-func (m *InstructionResponse) GetResponse() isInstructionResponse_Response {
-	if m != nil {
-		return m.Response
-	}
-	return nil
-}
-
-func (m *InstructionResponse) GetProcessBundle() *ProcessBundleResponse {
-	if x, ok := m.GetResponse().(*InstructionResponse_ProcessBundle); ok {
-		return x.ProcessBundle
-	}
-	return nil
-}
-
-func (m *InstructionResponse) GetProcessBundleProgress() *ProcessBundleProgressResponse {
-	if x, ok := m.GetResponse().(*InstructionResponse_ProcessBundleProgress); ok {
-		return x.ProcessBundleProgress
-	}
-	return nil
-}
-
-func (m *InstructionResponse) GetProcessBundleSplit() *ProcessBundleSplitResponse {
-	if x, ok := m.GetResponse().(*InstructionResponse_ProcessBundleSplit); ok {
-		return x.ProcessBundleSplit
-	}
-	return nil
-}
-
-func (m *InstructionResponse) GetFinalizeBundle() *FinalizeBundleResponse {
-	if x, ok := m.GetResponse().(*InstructionResponse_FinalizeBundle); ok {
-		return x.FinalizeBundle
-	}
-	return nil
-}
-
-func (m *InstructionResponse) GetProcessBundleProgressMetadata() *ProcessBundleProgressMetadataResponse {
-	if x, ok := m.GetResponse().(*InstructionResponse_ProcessBundleProgressMetadata); ok {
-		return x.ProcessBundleProgressMetadata
-	}
-	return nil
-}
-
-func (m *InstructionResponse) GetRegister() *RegisterResponse {
-	if x, ok := m.GetResponse().(*InstructionResponse_Register); ok {
-		return x.Register
-	}
-	return nil
-}
-
-// XXX_OneofWrappers is for the internal use of the proto package.
-func (*InstructionResponse) XXX_OneofWrappers() []interface{} {
-	return []interface{}{
-		(*InstructionResponse_ProcessBundle)(nil),
-		(*InstructionResponse_ProcessBundleProgress)(nil),
-		(*InstructionResponse_ProcessBundleSplit)(nil),
-		(*InstructionResponse_FinalizeBundle)(nil),
-		(*InstructionResponse_ProcessBundleProgressMetadata)(nil),
-		(*InstructionResponse_Register)(nil),
-	}
-}
-
 // A list of objects which can be referred to by the runner in
 // future requests.
 // Stable
 type RegisterRequest struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
 	// (Optional) The set of descriptors used to process bundles.
 	ProcessBundleDescriptor []*ProcessBundleDescriptor `protobuf:"bytes,1,rep,name=process_bundle_descriptor,json=processBundleDescriptor,proto3" json:"process_bundle_descriptor,omitempty"`
-	XXX_NoUnkeyedLiteral    struct{}                   `json:"-"`
-	XXX_unrecognized        []byte                     `json:"-"`
-	XXX_sizecache           int32                      `json:"-"`
 }
 
-func (m *RegisterRequest) Reset()         { *m = RegisterRequest{} }
-func (m *RegisterRequest) String() string { return proto.CompactTextString(m) }
-func (*RegisterRequest) ProtoMessage()    {}
+func (x *RegisterRequest) Reset() {
+	*x = RegisterRequest{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_fn_api_proto_msgTypes[4]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *RegisterRequest) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*RegisterRequest) ProtoMessage() {}
+
+func (x *RegisterRequest) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_fn_api_proto_msgTypes[4]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use RegisterRequest.ProtoReflect.Descriptor instead.
 func (*RegisterRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_6d954c03a4758710, []int{4}
+	return file_beam_fn_api_proto_rawDescGZIP(), []int{4}
 }
 
-func (m *RegisterRequest) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_RegisterRequest.Unmarshal(m, b)
-}
-func (m *RegisterRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_RegisterRequest.Marshal(b, m, deterministic)
-}
-func (m *RegisterRequest) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_RegisterRequest.Merge(m, src)
-}
-func (m *RegisterRequest) XXX_Size() int {
-	return xxx_messageInfo_RegisterRequest.Size(m)
-}
-func (m *RegisterRequest) XXX_DiscardUnknown() {
-	xxx_messageInfo_RegisterRequest.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_RegisterRequest proto.InternalMessageInfo
-
-func (m *RegisterRequest) GetProcessBundleDescriptor() []*ProcessBundleDescriptor {
-	if m != nil {
-		return m.ProcessBundleDescriptor
+func (x *RegisterRequest) GetProcessBundleDescriptor() []*ProcessBundleDescriptor {
+	if x != nil {
+		return x.ProcessBundleDescriptor
 	}
 	return nil
 }
 
 // Stable
 type RegisterResponse struct {
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
 }
 
-func (m *RegisterResponse) Reset()         { *m = RegisterResponse{} }
-func (m *RegisterResponse) String() string { return proto.CompactTextString(m) }
-func (*RegisterResponse) ProtoMessage()    {}
+func (x *RegisterResponse) Reset() {
+	*x = RegisterResponse{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_fn_api_proto_msgTypes[5]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *RegisterResponse) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*RegisterResponse) ProtoMessage() {}
+
+func (x *RegisterResponse) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_fn_api_proto_msgTypes[5]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use RegisterResponse.ProtoReflect.Descriptor instead.
 func (*RegisterResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_6d954c03a4758710, []int{5}
+	return file_beam_fn_api_proto_rawDescGZIP(), []int{5}
 }
 
-func (m *RegisterResponse) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_RegisterResponse.Unmarshal(m, b)
-}
-func (m *RegisterResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_RegisterResponse.Marshal(b, m, deterministic)
-}
-func (m *RegisterResponse) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_RegisterResponse.Merge(m, src)
-}
-func (m *RegisterResponse) XXX_Size() int {
-	return xxx_messageInfo_RegisterResponse.Size(m)
-}
-func (m *RegisterResponse) XXX_DiscardUnknown() {
-	xxx_messageInfo_RegisterResponse.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_RegisterResponse proto.InternalMessageInfo
-
 // Definitions that should be used to construct the bundle processing graph.
 type ProcessBundleDescriptor struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
 	// (Required) A pipeline level unique id which can be used as a reference to
 	// refer to this.
 	Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"`
@@ -596,88 +674,92 @@
 	// A descriptor describing the end point to use for Data API for user timers.
 	// Required if the ProcessBundleDescriptor contains any transforms that have user timers.
 	TimerApiServiceDescriptor *pipeline_v1.ApiServiceDescriptor `protobuf:"bytes,8,opt,name=timer_api_service_descriptor,json=timerApiServiceDescriptor,proto3" json:"timer_api_service_descriptor,omitempty"`
-	XXX_NoUnkeyedLiteral      struct{}                          `json:"-"`
-	XXX_unrecognized          []byte                            `json:"-"`
-	XXX_sizecache             int32                             `json:"-"`
 }
 
-func (m *ProcessBundleDescriptor) Reset()         { *m = ProcessBundleDescriptor{} }
-func (m *ProcessBundleDescriptor) String() string { return proto.CompactTextString(m) }
-func (*ProcessBundleDescriptor) ProtoMessage()    {}
+func (x *ProcessBundleDescriptor) Reset() {
+	*x = ProcessBundleDescriptor{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_fn_api_proto_msgTypes[6]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *ProcessBundleDescriptor) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*ProcessBundleDescriptor) ProtoMessage() {}
+
+func (x *ProcessBundleDescriptor) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_fn_api_proto_msgTypes[6]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use ProcessBundleDescriptor.ProtoReflect.Descriptor instead.
 func (*ProcessBundleDescriptor) Descriptor() ([]byte, []int) {
-	return fileDescriptor_6d954c03a4758710, []int{6}
+	return file_beam_fn_api_proto_rawDescGZIP(), []int{6}
 }
 
-func (m *ProcessBundleDescriptor) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_ProcessBundleDescriptor.Unmarshal(m, b)
-}
-func (m *ProcessBundleDescriptor) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_ProcessBundleDescriptor.Marshal(b, m, deterministic)
-}
-func (m *ProcessBundleDescriptor) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ProcessBundleDescriptor.Merge(m, src)
-}
-func (m *ProcessBundleDescriptor) XXX_Size() int {
-	return xxx_messageInfo_ProcessBundleDescriptor.Size(m)
-}
-func (m *ProcessBundleDescriptor) XXX_DiscardUnknown() {
-	xxx_messageInfo_ProcessBundleDescriptor.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_ProcessBundleDescriptor proto.InternalMessageInfo
-
-func (m *ProcessBundleDescriptor) GetId() string {
-	if m != nil {
-		return m.Id
+func (x *ProcessBundleDescriptor) GetId() string {
+	if x != nil {
+		return x.Id
 	}
 	return ""
 }
 
-func (m *ProcessBundleDescriptor) GetTransforms() map[string]*pipeline_v1.PTransform {
-	if m != nil {
-		return m.Transforms
+func (x *ProcessBundleDescriptor) GetTransforms() map[string]*pipeline_v1.PTransform {
+	if x != nil {
+		return x.Transforms
 	}
 	return nil
 }
 
-func (m *ProcessBundleDescriptor) GetPcollections() map[string]*pipeline_v1.PCollection {
-	if m != nil {
-		return m.Pcollections
+func (x *ProcessBundleDescriptor) GetPcollections() map[string]*pipeline_v1.PCollection {
+	if x != nil {
+		return x.Pcollections
 	}
 	return nil
 }
 
-func (m *ProcessBundleDescriptor) GetWindowingStrategies() map[string]*pipeline_v1.WindowingStrategy {
-	if m != nil {
-		return m.WindowingStrategies
+func (x *ProcessBundleDescriptor) GetWindowingStrategies() map[string]*pipeline_v1.WindowingStrategy {
+	if x != nil {
+		return x.WindowingStrategies
 	}
 	return nil
 }
 
-func (m *ProcessBundleDescriptor) GetCoders() map[string]*pipeline_v1.Coder {
-	if m != nil {
-		return m.Coders
+func (x *ProcessBundleDescriptor) GetCoders() map[string]*pipeline_v1.Coder {
+	if x != nil {
+		return x.Coders
 	}
 	return nil
 }
 
-func (m *ProcessBundleDescriptor) GetEnvironments() map[string]*pipeline_v1.Environment {
-	if m != nil {
-		return m.Environments
+func (x *ProcessBundleDescriptor) GetEnvironments() map[string]*pipeline_v1.Environment {
+	if x != nil {
+		return x.Environments
 	}
 	return nil
 }
 
-func (m *ProcessBundleDescriptor) GetStateApiServiceDescriptor() *pipeline_v1.ApiServiceDescriptor {
-	if m != nil {
-		return m.StateApiServiceDescriptor
+func (x *ProcessBundleDescriptor) GetStateApiServiceDescriptor() *pipeline_v1.ApiServiceDescriptor {
+	if x != nil {
+		return x.StateApiServiceDescriptor
 	}
 	return nil
 }
 
-func (m *ProcessBundleDescriptor) GetTimerApiServiceDescriptor() *pipeline_v1.ApiServiceDescriptor {
-	if m != nil {
-		return m.TimerApiServiceDescriptor
+func (x *ProcessBundleDescriptor) GetTimerApiServiceDescriptor() *pipeline_v1.ApiServiceDescriptor {
+	if x != nil {
+		return x.TimerApiServiceDescriptor
 	}
 	return nil
 }
@@ -687,6 +769,10 @@
 // https://docs.google.com/document/d/1tUDb45sStdR8u7-jBkGdw3OGFK7aa2-V7eo86zYSE_4/edit#heading=h.9g3g5weg2u9
 // for further details.
 type BundleApplication struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
 	// (Required) The transform to which to pass the element
 	TransformId string `protobuf:"bytes,1,opt,name=transform_id,json=transformId,proto3" json:"transform_id,omitempty"`
 	// (Required) Name of the transform's input to which to pass the element.
@@ -704,68 +790,72 @@
 	// Whether this application potentially produces an unbounded
 	// amount of data. Note that this should only be set to BOUNDED if and
 	// only if the application is known to produce a finite amount of output.
-	IsBounded            pipeline_v1.IsBounded_Enum `protobuf:"varint,5,opt,name=is_bounded,json=isBounded,proto3,enum=org.apache.beam.model.pipeline.v1.IsBounded_Enum" json:"is_bounded,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}                   `json:"-"`
-	XXX_unrecognized     []byte                     `json:"-"`
-	XXX_sizecache        int32                      `json:"-"`
+	IsBounded pipeline_v1.IsBounded_Enum `protobuf:"varint,5,opt,name=is_bounded,json=isBounded,proto3,enum=org.apache.beam.model.pipeline.v1.IsBounded_Enum" json:"is_bounded,omitempty"`
 }
 
-func (m *BundleApplication) Reset()         { *m = BundleApplication{} }
-func (m *BundleApplication) String() string { return proto.CompactTextString(m) }
-func (*BundleApplication) ProtoMessage()    {}
+func (x *BundleApplication) Reset() {
+	*x = BundleApplication{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_fn_api_proto_msgTypes[7]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *BundleApplication) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*BundleApplication) ProtoMessage() {}
+
+func (x *BundleApplication) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_fn_api_proto_msgTypes[7]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use BundleApplication.ProtoReflect.Descriptor instead.
 func (*BundleApplication) Descriptor() ([]byte, []int) {
-	return fileDescriptor_6d954c03a4758710, []int{7}
+	return file_beam_fn_api_proto_rawDescGZIP(), []int{7}
 }
 
-func (m *BundleApplication) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_BundleApplication.Unmarshal(m, b)
-}
-func (m *BundleApplication) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_BundleApplication.Marshal(b, m, deterministic)
-}
-func (m *BundleApplication) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_BundleApplication.Merge(m, src)
-}
-func (m *BundleApplication) XXX_Size() int {
-	return xxx_messageInfo_BundleApplication.Size(m)
-}
-func (m *BundleApplication) XXX_DiscardUnknown() {
-	xxx_messageInfo_BundleApplication.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_BundleApplication proto.InternalMessageInfo
-
-func (m *BundleApplication) GetTransformId() string {
-	if m != nil {
-		return m.TransformId
+func (x *BundleApplication) GetTransformId() string {
+	if x != nil {
+		return x.TransformId
 	}
 	return ""
 }
 
-func (m *BundleApplication) GetInputId() string {
-	if m != nil {
-		return m.InputId
+func (x *BundleApplication) GetInputId() string {
+	if x != nil {
+		return x.InputId
 	}
 	return ""
 }
 
-func (m *BundleApplication) GetElement() []byte {
-	if m != nil {
-		return m.Element
+func (x *BundleApplication) GetElement() []byte {
+	if x != nil {
+		return x.Element
 	}
 	return nil
 }
 
-func (m *BundleApplication) GetOutputWatermarks() map[string]*timestamp.Timestamp {
-	if m != nil {
-		return m.OutputWatermarks
+func (x *BundleApplication) GetOutputWatermarks() map[string]*timestamp.Timestamp {
+	if x != nil {
+		return x.OutputWatermarks
 	}
 	return nil
 }
 
-func (m *BundleApplication) GetIsBounded() pipeline_v1.IsBounded_Enum {
-	if m != nil {
-		return m.IsBounded
+func (x *BundleApplication) GetIsBounded() pipeline_v1.IsBounded_Enum {
+	if x != nil {
+		return x.IsBounded
 	}
 	return pipeline_v1.IsBounded_UNSPECIFIED
 }
@@ -774,52 +864,60 @@
 // Either an absolute timestamp or a relative timestamp can represent a
 // scheduled execution time.
 type DelayedBundleApplication struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
 	// (Required) The application that should be scheduled.
 	Application *BundleApplication `protobuf:"bytes,1,opt,name=application,proto3" json:"application,omitempty"`
 	// Recommended time delay at which the application should be scheduled to
 	// execute by the runner. Time delay that equals 0 may be scheduled to execute
 	// immediately. The unit of time delay should be microsecond.
-	RequestedTimeDelay   *duration.Duration `protobuf:"bytes,2,opt,name=requested_time_delay,json=requestedTimeDelay,proto3" json:"requested_time_delay,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}           `json:"-"`
-	XXX_unrecognized     []byte             `json:"-"`
-	XXX_sizecache        int32              `json:"-"`
+	RequestedTimeDelay *duration.Duration `protobuf:"bytes,2,opt,name=requested_time_delay,json=requestedTimeDelay,proto3" json:"requested_time_delay,omitempty"`
 }
 
-func (m *DelayedBundleApplication) Reset()         { *m = DelayedBundleApplication{} }
-func (m *DelayedBundleApplication) String() string { return proto.CompactTextString(m) }
-func (*DelayedBundleApplication) ProtoMessage()    {}
+func (x *DelayedBundleApplication) Reset() {
+	*x = DelayedBundleApplication{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_fn_api_proto_msgTypes[8]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *DelayedBundleApplication) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*DelayedBundleApplication) ProtoMessage() {}
+
+func (x *DelayedBundleApplication) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_fn_api_proto_msgTypes[8]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use DelayedBundleApplication.ProtoReflect.Descriptor instead.
 func (*DelayedBundleApplication) Descriptor() ([]byte, []int) {
-	return fileDescriptor_6d954c03a4758710, []int{8}
+	return file_beam_fn_api_proto_rawDescGZIP(), []int{8}
 }
 
-func (m *DelayedBundleApplication) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_DelayedBundleApplication.Unmarshal(m, b)
-}
-func (m *DelayedBundleApplication) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_DelayedBundleApplication.Marshal(b, m, deterministic)
-}
-func (m *DelayedBundleApplication) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_DelayedBundleApplication.Merge(m, src)
-}
-func (m *DelayedBundleApplication) XXX_Size() int {
-	return xxx_messageInfo_DelayedBundleApplication.Size(m)
-}
-func (m *DelayedBundleApplication) XXX_DiscardUnknown() {
-	xxx_messageInfo_DelayedBundleApplication.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_DelayedBundleApplication proto.InternalMessageInfo
-
-func (m *DelayedBundleApplication) GetApplication() *BundleApplication {
-	if m != nil {
-		return m.Application
+func (x *DelayedBundleApplication) GetApplication() *BundleApplication {
+	if x != nil {
+		return x.Application
 	}
 	return nil
 }
 
-func (m *DelayedBundleApplication) GetRequestedTimeDelay() *duration.Duration {
-	if m != nil {
-		return m.RequestedTimeDelay
+func (x *DelayedBundleApplication) GetRequestedTimeDelay() *duration.Duration {
+	if x != nil {
+		return x.RequestedTimeDelay
 	}
 	return nil
 }
@@ -827,236 +925,69 @@
 // A request to process a given bundle.
 // Stable
 type ProcessBundleRequest struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
 	// (Required) A reference to the process bundle descriptor that must be
 	// instantiated and executed by the SDK harness.
 	ProcessBundleDescriptorId string `protobuf:"bytes,1,opt,name=process_bundle_descriptor_id,json=processBundleDescriptorId,proto3" json:"process_bundle_descriptor_id,omitempty"`
 	// (Optional) A list of cache tokens that can be used by an SDK to reuse
 	// cached data returned by the State API across multiple bundles.
-	CacheTokens          []*ProcessBundleRequest_CacheToken `protobuf:"bytes,2,rep,name=cache_tokens,json=cacheTokens,proto3" json:"cache_tokens,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}                           `json:"-"`
-	XXX_unrecognized     []byte                             `json:"-"`
-	XXX_sizecache        int32                              `json:"-"`
+	CacheTokens []*ProcessBundleRequest_CacheToken `protobuf:"bytes,2,rep,name=cache_tokens,json=cacheTokens,proto3" json:"cache_tokens,omitempty"`
 }
 
-func (m *ProcessBundleRequest) Reset()         { *m = ProcessBundleRequest{} }
-func (m *ProcessBundleRequest) String() string { return proto.CompactTextString(m) }
-func (*ProcessBundleRequest) ProtoMessage()    {}
+func (x *ProcessBundleRequest) Reset() {
+	*x = ProcessBundleRequest{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_fn_api_proto_msgTypes[9]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *ProcessBundleRequest) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*ProcessBundleRequest) ProtoMessage() {}
+
+func (x *ProcessBundleRequest) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_fn_api_proto_msgTypes[9]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use ProcessBundleRequest.ProtoReflect.Descriptor instead.
 func (*ProcessBundleRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_6d954c03a4758710, []int{9}
+	return file_beam_fn_api_proto_rawDescGZIP(), []int{9}
 }
 
-func (m *ProcessBundleRequest) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_ProcessBundleRequest.Unmarshal(m, b)
-}
-func (m *ProcessBundleRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_ProcessBundleRequest.Marshal(b, m, deterministic)
-}
-func (m *ProcessBundleRequest) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ProcessBundleRequest.Merge(m, src)
-}
-func (m *ProcessBundleRequest) XXX_Size() int {
-	return xxx_messageInfo_ProcessBundleRequest.Size(m)
-}
-func (m *ProcessBundleRequest) XXX_DiscardUnknown() {
-	xxx_messageInfo_ProcessBundleRequest.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_ProcessBundleRequest proto.InternalMessageInfo
-
-func (m *ProcessBundleRequest) GetProcessBundleDescriptorId() string {
-	if m != nil {
-		return m.ProcessBundleDescriptorId
+func (x *ProcessBundleRequest) GetProcessBundleDescriptorId() string {
+	if x != nil {
+		return x.ProcessBundleDescriptorId
 	}
 	return ""
 }
 
-func (m *ProcessBundleRequest) GetCacheTokens() []*ProcessBundleRequest_CacheToken {
-	if m != nil {
-		return m.CacheTokens
+func (x *ProcessBundleRequest) GetCacheTokens() []*ProcessBundleRequest_CacheToken {
+	if x != nil {
+		return x.CacheTokens
 	}
 	return nil
 }
 
-// A cache token which can be used by an SDK to check for the validity
-// of cached elements which have a cache token associated.
-type ProcessBundleRequest_CacheToken struct {
-	// The scope of a cache token.
-	//
-	// Types that are valid to be assigned to Type:
-	//	*ProcessBundleRequest_CacheToken_UserState_
-	//	*ProcessBundleRequest_CacheToken_SideInput_
-	Type isProcessBundleRequest_CacheToken_Type `protobuf_oneof:"type"`
-	// The cache token identifier which should be globally unique.
-	Token                []byte   `protobuf:"bytes,10,opt,name=token,proto3" json:"token,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *ProcessBundleRequest_CacheToken) Reset()         { *m = ProcessBundleRequest_CacheToken{} }
-func (m *ProcessBundleRequest_CacheToken) String() string { return proto.CompactTextString(m) }
-func (*ProcessBundleRequest_CacheToken) ProtoMessage()    {}
-func (*ProcessBundleRequest_CacheToken) Descriptor() ([]byte, []int) {
-	return fileDescriptor_6d954c03a4758710, []int{9, 0}
-}
-
-func (m *ProcessBundleRequest_CacheToken) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_ProcessBundleRequest_CacheToken.Unmarshal(m, b)
-}
-func (m *ProcessBundleRequest_CacheToken) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_ProcessBundleRequest_CacheToken.Marshal(b, m, deterministic)
-}
-func (m *ProcessBundleRequest_CacheToken) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ProcessBundleRequest_CacheToken.Merge(m, src)
-}
-func (m *ProcessBundleRequest_CacheToken) XXX_Size() int {
-	return xxx_messageInfo_ProcessBundleRequest_CacheToken.Size(m)
-}
-func (m *ProcessBundleRequest_CacheToken) XXX_DiscardUnknown() {
-	xxx_messageInfo_ProcessBundleRequest_CacheToken.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_ProcessBundleRequest_CacheToken proto.InternalMessageInfo
-
-type isProcessBundleRequest_CacheToken_Type interface {
-	isProcessBundleRequest_CacheToken_Type()
-}
-
-type ProcessBundleRequest_CacheToken_UserState_ struct {
-	UserState *ProcessBundleRequest_CacheToken_UserState `protobuf:"bytes,1,opt,name=user_state,json=userState,proto3,oneof"`
-}
-
-type ProcessBundleRequest_CacheToken_SideInput_ struct {
-	SideInput *ProcessBundleRequest_CacheToken_SideInput `protobuf:"bytes,2,opt,name=side_input,json=sideInput,proto3,oneof"`
-}
-
-func (*ProcessBundleRequest_CacheToken_UserState_) isProcessBundleRequest_CacheToken_Type() {}
-
-func (*ProcessBundleRequest_CacheToken_SideInput_) isProcessBundleRequest_CacheToken_Type() {}
-
-func (m *ProcessBundleRequest_CacheToken) GetType() isProcessBundleRequest_CacheToken_Type {
-	if m != nil {
-		return m.Type
-	}
-	return nil
-}
-
-func (m *ProcessBundleRequest_CacheToken) GetUserState() *ProcessBundleRequest_CacheToken_UserState {
-	if x, ok := m.GetType().(*ProcessBundleRequest_CacheToken_UserState_); ok {
-		return x.UserState
-	}
-	return nil
-}
-
-func (m *ProcessBundleRequest_CacheToken) GetSideInput() *ProcessBundleRequest_CacheToken_SideInput {
-	if x, ok := m.GetType().(*ProcessBundleRequest_CacheToken_SideInput_); ok {
-		return x.SideInput
-	}
-	return nil
-}
-
-func (m *ProcessBundleRequest_CacheToken) GetToken() []byte {
-	if m != nil {
-		return m.Token
-	}
-	return nil
-}
-
-// XXX_OneofWrappers is for the internal use of the proto package.
-func (*ProcessBundleRequest_CacheToken) XXX_OneofWrappers() []interface{} {
-	return []interface{}{
-		(*ProcessBundleRequest_CacheToken_UserState_)(nil),
-		(*ProcessBundleRequest_CacheToken_SideInput_)(nil),
-	}
-}
-
-// A flag to indicate a cache token is valid for all user state.
-type ProcessBundleRequest_CacheToken_UserState struct {
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *ProcessBundleRequest_CacheToken_UserState) Reset() {
-	*m = ProcessBundleRequest_CacheToken_UserState{}
-}
-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_6d954c03a4758710, []int{9, 0, 0}
-}
-
-func (m *ProcessBundleRequest_CacheToken_UserState) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_ProcessBundleRequest_CacheToken_UserState.Unmarshal(m, b)
-}
-func (m *ProcessBundleRequest_CacheToken_UserState) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_ProcessBundleRequest_CacheToken_UserState.Marshal(b, m, deterministic)
-}
-func (m *ProcessBundleRequest_CacheToken_UserState) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ProcessBundleRequest_CacheToken_UserState.Merge(m, src)
-}
-func (m *ProcessBundleRequest_CacheToken_UserState) XXX_Size() int {
-	return xxx_messageInfo_ProcessBundleRequest_CacheToken_UserState.Size(m)
-}
-func (m *ProcessBundleRequest_CacheToken_UserState) XXX_DiscardUnknown() {
-	xxx_messageInfo_ProcessBundleRequest_CacheToken_UserState.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_ProcessBundleRequest_CacheToken_UserState proto.InternalMessageInfo
-
-// A flag to indicate a cache token is valid for a side input.
-type ProcessBundleRequest_CacheToken_SideInput struct {
-	// (Required) The id of the PTransform containing a side input.
-	TransformId string `protobuf:"bytes,1,opt,name=transform_id,json=transformId,proto3" json:"transform_id,omitempty"`
-	// (Required) The id of the side input.
-	SideInputId          string   `protobuf:"bytes,2,opt,name=side_input_id,json=sideInputId,proto3" json:"side_input_id,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *ProcessBundleRequest_CacheToken_SideInput) Reset() {
-	*m = ProcessBundleRequest_CacheToken_SideInput{}
-}
-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_6d954c03a4758710, []int{9, 0, 1}
-}
-
-func (m *ProcessBundleRequest_CacheToken_SideInput) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_ProcessBundleRequest_CacheToken_SideInput.Unmarshal(m, b)
-}
-func (m *ProcessBundleRequest_CacheToken_SideInput) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_ProcessBundleRequest_CacheToken_SideInput.Marshal(b, m, deterministic)
-}
-func (m *ProcessBundleRequest_CacheToken_SideInput) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ProcessBundleRequest_CacheToken_SideInput.Merge(m, src)
-}
-func (m *ProcessBundleRequest_CacheToken_SideInput) XXX_Size() int {
-	return xxx_messageInfo_ProcessBundleRequest_CacheToken_SideInput.Size(m)
-}
-func (m *ProcessBundleRequest_CacheToken_SideInput) XXX_DiscardUnknown() {
-	xxx_messageInfo_ProcessBundleRequest_CacheToken_SideInput.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_ProcessBundleRequest_CacheToken_SideInput proto.InternalMessageInfo
-
-func (m *ProcessBundleRequest_CacheToken_SideInput) GetTransformId() string {
-	if m != nil {
-		return m.TransformId
-	}
-	return ""
-}
-
-func (m *ProcessBundleRequest_CacheToken_SideInput) GetSideInputId() string {
-	if m != nil {
-		return m.SideInputId
-	}
-	return ""
-}
-
 type ProcessBundleResponse struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
 	// (Optional) Specifies that the bundle has not been completed and the
 	// following applications need to be scheduled and executed in the future.
 	// A runner that does not yet support residual roots MUST still check that
@@ -1075,66 +1006,70 @@
 	// An SDK can report metrics using an identifier that only contains the
 	// associated payload. A runner who wants to receive the full metrics
 	// information can request all the monitoring metadata via a
-	// ProcessBundleProgressMetadataRequest providing a list of ids as necessary.
+	// MonitoringInfosMetadataRequest providing a list of ids as necessary.
 	//
 	// The SDK is allowed to reuse the identifiers across multiple bundles as long
 	// as the MonitoringInfo could be reconstructed fully by overwriting its
 	// payload field with the bytes specified here.
-	MonitoringData       map[string][]byte `protobuf:"bytes,5,rep,name=monitoring_data,json=monitoringData,proto3" json:"monitoring_data,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"`
-	XXX_NoUnkeyedLiteral struct{}          `json:"-"`
-	XXX_unrecognized     []byte            `json:"-"`
-	XXX_sizecache        int32             `json:"-"`
+	MonitoringData map[string][]byte `protobuf:"bytes,5,rep,name=monitoring_data,json=monitoringData,proto3" json:"monitoring_data,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"`
 }
 
-func (m *ProcessBundleResponse) Reset()         { *m = ProcessBundleResponse{} }
-func (m *ProcessBundleResponse) String() string { return proto.CompactTextString(m) }
-func (*ProcessBundleResponse) ProtoMessage()    {}
+func (x *ProcessBundleResponse) Reset() {
+	*x = ProcessBundleResponse{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_fn_api_proto_msgTypes[10]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *ProcessBundleResponse) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*ProcessBundleResponse) ProtoMessage() {}
+
+func (x *ProcessBundleResponse) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_fn_api_proto_msgTypes[10]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use ProcessBundleResponse.ProtoReflect.Descriptor instead.
 func (*ProcessBundleResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_6d954c03a4758710, []int{10}
+	return file_beam_fn_api_proto_rawDescGZIP(), []int{10}
 }
 
-func (m *ProcessBundleResponse) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_ProcessBundleResponse.Unmarshal(m, b)
-}
-func (m *ProcessBundleResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_ProcessBundleResponse.Marshal(b, m, deterministic)
-}
-func (m *ProcessBundleResponse) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ProcessBundleResponse.Merge(m, src)
-}
-func (m *ProcessBundleResponse) XXX_Size() int {
-	return xxx_messageInfo_ProcessBundleResponse.Size(m)
-}
-func (m *ProcessBundleResponse) XXX_DiscardUnknown() {
-	xxx_messageInfo_ProcessBundleResponse.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_ProcessBundleResponse proto.InternalMessageInfo
-
-func (m *ProcessBundleResponse) GetResidualRoots() []*DelayedBundleApplication {
-	if m != nil {
-		return m.ResidualRoots
+func (x *ProcessBundleResponse) GetResidualRoots() []*DelayedBundleApplication {
+	if x != nil {
+		return x.ResidualRoots
 	}
 	return nil
 }
 
-func (m *ProcessBundleResponse) GetMonitoringInfos() []*pipeline_v1.MonitoringInfo {
-	if m != nil {
-		return m.MonitoringInfos
+func (x *ProcessBundleResponse) GetMonitoringInfos() []*pipeline_v1.MonitoringInfo {
+	if x != nil {
+		return x.MonitoringInfos
 	}
 	return nil
 }
 
-func (m *ProcessBundleResponse) GetRequiresFinalization() bool {
-	if m != nil {
-		return m.RequiresFinalization
+func (x *ProcessBundleResponse) GetRequiresFinalization() bool {
+	if x != nil {
+		return x.RequiresFinalization
 	}
 	return false
 }
 
-func (m *ProcessBundleResponse) GetMonitoringData() map[string][]byte {
-	if m != nil {
-		return m.MonitoringData
+func (x *ProcessBundleResponse) GetMonitoringData() map[string][]byte {
+	if x != nil {
+		return x.MonitoringData
 	}
 	return nil
 }
@@ -1143,97 +1078,117 @@
 // This is an optional request to be handled and is used to support advanced
 // SDK features such as SplittableDoFn, user level metrics etc.
 type ProcessBundleProgressRequest struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
 	// (Required) A reference to an active process bundle request with the given
 	// instruction id.
-	InstructionId        string   `protobuf:"bytes,1,opt,name=instruction_id,json=instructionId,proto3" json:"instruction_id,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
+	InstructionId string `protobuf:"bytes,1,opt,name=instruction_id,json=instructionId,proto3" json:"instruction_id,omitempty"`
 }
 
-func (m *ProcessBundleProgressRequest) Reset()         { *m = ProcessBundleProgressRequest{} }
-func (m *ProcessBundleProgressRequest) String() string { return proto.CompactTextString(m) }
-func (*ProcessBundleProgressRequest) ProtoMessage()    {}
+func (x *ProcessBundleProgressRequest) Reset() {
+	*x = ProcessBundleProgressRequest{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_fn_api_proto_msgTypes[11]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *ProcessBundleProgressRequest) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*ProcessBundleProgressRequest) ProtoMessage() {}
+
+func (x *ProcessBundleProgressRequest) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_fn_api_proto_msgTypes[11]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use ProcessBundleProgressRequest.ProtoReflect.Descriptor instead.
 func (*ProcessBundleProgressRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_6d954c03a4758710, []int{11}
+	return file_beam_fn_api_proto_rawDescGZIP(), []int{11}
 }
 
-func (m *ProcessBundleProgressRequest) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_ProcessBundleProgressRequest.Unmarshal(m, b)
-}
-func (m *ProcessBundleProgressRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_ProcessBundleProgressRequest.Marshal(b, m, deterministic)
-}
-func (m *ProcessBundleProgressRequest) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ProcessBundleProgressRequest.Merge(m, src)
-}
-func (m *ProcessBundleProgressRequest) XXX_Size() int {
-	return xxx_messageInfo_ProcessBundleProgressRequest.Size(m)
-}
-func (m *ProcessBundleProgressRequest) XXX_DiscardUnknown() {
-	xxx_messageInfo_ProcessBundleProgressRequest.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_ProcessBundleProgressRequest proto.InternalMessageInfo
-
-func (m *ProcessBundleProgressRequest) GetInstructionId() string {
-	if m != nil {
-		return m.InstructionId
+func (x *ProcessBundleProgressRequest) GetInstructionId() string {
+	if x != nil {
+		return x.InstructionId
 	}
 	return ""
 }
 
-// A request to provide full MonitoringInfo for a given bundle.
+// A request to provide full MonitoringInfo for a given id.
 //
 // An SDK can report metrics using an identifier that only contains the
 // associated payload. A runner who wants to receive the full metrics
 // information can request all the monitoring metadata via a
-// ProcessBundleProgressMetadataRequest providing a list of ids as necessary.
+// MonitoringInfosMetadataRequest providing a list of ids as necessary.
 //
-// The SDK is allowed to reuse the identifiers across multiple bundles as long
-// as the MonitoringInfo could be reconstructed fully by overwriting its
-// payload field with the bytes specified here.
-type ProcessBundleProgressMetadataRequest struct {
-	// A list of ids for which the full MonitoringInfo is requested for.
-	MonitoringInfoId     []string `protobuf:"bytes,1,rep,name=monitoring_info_id,json=monitoringInfoId,proto3" json:"monitoring_info_id,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
+// The MonitoringInfo ids are scoped to the associated control connection. For
+// example, an SDK may reuse the ids across multiple bundles.
+type MonitoringInfosMetadataRequest struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	// A list of ids for which MonitoringInfo are requested. All but the payload
+	// field will be populated.
+	MonitoringInfoId []string `protobuf:"bytes,1,rep,name=monitoring_info_id,json=monitoringInfoId,proto3" json:"monitoring_info_id,omitempty"`
 }
 
-func (m *ProcessBundleProgressMetadataRequest) Reset()         { *m = ProcessBundleProgressMetadataRequest{} }
-func (m *ProcessBundleProgressMetadataRequest) String() string { return proto.CompactTextString(m) }
-func (*ProcessBundleProgressMetadataRequest) ProtoMessage()    {}
-func (*ProcessBundleProgressMetadataRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_6d954c03a4758710, []int{12}
+func (x *MonitoringInfosMetadataRequest) Reset() {
+	*x = MonitoringInfosMetadataRequest{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_fn_api_proto_msgTypes[12]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
 }
 
-func (m *ProcessBundleProgressMetadataRequest) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_ProcessBundleProgressMetadataRequest.Unmarshal(m, b)
-}
-func (m *ProcessBundleProgressMetadataRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_ProcessBundleProgressMetadataRequest.Marshal(b, m, deterministic)
-}
-func (m *ProcessBundleProgressMetadataRequest) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ProcessBundleProgressMetadataRequest.Merge(m, src)
-}
-func (m *ProcessBundleProgressMetadataRequest) XXX_Size() int {
-	return xxx_messageInfo_ProcessBundleProgressMetadataRequest.Size(m)
-}
-func (m *ProcessBundleProgressMetadataRequest) XXX_DiscardUnknown() {
-	xxx_messageInfo_ProcessBundleProgressMetadataRequest.DiscardUnknown(m)
+func (x *MonitoringInfosMetadataRequest) String() string {
+	return protoimpl.X.MessageStringOf(x)
 }
 
-var xxx_messageInfo_ProcessBundleProgressMetadataRequest proto.InternalMessageInfo
+func (*MonitoringInfosMetadataRequest) ProtoMessage() {}
 
-func (m *ProcessBundleProgressMetadataRequest) GetMonitoringInfoId() []string {
-	if m != nil {
-		return m.MonitoringInfoId
+func (x *MonitoringInfosMetadataRequest) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_fn_api_proto_msgTypes[12]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use MonitoringInfosMetadataRequest.ProtoReflect.Descriptor instead.
+func (*MonitoringInfosMetadataRequest) Descriptor() ([]byte, []int) {
+	return file_beam_fn_api_proto_rawDescGZIP(), []int{12}
+}
+
+func (x *MonitoringInfosMetadataRequest) GetMonitoringInfoId() []string {
+	if x != nil {
+		return x.MonitoringInfoId
 	}
 	return nil
 }
 
 type ProcessBundleProgressResponse struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
 	// DEPRECATED (Required) The list of metrics or other MonitoredState
 	// collected while processing this bundle.
 	MonitoringInfos []*pipeline_v1.MonitoringInfo `protobuf:"bytes,3,rep,name=monitoring_infos,json=monitoringInfos,proto3" json:"monitoring_infos,omitempty"`
@@ -1242,52 +1197,56 @@
 	// An SDK can report metrics using an identifier that only contains the
 	// associated payload. A runner who wants to receive the full metrics
 	// information can request all the monitoring metadata via a
-	// ProcessBundleProgressMetadataRequest providing a list of ids as necessary.
+	// MonitoringInfosMetadataRequest providing a list of ids as necessary.
 	//
 	// The SDK is allowed to reuse the identifiers across multiple bundles as long
 	// as the MonitoringInfo could be reconstructed fully by overwriting its
 	// payload field with the bytes specified here.
-	MonitoringData       map[string][]byte `protobuf:"bytes,5,rep,name=monitoring_data,json=monitoringData,proto3" json:"monitoring_data,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"`
-	XXX_NoUnkeyedLiteral struct{}          `json:"-"`
-	XXX_unrecognized     []byte            `json:"-"`
-	XXX_sizecache        int32             `json:"-"`
+	MonitoringData map[string][]byte `protobuf:"bytes,5,rep,name=monitoring_data,json=monitoringData,proto3" json:"monitoring_data,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"`
 }
 
-func (m *ProcessBundleProgressResponse) Reset()         { *m = ProcessBundleProgressResponse{} }
-func (m *ProcessBundleProgressResponse) String() string { return proto.CompactTextString(m) }
-func (*ProcessBundleProgressResponse) ProtoMessage()    {}
+func (x *ProcessBundleProgressResponse) Reset() {
+	*x = ProcessBundleProgressResponse{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_fn_api_proto_msgTypes[13]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *ProcessBundleProgressResponse) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*ProcessBundleProgressResponse) ProtoMessage() {}
+
+func (x *ProcessBundleProgressResponse) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_fn_api_proto_msgTypes[13]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use ProcessBundleProgressResponse.ProtoReflect.Descriptor instead.
 func (*ProcessBundleProgressResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_6d954c03a4758710, []int{13}
+	return file_beam_fn_api_proto_rawDescGZIP(), []int{13}
 }
 
-func (m *ProcessBundleProgressResponse) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_ProcessBundleProgressResponse.Unmarshal(m, b)
-}
-func (m *ProcessBundleProgressResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_ProcessBundleProgressResponse.Marshal(b, m, deterministic)
-}
-func (m *ProcessBundleProgressResponse) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ProcessBundleProgressResponse.Merge(m, src)
-}
-func (m *ProcessBundleProgressResponse) XXX_Size() int {
-	return xxx_messageInfo_ProcessBundleProgressResponse.Size(m)
-}
-func (m *ProcessBundleProgressResponse) XXX_DiscardUnknown() {
-	xxx_messageInfo_ProcessBundleProgressResponse.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_ProcessBundleProgressResponse proto.InternalMessageInfo
-
-func (m *ProcessBundleProgressResponse) GetMonitoringInfos() []*pipeline_v1.MonitoringInfo {
-	if m != nil {
-		return m.MonitoringInfos
+func (x *ProcessBundleProgressResponse) GetMonitoringInfos() []*pipeline_v1.MonitoringInfo {
+	if x != nil {
+		return x.MonitoringInfos
 	}
 	return nil
 }
 
-func (m *ProcessBundleProgressResponse) GetMonitoringData() map[string][]byte {
-	if m != nil {
-		return m.MonitoringData
+func (x *ProcessBundleProgressResponse) GetMonitoringData() map[string][]byte {
+	if x != nil {
+		return x.MonitoringData
 	}
 	return nil
 }
@@ -1298,53 +1257,65 @@
 // An SDK can report metrics using an identifier that only contains the
 // associated payload. A runner who wants to receive the full metrics
 // information can request all the monitoring metadata via a
-// ProcessBundleProgressMetadataRequest providing a list of ids as necessary.
+// MonitoringInfosMetadataRequest providing a list of ids as necessary.
 //
-// The SDK is allowed to reuse the identifiers across multiple bundles as long
-// as the MonitoringInfo could be reconstructed fully by overwriting its
-// payload field with the bytes specified here.
-type ProcessBundleProgressMetadataResponse struct {
-	// A mapping from an identifier to the full metrics information.
-	MonitoringInfo       map[string]*pipeline_v1.MonitoringInfo `protobuf:"bytes,1,rep,name=monitoring_info,json=monitoringInfo,proto3" json:"monitoring_info,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"`
-	XXX_NoUnkeyedLiteral struct{}                               `json:"-"`
-	XXX_unrecognized     []byte                                 `json:"-"`
-	XXX_sizecache        int32                                  `json:"-"`
+// The MonitoringInfo ids are scoped to the associated control connection. For
+// example an SDK may reuse the ids across multiple bundles.
+type MonitoringInfosMetadataResponse struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	// A mapping from a requested identifier to a MonitoringInfo. All fields
+	// except for the payload of the MonitoringInfo will be specified.
+	MonitoringInfo map[string]*pipeline_v1.MonitoringInfo `protobuf:"bytes,1,rep,name=monitoring_info,json=monitoringInfo,proto3" json:"monitoring_info,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"`
 }
 
-func (m *ProcessBundleProgressMetadataResponse) Reset()         { *m = ProcessBundleProgressMetadataResponse{} }
-func (m *ProcessBundleProgressMetadataResponse) String() string { return proto.CompactTextString(m) }
-func (*ProcessBundleProgressMetadataResponse) ProtoMessage()    {}
-func (*ProcessBundleProgressMetadataResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_6d954c03a4758710, []int{14}
+func (x *MonitoringInfosMetadataResponse) Reset() {
+	*x = MonitoringInfosMetadataResponse{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_fn_api_proto_msgTypes[14]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
 }
 
-func (m *ProcessBundleProgressMetadataResponse) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_ProcessBundleProgressMetadataResponse.Unmarshal(m, b)
-}
-func (m *ProcessBundleProgressMetadataResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_ProcessBundleProgressMetadataResponse.Marshal(b, m, deterministic)
-}
-func (m *ProcessBundleProgressMetadataResponse) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ProcessBundleProgressMetadataResponse.Merge(m, src)
-}
-func (m *ProcessBundleProgressMetadataResponse) XXX_Size() int {
-	return xxx_messageInfo_ProcessBundleProgressMetadataResponse.Size(m)
-}
-func (m *ProcessBundleProgressMetadataResponse) XXX_DiscardUnknown() {
-	xxx_messageInfo_ProcessBundleProgressMetadataResponse.DiscardUnknown(m)
+func (x *MonitoringInfosMetadataResponse) String() string {
+	return protoimpl.X.MessageStringOf(x)
 }
 
-var xxx_messageInfo_ProcessBundleProgressMetadataResponse proto.InternalMessageInfo
+func (*MonitoringInfosMetadataResponse) ProtoMessage() {}
 
-func (m *ProcessBundleProgressMetadataResponse) GetMonitoringInfo() map[string]*pipeline_v1.MonitoringInfo {
-	if m != nil {
-		return m.MonitoringInfo
+func (x *MonitoringInfosMetadataResponse) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_fn_api_proto_msgTypes[14]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use MonitoringInfosMetadataResponse.ProtoReflect.Descriptor instead.
+func (*MonitoringInfosMetadataResponse) Descriptor() ([]byte, []int) {
+	return file_beam_fn_api_proto_rawDescGZIP(), []int{14}
+}
+
+func (x *MonitoringInfosMetadataResponse) GetMonitoringInfo() map[string]*pipeline_v1.MonitoringInfo {
+	if x != nil {
+		return x.MonitoringInfo
 	}
 	return nil
 }
 
 // Represents a request to the SDK to split a currently active bundle.
 type ProcessBundleSplitRequest struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
 	// (Required) A reference to an active process bundle request with the given
 	// instruction id.
 	InstructionId string `protobuf:"bytes,1,opt,name=instruction_id,json=instructionId,proto3" json:"instruction_id,omitempty"`
@@ -1353,122 +1324,55 @@
 	// Currently only splits at GRPC read operations are supported.
 	// This may, of course, limit the amount of work downstream operations
 	// receive.
-	DesiredSplits        map[string]*ProcessBundleSplitRequest_DesiredSplit `protobuf:"bytes,3,rep,name=desired_splits,json=desiredSplits,proto3" json:"desired_splits,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"`
-	XXX_NoUnkeyedLiteral struct{}                                           `json:"-"`
-	XXX_unrecognized     []byte                                             `json:"-"`
-	XXX_sizecache        int32                                              `json:"-"`
+	DesiredSplits map[string]*ProcessBundleSplitRequest_DesiredSplit `protobuf:"bytes,3,rep,name=desired_splits,json=desiredSplits,proto3" json:"desired_splits,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"`
 }
 
-func (m *ProcessBundleSplitRequest) Reset()         { *m = ProcessBundleSplitRequest{} }
-func (m *ProcessBundleSplitRequest) String() string { return proto.CompactTextString(m) }
-func (*ProcessBundleSplitRequest) ProtoMessage()    {}
+func (x *ProcessBundleSplitRequest) Reset() {
+	*x = ProcessBundleSplitRequest{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_fn_api_proto_msgTypes[15]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *ProcessBundleSplitRequest) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*ProcessBundleSplitRequest) ProtoMessage() {}
+
+func (x *ProcessBundleSplitRequest) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_fn_api_proto_msgTypes[15]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use ProcessBundleSplitRequest.ProtoReflect.Descriptor instead.
 func (*ProcessBundleSplitRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_6d954c03a4758710, []int{15}
+	return file_beam_fn_api_proto_rawDescGZIP(), []int{15}
 }
 
-func (m *ProcessBundleSplitRequest) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_ProcessBundleSplitRequest.Unmarshal(m, b)
-}
-func (m *ProcessBundleSplitRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_ProcessBundleSplitRequest.Marshal(b, m, deterministic)
-}
-func (m *ProcessBundleSplitRequest) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ProcessBundleSplitRequest.Merge(m, src)
-}
-func (m *ProcessBundleSplitRequest) XXX_Size() int {
-	return xxx_messageInfo_ProcessBundleSplitRequest.Size(m)
-}
-func (m *ProcessBundleSplitRequest) XXX_DiscardUnknown() {
-	xxx_messageInfo_ProcessBundleSplitRequest.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_ProcessBundleSplitRequest proto.InternalMessageInfo
-
-func (m *ProcessBundleSplitRequest) GetInstructionId() string {
-	if m != nil {
-		return m.InstructionId
+func (x *ProcessBundleSplitRequest) GetInstructionId() string {
+	if x != nil {
+		return x.InstructionId
 	}
 	return ""
 }
 
-func (m *ProcessBundleSplitRequest) GetDesiredSplits() map[string]*ProcessBundleSplitRequest_DesiredSplit {
-	if m != nil {
-		return m.DesiredSplits
+func (x *ProcessBundleSplitRequest) GetDesiredSplits() map[string]*ProcessBundleSplitRequest_DesiredSplit {
+	if x != nil {
+		return x.DesiredSplits
 	}
 	return nil
 }
 
-// A message specifying the desired split for a single transform.
-type ProcessBundleSplitRequest_DesiredSplit struct {
-	// (Required) The fraction of known work remaining in this bundle
-	// for this transform that should be kept by the SDK after this split.
-	//
-	// Set to 0 to "checkpoint" as soon as possible (keeping as little work as
-	// possible and returning the remainder).
-	FractionOfRemainder float64 `protobuf:"fixed64,1,opt,name=fraction_of_remainder,json=fractionOfRemainder,proto3" json:"fraction_of_remainder,omitempty"`
-	// A set of allowed element indices where the SDK may split. When this is
-	// empty, there are no constraints on where to split.
-	// Specifically, the first_residual_element of a split result must be an
-	// allowed split point, and the last_primary_element must immediately
-	// preceded an allowed split point.
-	AllowedSplitPoints []int64 `protobuf:"varint,3,rep,packed,name=allowed_split_points,json=allowedSplitPoints,proto3" json:"allowed_split_points,omitempty"`
-	// (Required for GrpcRead operations) Number of total elements expected
-	// to be sent to this GrpcRead operation, required to correctly account
-	// for unreceived data when determining where to split.
-	EstimatedInputElements int64    `protobuf:"varint,2,opt,name=estimated_input_elements,json=estimatedInputElements,proto3" json:"estimated_input_elements,omitempty"`
-	XXX_NoUnkeyedLiteral   struct{} `json:"-"`
-	XXX_unrecognized       []byte   `json:"-"`
-	XXX_sizecache          int32    `json:"-"`
-}
-
-func (m *ProcessBundleSplitRequest_DesiredSplit) Reset() {
-	*m = ProcessBundleSplitRequest_DesiredSplit{}
-}
-func (m *ProcessBundleSplitRequest_DesiredSplit) String() string { return proto.CompactTextString(m) }
-func (*ProcessBundleSplitRequest_DesiredSplit) ProtoMessage()    {}
-func (*ProcessBundleSplitRequest_DesiredSplit) Descriptor() ([]byte, []int) {
-	return fileDescriptor_6d954c03a4758710, []int{15, 0}
-}
-
-func (m *ProcessBundleSplitRequest_DesiredSplit) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_ProcessBundleSplitRequest_DesiredSplit.Unmarshal(m, b)
-}
-func (m *ProcessBundleSplitRequest_DesiredSplit) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_ProcessBundleSplitRequest_DesiredSplit.Marshal(b, m, deterministic)
-}
-func (m *ProcessBundleSplitRequest_DesiredSplit) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ProcessBundleSplitRequest_DesiredSplit.Merge(m, src)
-}
-func (m *ProcessBundleSplitRequest_DesiredSplit) XXX_Size() int {
-	return xxx_messageInfo_ProcessBundleSplitRequest_DesiredSplit.Size(m)
-}
-func (m *ProcessBundleSplitRequest_DesiredSplit) XXX_DiscardUnknown() {
-	xxx_messageInfo_ProcessBundleSplitRequest_DesiredSplit.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_ProcessBundleSplitRequest_DesiredSplit proto.InternalMessageInfo
-
-func (m *ProcessBundleSplitRequest_DesiredSplit) GetFractionOfRemainder() float64 {
-	if m != nil {
-		return m.FractionOfRemainder
-	}
-	return 0
-}
-
-func (m *ProcessBundleSplitRequest_DesiredSplit) GetAllowedSplitPoints() []int64 {
-	if m != nil {
-		return m.AllowedSplitPoints
-	}
-	return nil
-}
-
-func (m *ProcessBundleSplitRequest_DesiredSplit) GetEstimatedInputElements() int64 {
-	if m != nil {
-		return m.EstimatedInputElements
-	}
-	return 0
-}
-
 // Represents a partition of the bundle: a "primary" and
 // a "residual", with the following properties:
 // - The work in primary and residual doesn't overlap, and combined, adds up
@@ -1479,6 +1383,10 @@
 //   the work under primary_roots.
 // For more rigorous definitions see https://s.apache.org/beam-breaking-fusion
 type ProcessBundleSplitResponse struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
 	// Root applications that should replace the current bundle.
 	PrimaryRoots []*BundleApplication `protobuf:"bytes,1,rep,name=primary_roots,json=primaryRoots,proto3" json:"primary_roots,omitempty"`
 	// Root applications that have been removed from the current bundle and
@@ -1488,419 +1396,213 @@
 	// Partitions of input data channels into primary and residual elements,
 	// if any. Should not include any elements represented in the bundle
 	// applications roots above.
-	ChannelSplits        []*ProcessBundleSplitResponse_ChannelSplit `protobuf:"bytes,3,rep,name=channel_splits,json=channelSplits,proto3" json:"channel_splits,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}                                   `json:"-"`
-	XXX_unrecognized     []byte                                     `json:"-"`
-	XXX_sizecache        int32                                      `json:"-"`
+	ChannelSplits []*ProcessBundleSplitResponse_ChannelSplit `protobuf:"bytes,3,rep,name=channel_splits,json=channelSplits,proto3" json:"channel_splits,omitempty"`
 }
 
-func (m *ProcessBundleSplitResponse) Reset()         { *m = ProcessBundleSplitResponse{} }
-func (m *ProcessBundleSplitResponse) String() string { return proto.CompactTextString(m) }
-func (*ProcessBundleSplitResponse) ProtoMessage()    {}
+func (x *ProcessBundleSplitResponse) Reset() {
+	*x = ProcessBundleSplitResponse{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_fn_api_proto_msgTypes[16]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *ProcessBundleSplitResponse) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*ProcessBundleSplitResponse) ProtoMessage() {}
+
+func (x *ProcessBundleSplitResponse) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_fn_api_proto_msgTypes[16]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use ProcessBundleSplitResponse.ProtoReflect.Descriptor instead.
 func (*ProcessBundleSplitResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_6d954c03a4758710, []int{16}
+	return file_beam_fn_api_proto_rawDescGZIP(), []int{16}
 }
 
-func (m *ProcessBundleSplitResponse) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_ProcessBundleSplitResponse.Unmarshal(m, b)
-}
-func (m *ProcessBundleSplitResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_ProcessBundleSplitResponse.Marshal(b, m, deterministic)
-}
-func (m *ProcessBundleSplitResponse) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ProcessBundleSplitResponse.Merge(m, src)
-}
-func (m *ProcessBundleSplitResponse) XXX_Size() int {
-	return xxx_messageInfo_ProcessBundleSplitResponse.Size(m)
-}
-func (m *ProcessBundleSplitResponse) XXX_DiscardUnknown() {
-	xxx_messageInfo_ProcessBundleSplitResponse.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_ProcessBundleSplitResponse proto.InternalMessageInfo
-
-func (m *ProcessBundleSplitResponse) GetPrimaryRoots() []*BundleApplication {
-	if m != nil {
-		return m.PrimaryRoots
+func (x *ProcessBundleSplitResponse) GetPrimaryRoots() []*BundleApplication {
+	if x != nil {
+		return x.PrimaryRoots
 	}
 	return nil
 }
 
-func (m *ProcessBundleSplitResponse) GetResidualRoots() []*DelayedBundleApplication {
-	if m != nil {
-		return m.ResidualRoots
+func (x *ProcessBundleSplitResponse) GetResidualRoots() []*DelayedBundleApplication {
+	if x != nil {
+		return x.ResidualRoots
 	}
 	return nil
 }
 
-func (m *ProcessBundleSplitResponse) GetChannelSplits() []*ProcessBundleSplitResponse_ChannelSplit {
-	if m != nil {
-		return m.ChannelSplits
+func (x *ProcessBundleSplitResponse) GetChannelSplits() []*ProcessBundleSplitResponse_ChannelSplit {
+	if x != nil {
+		return x.ChannelSplits
 	}
 	return nil
 }
 
-// Represents contiguous portions of the data channel that are either
-// entirely processed or entirely unprocessed and belong to the primary
-// or residual respectively.
-//
-// This affords both a more efficient representation over the FnAPI
-// (if the bundle is large) and often a more efficient representation
-// on the runner side (e.g. if the set of elements can be represented
-// as some range in an underlying dataset).
-type ProcessBundleSplitResponse_ChannelSplit struct {
-	// (Required) The grpc read transform reading this channel.
-	TransformId string `protobuf:"bytes,1,opt,name=transform_id,json=transformId,proto3" json:"transform_id,omitempty"`
-	// The last element of the input channel that should be entirely considered
-	// part of the primary, identified by its absolute index in the (ordered)
-	// channel.
-	LastPrimaryElement int64 `protobuf:"varint,2,opt,name=last_primary_element,json=lastPrimaryElement,proto3" json:"last_primary_element,omitempty"`
-	// The first element of the input channel that should be entirely considered
-	// part of the residual, identified by its absolute index in the (ordered)
-	// channel.
-	FirstResidualElement int64    `protobuf:"varint,3,opt,name=first_residual_element,json=firstResidualElement,proto3" json:"first_residual_element,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *ProcessBundleSplitResponse_ChannelSplit) Reset() {
-	*m = ProcessBundleSplitResponse_ChannelSplit{}
-}
-func (m *ProcessBundleSplitResponse_ChannelSplit) String() string { return proto.CompactTextString(m) }
-func (*ProcessBundleSplitResponse_ChannelSplit) ProtoMessage()    {}
-func (*ProcessBundleSplitResponse_ChannelSplit) Descriptor() ([]byte, []int) {
-	return fileDescriptor_6d954c03a4758710, []int{16, 0}
-}
-
-func (m *ProcessBundleSplitResponse_ChannelSplit) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_ProcessBundleSplitResponse_ChannelSplit.Unmarshal(m, b)
-}
-func (m *ProcessBundleSplitResponse_ChannelSplit) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_ProcessBundleSplitResponse_ChannelSplit.Marshal(b, m, deterministic)
-}
-func (m *ProcessBundleSplitResponse_ChannelSplit) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ProcessBundleSplitResponse_ChannelSplit.Merge(m, src)
-}
-func (m *ProcessBundleSplitResponse_ChannelSplit) XXX_Size() int {
-	return xxx_messageInfo_ProcessBundleSplitResponse_ChannelSplit.Size(m)
-}
-func (m *ProcessBundleSplitResponse_ChannelSplit) XXX_DiscardUnknown() {
-	xxx_messageInfo_ProcessBundleSplitResponse_ChannelSplit.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_ProcessBundleSplitResponse_ChannelSplit proto.InternalMessageInfo
-
-func (m *ProcessBundleSplitResponse_ChannelSplit) GetTransformId() string {
-	if m != nil {
-		return m.TransformId
-	}
-	return ""
-}
-
-func (m *ProcessBundleSplitResponse_ChannelSplit) GetLastPrimaryElement() int64 {
-	if m != nil {
-		return m.LastPrimaryElement
-	}
-	return 0
-}
-
-func (m *ProcessBundleSplitResponse_ChannelSplit) GetFirstResidualElement() int64 {
-	if m != nil {
-		return m.FirstResidualElement
-	}
-	return 0
-}
-
 type FinalizeBundleRequest struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
 	// (Required) A reference to a completed process bundle request with the given
 	// instruction id.
-	InstructionId        string   `protobuf:"bytes,1,opt,name=instruction_id,json=instructionId,proto3" json:"instruction_id,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
+	InstructionId string `protobuf:"bytes,1,opt,name=instruction_id,json=instructionId,proto3" json:"instruction_id,omitempty"`
 }
 
-func (m *FinalizeBundleRequest) Reset()         { *m = FinalizeBundleRequest{} }
-func (m *FinalizeBundleRequest) String() string { return proto.CompactTextString(m) }
-func (*FinalizeBundleRequest) ProtoMessage()    {}
+func (x *FinalizeBundleRequest) Reset() {
+	*x = FinalizeBundleRequest{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_fn_api_proto_msgTypes[17]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *FinalizeBundleRequest) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*FinalizeBundleRequest) ProtoMessage() {}
+
+func (x *FinalizeBundleRequest) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_fn_api_proto_msgTypes[17]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use FinalizeBundleRequest.ProtoReflect.Descriptor instead.
 func (*FinalizeBundleRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_6d954c03a4758710, []int{17}
+	return file_beam_fn_api_proto_rawDescGZIP(), []int{17}
 }
 
-func (m *FinalizeBundleRequest) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_FinalizeBundleRequest.Unmarshal(m, b)
-}
-func (m *FinalizeBundleRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_FinalizeBundleRequest.Marshal(b, m, deterministic)
-}
-func (m *FinalizeBundleRequest) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_FinalizeBundleRequest.Merge(m, src)
-}
-func (m *FinalizeBundleRequest) XXX_Size() int {
-	return xxx_messageInfo_FinalizeBundleRequest.Size(m)
-}
-func (m *FinalizeBundleRequest) XXX_DiscardUnknown() {
-	xxx_messageInfo_FinalizeBundleRequest.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_FinalizeBundleRequest proto.InternalMessageInfo
-
-func (m *FinalizeBundleRequest) GetInstructionId() string {
-	if m != nil {
-		return m.InstructionId
+func (x *FinalizeBundleRequest) GetInstructionId() string {
+	if x != nil {
+		return x.InstructionId
 	}
 	return ""
 }
 
 type FinalizeBundleResponse struct {
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
 }
 
-func (m *FinalizeBundleResponse) Reset()         { *m = FinalizeBundleResponse{} }
-func (m *FinalizeBundleResponse) String() string { return proto.CompactTextString(m) }
-func (*FinalizeBundleResponse) ProtoMessage()    {}
+func (x *FinalizeBundleResponse) Reset() {
+	*x = FinalizeBundleResponse{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_fn_api_proto_msgTypes[18]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *FinalizeBundleResponse) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*FinalizeBundleResponse) ProtoMessage() {}
+
+func (x *FinalizeBundleResponse) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_fn_api_proto_msgTypes[18]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use FinalizeBundleResponse.ProtoReflect.Descriptor instead.
 func (*FinalizeBundleResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_6d954c03a4758710, []int{18}
+	return file_beam_fn_api_proto_rawDescGZIP(), []int{18}
 }
 
-func (m *FinalizeBundleResponse) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_FinalizeBundleResponse.Unmarshal(m, b)
-}
-func (m *FinalizeBundleResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_FinalizeBundleResponse.Marshal(b, m, deterministic)
-}
-func (m *FinalizeBundleResponse) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_FinalizeBundleResponse.Merge(m, src)
-}
-func (m *FinalizeBundleResponse) XXX_Size() int {
-	return xxx_messageInfo_FinalizeBundleResponse.Size(m)
-}
-func (m *FinalizeBundleResponse) XXX_DiscardUnknown() {
-	xxx_messageInfo_FinalizeBundleResponse.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_FinalizeBundleResponse proto.InternalMessageInfo
-
 // Messages used to represent logical byte streams.
 // Stable
 type Elements struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
 	// (Optional) A list containing parts of logical byte streams.
 	Data []*Elements_Data `protobuf:"bytes,1,rep,name=data,proto3" json:"data,omitempty"`
 	// (Optional)  A list of timer byte streams.
-	Timers               []*Elements_Timers `protobuf:"bytes,2,rep,name=timers,proto3" json:"timers,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}           `json:"-"`
-	XXX_unrecognized     []byte             `json:"-"`
-	XXX_sizecache        int32              `json:"-"`
+	Timers []*Elements_Timers `protobuf:"bytes,2,rep,name=timers,proto3" json:"timers,omitempty"`
 }
 
-func (m *Elements) Reset()         { *m = Elements{} }
-func (m *Elements) String() string { return proto.CompactTextString(m) }
-func (*Elements) ProtoMessage()    {}
+func (x *Elements) Reset() {
+	*x = Elements{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_fn_api_proto_msgTypes[19]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *Elements) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*Elements) ProtoMessage() {}
+
+func (x *Elements) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_fn_api_proto_msgTypes[19]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use Elements.ProtoReflect.Descriptor instead.
 func (*Elements) Descriptor() ([]byte, []int) {
-	return fileDescriptor_6d954c03a4758710, []int{19}
+	return file_beam_fn_api_proto_rawDescGZIP(), []int{19}
 }
 
-func (m *Elements) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_Elements.Unmarshal(m, b)
-}
-func (m *Elements) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_Elements.Marshal(b, m, deterministic)
-}
-func (m *Elements) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Elements.Merge(m, src)
-}
-func (m *Elements) XXX_Size() int {
-	return xxx_messageInfo_Elements.Size(m)
-}
-func (m *Elements) XXX_DiscardUnknown() {
-	xxx_messageInfo_Elements.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_Elements proto.InternalMessageInfo
-
-func (m *Elements) GetData() []*Elements_Data {
-	if m != nil {
-		return m.Data
+func (x *Elements) GetData() []*Elements_Data {
+	if x != nil {
+		return x.Data
 	}
 	return nil
 }
 
-func (m *Elements) GetTimers() []*Elements_Timers {
-	if m != nil {
-		return m.Timers
+func (x *Elements) GetTimers() []*Elements_Timers {
+	if x != nil {
+		return x.Timers
 	}
 	return nil
 }
 
-// Represents multiple encoded elements in nested context for a given named
-// instruction and transform.
-type Elements_Data struct {
-	// (Required) A reference to an active instruction request with the given
-	// instruction id.
-	InstructionId string `protobuf:"bytes,1,opt,name=instruction_id,json=instructionId,proto3" json:"instruction_id,omitempty"`
-	// (Required) A definition representing a consumer or producer of this data.
-	// If received by a harness, this represents the consumer within that
-	// harness that should consume these bytes. If sent by a harness, this
-	// represents the producer of these bytes.
-	//
-	// Note that a single element may span multiple Data messages.
-	//
-	// Note that a sending/receiving pair should share the same identifier.
-	TransformId string `protobuf:"bytes,2,opt,name=transform_id,json=transformId,proto3" json:"transform_id,omitempty"`
-	// (Optional) Represents a part of a logical byte stream. Elements within
-	// the logical byte stream are encoded in the nested context and
-	// concatenated together.
-	Data []byte `protobuf:"bytes,3,opt,name=data,proto3" json:"data,omitempty"`
-	// (Optional) Set this bit to indicate the this is the last data block
-	// for the given instruction and transform, ending the stream.
-	IsLast               bool     `protobuf:"varint,4,opt,name=is_last,json=isLast,proto3" json:"is_last,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *Elements_Data) Reset()         { *m = Elements_Data{} }
-func (m *Elements_Data) String() string { return proto.CompactTextString(m) }
-func (*Elements_Data) ProtoMessage()    {}
-func (*Elements_Data) Descriptor() ([]byte, []int) {
-	return fileDescriptor_6d954c03a4758710, []int{19, 0}
-}
-
-func (m *Elements_Data) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_Elements_Data.Unmarshal(m, b)
-}
-func (m *Elements_Data) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_Elements_Data.Marshal(b, m, deterministic)
-}
-func (m *Elements_Data) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Elements_Data.Merge(m, src)
-}
-func (m *Elements_Data) XXX_Size() int {
-	return xxx_messageInfo_Elements_Data.Size(m)
-}
-func (m *Elements_Data) XXX_DiscardUnknown() {
-	xxx_messageInfo_Elements_Data.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_Elements_Data proto.InternalMessageInfo
-
-func (m *Elements_Data) GetInstructionId() string {
-	if m != nil {
-		return m.InstructionId
-	}
-	return ""
-}
-
-func (m *Elements_Data) GetTransformId() string {
-	if m != nil {
-		return m.TransformId
-	}
-	return ""
-}
-
-func (m *Elements_Data) GetData() []byte {
-	if m != nil {
-		return m.Data
-	}
-	return nil
-}
-
-func (m *Elements_Data) GetIsLast() bool {
-	if m != nil {
-		return m.IsLast
-	}
-	return false
-}
-
-// Represent the encoded user timer for a given instruction, transform and
-// timer id.
-type Elements_Timers struct {
-	// (Required) A reference to an active instruction request with the given
-	// instruction id.
-	InstructionId string `protobuf:"bytes,1,opt,name=instruction_id,json=instructionId,proto3" json:"instruction_id,omitempty"`
-	// (Required) A definition representing a consumer or producer of this data.
-	// If received by a harness, this represents the consumer within that
-	// harness that should consume these timers. If sent by a harness, this
-	// represents the producer of these timers.
-	TransformId string `protobuf:"bytes,2,opt,name=transform_id,json=transformId,proto3" json:"transform_id,omitempty"`
-	// (Required) The local timer family name used to identify the associated
-	// timer family specification
-	TimerFamilyId string `protobuf:"bytes,3,opt,name=timer_family_id,json=timerFamilyId,proto3" json:"timer_family_id,omitempty"`
-	// (Optional) Represents a logical byte stream of timers. Encoded according
-	// to the coder in the timer spec.
-	Timers []byte `protobuf:"bytes,4,opt,name=timers,proto3" json:"timers,omitempty"`
-	// (Optional) Set this bit to indicate the this is the last data block
-	// for the given instruction and transform, ending the stream.
-	IsLast               bool     `protobuf:"varint,5,opt,name=is_last,json=isLast,proto3" json:"is_last,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *Elements_Timers) Reset()         { *m = Elements_Timers{} }
-func (m *Elements_Timers) String() string { return proto.CompactTextString(m) }
-func (*Elements_Timers) ProtoMessage()    {}
-func (*Elements_Timers) Descriptor() ([]byte, []int) {
-	return fileDescriptor_6d954c03a4758710, []int{19, 1}
-}
-
-func (m *Elements_Timers) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_Elements_Timers.Unmarshal(m, b)
-}
-func (m *Elements_Timers) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_Elements_Timers.Marshal(b, m, deterministic)
-}
-func (m *Elements_Timers) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Elements_Timers.Merge(m, src)
-}
-func (m *Elements_Timers) XXX_Size() int {
-	return xxx_messageInfo_Elements_Timers.Size(m)
-}
-func (m *Elements_Timers) XXX_DiscardUnknown() {
-	xxx_messageInfo_Elements_Timers.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_Elements_Timers proto.InternalMessageInfo
-
-func (m *Elements_Timers) GetInstructionId() string {
-	if m != nil {
-		return m.InstructionId
-	}
-	return ""
-}
-
-func (m *Elements_Timers) GetTransformId() string {
-	if m != nil {
-		return m.TransformId
-	}
-	return ""
-}
-
-func (m *Elements_Timers) GetTimerFamilyId() string {
-	if m != nil {
-		return m.TimerFamilyId
-	}
-	return ""
-}
-
-func (m *Elements_Timers) GetTimers() []byte {
-	if m != nil {
-		return m.Timers
-	}
-	return nil
-}
-
-func (m *Elements_Timers) GetIsLast() bool {
-	if m != nil {
-		return m.IsLast
-	}
-	return false
-}
-
 type StateRequest struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
 	// (Required) A unique identifier provided by the SDK which represents this
 	// requests execution. The StateResponse corresponding with this request
 	// will have the matching id.
@@ -1913,58 +1615,90 @@
 	StateKey *StateKey `protobuf:"bytes,3,opt,name=state_key,json=stateKey,proto3" json:"state_key,omitempty"`
 	// (Required) The action to take on this request.
 	//
-	// Types that are valid to be assigned to Request:
+	// Types that are assignable to Request:
 	//	*StateRequest_Get
 	//	*StateRequest_Append
 	//	*StateRequest_Clear
-	Request              isStateRequest_Request `protobuf_oneof:"request"`
-	XXX_NoUnkeyedLiteral struct{}               `json:"-"`
-	XXX_unrecognized     []byte                 `json:"-"`
-	XXX_sizecache        int32                  `json:"-"`
+	Request isStateRequest_Request `protobuf_oneof:"request"`
 }
 
-func (m *StateRequest) Reset()         { *m = StateRequest{} }
-func (m *StateRequest) String() string { return proto.CompactTextString(m) }
-func (*StateRequest) ProtoMessage()    {}
+func (x *StateRequest) Reset() {
+	*x = StateRequest{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_fn_api_proto_msgTypes[20]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *StateRequest) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*StateRequest) ProtoMessage() {}
+
+func (x *StateRequest) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_fn_api_proto_msgTypes[20]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use StateRequest.ProtoReflect.Descriptor instead.
 func (*StateRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_6d954c03a4758710, []int{20}
+	return file_beam_fn_api_proto_rawDescGZIP(), []int{20}
 }
 
-func (m *StateRequest) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_StateRequest.Unmarshal(m, b)
-}
-func (m *StateRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_StateRequest.Marshal(b, m, deterministic)
-}
-func (m *StateRequest) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_StateRequest.Merge(m, src)
-}
-func (m *StateRequest) XXX_Size() int {
-	return xxx_messageInfo_StateRequest.Size(m)
-}
-func (m *StateRequest) XXX_DiscardUnknown() {
-	xxx_messageInfo_StateRequest.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_StateRequest proto.InternalMessageInfo
-
-func (m *StateRequest) GetId() string {
-	if m != nil {
-		return m.Id
+func (x *StateRequest) GetId() string {
+	if x != nil {
+		return x.Id
 	}
 	return ""
 }
 
-func (m *StateRequest) GetInstructionId() string {
-	if m != nil {
-		return m.InstructionId
+func (x *StateRequest) GetInstructionId() string {
+	if x != nil {
+		return x.InstructionId
 	}
 	return ""
 }
 
-func (m *StateRequest) GetStateKey() *StateKey {
+func (x *StateRequest) GetStateKey() *StateKey {
+	if x != nil {
+		return x.StateKey
+	}
+	return nil
+}
+
+func (m *StateRequest) GetRequest() isStateRequest_Request {
 	if m != nil {
-		return m.StateKey
+		return m.Request
+	}
+	return nil
+}
+
+func (x *StateRequest) GetGet() *StateGetRequest {
+	if x, ok := x.GetRequest().(*StateRequest_Get); ok {
+		return x.Get
+	}
+	return nil
+}
+
+func (x *StateRequest) GetAppend() *StateAppendRequest {
+	if x, ok := x.GetRequest().(*StateRequest_Append); ok {
+		return x.Append
+	}
+	return nil
+}
+
+func (x *StateRequest) GetClear() *StateClearRequest {
+	if x, ok := x.GetRequest().(*StateRequest_Clear); ok {
+		return x.Clear
 	}
 	return nil
 }
@@ -1974,14 +1708,17 @@
 }
 
 type StateRequest_Get struct {
+	// A request to get state.
 	Get *StateGetRequest `protobuf:"bytes,1000,opt,name=get,proto3,oneof"`
 }
 
 type StateRequest_Append struct {
+	// A request to append to state.
 	Append *StateAppendRequest `protobuf:"bytes,1001,opt,name=append,proto3,oneof"`
 }
 
 type StateRequest_Clear struct {
+	// A request to clear state.
 	Clear *StateClearRequest `protobuf:"bytes,1002,opt,name=clear,proto3,oneof"`
 }
 
@@ -1991,44 +1728,11 @@
 
 func (*StateRequest_Clear) isStateRequest_Request() {}
 
-func (m *StateRequest) GetRequest() isStateRequest_Request {
-	if m != nil {
-		return m.Request
-	}
-	return nil
-}
-
-func (m *StateRequest) GetGet() *StateGetRequest {
-	if x, ok := m.GetRequest().(*StateRequest_Get); ok {
-		return x.Get
-	}
-	return nil
-}
-
-func (m *StateRequest) GetAppend() *StateAppendRequest {
-	if x, ok := m.GetRequest().(*StateRequest_Append); ok {
-		return x.Append
-	}
-	return nil
-}
-
-func (m *StateRequest) GetClear() *StateClearRequest {
-	if x, ok := m.GetRequest().(*StateRequest_Clear); ok {
-		return x.Clear
-	}
-	return nil
-}
-
-// XXX_OneofWrappers is for the internal use of the proto package.
-func (*StateRequest) XXX_OneofWrappers() []interface{} {
-	return []interface{}{
-		(*StateRequest_Get)(nil),
-		(*StateRequest_Append)(nil),
-		(*StateRequest_Clear)(nil),
-	}
-}
-
 type StateResponse struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
 	// (Required) A reference provided by the SDK which represents a requests
 	// execution. The StateResponse must have the matching id when responding
 	// to the SDK.
@@ -2039,68 +1743,103 @@
 	Error string `protobuf:"bytes,2,opt,name=error,proto3" json:"error,omitempty"`
 	// A corresponding response matching the request will be populated.
 	//
-	// Types that are valid to be assigned to Response:
+	// Types that are assignable to Response:
 	//	*StateResponse_Get
 	//	*StateResponse_Append
 	//	*StateResponse_Clear
-	Response             isStateResponse_Response `protobuf_oneof:"response"`
-	XXX_NoUnkeyedLiteral struct{}                 `json:"-"`
-	XXX_unrecognized     []byte                   `json:"-"`
-	XXX_sizecache        int32                    `json:"-"`
+	Response isStateResponse_Response `protobuf_oneof:"response"`
 }
 
-func (m *StateResponse) Reset()         { *m = StateResponse{} }
-func (m *StateResponse) String() string { return proto.CompactTextString(m) }
-func (*StateResponse) ProtoMessage()    {}
+func (x *StateResponse) Reset() {
+	*x = StateResponse{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_fn_api_proto_msgTypes[21]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *StateResponse) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*StateResponse) ProtoMessage() {}
+
+func (x *StateResponse) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_fn_api_proto_msgTypes[21]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use StateResponse.ProtoReflect.Descriptor instead.
 func (*StateResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_6d954c03a4758710, []int{21}
+	return file_beam_fn_api_proto_rawDescGZIP(), []int{21}
 }
 
-func (m *StateResponse) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_StateResponse.Unmarshal(m, b)
-}
-func (m *StateResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_StateResponse.Marshal(b, m, deterministic)
-}
-func (m *StateResponse) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_StateResponse.Merge(m, src)
-}
-func (m *StateResponse) XXX_Size() int {
-	return xxx_messageInfo_StateResponse.Size(m)
-}
-func (m *StateResponse) XXX_DiscardUnknown() {
-	xxx_messageInfo_StateResponse.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_StateResponse proto.InternalMessageInfo
-
-func (m *StateResponse) GetId() string {
-	if m != nil {
-		return m.Id
+func (x *StateResponse) GetId() string {
+	if x != nil {
+		return x.Id
 	}
 	return ""
 }
 
-func (m *StateResponse) GetError() string {
-	if m != nil {
-		return m.Error
+func (x *StateResponse) GetError() string {
+	if x != nil {
+		return x.Error
 	}
 	return ""
 }
 
+func (m *StateResponse) GetResponse() isStateResponse_Response {
+	if m != nil {
+		return m.Response
+	}
+	return nil
+}
+
+func (x *StateResponse) GetGet() *StateGetResponse {
+	if x, ok := x.GetResponse().(*StateResponse_Get); ok {
+		return x.Get
+	}
+	return nil
+}
+
+func (x *StateResponse) GetAppend() *StateAppendResponse {
+	if x, ok := x.GetResponse().(*StateResponse_Append); ok {
+		return x.Append
+	}
+	return nil
+}
+
+func (x *StateResponse) GetClear() *StateClearResponse {
+	if x, ok := x.GetResponse().(*StateResponse_Clear); ok {
+		return x.Clear
+	}
+	return nil
+}
+
 type isStateResponse_Response interface {
 	isStateResponse_Response()
 }
 
 type StateResponse_Get struct {
+	// A response to getting state.
 	Get *StateGetResponse `protobuf:"bytes,1000,opt,name=get,proto3,oneof"`
 }
 
 type StateResponse_Append struct {
+	// A response to appending to state.
 	Append *StateAppendResponse `protobuf:"bytes,1001,opt,name=append,proto3,oneof"`
 }
 
 type StateResponse_Clear struct {
+	// A response to clearing state.
 	Clear *StateClearResponse `protobuf:"bytes,1002,opt,name=clear,proto3,oneof"`
 }
 
@@ -2110,82 +1849,95 @@
 
 func (*StateResponse_Clear) isStateResponse_Response() {}
 
-func (m *StateResponse) GetResponse() isStateResponse_Response {
-	if m != nil {
-		return m.Response
-	}
-	return nil
-}
-
-func (m *StateResponse) GetGet() *StateGetResponse {
-	if x, ok := m.GetResponse().(*StateResponse_Get); ok {
-		return x.Get
-	}
-	return nil
-}
-
-func (m *StateResponse) GetAppend() *StateAppendResponse {
-	if x, ok := m.GetResponse().(*StateResponse_Append); ok {
-		return x.Append
-	}
-	return nil
-}
-
-func (m *StateResponse) GetClear() *StateClearResponse {
-	if x, ok := m.GetResponse().(*StateResponse_Clear); ok {
-		return x.Clear
-	}
-	return nil
-}
-
-// XXX_OneofWrappers is for the internal use of the proto package.
-func (*StateResponse) XXX_OneofWrappers() []interface{} {
-	return []interface{}{
-		(*StateResponse_Get)(nil),
-		(*StateResponse_Append)(nil),
-		(*StateResponse_Clear)(nil),
-	}
-}
-
 type StateKey struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
 	// (Required) One of the following state keys must be set.
 	//
-	// Types that are valid to be assigned to Type:
+	// Types that are assignable to Type:
 	//	*StateKey_Runner_
 	//	*StateKey_MultimapSideInput_
 	//	*StateKey_BagUserState_
 	//	*StateKey_IterableSideInput_
 	//	*StateKey_MultimapKeysSideInput_
-	Type                 isStateKey_Type `protobuf_oneof:"type"`
-	XXX_NoUnkeyedLiteral struct{}        `json:"-"`
-	XXX_unrecognized     []byte          `json:"-"`
-	XXX_sizecache        int32           `json:"-"`
+	Type isStateKey_Type `protobuf_oneof:"type"`
 }
 
-func (m *StateKey) Reset()         { *m = StateKey{} }
-func (m *StateKey) String() string { return proto.CompactTextString(m) }
-func (*StateKey) ProtoMessage()    {}
+func (x *StateKey) Reset() {
+	*x = StateKey{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_fn_api_proto_msgTypes[22]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *StateKey) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*StateKey) ProtoMessage() {}
+
+func (x *StateKey) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_fn_api_proto_msgTypes[22]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use StateKey.ProtoReflect.Descriptor instead.
 func (*StateKey) Descriptor() ([]byte, []int) {
-	return fileDescriptor_6d954c03a4758710, []int{22}
+	return file_beam_fn_api_proto_rawDescGZIP(), []int{22}
 }
 
-func (m *StateKey) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_StateKey.Unmarshal(m, b)
-}
-func (m *StateKey) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_StateKey.Marshal(b, m, deterministic)
-}
-func (m *StateKey) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_StateKey.Merge(m, src)
-}
-func (m *StateKey) XXX_Size() int {
-	return xxx_messageInfo_StateKey.Size(m)
-}
-func (m *StateKey) XXX_DiscardUnknown() {
-	xxx_messageInfo_StateKey.DiscardUnknown(m)
+func (m *StateKey) GetType() isStateKey_Type {
+	if m != nil {
+		return m.Type
+	}
+	return nil
 }
 
-var xxx_messageInfo_StateKey proto.InternalMessageInfo
+func (x *StateKey) GetRunner() *StateKey_Runner {
+	if x, ok := x.GetType().(*StateKey_Runner_); ok {
+		return x.Runner
+	}
+	return nil
+}
+
+func (x *StateKey) GetMultimapSideInput() *StateKey_MultimapSideInput {
+	if x, ok := x.GetType().(*StateKey_MultimapSideInput_); ok {
+		return x.MultimapSideInput
+	}
+	return nil
+}
+
+func (x *StateKey) GetBagUserState() *StateKey_BagUserState {
+	if x, ok := x.GetType().(*StateKey_BagUserState_); ok {
+		return x.BagUserState
+	}
+	return nil
+}
+
+func (x *StateKey) GetIterableSideInput() *StateKey_IterableSideInput {
+	if x, ok := x.GetType().(*StateKey_IterableSideInput_); ok {
+		return x.IterableSideInput
+	}
+	return nil
+}
+
+func (x *StateKey) GetMultimapKeysSideInput() *StateKey_MultimapKeysSideInput {
+	if x, ok := x.GetType().(*StateKey_MultimapKeysSideInput_); ok {
+		return x.MultimapKeysSideInput
+	}
+	return nil
+}
 
 type isStateKey_Type interface {
 	isStateKey_Type()
@@ -2208,7 +1960,7 @@
 }
 
 type StateKey_MultimapKeysSideInput_ struct {
-	MultimapKeysSideInput *StateKey_MultimapKeysSideInput `protobuf:"bytes,5,opt,name=multimap_keys_side_input,json=multimapKeysSideInput,proto3,oneof"`
+	MultimapKeysSideInput *StateKey_MultimapKeysSideInput `protobuf:"bytes,5,opt,name=multimap_keys_side_input,json=multimapKeysSideInput,proto3,oneof"` // TODO: represent a state key for user map state
 }
 
 func (*StateKey_Runner_) isStateKey_Type() {}
@@ -2221,434 +1973,55 @@
 
 func (*StateKey_MultimapKeysSideInput_) isStateKey_Type() {}
 
-func (m *StateKey) GetType() isStateKey_Type {
-	if m != nil {
-		return m.Type
-	}
-	return nil
-}
-
-func (m *StateKey) GetRunner() *StateKey_Runner {
-	if x, ok := m.GetType().(*StateKey_Runner_); ok {
-		return x.Runner
-	}
-	return nil
-}
-
-func (m *StateKey) GetMultimapSideInput() *StateKey_MultimapSideInput {
-	if x, ok := m.GetType().(*StateKey_MultimapSideInput_); ok {
-		return x.MultimapSideInput
-	}
-	return nil
-}
-
-func (m *StateKey) GetBagUserState() *StateKey_BagUserState {
-	if x, ok := m.GetType().(*StateKey_BagUserState_); ok {
-		return x.BagUserState
-	}
-	return nil
-}
-
-func (m *StateKey) GetIterableSideInput() *StateKey_IterableSideInput {
-	if x, ok := m.GetType().(*StateKey_IterableSideInput_); ok {
-		return x.IterableSideInput
-	}
-	return nil
-}
-
-func (m *StateKey) GetMultimapKeysSideInput() *StateKey_MultimapKeysSideInput {
-	if x, ok := m.GetType().(*StateKey_MultimapKeysSideInput_); ok {
-		return x.MultimapKeysSideInput
-	}
-	return nil
-}
-
-// XXX_OneofWrappers is for the internal use of the proto package.
-func (*StateKey) XXX_OneofWrappers() []interface{} {
-	return []interface{}{
-		(*StateKey_Runner_)(nil),
-		(*StateKey_MultimapSideInput_)(nil),
-		(*StateKey_BagUserState_)(nil),
-		(*StateKey_IterableSideInput_)(nil),
-		(*StateKey_MultimapKeysSideInput_)(nil),
-	}
-}
-
-type StateKey_Runner struct {
-	// (Required) Opaque information supplied by the runner. Used to support
-	// remote references.
-	// https://s.apache.org/beam-fn-api-send-and-receive-data
-	//
-	// Used by state backed iterable. And in this use case, request type can
-	// only be of type get. Details see:
-	// https://s.apache.org/beam-fn-api-state-backed-iterables
-	Key                  []byte   `protobuf:"bytes,1,opt,name=key,proto3" json:"key,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *StateKey_Runner) Reset()         { *m = StateKey_Runner{} }
-func (m *StateKey_Runner) String() string { return proto.CompactTextString(m) }
-func (*StateKey_Runner) ProtoMessage()    {}
-func (*StateKey_Runner) Descriptor() ([]byte, []int) {
-	return fileDescriptor_6d954c03a4758710, []int{22, 0}
-}
-
-func (m *StateKey_Runner) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_StateKey_Runner.Unmarshal(m, b)
-}
-func (m *StateKey_Runner) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_StateKey_Runner.Marshal(b, m, deterministic)
-}
-func (m *StateKey_Runner) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_StateKey_Runner.Merge(m, src)
-}
-func (m *StateKey_Runner) XXX_Size() int {
-	return xxx_messageInfo_StateKey_Runner.Size(m)
-}
-func (m *StateKey_Runner) XXX_DiscardUnknown() {
-	xxx_messageInfo_StateKey_Runner.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_StateKey_Runner proto.InternalMessageInfo
-
-func (m *StateKey_Runner) GetKey() []byte {
-	if m != nil {
-		return m.Key
-	}
-	return nil
-}
-
-// Represents a request for the values associated with a specified window
-// in a PCollection. See
-// https://s.apache.org/beam-fn-state-api-and-bundle-processing for further
-// details.
-//
-// Can only be used to perform StateGetRequests on side inputs of the URN
-// beam:side_input:iterable:v1.
-//
-// For a PCollection<V>, the response data stream will be a concatenation
-// of all V's. See https://s.apache.org/beam-fn-api-send-and-receive-data
-// for further details.
-type StateKey_IterableSideInput struct {
-	// (Required) The id of the PTransform containing a side input.
-	TransformId string `protobuf:"bytes,1,opt,name=transform_id,json=transformId,proto3" json:"transform_id,omitempty"`
-	// (Required) The id of the side input.
-	SideInputId string `protobuf:"bytes,2,opt,name=side_input_id,json=sideInputId,proto3" json:"side_input_id,omitempty"`
-	// (Required) The window (after mapping the currently executing elements
-	// window into the side input windows domain) encoded in a nested context.
-	Window               []byte   `protobuf:"bytes,3,opt,name=window,proto3" json:"window,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *StateKey_IterableSideInput) Reset()         { *m = StateKey_IterableSideInput{} }
-func (m *StateKey_IterableSideInput) String() string { return proto.CompactTextString(m) }
-func (*StateKey_IterableSideInput) ProtoMessage()    {}
-func (*StateKey_IterableSideInput) Descriptor() ([]byte, []int) {
-	return fileDescriptor_6d954c03a4758710, []int{22, 1}
-}
-
-func (m *StateKey_IterableSideInput) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_StateKey_IterableSideInput.Unmarshal(m, b)
-}
-func (m *StateKey_IterableSideInput) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_StateKey_IterableSideInput.Marshal(b, m, deterministic)
-}
-func (m *StateKey_IterableSideInput) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_StateKey_IterableSideInput.Merge(m, src)
-}
-func (m *StateKey_IterableSideInput) XXX_Size() int {
-	return xxx_messageInfo_StateKey_IterableSideInput.Size(m)
-}
-func (m *StateKey_IterableSideInput) XXX_DiscardUnknown() {
-	xxx_messageInfo_StateKey_IterableSideInput.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_StateKey_IterableSideInput proto.InternalMessageInfo
-
-func (m *StateKey_IterableSideInput) GetTransformId() string {
-	if m != nil {
-		return m.TransformId
-	}
-	return ""
-}
-
-func (m *StateKey_IterableSideInput) GetSideInputId() string {
-	if m != nil {
-		return m.SideInputId
-	}
-	return ""
-}
-
-func (m *StateKey_IterableSideInput) GetWindow() []byte {
-	if m != nil {
-		return m.Window
-	}
-	return nil
-}
-
-// Represents a request for the values associated with a specified user key
-// and window in a PCollection. See
-// https://s.apache.org/beam-fn-state-api-and-bundle-processing for further
-// details.
-//
-// Can only be used to perform StateGetRequests on side inputs of the URN
-// beam:side_input:multimap:v1.
-//
-// For a PCollection<KV<K, V>>, the response data stream will be a
-// concatenation of all V's associated with the specified key K. See
-// https://s.apache.org/beam-fn-api-send-and-receive-data for further
-// details.
-type StateKey_MultimapSideInput struct {
-	// (Required) The id of the PTransform containing a side input.
-	TransformId string `protobuf:"bytes,1,opt,name=transform_id,json=transformId,proto3" json:"transform_id,omitempty"`
-	// (Required) The id of the side input.
-	SideInputId string `protobuf:"bytes,2,opt,name=side_input_id,json=sideInputId,proto3" json:"side_input_id,omitempty"`
-	// (Required) The window (after mapping the currently executing elements
-	// window into the side input windows domain) encoded in a nested context.
-	Window []byte `protobuf:"bytes,3,opt,name=window,proto3" json:"window,omitempty"`
-	// (Required) The key encoded in a nested context.
-	Key                  []byte   `protobuf:"bytes,4,opt,name=key,proto3" json:"key,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *StateKey_MultimapSideInput) Reset()         { *m = StateKey_MultimapSideInput{} }
-func (m *StateKey_MultimapSideInput) String() string { return proto.CompactTextString(m) }
-func (*StateKey_MultimapSideInput) ProtoMessage()    {}
-func (*StateKey_MultimapSideInput) Descriptor() ([]byte, []int) {
-	return fileDescriptor_6d954c03a4758710, []int{22, 2}
-}
-
-func (m *StateKey_MultimapSideInput) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_StateKey_MultimapSideInput.Unmarshal(m, b)
-}
-func (m *StateKey_MultimapSideInput) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_StateKey_MultimapSideInput.Marshal(b, m, deterministic)
-}
-func (m *StateKey_MultimapSideInput) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_StateKey_MultimapSideInput.Merge(m, src)
-}
-func (m *StateKey_MultimapSideInput) XXX_Size() int {
-	return xxx_messageInfo_StateKey_MultimapSideInput.Size(m)
-}
-func (m *StateKey_MultimapSideInput) XXX_DiscardUnknown() {
-	xxx_messageInfo_StateKey_MultimapSideInput.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_StateKey_MultimapSideInput proto.InternalMessageInfo
-
-func (m *StateKey_MultimapSideInput) GetTransformId() string {
-	if m != nil {
-		return m.TransformId
-	}
-	return ""
-}
-
-func (m *StateKey_MultimapSideInput) GetSideInputId() string {
-	if m != nil {
-		return m.SideInputId
-	}
-	return ""
-}
-
-func (m *StateKey_MultimapSideInput) GetWindow() []byte {
-	if m != nil {
-		return m.Window
-	}
-	return nil
-}
-
-func (m *StateKey_MultimapSideInput) GetKey() []byte {
-	if m != nil {
-		return m.Key
-	}
-	return nil
-}
-
-// Represents a request for the keys associated with a specified window in a PCollection. See
-// https://s.apache.org/beam-fn-state-api-and-bundle-processing for further
-// details.
-//
-// Can only be used to perform StateGetRequests on side inputs of the URN
-// beam:side_input:multimap:v1.
-//
-// For a PCollection<KV<K, V>>, the response data stream will be a
-// concatenation of all K's associated with the specified window. See
-// https://s.apache.org/beam-fn-api-send-and-receive-data for further
-// details.
-type StateKey_MultimapKeysSideInput struct {
-	// (Required) The id of the PTransform containing a side input.
-	TransformId string `protobuf:"bytes,1,opt,name=transform_id,json=transformId,proto3" json:"transform_id,omitempty"`
-	// (Required) The id of the side input.
-	SideInputId string `protobuf:"bytes,2,opt,name=side_input_id,json=sideInputId,proto3" json:"side_input_id,omitempty"`
-	// (Required) The window (after mapping the currently executing elements
-	// window into the side input windows domain) encoded in a nested context.
-	Window               []byte   `protobuf:"bytes,3,opt,name=window,proto3" json:"window,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *StateKey_MultimapKeysSideInput) Reset()         { *m = StateKey_MultimapKeysSideInput{} }
-func (m *StateKey_MultimapKeysSideInput) String() string { return proto.CompactTextString(m) }
-func (*StateKey_MultimapKeysSideInput) ProtoMessage()    {}
-func (*StateKey_MultimapKeysSideInput) Descriptor() ([]byte, []int) {
-	return fileDescriptor_6d954c03a4758710, []int{22, 3}
-}
-
-func (m *StateKey_MultimapKeysSideInput) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_StateKey_MultimapKeysSideInput.Unmarshal(m, b)
-}
-func (m *StateKey_MultimapKeysSideInput) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_StateKey_MultimapKeysSideInput.Marshal(b, m, deterministic)
-}
-func (m *StateKey_MultimapKeysSideInput) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_StateKey_MultimapKeysSideInput.Merge(m, src)
-}
-func (m *StateKey_MultimapKeysSideInput) XXX_Size() int {
-	return xxx_messageInfo_StateKey_MultimapKeysSideInput.Size(m)
-}
-func (m *StateKey_MultimapKeysSideInput) XXX_DiscardUnknown() {
-	xxx_messageInfo_StateKey_MultimapKeysSideInput.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_StateKey_MultimapKeysSideInput proto.InternalMessageInfo
-
-func (m *StateKey_MultimapKeysSideInput) GetTransformId() string {
-	if m != nil {
-		return m.TransformId
-	}
-	return ""
-}
-
-func (m *StateKey_MultimapKeysSideInput) GetSideInputId() string {
-	if m != nil {
-		return m.SideInputId
-	}
-	return ""
-}
-
-func (m *StateKey_MultimapKeysSideInput) GetWindow() []byte {
-	if m != nil {
-		return m.Window
-	}
-	return nil
-}
-
-type StateKey_BagUserState struct {
-	// (Required) The id of the PTransform containing user state.
-	TransformId string `protobuf:"bytes,1,opt,name=transform_id,json=transformId,proto3" json:"transform_id,omitempty"`
-	// (Required) The id of the user state.
-	UserStateId string `protobuf:"bytes,2,opt,name=user_state_id,json=userStateId,proto3" json:"user_state_id,omitempty"`
-	// (Required) The window encoded in a nested context.
-	Window []byte `protobuf:"bytes,3,opt,name=window,proto3" json:"window,omitempty"`
-	// (Required) The key of the currently executing element encoded in a
-	// nested context.
-	Key                  []byte   `protobuf:"bytes,4,opt,name=key,proto3" json:"key,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *StateKey_BagUserState) Reset()         { *m = StateKey_BagUserState{} }
-func (m *StateKey_BagUserState) String() string { return proto.CompactTextString(m) }
-func (*StateKey_BagUserState) ProtoMessage()    {}
-func (*StateKey_BagUserState) Descriptor() ([]byte, []int) {
-	return fileDescriptor_6d954c03a4758710, []int{22, 4}
-}
-
-func (m *StateKey_BagUserState) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_StateKey_BagUserState.Unmarshal(m, b)
-}
-func (m *StateKey_BagUserState) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_StateKey_BagUserState.Marshal(b, m, deterministic)
-}
-func (m *StateKey_BagUserState) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_StateKey_BagUserState.Merge(m, src)
-}
-func (m *StateKey_BagUserState) XXX_Size() int {
-	return xxx_messageInfo_StateKey_BagUserState.Size(m)
-}
-func (m *StateKey_BagUserState) XXX_DiscardUnknown() {
-	xxx_messageInfo_StateKey_BagUserState.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_StateKey_BagUserState proto.InternalMessageInfo
-
-func (m *StateKey_BagUserState) GetTransformId() string {
-	if m != nil {
-		return m.TransformId
-	}
-	return ""
-}
-
-func (m *StateKey_BagUserState) GetUserStateId() string {
-	if m != nil {
-		return m.UserStateId
-	}
-	return ""
-}
-
-func (m *StateKey_BagUserState) GetWindow() []byte {
-	if m != nil {
-		return m.Window
-	}
-	return nil
-}
-
-func (m *StateKey_BagUserState) GetKey() []byte {
-	if m != nil {
-		return m.Key
-	}
-	return nil
-}
-
 // A request to get state.
 type StateGetRequest struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
 	// (Optional) If specified, signals to the runner that the response
 	// should resume from the following continuation token.
 	//
 	// If unspecified, signals to the runner that the response should start
 	// from the beginning of the logical continuable stream.
-	ContinuationToken    []byte   `protobuf:"bytes,1,opt,name=continuation_token,json=continuationToken,proto3" json:"continuation_token,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
+	ContinuationToken []byte `protobuf:"bytes,1,opt,name=continuation_token,json=continuationToken,proto3" json:"continuation_token,omitempty"`
 }
 
-func (m *StateGetRequest) Reset()         { *m = StateGetRequest{} }
-func (m *StateGetRequest) String() string { return proto.CompactTextString(m) }
-func (*StateGetRequest) ProtoMessage()    {}
+func (x *StateGetRequest) Reset() {
+	*x = StateGetRequest{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_fn_api_proto_msgTypes[23]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *StateGetRequest) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*StateGetRequest) ProtoMessage() {}
+
+func (x *StateGetRequest) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_fn_api_proto_msgTypes[23]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use StateGetRequest.ProtoReflect.Descriptor instead.
 func (*StateGetRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_6d954c03a4758710, []int{23}
+	return file_beam_fn_api_proto_rawDescGZIP(), []int{23}
 }
 
-func (m *StateGetRequest) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_StateGetRequest.Unmarshal(m, b)
-}
-func (m *StateGetRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_StateGetRequest.Marshal(b, m, deterministic)
-}
-func (m *StateGetRequest) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_StateGetRequest.Merge(m, src)
-}
-func (m *StateGetRequest) XXX_Size() int {
-	return xxx_messageInfo_StateGetRequest.Size(m)
-}
-func (m *StateGetRequest) XXX_DiscardUnknown() {
-	xxx_messageInfo_StateGetRequest.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_StateGetRequest proto.InternalMessageInfo
-
-func (m *StateGetRequest) GetContinuationToken() []byte {
-	if m != nil {
-		return m.ContinuationToken
+func (x *StateGetRequest) GetContinuationToken() []byte {
+	if x != nil {
+		return x.ContinuationToken
 	}
 	return nil
 }
@@ -2656,6 +2029,10 @@
 // A response to get state representing a logical byte stream which can be
 // continued using the state API.
 type StateGetResponse struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
 	// (Optional) If specified, represents a token which can be used with the
 	// state API to get the next chunk of this logical byte stream. The end of
 	// the logical byte stream is signalled by this field being unset.
@@ -2663,192 +2040,229 @@
 	// Represents a part of a logical byte stream. Elements within
 	// the logical byte stream are encoded in the nested context and
 	// concatenated together.
-	Data                 []byte   `protobuf:"bytes,2,opt,name=data,proto3" json:"data,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
+	Data []byte `protobuf:"bytes,2,opt,name=data,proto3" json:"data,omitempty"`
 }
 
-func (m *StateGetResponse) Reset()         { *m = StateGetResponse{} }
-func (m *StateGetResponse) String() string { return proto.CompactTextString(m) }
-func (*StateGetResponse) ProtoMessage()    {}
+func (x *StateGetResponse) Reset() {
+	*x = StateGetResponse{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_fn_api_proto_msgTypes[24]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *StateGetResponse) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*StateGetResponse) ProtoMessage() {}
+
+func (x *StateGetResponse) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_fn_api_proto_msgTypes[24]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use StateGetResponse.ProtoReflect.Descriptor instead.
 func (*StateGetResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_6d954c03a4758710, []int{24}
+	return file_beam_fn_api_proto_rawDescGZIP(), []int{24}
 }
 
-func (m *StateGetResponse) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_StateGetResponse.Unmarshal(m, b)
-}
-func (m *StateGetResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_StateGetResponse.Marshal(b, m, deterministic)
-}
-func (m *StateGetResponse) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_StateGetResponse.Merge(m, src)
-}
-func (m *StateGetResponse) XXX_Size() int {
-	return xxx_messageInfo_StateGetResponse.Size(m)
-}
-func (m *StateGetResponse) XXX_DiscardUnknown() {
-	xxx_messageInfo_StateGetResponse.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_StateGetResponse proto.InternalMessageInfo
-
-func (m *StateGetResponse) GetContinuationToken() []byte {
-	if m != nil {
-		return m.ContinuationToken
+func (x *StateGetResponse) GetContinuationToken() []byte {
+	if x != nil {
+		return x.ContinuationToken
 	}
 	return nil
 }
 
-func (m *StateGetResponse) GetData() []byte {
-	if m != nil {
-		return m.Data
+func (x *StateGetResponse) GetData() []byte {
+	if x != nil {
+		return x.Data
 	}
 	return nil
 }
 
 // A request to append state.
 type StateAppendRequest struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
 	// Represents a part of a logical byte stream. Elements within
 	// the logical byte stream are encoded in the nested context and
 	// multiple append requests are concatenated together.
-	Data                 []byte   `protobuf:"bytes,1,opt,name=data,proto3" json:"data,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
+	Data []byte `protobuf:"bytes,1,opt,name=data,proto3" json:"data,omitempty"`
 }
 
-func (m *StateAppendRequest) Reset()         { *m = StateAppendRequest{} }
-func (m *StateAppendRequest) String() string { return proto.CompactTextString(m) }
-func (*StateAppendRequest) ProtoMessage()    {}
+func (x *StateAppendRequest) Reset() {
+	*x = StateAppendRequest{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_fn_api_proto_msgTypes[25]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *StateAppendRequest) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*StateAppendRequest) ProtoMessage() {}
+
+func (x *StateAppendRequest) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_fn_api_proto_msgTypes[25]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use StateAppendRequest.ProtoReflect.Descriptor instead.
 func (*StateAppendRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_6d954c03a4758710, []int{25}
+	return file_beam_fn_api_proto_rawDescGZIP(), []int{25}
 }
 
-func (m *StateAppendRequest) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_StateAppendRequest.Unmarshal(m, b)
-}
-func (m *StateAppendRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_StateAppendRequest.Marshal(b, m, deterministic)
-}
-func (m *StateAppendRequest) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_StateAppendRequest.Merge(m, src)
-}
-func (m *StateAppendRequest) XXX_Size() int {
-	return xxx_messageInfo_StateAppendRequest.Size(m)
-}
-func (m *StateAppendRequest) XXX_DiscardUnknown() {
-	xxx_messageInfo_StateAppendRequest.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_StateAppendRequest proto.InternalMessageInfo
-
-func (m *StateAppendRequest) GetData() []byte {
-	if m != nil {
-		return m.Data
+func (x *StateAppendRequest) GetData() []byte {
+	if x != nil {
+		return x.Data
 	}
 	return nil
 }
 
 // A response to append state.
 type StateAppendResponse struct {
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
 }
 
-func (m *StateAppendResponse) Reset()         { *m = StateAppendResponse{} }
-func (m *StateAppendResponse) String() string { return proto.CompactTextString(m) }
-func (*StateAppendResponse) ProtoMessage()    {}
+func (x *StateAppendResponse) Reset() {
+	*x = StateAppendResponse{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_fn_api_proto_msgTypes[26]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *StateAppendResponse) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*StateAppendResponse) ProtoMessage() {}
+
+func (x *StateAppendResponse) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_fn_api_proto_msgTypes[26]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use StateAppendResponse.ProtoReflect.Descriptor instead.
 func (*StateAppendResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_6d954c03a4758710, []int{26}
+	return file_beam_fn_api_proto_rawDescGZIP(), []int{26}
 }
 
-func (m *StateAppendResponse) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_StateAppendResponse.Unmarshal(m, b)
-}
-func (m *StateAppendResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_StateAppendResponse.Marshal(b, m, deterministic)
-}
-func (m *StateAppendResponse) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_StateAppendResponse.Merge(m, src)
-}
-func (m *StateAppendResponse) XXX_Size() int {
-	return xxx_messageInfo_StateAppendResponse.Size(m)
-}
-func (m *StateAppendResponse) XXX_DiscardUnknown() {
-	xxx_messageInfo_StateAppendResponse.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_StateAppendResponse proto.InternalMessageInfo
-
 // A request to clear state.
 type StateClearRequest struct {
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
 }
 
-func (m *StateClearRequest) Reset()         { *m = StateClearRequest{} }
-func (m *StateClearRequest) String() string { return proto.CompactTextString(m) }
-func (*StateClearRequest) ProtoMessage()    {}
+func (x *StateClearRequest) Reset() {
+	*x = StateClearRequest{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_fn_api_proto_msgTypes[27]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *StateClearRequest) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*StateClearRequest) ProtoMessage() {}
+
+func (x *StateClearRequest) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_fn_api_proto_msgTypes[27]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use StateClearRequest.ProtoReflect.Descriptor instead.
 func (*StateClearRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_6d954c03a4758710, []int{27}
+	return file_beam_fn_api_proto_rawDescGZIP(), []int{27}
 }
 
-func (m *StateClearRequest) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_StateClearRequest.Unmarshal(m, b)
-}
-func (m *StateClearRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_StateClearRequest.Marshal(b, m, deterministic)
-}
-func (m *StateClearRequest) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_StateClearRequest.Merge(m, src)
-}
-func (m *StateClearRequest) XXX_Size() int {
-	return xxx_messageInfo_StateClearRequest.Size(m)
-}
-func (m *StateClearRequest) XXX_DiscardUnknown() {
-	xxx_messageInfo_StateClearRequest.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_StateClearRequest proto.InternalMessageInfo
-
 // A response to clear state.
 type StateClearResponse struct {
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
 }
 
-func (m *StateClearResponse) Reset()         { *m = StateClearResponse{} }
-func (m *StateClearResponse) String() string { return proto.CompactTextString(m) }
-func (*StateClearResponse) ProtoMessage()    {}
+func (x *StateClearResponse) Reset() {
+	*x = StateClearResponse{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_fn_api_proto_msgTypes[28]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *StateClearResponse) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*StateClearResponse) ProtoMessage() {}
+
+func (x *StateClearResponse) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_fn_api_proto_msgTypes[28]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use StateClearResponse.ProtoReflect.Descriptor instead.
 func (*StateClearResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_6d954c03a4758710, []int{28}
+	return file_beam_fn_api_proto_rawDescGZIP(), []int{28}
 }
 
-func (m *StateClearResponse) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_StateClearResponse.Unmarshal(m, b)
-}
-func (m *StateClearResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_StateClearResponse.Marshal(b, m, deterministic)
-}
-func (m *StateClearResponse) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_StateClearResponse.Merge(m, src)
-}
-func (m *StateClearResponse) XXX_Size() int {
-	return xxx_messageInfo_StateClearResponse.Size(m)
-}
-func (m *StateClearResponse) XXX_DiscardUnknown() {
-	xxx_messageInfo_StateClearResponse.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_StateClearResponse proto.InternalMessageInfo
-
 // A log entry
 type LogEntry struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
 	// (Required) The severity of the log statement.
 	Severity LogEntry_Severity_Enum `protobuf:"varint,1,opt,name=severity,proto3,enum=org.apache.beam.model.fn_execution.v1.LogEntry_Severity_Enum" json:"severity,omitempty"`
 	// (Required) The time at which this log statement occurred.
@@ -2873,131 +2287,1427 @@
 	//   file.cc:382 (C++)
 	LogLocation string `protobuf:"bytes,7,opt,name=log_location,json=logLocation,proto3" json:"log_location,omitempty"`
 	// (Optional) The name of the thread this log statement is associated with.
-	Thread               string   `protobuf:"bytes,8,opt,name=thread,proto3" json:"thread,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
+	Thread string `protobuf:"bytes,8,opt,name=thread,proto3" json:"thread,omitempty"`
 }
 
-func (m *LogEntry) Reset()         { *m = LogEntry{} }
-func (m *LogEntry) String() string { return proto.CompactTextString(m) }
-func (*LogEntry) ProtoMessage()    {}
+func (x *LogEntry) Reset() {
+	*x = LogEntry{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_fn_api_proto_msgTypes[29]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *LogEntry) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*LogEntry) ProtoMessage() {}
+
+func (x *LogEntry) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_fn_api_proto_msgTypes[29]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use LogEntry.ProtoReflect.Descriptor instead.
 func (*LogEntry) Descriptor() ([]byte, []int) {
-	return fileDescriptor_6d954c03a4758710, []int{29}
+	return file_beam_fn_api_proto_rawDescGZIP(), []int{29}
 }
 
-func (m *LogEntry) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_LogEntry.Unmarshal(m, b)
-}
-func (m *LogEntry) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_LogEntry.Marshal(b, m, deterministic)
-}
-func (m *LogEntry) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_LogEntry.Merge(m, src)
-}
-func (m *LogEntry) XXX_Size() int {
-	return xxx_messageInfo_LogEntry.Size(m)
-}
-func (m *LogEntry) XXX_DiscardUnknown() {
-	xxx_messageInfo_LogEntry.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_LogEntry proto.InternalMessageInfo
-
-func (m *LogEntry) GetSeverity() LogEntry_Severity_Enum {
-	if m != nil {
-		return m.Severity
+func (x *LogEntry) GetSeverity() LogEntry_Severity_Enum {
+	if x != nil {
+		return x.Severity
 	}
 	return LogEntry_Severity_UNSPECIFIED
 }
 
-func (m *LogEntry) GetTimestamp() *timestamp.Timestamp {
-	if m != nil {
-		return m.Timestamp
+func (x *LogEntry) GetTimestamp() *timestamp.Timestamp {
+	if x != nil {
+		return x.Timestamp
 	}
 	return nil
 }
 
-func (m *LogEntry) GetMessage() string {
-	if m != nil {
-		return m.Message
+func (x *LogEntry) GetMessage() string {
+	if x != nil {
+		return x.Message
 	}
 	return ""
 }
 
-func (m *LogEntry) GetTrace() string {
-	if m != nil {
-		return m.Trace
+func (x *LogEntry) GetTrace() string {
+	if x != nil {
+		return x.Trace
 	}
 	return ""
 }
 
-func (m *LogEntry) GetInstructionId() string {
-	if m != nil {
-		return m.InstructionId
+func (x *LogEntry) GetInstructionId() string {
+	if x != nil {
+		return x.InstructionId
 	}
 	return ""
 }
 
-func (m *LogEntry) GetTransformId() string {
-	if m != nil {
-		return m.TransformId
+func (x *LogEntry) GetTransformId() string {
+	if x != nil {
+		return x.TransformId
 	}
 	return ""
 }
 
-func (m *LogEntry) GetLogLocation() string {
-	if m != nil {
-		return m.LogLocation
+func (x *LogEntry) GetLogLocation() string {
+	if x != nil {
+		return x.LogLocation
 	}
 	return ""
 }
 
-func (m *LogEntry) GetThread() string {
-	if m != nil {
-		return m.Thread
+func (x *LogEntry) GetThread() string {
+	if x != nil {
+		return x.Thread
 	}
 	return ""
 }
 
+type LogControl struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+}
+
+func (x *LogControl) Reset() {
+	*x = LogControl{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_fn_api_proto_msgTypes[30]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *LogControl) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*LogControl) ProtoMessage() {}
+
+func (x *LogControl) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_fn_api_proto_msgTypes[30]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use LogControl.ProtoReflect.Descriptor instead.
+func (*LogControl) Descriptor() ([]byte, []int) {
+	return file_beam_fn_api_proto_rawDescGZIP(), []int{30}
+}
+
+type StartWorkerRequest struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	WorkerId          string                            `protobuf:"bytes,1,opt,name=worker_id,json=workerId,proto3" json:"worker_id,omitempty"`
+	ControlEndpoint   *pipeline_v1.ApiServiceDescriptor `protobuf:"bytes,2,opt,name=control_endpoint,json=controlEndpoint,proto3" json:"control_endpoint,omitempty"`
+	LoggingEndpoint   *pipeline_v1.ApiServiceDescriptor `protobuf:"bytes,3,opt,name=logging_endpoint,json=loggingEndpoint,proto3" json:"logging_endpoint,omitempty"`
+	ArtifactEndpoint  *pipeline_v1.ApiServiceDescriptor `protobuf:"bytes,4,opt,name=artifact_endpoint,json=artifactEndpoint,proto3" json:"artifact_endpoint,omitempty"`
+	ProvisionEndpoint *pipeline_v1.ApiServiceDescriptor `protobuf:"bytes,5,opt,name=provision_endpoint,json=provisionEndpoint,proto3" json:"provision_endpoint,omitempty"`
+	Params            map[string]string                 `protobuf:"bytes,10,rep,name=params,proto3" json:"params,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"`
+}
+
+func (x *StartWorkerRequest) Reset() {
+	*x = StartWorkerRequest{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_fn_api_proto_msgTypes[31]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *StartWorkerRequest) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*StartWorkerRequest) ProtoMessage() {}
+
+func (x *StartWorkerRequest) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_fn_api_proto_msgTypes[31]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use StartWorkerRequest.ProtoReflect.Descriptor instead.
+func (*StartWorkerRequest) Descriptor() ([]byte, []int) {
+	return file_beam_fn_api_proto_rawDescGZIP(), []int{31}
+}
+
+func (x *StartWorkerRequest) GetWorkerId() string {
+	if x != nil {
+		return x.WorkerId
+	}
+	return ""
+}
+
+func (x *StartWorkerRequest) GetControlEndpoint() *pipeline_v1.ApiServiceDescriptor {
+	if x != nil {
+		return x.ControlEndpoint
+	}
+	return nil
+}
+
+func (x *StartWorkerRequest) GetLoggingEndpoint() *pipeline_v1.ApiServiceDescriptor {
+	if x != nil {
+		return x.LoggingEndpoint
+	}
+	return nil
+}
+
+func (x *StartWorkerRequest) GetArtifactEndpoint() *pipeline_v1.ApiServiceDescriptor {
+	if x != nil {
+		return x.ArtifactEndpoint
+	}
+	return nil
+}
+
+func (x *StartWorkerRequest) GetProvisionEndpoint() *pipeline_v1.ApiServiceDescriptor {
+	if x != nil {
+		return x.ProvisionEndpoint
+	}
+	return nil
+}
+
+func (x *StartWorkerRequest) GetParams() map[string]string {
+	if x != nil {
+		return x.Params
+	}
+	return nil
+}
+
+type StartWorkerResponse struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	Error string `protobuf:"bytes,1,opt,name=error,proto3" json:"error,omitempty"`
+}
+
+func (x *StartWorkerResponse) Reset() {
+	*x = StartWorkerResponse{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_fn_api_proto_msgTypes[32]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *StartWorkerResponse) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*StartWorkerResponse) ProtoMessage() {}
+
+func (x *StartWorkerResponse) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_fn_api_proto_msgTypes[32]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use StartWorkerResponse.ProtoReflect.Descriptor instead.
+func (*StartWorkerResponse) Descriptor() ([]byte, []int) {
+	return file_beam_fn_api_proto_rawDescGZIP(), []int{32}
+}
+
+func (x *StartWorkerResponse) GetError() string {
+	if x != nil {
+		return x.Error
+	}
+	return ""
+}
+
+type StopWorkerRequest struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	WorkerId string `protobuf:"bytes,1,opt,name=worker_id,json=workerId,proto3" json:"worker_id,omitempty"`
+}
+
+func (x *StopWorkerRequest) Reset() {
+	*x = StopWorkerRequest{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_fn_api_proto_msgTypes[33]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *StopWorkerRequest) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*StopWorkerRequest) ProtoMessage() {}
+
+func (x *StopWorkerRequest) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_fn_api_proto_msgTypes[33]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use StopWorkerRequest.ProtoReflect.Descriptor instead.
+func (*StopWorkerRequest) Descriptor() ([]byte, []int) {
+	return file_beam_fn_api_proto_rawDescGZIP(), []int{33}
+}
+
+func (x *StopWorkerRequest) GetWorkerId() string {
+	if x != nil {
+		return x.WorkerId
+	}
+	return ""
+}
+
+type StopWorkerResponse struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	Error string `protobuf:"bytes,1,opt,name=error,proto3" json:"error,omitempty"`
+}
+
+func (x *StopWorkerResponse) Reset() {
+	*x = StopWorkerResponse{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_fn_api_proto_msgTypes[34]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *StopWorkerResponse) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*StopWorkerResponse) ProtoMessage() {}
+
+func (x *StopWorkerResponse) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_fn_api_proto_msgTypes[34]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use StopWorkerResponse.ProtoReflect.Descriptor instead.
+func (*StopWorkerResponse) Descriptor() ([]byte, []int) {
+	return file_beam_fn_api_proto_rawDescGZIP(), []int{34}
+}
+
+func (x *StopWorkerResponse) GetError() string {
+	if x != nil {
+		return x.Error
+	}
+	return ""
+}
+
+// Request from runner to SDK Harness asking for its status. For more details see
+// https://s.apache.org/beam-fn-api-harness-status
+type WorkerStatusRequest struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	// (Required) Unique ID identifying this request.
+	Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"`
+}
+
+func (x *WorkerStatusRequest) Reset() {
+	*x = WorkerStatusRequest{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_fn_api_proto_msgTypes[35]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *WorkerStatusRequest) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*WorkerStatusRequest) ProtoMessage() {}
+
+func (x *WorkerStatusRequest) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_fn_api_proto_msgTypes[35]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use WorkerStatusRequest.ProtoReflect.Descriptor instead.
+func (*WorkerStatusRequest) Descriptor() ([]byte, []int) {
+	return file_beam_fn_api_proto_rawDescGZIP(), []int{35}
+}
+
+func (x *WorkerStatusRequest) GetId() string {
+	if x != nil {
+		return x.Id
+	}
+	return ""
+}
+
+// Response from SDK Harness to runner containing the debug related status info.
+type WorkerStatusResponse struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	// (Required) Unique ID from the original request.
+	Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"`
+	// (Optional) Error message if exception encountered generating the status response.
+	Error string `protobuf:"bytes,2,opt,name=error,proto3" json:"error,omitempty"`
+	// (Optional) Status debugging info reported by SDK harness worker. Content and
+	// format is not strongly enforced but should be print-friendly and
+	// appropriate as an HTTP response body for end user. For details of the preferred
+	// info to include in the message see
+	// https://s.apache.org/beam-fn-api-harness-status
+	StatusInfo string `protobuf:"bytes,3,opt,name=status_info,json=statusInfo,proto3" json:"status_info,omitempty"`
+}
+
+func (x *WorkerStatusResponse) Reset() {
+	*x = WorkerStatusResponse{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_fn_api_proto_msgTypes[36]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *WorkerStatusResponse) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*WorkerStatusResponse) ProtoMessage() {}
+
+func (x *WorkerStatusResponse) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_fn_api_proto_msgTypes[36]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use WorkerStatusResponse.ProtoReflect.Descriptor instead.
+func (*WorkerStatusResponse) Descriptor() ([]byte, []int) {
+	return file_beam_fn_api_proto_rawDescGZIP(), []int{36}
+}
+
+func (x *WorkerStatusResponse) GetId() string {
+	if x != nil {
+		return x.Id
+	}
+	return ""
+}
+
+func (x *WorkerStatusResponse) GetError() string {
+	if x != nil {
+		return x.Error
+	}
+	return ""
+}
+
+func (x *WorkerStatusResponse) GetStatusInfo() string {
+	if x != nil {
+		return x.StatusInfo
+	}
+	return ""
+}
+
+// A cache token which can be used by an SDK to check for the validity
+// of cached elements which have a cache token associated.
+type ProcessBundleRequest_CacheToken struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	// The scope of a cache token.
+	//
+	// Types that are assignable to Type:
+	//	*ProcessBundleRequest_CacheToken_UserState_
+	//	*ProcessBundleRequest_CacheToken_SideInput_
+	Type isProcessBundleRequest_CacheToken_Type `protobuf_oneof:"type"`
+	// The cache token identifier which should be globally unique.
+	Token []byte `protobuf:"bytes,10,opt,name=token,proto3" json:"token,omitempty"`
+}
+
+func (x *ProcessBundleRequest_CacheToken) Reset() {
+	*x = ProcessBundleRequest_CacheToken{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_fn_api_proto_msgTypes[43]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *ProcessBundleRequest_CacheToken) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*ProcessBundleRequest_CacheToken) ProtoMessage() {}
+
+func (x *ProcessBundleRequest_CacheToken) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_fn_api_proto_msgTypes[43]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use ProcessBundleRequest_CacheToken.ProtoReflect.Descriptor instead.
+func (*ProcessBundleRequest_CacheToken) Descriptor() ([]byte, []int) {
+	return file_beam_fn_api_proto_rawDescGZIP(), []int{9, 0}
+}
+
+func (m *ProcessBundleRequest_CacheToken) GetType() isProcessBundleRequest_CacheToken_Type {
+	if m != nil {
+		return m.Type
+	}
+	return nil
+}
+
+func (x *ProcessBundleRequest_CacheToken) GetUserState() *ProcessBundleRequest_CacheToken_UserState {
+	if x, ok := x.GetType().(*ProcessBundleRequest_CacheToken_UserState_); ok {
+		return x.UserState
+	}
+	return nil
+}
+
+func (x *ProcessBundleRequest_CacheToken) GetSideInput() *ProcessBundleRequest_CacheToken_SideInput {
+	if x, ok := x.GetType().(*ProcessBundleRequest_CacheToken_SideInput_); ok {
+		return x.SideInput
+	}
+	return nil
+}
+
+func (x *ProcessBundleRequest_CacheToken) GetToken() []byte {
+	if x != nil {
+		return x.Token
+	}
+	return nil
+}
+
+type isProcessBundleRequest_CacheToken_Type interface {
+	isProcessBundleRequest_CacheToken_Type()
+}
+
+type ProcessBundleRequest_CacheToken_UserState_ struct {
+	UserState *ProcessBundleRequest_CacheToken_UserState `protobuf:"bytes,1,opt,name=user_state,json=userState,proto3,oneof"`
+}
+
+type ProcessBundleRequest_CacheToken_SideInput_ struct {
+	SideInput *ProcessBundleRequest_CacheToken_SideInput `protobuf:"bytes,2,opt,name=side_input,json=sideInput,proto3,oneof"`
+}
+
+func (*ProcessBundleRequest_CacheToken_UserState_) isProcessBundleRequest_CacheToken_Type() {}
+
+func (*ProcessBundleRequest_CacheToken_SideInput_) isProcessBundleRequest_CacheToken_Type() {}
+
+// A flag to indicate a cache token is valid for all user state.
+type ProcessBundleRequest_CacheToken_UserState struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+}
+
+func (x *ProcessBundleRequest_CacheToken_UserState) Reset() {
+	*x = ProcessBundleRequest_CacheToken_UserState{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_fn_api_proto_msgTypes[44]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *ProcessBundleRequest_CacheToken_UserState) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*ProcessBundleRequest_CacheToken_UserState) ProtoMessage() {}
+
+func (x *ProcessBundleRequest_CacheToken_UserState) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_fn_api_proto_msgTypes[44]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use ProcessBundleRequest_CacheToken_UserState.ProtoReflect.Descriptor instead.
+func (*ProcessBundleRequest_CacheToken_UserState) Descriptor() ([]byte, []int) {
+	return file_beam_fn_api_proto_rawDescGZIP(), []int{9, 0, 0}
+}
+
+// A flag to indicate a cache token is valid for a side input.
+type ProcessBundleRequest_CacheToken_SideInput struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	// (Required) The id of the PTransform containing a side input.
+	TransformId string `protobuf:"bytes,1,opt,name=transform_id,json=transformId,proto3" json:"transform_id,omitempty"`
+	// (Required) The id of the side input.
+	SideInputId string `protobuf:"bytes,2,opt,name=side_input_id,json=sideInputId,proto3" json:"side_input_id,omitempty"`
+}
+
+func (x *ProcessBundleRequest_CacheToken_SideInput) Reset() {
+	*x = ProcessBundleRequest_CacheToken_SideInput{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_fn_api_proto_msgTypes[45]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *ProcessBundleRequest_CacheToken_SideInput) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*ProcessBundleRequest_CacheToken_SideInput) ProtoMessage() {}
+
+func (x *ProcessBundleRequest_CacheToken_SideInput) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_fn_api_proto_msgTypes[45]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use ProcessBundleRequest_CacheToken_SideInput.ProtoReflect.Descriptor instead.
+func (*ProcessBundleRequest_CacheToken_SideInput) Descriptor() ([]byte, []int) {
+	return file_beam_fn_api_proto_rawDescGZIP(), []int{9, 0, 1}
+}
+
+func (x *ProcessBundleRequest_CacheToken_SideInput) GetTransformId() string {
+	if x != nil {
+		return x.TransformId
+	}
+	return ""
+}
+
+func (x *ProcessBundleRequest_CacheToken_SideInput) GetSideInputId() string {
+	if x != nil {
+		return x.SideInputId
+	}
+	return ""
+}
+
+// A message specifying the desired split for a single transform.
+type ProcessBundleSplitRequest_DesiredSplit struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	// (Required) The fraction of known work remaining in this bundle
+	// for this transform that should be kept by the SDK after this split.
+	//
+	// Set to 0 to "checkpoint" as soon as possible (keeping as little work as
+	// possible and returning the remainder).
+	FractionOfRemainder float64 `protobuf:"fixed64,1,opt,name=fraction_of_remainder,json=fractionOfRemainder,proto3" json:"fraction_of_remainder,omitempty"`
+	// A set of allowed element indices where the SDK may split. When this is
+	// empty, there are no constraints on where to split.
+	AllowedSplitPoints []int64 `protobuf:"varint,3,rep,packed,name=allowed_split_points,json=allowedSplitPoints,proto3" json:"allowed_split_points,omitempty"`
+	// (Required for GrpcRead operations) Number of total elements expected
+	// to be sent to this GrpcRead operation, required to correctly account
+	// for unreceived data when determining where to split.
+	EstimatedInputElements int64 `protobuf:"varint,2,opt,name=estimated_input_elements,json=estimatedInputElements,proto3" json:"estimated_input_elements,omitempty"`
+}
+
+func (x *ProcessBundleSplitRequest_DesiredSplit) Reset() {
+	*x = ProcessBundleSplitRequest_DesiredSplit{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_fn_api_proto_msgTypes[49]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *ProcessBundleSplitRequest_DesiredSplit) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*ProcessBundleSplitRequest_DesiredSplit) ProtoMessage() {}
+
+func (x *ProcessBundleSplitRequest_DesiredSplit) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_fn_api_proto_msgTypes[49]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use ProcessBundleSplitRequest_DesiredSplit.ProtoReflect.Descriptor instead.
+func (*ProcessBundleSplitRequest_DesiredSplit) Descriptor() ([]byte, []int) {
+	return file_beam_fn_api_proto_rawDescGZIP(), []int{15, 0}
+}
+
+func (x *ProcessBundleSplitRequest_DesiredSplit) GetFractionOfRemainder() float64 {
+	if x != nil {
+		return x.FractionOfRemainder
+	}
+	return 0
+}
+
+func (x *ProcessBundleSplitRequest_DesiredSplit) GetAllowedSplitPoints() []int64 {
+	if x != nil {
+		return x.AllowedSplitPoints
+	}
+	return nil
+}
+
+func (x *ProcessBundleSplitRequest_DesiredSplit) GetEstimatedInputElements() int64 {
+	if x != nil {
+		return x.EstimatedInputElements
+	}
+	return 0
+}
+
+// Represents contiguous portions of the data channel that are either
+// entirely processed or entirely unprocessed and belong to the primary
+// or residual respectively.
+//
+// This affords both a more efficient representation over the FnAPI
+// (if the bundle is large) and often a more efficient representation
+// on the runner side (e.g. if the set of elements can be represented
+// as some range in an underlying dataset).
+type ProcessBundleSplitResponse_ChannelSplit struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	// (Required) The grpc read transform reading this channel.
+	TransformId string `protobuf:"bytes,1,opt,name=transform_id,json=transformId,proto3" json:"transform_id,omitempty"`
+	// The last element of the input channel that should be entirely considered
+	// part of the primary, identified by its absolute index in the (ordered)
+	// channel.
+	LastPrimaryElement int64 `protobuf:"varint,2,opt,name=last_primary_element,json=lastPrimaryElement,proto3" json:"last_primary_element,omitempty"`
+	// The first element of the input channel that should be entirely considered
+	// part of the residual, identified by its absolute index in the (ordered)
+	// channel.
+	FirstResidualElement int64 `protobuf:"varint,3,opt,name=first_residual_element,json=firstResidualElement,proto3" json:"first_residual_element,omitempty"`
+}
+
+func (x *ProcessBundleSplitResponse_ChannelSplit) Reset() {
+	*x = ProcessBundleSplitResponse_ChannelSplit{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_fn_api_proto_msgTypes[51]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *ProcessBundleSplitResponse_ChannelSplit) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*ProcessBundleSplitResponse_ChannelSplit) ProtoMessage() {}
+
+func (x *ProcessBundleSplitResponse_ChannelSplit) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_fn_api_proto_msgTypes[51]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use ProcessBundleSplitResponse_ChannelSplit.ProtoReflect.Descriptor instead.
+func (*ProcessBundleSplitResponse_ChannelSplit) Descriptor() ([]byte, []int) {
+	return file_beam_fn_api_proto_rawDescGZIP(), []int{16, 0}
+}
+
+func (x *ProcessBundleSplitResponse_ChannelSplit) GetTransformId() string {
+	if x != nil {
+		return x.TransformId
+	}
+	return ""
+}
+
+func (x *ProcessBundleSplitResponse_ChannelSplit) GetLastPrimaryElement() int64 {
+	if x != nil {
+		return x.LastPrimaryElement
+	}
+	return 0
+}
+
+func (x *ProcessBundleSplitResponse_ChannelSplit) GetFirstResidualElement() int64 {
+	if x != nil {
+		return x.FirstResidualElement
+	}
+	return 0
+}
+
+// Represents multiple encoded elements in nested context for a given named
+// instruction and transform.
+type Elements_Data struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	// (Required) A reference to an active instruction request with the given
+	// instruction id.
+	InstructionId string `protobuf:"bytes,1,opt,name=instruction_id,json=instructionId,proto3" json:"instruction_id,omitempty"`
+	// (Required) A definition representing a consumer or producer of this data.
+	// If received by a harness, this represents the consumer within that
+	// harness that should consume these bytes. If sent by a harness, this
+	// represents the producer of these bytes.
+	//
+	// Note that a single element may span multiple Data messages.
+	//
+	// Note that a sending/receiving pair should share the same identifier.
+	TransformId string `protobuf:"bytes,2,opt,name=transform_id,json=transformId,proto3" json:"transform_id,omitempty"`
+	// (Optional) Represents a part of a logical byte stream. Elements within
+	// the logical byte stream are encoded in the nested context and
+	// concatenated together.
+	Data []byte `protobuf:"bytes,3,opt,name=data,proto3" json:"data,omitempty"`
+	// (Optional) Set this bit to indicate the this is the last data block
+	// for the given instruction and transform, ending the stream.
+	IsLast bool `protobuf:"varint,4,opt,name=is_last,json=isLast,proto3" json:"is_last,omitempty"`
+}
+
+func (x *Elements_Data) Reset() {
+	*x = Elements_Data{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_fn_api_proto_msgTypes[52]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *Elements_Data) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*Elements_Data) ProtoMessage() {}
+
+func (x *Elements_Data) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_fn_api_proto_msgTypes[52]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use Elements_Data.ProtoReflect.Descriptor instead.
+func (*Elements_Data) Descriptor() ([]byte, []int) {
+	return file_beam_fn_api_proto_rawDescGZIP(), []int{19, 0}
+}
+
+func (x *Elements_Data) GetInstructionId() string {
+	if x != nil {
+		return x.InstructionId
+	}
+	return ""
+}
+
+func (x *Elements_Data) GetTransformId() string {
+	if x != nil {
+		return x.TransformId
+	}
+	return ""
+}
+
+func (x *Elements_Data) GetData() []byte {
+	if x != nil {
+		return x.Data
+	}
+	return nil
+}
+
+func (x *Elements_Data) GetIsLast() bool {
+	if x != nil {
+		return x.IsLast
+	}
+	return false
+}
+
+// Represent the encoded user timer for a given instruction, transform and
+// timer id.
+type Elements_Timers struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	// (Required) A reference to an active instruction request with the given
+	// instruction id.
+	InstructionId string `protobuf:"bytes,1,opt,name=instruction_id,json=instructionId,proto3" json:"instruction_id,omitempty"`
+	// (Required) A definition representing a consumer or producer of this data.
+	// If received by a harness, this represents the consumer within that
+	// harness that should consume these timers. If sent by a harness, this
+	// represents the producer of these timers.
+	TransformId string `protobuf:"bytes,2,opt,name=transform_id,json=transformId,proto3" json:"transform_id,omitempty"`
+	// (Required) The local timer family name used to identify the associated
+	// timer family specification
+	TimerFamilyId string `protobuf:"bytes,3,opt,name=timer_family_id,json=timerFamilyId,proto3" json:"timer_family_id,omitempty"`
+	// (Optional) Represents a logical byte stream of timers. Encoded according
+	// to the coder in the timer spec.
+	Timers []byte `protobuf:"bytes,4,opt,name=timers,proto3" json:"timers,omitempty"`
+	// (Optional) Set this bit to indicate the this is the last data block
+	// for the given instruction and transform, ending the stream.
+	IsLast bool `protobuf:"varint,5,opt,name=is_last,json=isLast,proto3" json:"is_last,omitempty"`
+}
+
+func (x *Elements_Timers) Reset() {
+	*x = Elements_Timers{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_fn_api_proto_msgTypes[53]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *Elements_Timers) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*Elements_Timers) ProtoMessage() {}
+
+func (x *Elements_Timers) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_fn_api_proto_msgTypes[53]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use Elements_Timers.ProtoReflect.Descriptor instead.
+func (*Elements_Timers) Descriptor() ([]byte, []int) {
+	return file_beam_fn_api_proto_rawDescGZIP(), []int{19, 1}
+}
+
+func (x *Elements_Timers) GetInstructionId() string {
+	if x != nil {
+		return x.InstructionId
+	}
+	return ""
+}
+
+func (x *Elements_Timers) GetTransformId() string {
+	if x != nil {
+		return x.TransformId
+	}
+	return ""
+}
+
+func (x *Elements_Timers) GetTimerFamilyId() string {
+	if x != nil {
+		return x.TimerFamilyId
+	}
+	return ""
+}
+
+func (x *Elements_Timers) GetTimers() []byte {
+	if x != nil {
+		return x.Timers
+	}
+	return nil
+}
+
+func (x *Elements_Timers) GetIsLast() bool {
+	if x != nil {
+		return x.IsLast
+	}
+	return false
+}
+
+type StateKey_Runner struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	// (Required) Opaque information supplied by the runner. Used to support
+	// remote references.
+	// https://s.apache.org/beam-fn-api-send-and-receive-data
+	//
+	// Used by state backed iterable. And in this use case, request type can
+	// only be of type get. Details see:
+	// https://s.apache.org/beam-fn-api-state-backed-iterables
+	Key []byte `protobuf:"bytes,1,opt,name=key,proto3" json:"key,omitempty"`
+}
+
+func (x *StateKey_Runner) Reset() {
+	*x = StateKey_Runner{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_fn_api_proto_msgTypes[54]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *StateKey_Runner) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*StateKey_Runner) ProtoMessage() {}
+
+func (x *StateKey_Runner) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_fn_api_proto_msgTypes[54]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use StateKey_Runner.ProtoReflect.Descriptor instead.
+func (*StateKey_Runner) Descriptor() ([]byte, []int) {
+	return file_beam_fn_api_proto_rawDescGZIP(), []int{22, 0}
+}
+
+func (x *StateKey_Runner) GetKey() []byte {
+	if x != nil {
+		return x.Key
+	}
+	return nil
+}
+
+// Represents a request for the values associated with a specified window
+// in a PCollection. See
+// https://s.apache.org/beam-fn-state-api-and-bundle-processing for further
+// details.
+//
+// Can only be used to perform StateGetRequests on side inputs of the URN
+// beam:side_input:iterable:v1.
+//
+// For a PCollection<V>, the response data stream will be a concatenation
+// of all V's. See https://s.apache.org/beam-fn-api-send-and-receive-data
+// for further details.
+type StateKey_IterableSideInput struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	// (Required) The id of the PTransform containing a side input.
+	TransformId string `protobuf:"bytes,1,opt,name=transform_id,json=transformId,proto3" json:"transform_id,omitempty"`
+	// (Required) The id of the side input.
+	SideInputId string `protobuf:"bytes,2,opt,name=side_input_id,json=sideInputId,proto3" json:"side_input_id,omitempty"`
+	// (Required) The window (after mapping the currently executing elements
+	// window into the side input windows domain) encoded in a nested context.
+	Window []byte `protobuf:"bytes,3,opt,name=window,proto3" json:"window,omitempty"`
+}
+
+func (x *StateKey_IterableSideInput) Reset() {
+	*x = StateKey_IterableSideInput{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_fn_api_proto_msgTypes[55]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *StateKey_IterableSideInput) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*StateKey_IterableSideInput) ProtoMessage() {}
+
+func (x *StateKey_IterableSideInput) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_fn_api_proto_msgTypes[55]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use StateKey_IterableSideInput.ProtoReflect.Descriptor instead.
+func (*StateKey_IterableSideInput) Descriptor() ([]byte, []int) {
+	return file_beam_fn_api_proto_rawDescGZIP(), []int{22, 1}
+}
+
+func (x *StateKey_IterableSideInput) GetTransformId() string {
+	if x != nil {
+		return x.TransformId
+	}
+	return ""
+}
+
+func (x *StateKey_IterableSideInput) GetSideInputId() string {
+	if x != nil {
+		return x.SideInputId
+	}
+	return ""
+}
+
+func (x *StateKey_IterableSideInput) GetWindow() []byte {
+	if x != nil {
+		return x.Window
+	}
+	return nil
+}
+
+// Represents a request for the values associated with a specified user key
+// and window in a PCollection. See
+// https://s.apache.org/beam-fn-state-api-and-bundle-processing for further
+// details.
+//
+// Can only be used to perform StateGetRequests on side inputs of the URN
+// beam:side_input:multimap:v1.
+//
+// For a PCollection<KV<K, V>>, the response data stream will be a
+// concatenation of all V's associated with the specified key K. See
+// https://s.apache.org/beam-fn-api-send-and-receive-data for further
+// details.
+type StateKey_MultimapSideInput struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	// (Required) The id of the PTransform containing a side input.
+	TransformId string `protobuf:"bytes,1,opt,name=transform_id,json=transformId,proto3" json:"transform_id,omitempty"`
+	// (Required) The id of the side input.
+	SideInputId string `protobuf:"bytes,2,opt,name=side_input_id,json=sideInputId,proto3" json:"side_input_id,omitempty"`
+	// (Required) The window (after mapping the currently executing elements
+	// window into the side input windows domain) encoded in a nested context.
+	Window []byte `protobuf:"bytes,3,opt,name=window,proto3" json:"window,omitempty"`
+	// (Required) The key encoded in a nested context.
+	Key []byte `protobuf:"bytes,4,opt,name=key,proto3" json:"key,omitempty"`
+}
+
+func (x *StateKey_MultimapSideInput) Reset() {
+	*x = StateKey_MultimapSideInput{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_fn_api_proto_msgTypes[56]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *StateKey_MultimapSideInput) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*StateKey_MultimapSideInput) ProtoMessage() {}
+
+func (x *StateKey_MultimapSideInput) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_fn_api_proto_msgTypes[56]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use StateKey_MultimapSideInput.ProtoReflect.Descriptor instead.
+func (*StateKey_MultimapSideInput) Descriptor() ([]byte, []int) {
+	return file_beam_fn_api_proto_rawDescGZIP(), []int{22, 2}
+}
+
+func (x *StateKey_MultimapSideInput) GetTransformId() string {
+	if x != nil {
+		return x.TransformId
+	}
+	return ""
+}
+
+func (x *StateKey_MultimapSideInput) GetSideInputId() string {
+	if x != nil {
+		return x.SideInputId
+	}
+	return ""
+}
+
+func (x *StateKey_MultimapSideInput) GetWindow() []byte {
+	if x != nil {
+		return x.Window
+	}
+	return nil
+}
+
+func (x *StateKey_MultimapSideInput) GetKey() []byte {
+	if x != nil {
+		return x.Key
+	}
+	return nil
+}
+
+// Represents a request for the keys associated with a specified window in a PCollection. See
+// https://s.apache.org/beam-fn-state-api-and-bundle-processing for further
+// details.
+//
+// Can only be used to perform StateGetRequests on side inputs of the URN
+// beam:side_input:multimap:v1.
+//
+// For a PCollection<KV<K, V>>, the response data stream will be a
+// concatenation of all K's associated with the specified window. See
+// https://s.apache.org/beam-fn-api-send-and-receive-data for further
+// details.
+type StateKey_MultimapKeysSideInput struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	// (Required) The id of the PTransform containing a side input.
+	TransformId string `protobuf:"bytes,1,opt,name=transform_id,json=transformId,proto3" json:"transform_id,omitempty"`
+	// (Required) The id of the side input.
+	SideInputId string `protobuf:"bytes,2,opt,name=side_input_id,json=sideInputId,proto3" json:"side_input_id,omitempty"`
+	// (Required) The window (after mapping the currently executing elements
+	// window into the side input windows domain) encoded in a nested context.
+	Window []byte `protobuf:"bytes,3,opt,name=window,proto3" json:"window,omitempty"`
+}
+
+func (x *StateKey_MultimapKeysSideInput) Reset() {
+	*x = StateKey_MultimapKeysSideInput{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_fn_api_proto_msgTypes[57]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *StateKey_MultimapKeysSideInput) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*StateKey_MultimapKeysSideInput) ProtoMessage() {}
+
+func (x *StateKey_MultimapKeysSideInput) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_fn_api_proto_msgTypes[57]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use StateKey_MultimapKeysSideInput.ProtoReflect.Descriptor instead.
+func (*StateKey_MultimapKeysSideInput) Descriptor() ([]byte, []int) {
+	return file_beam_fn_api_proto_rawDescGZIP(), []int{22, 3}
+}
+
+func (x *StateKey_MultimapKeysSideInput) GetTransformId() string {
+	if x != nil {
+		return x.TransformId
+	}
+	return ""
+}
+
+func (x *StateKey_MultimapKeysSideInput) GetSideInputId() string {
+	if x != nil {
+		return x.SideInputId
+	}
+	return ""
+}
+
+func (x *StateKey_MultimapKeysSideInput) GetWindow() []byte {
+	if x != nil {
+		return x.Window
+	}
+	return nil
+}
+
+type StateKey_BagUserState struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	// (Required) The id of the PTransform containing user state.
+	TransformId string `protobuf:"bytes,1,opt,name=transform_id,json=transformId,proto3" json:"transform_id,omitempty"`
+	// (Required) The id of the user state.
+	UserStateId string `protobuf:"bytes,2,opt,name=user_state_id,json=userStateId,proto3" json:"user_state_id,omitempty"`
+	// (Required) The window encoded in a nested context.
+	Window []byte `protobuf:"bytes,3,opt,name=window,proto3" json:"window,omitempty"`
+	// (Required) The key of the currently executing element encoded in a
+	// nested context.
+	Key []byte `protobuf:"bytes,4,opt,name=key,proto3" json:"key,omitempty"`
+}
+
+func (x *StateKey_BagUserState) Reset() {
+	*x = StateKey_BagUserState{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_fn_api_proto_msgTypes[58]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *StateKey_BagUserState) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*StateKey_BagUserState) ProtoMessage() {}
+
+func (x *StateKey_BagUserState) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_fn_api_proto_msgTypes[58]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use StateKey_BagUserState.ProtoReflect.Descriptor instead.
+func (*StateKey_BagUserState) Descriptor() ([]byte, []int) {
+	return file_beam_fn_api_proto_rawDescGZIP(), []int{22, 4}
+}
+
+func (x *StateKey_BagUserState) GetTransformId() string {
+	if x != nil {
+		return x.TransformId
+	}
+	return ""
+}
+
+func (x *StateKey_BagUserState) GetUserStateId() string {
+	if x != nil {
+		return x.UserStateId
+	}
+	return ""
+}
+
+func (x *StateKey_BagUserState) GetWindow() []byte {
+	if x != nil {
+		return x.Window
+	}
+	return nil
+}
+
+func (x *StateKey_BagUserState) GetKey() []byte {
+	if x != nil {
+		return x.Key
+	}
+	return nil
+}
+
 // A list of log entries, enables buffering and batching of multiple
 // log messages using the logging API.
 type LogEntry_List struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
 	// (Required) One or or more log messages.
-	LogEntries           []*LogEntry `protobuf:"bytes,1,rep,name=log_entries,json=logEntries,proto3" json:"log_entries,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}    `json:"-"`
-	XXX_unrecognized     []byte      `json:"-"`
-	XXX_sizecache        int32       `json:"-"`
+	LogEntries []*LogEntry `protobuf:"bytes,1,rep,name=log_entries,json=logEntries,proto3" json:"log_entries,omitempty"`
 }
 
-func (m *LogEntry_List) Reset()         { *m = LogEntry_List{} }
-func (m *LogEntry_List) String() string { return proto.CompactTextString(m) }
-func (*LogEntry_List) ProtoMessage()    {}
+func (x *LogEntry_List) Reset() {
+	*x = LogEntry_List{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_fn_api_proto_msgTypes[59]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *LogEntry_List) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*LogEntry_List) ProtoMessage() {}
+
+func (x *LogEntry_List) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_fn_api_proto_msgTypes[59]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use LogEntry_List.ProtoReflect.Descriptor instead.
 func (*LogEntry_List) Descriptor() ([]byte, []int) {
-	return fileDescriptor_6d954c03a4758710, []int{29, 0}
+	return file_beam_fn_api_proto_rawDescGZIP(), []int{29, 0}
 }
 
-func (m *LogEntry_List) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_LogEntry_List.Unmarshal(m, b)
-}
-func (m *LogEntry_List) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_LogEntry_List.Marshal(b, m, deterministic)
-}
-func (m *LogEntry_List) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_LogEntry_List.Merge(m, src)
-}
-func (m *LogEntry_List) XXX_Size() int {
-	return xxx_messageInfo_LogEntry_List.Size(m)
-}
-func (m *LogEntry_List) XXX_DiscardUnknown() {
-	xxx_messageInfo_LogEntry_List.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_LogEntry_List proto.InternalMessageInfo
-
-func (m *LogEntry_List) GetLogEntries() []*LogEntry {
-	if m != nil {
-		return m.LogEntries
+func (x *LogEntry_List) GetLogEntries() []*LogEntry {
+	if x != nil {
+		return x.LogEntries
 	}
 	return nil
 }
@@ -3016,636 +3726,1692 @@
 // can provide filtering and searching across log types. Users of the API are
 // free not to use all severity levels in their log messages.
 type LogEntry_Severity struct {
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
 }
 
-func (m *LogEntry_Severity) Reset()         { *m = LogEntry_Severity{} }
-func (m *LogEntry_Severity) String() string { return proto.CompactTextString(m) }
-func (*LogEntry_Severity) ProtoMessage()    {}
+func (x *LogEntry_Severity) Reset() {
+	*x = LogEntry_Severity{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_fn_api_proto_msgTypes[60]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *LogEntry_Severity) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*LogEntry_Severity) ProtoMessage() {}
+
+func (x *LogEntry_Severity) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_fn_api_proto_msgTypes[60]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use LogEntry_Severity.ProtoReflect.Descriptor instead.
 func (*LogEntry_Severity) Descriptor() ([]byte, []int) {
-	return fileDescriptor_6d954c03a4758710, []int{29, 1}
+	return file_beam_fn_api_proto_rawDescGZIP(), []int{29, 1}
 }
 
-func (m *LogEntry_Severity) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_LogEntry_Severity.Unmarshal(m, b)
-}
-func (m *LogEntry_Severity) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_LogEntry_Severity.Marshal(b, m, deterministic)
-}
-func (m *LogEntry_Severity) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_LogEntry_Severity.Merge(m, src)
-}
-func (m *LogEntry_Severity) XXX_Size() int {
-	return xxx_messageInfo_LogEntry_Severity.Size(m)
-}
-func (m *LogEntry_Severity) XXX_DiscardUnknown() {
-	xxx_messageInfo_LogEntry_Severity.DiscardUnknown(m)
+var File_beam_fn_api_proto protoreflect.FileDescriptor
+
+var file_beam_fn_api_proto_rawDesc = []byte{
+	0x0a, 0x11, 0x62, 0x65, 0x61, 0x6d, 0x5f, 0x66, 0x6e, 0x5f, 0x61, 0x70, 0x69, 0x2e, 0x70, 0x72,
+	0x6f, 0x74, 0x6f, 0x12, 0x25, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e,
+	0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78,
+	0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x1a, 0x15, 0x62, 0x65, 0x61, 0x6d,
+	0x5f, 0x72, 0x75, 0x6e, 0x6e, 0x65, 0x72, 0x5f, 0x61, 0x70, 0x69, 0x2e, 0x70, 0x72, 0x6f, 0x74,
+	0x6f, 0x1a, 0x0f, 0x65, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x73, 0x2e, 0x70, 0x72, 0x6f,
+	0x74, 0x6f, 0x1a, 0x20, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f,
+	0x62, 0x75, 0x66, 0x2f, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x2e, 0x70,
+	0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x1f, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2f, 0x70, 0x72, 0x6f,
+	0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x2e,
+	0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x1e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2f, 0x70, 0x72,
+	0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x64, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e,
+	0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x0d, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x2e, 0x70,
+	0x72, 0x6f, 0x74, 0x6f, 0x22, 0x9a, 0x01, 0x0a, 0x0e, 0x52, 0x65, 0x6d, 0x6f, 0x74, 0x65, 0x47,
+	0x72, 0x70, 0x63, 0x50, 0x6f, 0x72, 0x74, 0x12, 0x6d, 0x0a, 0x16, 0x61, 0x70, 0x69, 0x5f, 0x73,
+	0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x5f, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f,
+	0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x37, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70,
+	0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e,
+	0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x70, 0x69, 0x53,
+	0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72,
+	0x52, 0x14, 0x61, 0x70, 0x69, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x44, 0x65, 0x73, 0x63,
+	0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x12, 0x19, 0x0a, 0x08, 0x63, 0x6f, 0x64, 0x65, 0x72, 0x5f,
+	0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x63, 0x6f, 0x64, 0x65, 0x72, 0x49,
+	0x64, 0x22, 0x64, 0x0a, 0x21, 0x47, 0x65, 0x74, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x42,
+	0x75, 0x6e, 0x64, 0x6c, 0x65, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x52,
+	0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x3f, 0x0a, 0x1c, 0x70, 0x72, 0x6f, 0x63, 0x65, 0x73,
+	0x73, 0x5f, 0x62, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x5f, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70,
+	0x74, 0x6f, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x19, 0x70, 0x72,
+	0x6f, 0x63, 0x65, 0x73, 0x73, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x44, 0x65, 0x73, 0x63, 0x72,
+	0x69, 0x70, 0x74, 0x6f, 0x72, 0x49, 0x64, 0x22, 0xda, 0x05, 0x0a, 0x12, 0x49, 0x6e, 0x73, 0x74,
+	0x72, 0x75, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x25,
+	0x0a, 0x0e, 0x69, 0x6e, 0x73, 0x74, 0x72, 0x75, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64,
+	0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x69, 0x6e, 0x73, 0x74, 0x72, 0x75, 0x63, 0x74,
+	0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x65, 0x0a, 0x0e, 0x70, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73,
+	0x5f, 0x62, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x18, 0xe9, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3b,
+	0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d,
+	0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74,
+	0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x42, 0x75,
+	0x6e, 0x64, 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x0d, 0x70,
+	0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x12, 0x7e, 0x0a, 0x17,
+	0x70, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x5f, 0x62, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x5f, 0x70,
+	0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x18, 0xea, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x43,
+	0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d,
+	0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74,
+	0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x42, 0x75,
+	0x6e, 0x64, 0x6c, 0x65, 0x50, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x52, 0x65, 0x71, 0x75,
+	0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x15, 0x70, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x42, 0x75,
+	0x6e, 0x64, 0x6c, 0x65, 0x50, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x12, 0x75, 0x0a, 0x14,
+	0x70, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x5f, 0x62, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x5f, 0x73,
+	0x70, 0x6c, 0x69, 0x74, 0x18, 0xeb, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x40, 0x2e, 0x6f, 0x72,
+	0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f,
+	0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e,
+	0x2e, 0x76, 0x31, 0x2e, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x42, 0x75, 0x6e, 0x64, 0x6c,
+	0x65, 0x53, 0x70, 0x6c, 0x69, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52,
+	0x12, 0x70, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x53, 0x70,
+	0x6c, 0x69, 0x74, 0x12, 0x68, 0x0a, 0x0f, 0x66, 0x69, 0x6e, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x5f,
+	0x62, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x18, 0xec, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3c, 0x2e,
+	0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e,
+	0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69,
+	0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x46, 0x69, 0x6e, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x42, 0x75,
+	0x6e, 0x64, 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x0e, 0x66,
+	0x69, 0x6e, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x12, 0x73, 0x0a,
+	0x10, 0x6d, 0x6f, 0x6e, 0x69, 0x74, 0x6f, 0x72, 0x69, 0x6e, 0x67, 0x5f, 0x69, 0x6e, 0x66, 0x6f,
+	0x73, 0x18, 0xed, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x45, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61,
+	0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c,
+	0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31,
+	0x2e, 0x4d, 0x6f, 0x6e, 0x69, 0x74, 0x6f, 0x72, 0x69, 0x6e, 0x67, 0x49, 0x6e, 0x66, 0x6f, 0x73,
+	0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48,
+	0x00, 0x52, 0x0f, 0x6d, 0x6f, 0x6e, 0x69, 0x74, 0x6f, 0x72, 0x69, 0x6e, 0x67, 0x49, 0x6e, 0x66,
+	0x6f, 0x73, 0x12, 0x55, 0x0a, 0x08, 0x72, 0x65, 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, 0x18, 0xe8,
+	0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x36, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63,
+	0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e,
+	0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65,
+	0x67, 0x69, 0x73, 0x74, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52,
+	0x08, 0x72, 0x65, 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, 0x42, 0x09, 0x0a, 0x07, 0x72, 0x65, 0x71,
+	0x75, 0x65, 0x73, 0x74, 0x22, 0xf8, 0x05, 0x0a, 0x13, 0x49, 0x6e, 0x73, 0x74, 0x72, 0x75, 0x63,
+	0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x25, 0x0a, 0x0e,
+	0x69, 0x6e, 0x73, 0x74, 0x72, 0x75, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01,
+	0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x69, 0x6e, 0x73, 0x74, 0x72, 0x75, 0x63, 0x74, 0x69, 0x6f,
+	0x6e, 0x49, 0x64, 0x12, 0x14, 0x0a, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x02, 0x20, 0x01,
+	0x28, 0x09, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x12, 0x66, 0x0a, 0x0e, 0x70, 0x72, 0x6f,
+	0x63, 0x65, 0x73, 0x73, 0x5f, 0x62, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x18, 0xe9, 0x07, 0x20, 0x01,
+	0x28, 0x0b, 0x32, 0x3c, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e,
+	0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78,
+	0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x72, 0x6f, 0x63, 0x65,
+	0x73, 0x73, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65,
+	0x48, 0x00, 0x52, 0x0d, 0x70, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x42, 0x75, 0x6e, 0x64, 0x6c,
+	0x65, 0x12, 0x7f, 0x0a, 0x17, 0x70, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x5f, 0x62, 0x75, 0x6e,
+	0x64, 0x6c, 0x65, 0x5f, 0x70, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x18, 0xea, 0x07, 0x20,
+	0x01, 0x28, 0x0b, 0x32, 0x44, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65,
+	0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65,
+	0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x72, 0x6f, 0x63,
+	0x65, 0x73, 0x73, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x50, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73,
+	0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x48, 0x00, 0x52, 0x15, 0x70, 0x72, 0x6f,
+	0x63, 0x65, 0x73, 0x73, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x50, 0x72, 0x6f, 0x67, 0x72, 0x65,
+	0x73, 0x73, 0x12, 0x76, 0x0a, 0x14, 0x70, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x5f, 0x62, 0x75,
+	0x6e, 0x64, 0x6c, 0x65, 0x5f, 0x73, 0x70, 0x6c, 0x69, 0x74, 0x18, 0xeb, 0x07, 0x20, 0x01, 0x28,
+	0x0b, 0x32, 0x41, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62,
+	0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65,
+	0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73,
+	0x73, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x53, 0x70, 0x6c, 0x69, 0x74, 0x52, 0x65, 0x73, 0x70,
+	0x6f, 0x6e, 0x73, 0x65, 0x48, 0x00, 0x52, 0x12, 0x70, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x42,
+	0x75, 0x6e, 0x64, 0x6c, 0x65, 0x53, 0x70, 0x6c, 0x69, 0x74, 0x12, 0x69, 0x0a, 0x0f, 0x66, 0x69,
+	0x6e, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x5f, 0x62, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x18, 0xec, 0x07,
+	0x20, 0x01, 0x28, 0x0b, 0x32, 0x3d, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68,
+	0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f,
+	0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x46, 0x69, 0x6e,
+	0x61, 0x6c, 0x69, 0x7a, 0x65, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f,
+	0x6e, 0x73, 0x65, 0x48, 0x00, 0x52, 0x0e, 0x66, 0x69, 0x6e, 0x61, 0x6c, 0x69, 0x7a, 0x65, 0x42,
+	0x75, 0x6e, 0x64, 0x6c, 0x65, 0x12, 0x74, 0x0a, 0x10, 0x6d, 0x6f, 0x6e, 0x69, 0x74, 0x6f, 0x72,
+	0x69, 0x6e, 0x67, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x73, 0x18, 0xed, 0x07, 0x20, 0x01, 0x28, 0x0b,
+	0x32, 0x46, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65,
+	0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65, 0x63,
+	0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x4d, 0x6f, 0x6e, 0x69, 0x74, 0x6f, 0x72,
+	0x69, 0x6e, 0x67, 0x49, 0x6e, 0x66, 0x6f, 0x73, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61,
+	0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x48, 0x00, 0x52, 0x0f, 0x6d, 0x6f, 0x6e, 0x69,
+	0x74, 0x6f, 0x72, 0x69, 0x6e, 0x67, 0x49, 0x6e, 0x66, 0x6f, 0x73, 0x12, 0x56, 0x0a, 0x08, 0x72,
+	0x65, 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, 0x18, 0xe8, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x37,
+	0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d,
+	0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74,
+	0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, 0x52,
+	0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x48, 0x00, 0x52, 0x08, 0x72, 0x65, 0x67, 0x69, 0x73,
+	0x74, 0x65, 0x72, 0x42, 0x0a, 0x0a, 0x08, 0x72, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22,
+	0x8d, 0x01, 0x0a, 0x0f, 0x52, 0x65, 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75,
+	0x65, 0x73, 0x74, 0x12, 0x7a, 0x0a, 0x19, 0x70, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x5f, 0x62,
+	0x75, 0x6e, 0x64, 0x6c, 0x65, 0x5f, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72,
+	0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x3e, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61,
+	0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66,
+	0x6e, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x50,
+	0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x44, 0x65, 0x73, 0x63,
+	0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x52, 0x17, 0x70, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x42,
+	0x75, 0x6e, 0x64, 0x6c, 0x65, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x22,
+	0x12, 0x0a, 0x10, 0x52, 0x65, 0x67, 0x69, 0x73, 0x74, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f,
+	0x6e, 0x73, 0x65, 0x22, 0x9d, 0x0b, 0x0a, 0x17, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x42,
+	0x75, 0x6e, 0x64, 0x6c, 0x65, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x12,
+	0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, 0x12,
+	0x6e, 0x0a, 0x0a, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x73, 0x18, 0x02, 0x20,
+	0x03, 0x28, 0x0b, 0x32, 0x4e, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65,
+	0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65,
+	0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x72, 0x6f, 0x63,
+	0x65, 0x73, 0x73, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70,
+	0x74, 0x6f, 0x72, 0x2e, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x73, 0x45, 0x6e,
+	0x74, 0x72, 0x79, 0x52, 0x0a, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x73, 0x12,
+	0x74, 0x0a, 0x0c, 0x70, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18,
+	0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x50, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63,
+	0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e,
+	0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x72,
+	0x6f, 0x63, 0x65, 0x73, 0x73, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x44, 0x65, 0x73, 0x63, 0x72,
+	0x69, 0x70, 0x74, 0x6f, 0x72, 0x2e, 0x50, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f,
+	0x6e, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0c, 0x70, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63,
+	0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x8a, 0x01, 0x0a, 0x14, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77,
+	0x69, 0x6e, 0x67, 0x5f, 0x73, 0x74, 0x72, 0x61, 0x74, 0x65, 0x67, 0x69, 0x65, 0x73, 0x18, 0x04,
+	0x20, 0x03, 0x28, 0x0b, 0x32, 0x57, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68,
+	0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f,
+	0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x72, 0x6f,
+	0x63, 0x65, 0x73, 0x73, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69,
+	0x70, 0x74, 0x6f, 0x72, 0x2e, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x69, 0x6e, 0x67, 0x53, 0x74,
+	0x72, 0x61, 0x74, 0x65, 0x67, 0x69, 0x65, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x13, 0x77,
+	0x69, 0x6e, 0x64, 0x6f, 0x77, 0x69, 0x6e, 0x67, 0x53, 0x74, 0x72, 0x61, 0x74, 0x65, 0x67, 0x69,
+	0x65, 0x73, 0x12, 0x62, 0x0a, 0x06, 0x63, 0x6f, 0x64, 0x65, 0x72, 0x73, 0x18, 0x05, 0x20, 0x03,
+	0x28, 0x0b, 0x32, 0x4a, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e,
+	0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78,
+	0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x72, 0x6f, 0x63, 0x65,
+	0x73, 0x73, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74,
+	0x6f, 0x72, 0x2e, 0x43, 0x6f, 0x64, 0x65, 0x72, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x06,
+	0x63, 0x6f, 0x64, 0x65, 0x72, 0x73, 0x12, 0x74, 0x0a, 0x0c, 0x65, 0x6e, 0x76, 0x69, 0x72, 0x6f,
+	0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x06, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x50, 0x2e, 0x6f,
+	0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d,
+	0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f,
+	0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x42, 0x75, 0x6e, 0x64,
+	0x6c, 0x65, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x2e, 0x45, 0x6e, 0x76,
+	0x69, 0x72, 0x6f, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0c,
+	0x65, 0x6e, 0x76, 0x69, 0x72, 0x6f, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x12, 0x78, 0x0a, 0x1c,
+	0x73, 0x74, 0x61, 0x74, 0x65, 0x5f, 0x61, 0x70, 0x69, 0x5f, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63,
+	0x65, 0x5f, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x18, 0x07, 0x20, 0x01,
+	0x28, 0x0b, 0x32, 0x37, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e,
+	0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c,
+	0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x70, 0x69, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63,
+	0x65, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x52, 0x19, 0x73, 0x74, 0x61,
+	0x74, 0x65, 0x41, 0x70, 0x69, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x44, 0x65, 0x73, 0x63,
+	0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x12, 0x78, 0x0a, 0x1c, 0x74, 0x69, 0x6d, 0x65, 0x72, 0x5f,
+	0x61, 0x70, 0x69, 0x5f, 0x73, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x5f, 0x64, 0x65, 0x73, 0x63,
+	0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x37, 0x2e, 0x6f,
+	0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d,
+	0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31,
+	0x2e, 0x41, 0x70, 0x69, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x44, 0x65, 0x73, 0x63, 0x72,
+	0x69, 0x70, 0x74, 0x6f, 0x72, 0x52, 0x19, 0x74, 0x69, 0x6d, 0x65, 0x72, 0x41, 0x70, 0x69, 0x53,
+	0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72,
+	0x1a, 0x6c, 0x0a, 0x0f, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x73, 0x45, 0x6e,
+	0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09,
+	0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x43, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02,
+	0x20, 0x01, 0x28, 0x0b, 0x32, 0x2d, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68,
+	0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70,
+	0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x66,
+	0x6f, 0x72, 0x6d, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x1a, 0x6f,
+	0x0a, 0x11, 0x50, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x45, 0x6e,
+	0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09,
+	0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x44, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02,
+	0x20, 0x01, 0x28, 0x0b, 0x32, 0x2e, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68,
+	0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70,
+	0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63,
+	0x74, 0x69, 0x6f, 0x6e, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x1a,
+	0x7c, 0x0a, 0x18, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x69, 0x6e, 0x67, 0x53, 0x74, 0x72, 0x61,
+	0x74, 0x65, 0x67, 0x69, 0x65, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b,
+	0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x4a, 0x0a,
+	0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x34, 0x2e, 0x6f,
+	0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d,
+	0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31,
+	0x2e, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x69, 0x6e, 0x67, 0x53, 0x74, 0x72, 0x61, 0x74, 0x65,
+	0x67, 0x79, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x1a, 0x63, 0x0a,
+	0x0b, 0x43, 0x6f, 0x64, 0x65, 0x72, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03,
+	0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x3e,
+	0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x28, 0x2e,
+	0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e,
+	0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76,
+	0x31, 0x2e, 0x43, 0x6f, 0x64, 0x65, 0x72, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02,
+	0x38, 0x01, 0x1a, 0x6f, 0x0a, 0x11, 0x45, 0x6e, 0x76, 0x69, 0x72, 0x6f, 0x6e, 0x6d, 0x65, 0x6e,
+	0x74, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01,
+	0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x44, 0x0a, 0x05, 0x76, 0x61, 0x6c,
+	0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2e, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61,
+	0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c,
+	0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x45, 0x6e, 0x76,
+	0x69, 0x72, 0x6f, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a,
+	0x02, 0x38, 0x01, 0x22, 0x9b, 0x03, 0x0a, 0x11, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x41, 0x70,
+	0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x21, 0x0a, 0x0c, 0x74, 0x72, 0x61,
+	0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52,
+	0x0b, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x49, 0x64, 0x12, 0x19, 0x0a, 0x08,
+	0x69, 0x6e, 0x70, 0x75, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07,
+	0x69, 0x6e, 0x70, 0x75, 0x74, 0x49, 0x64, 0x12, 0x18, 0x0a, 0x07, 0x65, 0x6c, 0x65, 0x6d, 0x65,
+	0x6e, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x07, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e,
+	0x74, 0x12, 0x7b, 0x0a, 0x11, 0x6f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x5f, 0x77, 0x61, 0x74, 0x65,
+	0x72, 0x6d, 0x61, 0x72, 0x6b, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x4e, 0x2e, 0x6f,
+	0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d,
+	0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f,
+	0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x41, 0x70, 0x70, 0x6c, 0x69,
+	0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x57, 0x61, 0x74,
+	0x65, 0x72, 0x6d, 0x61, 0x72, 0x6b, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x10, 0x6f, 0x75,
+	0x74, 0x70, 0x75, 0x74, 0x57, 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, 0x6b, 0x73, 0x12, 0x50,
+	0x0a, 0x0a, 0x69, 0x73, 0x5f, 0x62, 0x6f, 0x75, 0x6e, 0x64, 0x65, 0x64, 0x18, 0x05, 0x20, 0x01,
+	0x28, 0x0e, 0x32, 0x31, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e,
+	0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c,
+	0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x49, 0x73, 0x42, 0x6f, 0x75, 0x6e, 0x64, 0x65, 0x64,
+	0x2e, 0x45, 0x6e, 0x75, 0x6d, 0x52, 0x09, 0x69, 0x73, 0x42, 0x6f, 0x75, 0x6e, 0x64, 0x65, 0x64,
+	0x1a, 0x5f, 0x0a, 0x15, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x57, 0x61, 0x74, 0x65, 0x72, 0x6d,
+	0x61, 0x72, 0x6b, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79,
+	0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x30, 0x0a, 0x05, 0x76,
+	0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f,
+	0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d,
+	0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38,
+	0x01, 0x22, 0xc3, 0x01, 0x0a, 0x18, 0x44, 0x65, 0x6c, 0x61, 0x79, 0x65, 0x64, 0x42, 0x75, 0x6e,
+	0x64, 0x6c, 0x65, 0x41, 0x70, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x5a,
+	0x0a, 0x0b, 0x61, 0x70, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20,
+	0x01, 0x28, 0x0b, 0x32, 0x38, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65,
+	0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65,
+	0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x42, 0x75, 0x6e, 0x64,
+	0x6c, 0x65, 0x41, 0x70, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0b, 0x61,
+	0x70, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x4b, 0x0a, 0x14, 0x72, 0x65,
+	0x71, 0x75, 0x65, 0x73, 0x74, 0x65, 0x64, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x5f, 0x64, 0x65, 0x6c,
+	0x61, 0x79, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c,
+	0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74,
+	0x69, 0x6f, 0x6e, 0x52, 0x12, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x65, 0x64, 0x54, 0x69,
+	0x6d, 0x65, 0x44, 0x65, 0x6c, 0x61, 0x79, 0x22, 0xb6, 0x04, 0x0a, 0x14, 0x50, 0x72, 0x6f, 0x63,
+	0x65, 0x73, 0x73, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74,
+	0x12, 0x3f, 0x0a, 0x1c, 0x70, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x5f, 0x62, 0x75, 0x6e, 0x64,
+	0x6c, 0x65, 0x5f, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x5f, 0x69, 0x64,
+	0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x19, 0x70, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x42,
+	0x75, 0x6e, 0x64, 0x6c, 0x65, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x49,
+	0x64, 0x12, 0x69, 0x0a, 0x0c, 0x63, 0x61, 0x63, 0x68, 0x65, 0x5f, 0x74, 0x6f, 0x6b, 0x65, 0x6e,
+	0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x46, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70,
+	0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e,
+	0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e,
+	0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x52, 0x65, 0x71,
+	0x75, 0x65, 0x73, 0x74, 0x2e, 0x43, 0x61, 0x63, 0x68, 0x65, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x52,
+	0x0b, 0x63, 0x61, 0x63, 0x68, 0x65, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x73, 0x1a, 0xf1, 0x02, 0x0a,
+	0x0a, 0x43, 0x61, 0x63, 0x68, 0x65, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x12, 0x71, 0x0a, 0x0a, 0x75,
+	0x73, 0x65, 0x72, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32,
+	0x50, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61,
+	0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75,
+	0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x42,
+	0x75, 0x6e, 0x64, 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x43, 0x61, 0x63,
+	0x68, 0x65, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x2e, 0x55, 0x73, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74,
+	0x65, 0x48, 0x00, 0x52, 0x09, 0x75, 0x73, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x71,
+	0x0a, 0x0a, 0x73, 0x69, 0x64, 0x65, 0x5f, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x18, 0x02, 0x20, 0x01,
+	0x28, 0x0b, 0x32, 0x50, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e,
+	0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78,
+	0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x72, 0x6f, 0x63, 0x65,
+	0x73, 0x73, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e,
+	0x43, 0x61, 0x63, 0x68, 0x65, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x2e, 0x53, 0x69, 0x64, 0x65, 0x49,
+	0x6e, 0x70, 0x75, 0x74, 0x48, 0x00, 0x52, 0x09, 0x73, 0x69, 0x64, 0x65, 0x49, 0x6e, 0x70, 0x75,
+	0x74, 0x12, 0x14, 0x0a, 0x05, 0x74, 0x6f, 0x6b, 0x65, 0x6e, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x0c,
+	0x52, 0x05, 0x74, 0x6f, 0x6b, 0x65, 0x6e, 0x1a, 0x0b, 0x0a, 0x09, 0x55, 0x73, 0x65, 0x72, 0x53,
+	0x74, 0x61, 0x74, 0x65, 0x1a, 0x52, 0x0a, 0x09, 0x53, 0x69, 0x64, 0x65, 0x49, 0x6e, 0x70, 0x75,
+	0x74, 0x12, 0x21, 0x0a, 0x0c, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x5f, 0x69,
+	0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f,
+	0x72, 0x6d, 0x49, 0x64, 0x12, 0x22, 0x0a, 0x0d, 0x73, 0x69, 0x64, 0x65, 0x5f, 0x69, 0x6e, 0x70,
+	0x75, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x73, 0x69, 0x64,
+	0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x49, 0x64, 0x42, 0x06, 0x0a, 0x04, 0x74, 0x79, 0x70, 0x65,
+	0x22, 0xd6, 0x03, 0x0a, 0x15, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x42, 0x75, 0x6e, 0x64,
+	0x6c, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x66, 0x0a, 0x0e, 0x72, 0x65,
+	0x73, 0x69, 0x64, 0x75, 0x61, 0x6c, 0x5f, 0x72, 0x6f, 0x6f, 0x74, 0x73, 0x18, 0x02, 0x20, 0x03,
+	0x28, 0x0b, 0x32, 0x3f, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e,
+	0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78,
+	0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x44, 0x65, 0x6c, 0x61, 0x79,
+	0x65, 0x64, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x41, 0x70, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74,
+	0x69, 0x6f, 0x6e, 0x52, 0x0d, 0x72, 0x65, 0x73, 0x69, 0x64, 0x75, 0x61, 0x6c, 0x52, 0x6f, 0x6f,
+	0x74, 0x73, 0x12, 0x5c, 0x0a, 0x10, 0x6d, 0x6f, 0x6e, 0x69, 0x74, 0x6f, 0x72, 0x69, 0x6e, 0x67,
+	0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x6f,
+	0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d,
+	0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31,
+	0x2e, 0x4d, 0x6f, 0x6e, 0x69, 0x74, 0x6f, 0x72, 0x69, 0x6e, 0x67, 0x49, 0x6e, 0x66, 0x6f, 0x52,
+	0x0f, 0x6d, 0x6f, 0x6e, 0x69, 0x74, 0x6f, 0x72, 0x69, 0x6e, 0x67, 0x49, 0x6e, 0x66, 0x6f, 0x73,
+	0x12, 0x33, 0x0a, 0x15, 0x72, 0x65, 0x71, 0x75, 0x69, 0x72, 0x65, 0x73, 0x5f, 0x66, 0x69, 0x6e,
+	0x61, 0x6c, 0x69, 0x7a, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52,
+	0x14, 0x72, 0x65, 0x71, 0x75, 0x69, 0x72, 0x65, 0x73, 0x46, 0x69, 0x6e, 0x61, 0x6c, 0x69, 0x7a,
+	0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x79, 0x0a, 0x0f, 0x6d, 0x6f, 0x6e, 0x69, 0x74, 0x6f, 0x72,
+	0x69, 0x6e, 0x67, 0x5f, 0x64, 0x61, 0x74, 0x61, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x50,
+	0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d,
+	0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74,
+	0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x42, 0x75,
+	0x6e, 0x64, 0x6c, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x4d, 0x6f, 0x6e,
+	0x69, 0x74, 0x6f, 0x72, 0x69, 0x6e, 0x67, 0x44, 0x61, 0x74, 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79,
+	0x52, 0x0e, 0x6d, 0x6f, 0x6e, 0x69, 0x74, 0x6f, 0x72, 0x69, 0x6e, 0x67, 0x44, 0x61, 0x74, 0x61,
+	0x1a, 0x41, 0x0a, 0x13, 0x4d, 0x6f, 0x6e, 0x69, 0x74, 0x6f, 0x72, 0x69, 0x6e, 0x67, 0x44, 0x61,
+	0x74, 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01,
+	0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c,
+	0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a,
+	0x02, 0x38, 0x01, 0x4a, 0x04, 0x08, 0x01, 0x10, 0x02, 0x22, 0x45, 0x0a, 0x1c, 0x50, 0x72, 0x6f,
+	0x63, 0x65, 0x73, 0x73, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x50, 0x72, 0x6f, 0x67, 0x72, 0x65,
+	0x73, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x25, 0x0a, 0x0e, 0x69, 0x6e, 0x73,
+	0x74, 0x72, 0x75, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28,
+	0x09, 0x52, 0x0d, 0x69, 0x6e, 0x73, 0x74, 0x72, 0x75, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64,
+	0x22, 0x4e, 0x0a, 0x1e, 0x4d, 0x6f, 0x6e, 0x69, 0x74, 0x6f, 0x72, 0x69, 0x6e, 0x67, 0x49, 0x6e,
+	0x66, 0x6f, 0x73, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x52, 0x65, 0x71, 0x75, 0x65,
+	0x73, 0x74, 0x12, 0x2c, 0x0a, 0x12, 0x6d, 0x6f, 0x6e, 0x69, 0x74, 0x6f, 0x72, 0x69, 0x6e, 0x67,
+	0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x03, 0x28, 0x09, 0x52, 0x10,
+	0x6d, 0x6f, 0x6e, 0x69, 0x74, 0x6f, 0x72, 0x69, 0x6e, 0x67, 0x49, 0x6e, 0x66, 0x6f, 0x49, 0x64,
+	0x22, 0xd6, 0x02, 0x0a, 0x1d, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x42, 0x75, 0x6e, 0x64,
+	0x6c, 0x65, 0x50, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e,
+	0x73, 0x65, 0x12, 0x5c, 0x0a, 0x10, 0x6d, 0x6f, 0x6e, 0x69, 0x74, 0x6f, 0x72, 0x69, 0x6e, 0x67,
+	0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x6f,
+	0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d,
+	0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31,
+	0x2e, 0x4d, 0x6f, 0x6e, 0x69, 0x74, 0x6f, 0x72, 0x69, 0x6e, 0x67, 0x49, 0x6e, 0x66, 0x6f, 0x52,
+	0x0f, 0x6d, 0x6f, 0x6e, 0x69, 0x74, 0x6f, 0x72, 0x69, 0x6e, 0x67, 0x49, 0x6e, 0x66, 0x6f, 0x73,
+	0x12, 0x81, 0x01, 0x0a, 0x0f, 0x6d, 0x6f, 0x6e, 0x69, 0x74, 0x6f, 0x72, 0x69, 0x6e, 0x67, 0x5f,
+	0x64, 0x61, 0x74, 0x61, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x58, 0x2e, 0x6f, 0x72, 0x67,
+	0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64,
+	0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e,
+	0x76, 0x31, 0x2e, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65,
+	0x50, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65,
+	0x2e, 0x4d, 0x6f, 0x6e, 0x69, 0x74, 0x6f, 0x72, 0x69, 0x6e, 0x67, 0x44, 0x61, 0x74, 0x61, 0x45,
+	0x6e, 0x74, 0x72, 0x79, 0x52, 0x0e, 0x6d, 0x6f, 0x6e, 0x69, 0x74, 0x6f, 0x72, 0x69, 0x6e, 0x67,
+	0x44, 0x61, 0x74, 0x61, 0x1a, 0x41, 0x0a, 0x13, 0x4d, 0x6f, 0x6e, 0x69, 0x74, 0x6f, 0x72, 0x69,
+	0x6e, 0x67, 0x44, 0x61, 0x74, 0x61, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b,
+	0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a,
+	0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x05, 0x76, 0x61,
+	0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x4a, 0x04, 0x08, 0x01, 0x10, 0x02, 0x4a, 0x04, 0x08,
+	0x02, 0x10, 0x03, 0x4a, 0x04, 0x08, 0x04, 0x10, 0x05, 0x22, 0x9d, 0x02, 0x0a, 0x1f, 0x4d, 0x6f,
+	0x6e, 0x69, 0x74, 0x6f, 0x72, 0x69, 0x6e, 0x67, 0x49, 0x6e, 0x66, 0x6f, 0x73, 0x4d, 0x65, 0x74,
+	0x61, 0x64, 0x61, 0x74, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x83, 0x01,
+	0x0a, 0x0f, 0x6d, 0x6f, 0x6e, 0x69, 0x74, 0x6f, 0x72, 0x69, 0x6e, 0x67, 0x5f, 0x69, 0x6e, 0x66,
+	0x6f, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x5a, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70,
+	0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e,
+	0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e,
+	0x4d, 0x6f, 0x6e, 0x69, 0x74, 0x6f, 0x72, 0x69, 0x6e, 0x67, 0x49, 0x6e, 0x66, 0x6f, 0x73, 0x4d,
+	0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e,
+	0x4d, 0x6f, 0x6e, 0x69, 0x74, 0x6f, 0x72, 0x69, 0x6e, 0x67, 0x49, 0x6e, 0x66, 0x6f, 0x45, 0x6e,
+	0x74, 0x72, 0x79, 0x52, 0x0e, 0x6d, 0x6f, 0x6e, 0x69, 0x74, 0x6f, 0x72, 0x69, 0x6e, 0x67, 0x49,
+	0x6e, 0x66, 0x6f, 0x1a, 0x74, 0x0a, 0x13, 0x4d, 0x6f, 0x6e, 0x69, 0x74, 0x6f, 0x72, 0x69, 0x6e,
+	0x67, 0x49, 0x6e, 0x66, 0x6f, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65,
+	0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x47, 0x0a, 0x05,
+	0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x6f, 0x72,
+	0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f,
+	0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e,
+	0x4d, 0x6f, 0x6e, 0x69, 0x74, 0x6f, 0x72, 0x69, 0x6e, 0x67, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x05,
+	0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x81, 0x04, 0x0a, 0x19, 0x50, 0x72,
+	0x6f, 0x63, 0x65, 0x73, 0x73, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x53, 0x70, 0x6c, 0x69, 0x74,
+	0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x25, 0x0a, 0x0e, 0x69, 0x6e, 0x73, 0x74, 0x72,
+	0x75, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52,
+	0x0d, 0x69, 0x6e, 0x73, 0x74, 0x72, 0x75, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x7a,
+	0x0a, 0x0e, 0x64, 0x65, 0x73, 0x69, 0x72, 0x65, 0x64, 0x5f, 0x73, 0x70, 0x6c, 0x69, 0x74, 0x73,
+	0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x53, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61,
+	0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66,
+	0x6e, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x50,
+	0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x53, 0x70, 0x6c, 0x69,
+	0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x44, 0x65, 0x73, 0x69, 0x72, 0x65, 0x64,
+	0x53, 0x70, 0x6c, 0x69, 0x74, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0d, 0x64, 0x65, 0x73,
+	0x69, 0x72, 0x65, 0x64, 0x53, 0x70, 0x6c, 0x69, 0x74, 0x73, 0x1a, 0xae, 0x01, 0x0a, 0x0c, 0x44,
+	0x65, 0x73, 0x69, 0x72, 0x65, 0x64, 0x53, 0x70, 0x6c, 0x69, 0x74, 0x12, 0x32, 0x0a, 0x15, 0x66,
+	0x72, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6f, 0x66, 0x5f, 0x72, 0x65, 0x6d, 0x61, 0x69,
+	0x6e, 0x64, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x01, 0x52, 0x13, 0x66, 0x72, 0x61, 0x63,
+	0x74, 0x69, 0x6f, 0x6e, 0x4f, 0x66, 0x52, 0x65, 0x6d, 0x61, 0x69, 0x6e, 0x64, 0x65, 0x72, 0x12,
+	0x30, 0x0a, 0x14, 0x61, 0x6c, 0x6c, 0x6f, 0x77, 0x65, 0x64, 0x5f, 0x73, 0x70, 0x6c, 0x69, 0x74,
+	0x5f, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x03, 0x52, 0x12, 0x61,
+	0x6c, 0x6c, 0x6f, 0x77, 0x65, 0x64, 0x53, 0x70, 0x6c, 0x69, 0x74, 0x50, 0x6f, 0x69, 0x6e, 0x74,
+	0x73, 0x12, 0x38, 0x0a, 0x18, 0x65, 0x73, 0x74, 0x69, 0x6d, 0x61, 0x74, 0x65, 0x64, 0x5f, 0x69,
+	0x6e, 0x70, 0x75, 0x74, 0x5f, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x02, 0x20,
+	0x01, 0x28, 0x03, 0x52, 0x16, 0x65, 0x73, 0x74, 0x69, 0x6d, 0x61, 0x74, 0x65, 0x64, 0x49, 0x6e,
+	0x70, 0x75, 0x74, 0x45, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x1a, 0x8f, 0x01, 0x0a, 0x12,
+	0x44, 0x65, 0x73, 0x69, 0x72, 0x65, 0x64, 0x53, 0x70, 0x6c, 0x69, 0x74, 0x73, 0x45, 0x6e, 0x74,
+	0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52,
+	0x03, 0x6b, 0x65, 0x79, 0x12, 0x63, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20,
+	0x01, 0x28, 0x0b, 0x32, 0x4d, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65,
+	0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65,
+	0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x72, 0x6f, 0x63,
+	0x65, 0x73, 0x73, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x53, 0x70, 0x6c, 0x69, 0x74, 0x52, 0x65,
+	0x71, 0x75, 0x65, 0x73, 0x74, 0x2e, 0x44, 0x65, 0x73, 0x69, 0x72, 0x65, 0x64, 0x53, 0x70, 0x6c,
+	0x69, 0x74, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0xf6, 0x03,
+	0x0a, 0x1a, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x53,
+	0x70, 0x6c, 0x69, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x5d, 0x0a, 0x0d,
+	0x70, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x5f, 0x72, 0x6f, 0x6f, 0x74, 0x73, 0x18, 0x01, 0x20,
+	0x03, 0x28, 0x0b, 0x32, 0x38, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65,
+	0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65,
+	0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x42, 0x75, 0x6e, 0x64,
+	0x6c, 0x65, 0x41, 0x70, 0x70, 0x6c, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0c, 0x70,
+	0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x52, 0x6f, 0x6f, 0x74, 0x73, 0x12, 0x66, 0x0a, 0x0e, 0x72,
+	0x65, 0x73, 0x69, 0x64, 0x75, 0x61, 0x6c, 0x5f, 0x72, 0x6f, 0x6f, 0x74, 0x73, 0x18, 0x02, 0x20,
+	0x03, 0x28, 0x0b, 0x32, 0x3f, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65,
+	0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65,
+	0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x44, 0x65, 0x6c, 0x61,
+	0x79, 0x65, 0x64, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x41, 0x70, 0x70, 0x6c, 0x69, 0x63, 0x61,
+	0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0d, 0x72, 0x65, 0x73, 0x69, 0x64, 0x75, 0x61, 0x6c, 0x52, 0x6f,
+	0x6f, 0x74, 0x73, 0x12, 0x75, 0x0a, 0x0e, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x5f, 0x73,
+	0x70, 0x6c, 0x69, 0x74, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x4e, 0x2e, 0x6f, 0x72,
+	0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f,
+	0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e,
+	0x2e, 0x76, 0x31, 0x2e, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x42, 0x75, 0x6e, 0x64, 0x6c,
+	0x65, 0x53, 0x70, 0x6c, 0x69, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x2e, 0x43,
+	0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x53, 0x70, 0x6c, 0x69, 0x74, 0x52, 0x0d, 0x63, 0x68, 0x61,
+	0x6e, 0x6e, 0x65, 0x6c, 0x53, 0x70, 0x6c, 0x69, 0x74, 0x73, 0x1a, 0x99, 0x01, 0x0a, 0x0c, 0x43,
+	0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x53, 0x70, 0x6c, 0x69, 0x74, 0x12, 0x21, 0x0a, 0x0c, 0x74,
+	0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28,
+	0x09, 0x52, 0x0b, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x49, 0x64, 0x12, 0x30,
+	0x0a, 0x14, 0x6c, 0x61, 0x73, 0x74, 0x5f, 0x70, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x5f, 0x65,
+	0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x12, 0x6c, 0x61,
+	0x73, 0x74, 0x50, 0x72, 0x69, 0x6d, 0x61, 0x72, 0x79, 0x45, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74,
+	0x12, 0x34, 0x0a, 0x16, 0x66, 0x69, 0x72, 0x73, 0x74, 0x5f, 0x72, 0x65, 0x73, 0x69, 0x64, 0x75,
+	0x61, 0x6c, 0x5f, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03,
+	0x52, 0x14, 0x66, 0x69, 0x72, 0x73, 0x74, 0x52, 0x65, 0x73, 0x69, 0x64, 0x75, 0x61, 0x6c, 0x45,
+	0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x22, 0x3e, 0x0a, 0x15, 0x46, 0x69, 0x6e, 0x61, 0x6c, 0x69,
+	0x7a, 0x65, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12,
+	0x25, 0x0a, 0x0e, 0x69, 0x6e, 0x73, 0x74, 0x72, 0x75, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69,
+	0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x69, 0x6e, 0x73, 0x74, 0x72, 0x75, 0x63,
+	0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x22, 0x18, 0x0a, 0x16, 0x46, 0x69, 0x6e, 0x61, 0x6c, 0x69,
+	0x7a, 0x65, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65,
+	0x22, 0xd1, 0x03, 0x0a, 0x08, 0x45, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x12, 0x48, 0x0a,
+	0x04, 0x64, 0x61, 0x74, 0x61, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x34, 0x2e, 0x6f, 0x72,
+	0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f,
+	0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e,
+	0x2e, 0x76, 0x31, 0x2e, 0x45, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x2e, 0x44, 0x61, 0x74,
+	0x61, 0x52, 0x04, 0x64, 0x61, 0x74, 0x61, 0x12, 0x4e, 0x0a, 0x06, 0x74, 0x69, 0x6d, 0x65, 0x72,
+	0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x36, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70,
+	0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e,
+	0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e,
+	0x45, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x72, 0x73, 0x52,
+	0x06, 0x74, 0x69, 0x6d, 0x65, 0x72, 0x73, 0x1a, 0x7d, 0x0a, 0x04, 0x44, 0x61, 0x74, 0x61, 0x12,
+	0x25, 0x0a, 0x0e, 0x69, 0x6e, 0x73, 0x74, 0x72, 0x75, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69,
+	0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x69, 0x6e, 0x73, 0x74, 0x72, 0x75, 0x63,
+	0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x21, 0x0a, 0x0c, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66,
+	0x6f, 0x72, 0x6d, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x74, 0x72,
+	0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x49, 0x64, 0x12, 0x12, 0x0a, 0x04, 0x64, 0x61, 0x74,
+	0x61, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x04, 0x64, 0x61, 0x74, 0x61, 0x12, 0x17, 0x0a,
+	0x07, 0x69, 0x73, 0x5f, 0x6c, 0x61, 0x73, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x06,
+	0x69, 0x73, 0x4c, 0x61, 0x73, 0x74, 0x1a, 0xab, 0x01, 0x0a, 0x06, 0x54, 0x69, 0x6d, 0x65, 0x72,
+	0x73, 0x12, 0x25, 0x0a, 0x0e, 0x69, 0x6e, 0x73, 0x74, 0x72, 0x75, 0x63, 0x74, 0x69, 0x6f, 0x6e,
+	0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x69, 0x6e, 0x73, 0x74, 0x72,
+	0x75, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x21, 0x0a, 0x0c, 0x74, 0x72, 0x61, 0x6e,
+	0x73, 0x66, 0x6f, 0x72, 0x6d, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b,
+	0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x49, 0x64, 0x12, 0x26, 0x0a, 0x0f, 0x74,
+	0x69, 0x6d, 0x65, 0x72, 0x5f, 0x66, 0x61, 0x6d, 0x69, 0x6c, 0x79, 0x5f, 0x69, 0x64, 0x18, 0x03,
+	0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x74, 0x69, 0x6d, 0x65, 0x72, 0x46, 0x61, 0x6d, 0x69, 0x6c,
+	0x79, 0x49, 0x64, 0x12, 0x16, 0x0a, 0x06, 0x74, 0x69, 0x6d, 0x65, 0x72, 0x73, 0x18, 0x04, 0x20,
+	0x01, 0x28, 0x0c, 0x52, 0x06, 0x74, 0x69, 0x6d, 0x65, 0x72, 0x73, 0x12, 0x17, 0x0a, 0x07, 0x69,
+	0x73, 0x5f, 0x6c, 0x61, 0x73, 0x74, 0x18, 0x05, 0x20, 0x01, 0x28, 0x08, 0x52, 0x06, 0x69, 0x73,
+	0x4c, 0x61, 0x73, 0x74, 0x22, 0x94, 0x03, 0x0a, 0x0c, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65,
+	0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28,
+	0x09, 0x52, 0x02, 0x69, 0x64, 0x12, 0x25, 0x0a, 0x0e, 0x69, 0x6e, 0x73, 0x74, 0x72, 0x75, 0x63,
+	0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x69,
+	0x6e, 0x73, 0x74, 0x72, 0x75, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12, 0x4c, 0x0a, 0x09,
+	0x73, 0x74, 0x61, 0x74, 0x65, 0x5f, 0x6b, 0x65, 0x79, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32,
+	0x2f, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61,
+	0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75,
+	0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x4b, 0x65, 0x79,
+	0x52, 0x08, 0x73, 0x74, 0x61, 0x74, 0x65, 0x4b, 0x65, 0x79, 0x12, 0x4b, 0x0a, 0x03, 0x67, 0x65,
+	0x74, 0x18, 0xe8, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x36, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61,
+	0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c,
+	0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31,
+	0x2e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x47, 0x65, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74,
+	0x48, 0x00, 0x52, 0x03, 0x67, 0x65, 0x74, 0x12, 0x54, 0x0a, 0x06, 0x61, 0x70, 0x70, 0x65, 0x6e,
+	0x64, 0x18, 0xe9, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x39, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61,
+	0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c,
+	0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31,
+	0x2e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x41, 0x70, 0x70, 0x65, 0x6e, 0x64, 0x52, 0x65, 0x71, 0x75,
+	0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x06, 0x61, 0x70, 0x70, 0x65, 0x6e, 0x64, 0x12, 0x51, 0x0a,
+	0x05, 0x63, 0x6c, 0x65, 0x61, 0x72, 0x18, 0xea, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x38, 0x2e,
+	0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e,
+	0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69,
+	0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x43, 0x6c, 0x65, 0x61, 0x72,
+	0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x05, 0x63, 0x6c, 0x65, 0x61, 0x72,
+	0x42, 0x09, 0x0a, 0x07, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x22, 0xba, 0x02, 0x0a, 0x0d,
+	0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x0e, 0x0a,
+	0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, 0x12, 0x14, 0x0a,
+	0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x65, 0x72,
+	0x72, 0x6f, 0x72, 0x12, 0x4c, 0x0a, 0x03, 0x67, 0x65, 0x74, 0x18, 0xe8, 0x07, 0x20, 0x01, 0x28,
+	0x0b, 0x32, 0x37, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62,
+	0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65,
+	0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x47,
+	0x65, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x48, 0x00, 0x52, 0x03, 0x67, 0x65,
+	0x74, 0x12, 0x55, 0x0a, 0x06, 0x61, 0x70, 0x70, 0x65, 0x6e, 0x64, 0x18, 0xe9, 0x07, 0x20, 0x01,
+	0x28, 0x0b, 0x32, 0x3a, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e,
+	0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78,
+	0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x65,
+	0x41, 0x70, 0x70, 0x65, 0x6e, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x48, 0x00,
+	0x52, 0x06, 0x61, 0x70, 0x70, 0x65, 0x6e, 0x64, 0x12, 0x52, 0x0a, 0x05, 0x63, 0x6c, 0x65, 0x61,
+	0x72, 0x18, 0xea, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x39, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61,
+	0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c,
+	0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31,
+	0x2e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x43, 0x6c, 0x65, 0x61, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f,
+	0x6e, 0x73, 0x65, 0x48, 0x00, 0x52, 0x05, 0x63, 0x6c, 0x65, 0x61, 0x72, 0x42, 0x0a, 0x0a, 0x08,
+	0x72, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0xc7, 0x08, 0x0a, 0x08, 0x53, 0x74, 0x61,
+	0x74, 0x65, 0x4b, 0x65, 0x79, 0x12, 0x50, 0x0a, 0x06, 0x72, 0x75, 0x6e, 0x6e, 0x65, 0x72, 0x18,
+	0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x36, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63,
+	0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e,
+	0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x74,
+	0x61, 0x74, 0x65, 0x4b, 0x65, 0x79, 0x2e, 0x52, 0x75, 0x6e, 0x6e, 0x65, 0x72, 0x48, 0x00, 0x52,
+	0x06, 0x72, 0x75, 0x6e, 0x6e, 0x65, 0x72, 0x12, 0x73, 0x0a, 0x13, 0x6d, 0x75, 0x6c, 0x74, 0x69,
+	0x6d, 0x61, 0x70, 0x5f, 0x73, 0x69, 0x64, 0x65, 0x5f, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x18, 0x02,
+	0x20, 0x01, 0x28, 0x0b, 0x32, 0x41, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68,
+	0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f,
+	0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x74, 0x61,
+	0x74, 0x65, 0x4b, 0x65, 0x79, 0x2e, 0x4d, 0x75, 0x6c, 0x74, 0x69, 0x6d, 0x61, 0x70, 0x53, 0x69,
+	0x64, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x48, 0x00, 0x52, 0x11, 0x6d, 0x75, 0x6c, 0x74, 0x69,
+	0x6d, 0x61, 0x70, 0x53, 0x69, 0x64, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x64, 0x0a, 0x0e,
+	0x62, 0x61, 0x67, 0x5f, 0x75, 0x73, 0x65, 0x72, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x03,
+	0x20, 0x01, 0x28, 0x0b, 0x32, 0x3c, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68,
+	0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f,
+	0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x74, 0x61,
+	0x74, 0x65, 0x4b, 0x65, 0x79, 0x2e, 0x42, 0x61, 0x67, 0x55, 0x73, 0x65, 0x72, 0x53, 0x74, 0x61,
+	0x74, 0x65, 0x48, 0x00, 0x52, 0x0c, 0x62, 0x61, 0x67, 0x55, 0x73, 0x65, 0x72, 0x53, 0x74, 0x61,
+	0x74, 0x65, 0x12, 0x73, 0x0a, 0x13, 0x69, 0x74, 0x65, 0x72, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x73,
+	0x69, 0x64, 0x65, 0x5f, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32,
+	0x41, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61,
+	0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75,
+	0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x4b, 0x65, 0x79,
+	0x2e, 0x49, 0x74, 0x65, 0x72, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x69, 0x64, 0x65, 0x49, 0x6e, 0x70,
+	0x75, 0x74, 0x48, 0x00, 0x52, 0x11, 0x69, 0x74, 0x65, 0x72, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x69,
+	0x64, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x80, 0x01, 0x0a, 0x18, 0x6d, 0x75, 0x6c, 0x74,
+	0x69, 0x6d, 0x61, 0x70, 0x5f, 0x6b, 0x65, 0x79, 0x73, 0x5f, 0x73, 0x69, 0x64, 0x65, 0x5f, 0x69,
+	0x6e, 0x70, 0x75, 0x74, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x45, 0x2e, 0x6f, 0x72, 0x67,
+	0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64,
+	0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e,
+	0x76, 0x31, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x4b, 0x65, 0x79, 0x2e, 0x4d, 0x75, 0x6c, 0x74,
+	0x69, 0x6d, 0x61, 0x70, 0x4b, 0x65, 0x79, 0x73, 0x53, 0x69, 0x64, 0x65, 0x49, 0x6e, 0x70, 0x75,
+	0x74, 0x48, 0x00, 0x52, 0x15, 0x6d, 0x75, 0x6c, 0x74, 0x69, 0x6d, 0x61, 0x70, 0x4b, 0x65, 0x79,
+	0x73, 0x53, 0x69, 0x64, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x1a, 0x1a, 0x0a, 0x06, 0x52, 0x75,
+	0x6e, 0x6e, 0x65, 0x72, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28,
+	0x0c, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x1a, 0x72, 0x0a, 0x11, 0x49, 0x74, 0x65, 0x72, 0x61, 0x62,
+	0x6c, 0x65, 0x53, 0x69, 0x64, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x21, 0x0a, 0x0c, 0x74,
+	0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28,
+	0x09, 0x52, 0x0b, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x49, 0x64, 0x12, 0x22,
+	0x0a, 0x0d, 0x73, 0x69, 0x64, 0x65, 0x5f, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x5f, 0x69, 0x64, 0x18,
+	0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x73, 0x69, 0x64, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74,
+	0x49, 0x64, 0x12, 0x16, 0x0a, 0x06, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x18, 0x03, 0x20, 0x01,
+	0x28, 0x0c, 0x52, 0x06, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x1a, 0x84, 0x01, 0x0a, 0x11, 0x4d,
+	0x75, 0x6c, 0x74, 0x69, 0x6d, 0x61, 0x70, 0x53, 0x69, 0x64, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74,
+	0x12, 0x21, 0x0a, 0x0c, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x5f, 0x69, 0x64,
+	0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72,
+	0x6d, 0x49, 0x64, 0x12, 0x22, 0x0a, 0x0d, 0x73, 0x69, 0x64, 0x65, 0x5f, 0x69, 0x6e, 0x70, 0x75,
+	0x74, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x73, 0x69, 0x64, 0x65,
+	0x49, 0x6e, 0x70, 0x75, 0x74, 0x49, 0x64, 0x12, 0x16, 0x0a, 0x06, 0x77, 0x69, 0x6e, 0x64, 0x6f,
+	0x77, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x06, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x12,
+	0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x03, 0x6b, 0x65,
+	0x79, 0x1a, 0x76, 0x0a, 0x15, 0x4d, 0x75, 0x6c, 0x74, 0x69, 0x6d, 0x61, 0x70, 0x4b, 0x65, 0x79,
+	0x73, 0x53, 0x69, 0x64, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x21, 0x0a, 0x0c, 0x74, 0x72,
+	0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09,
+	0x52, 0x0b, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x49, 0x64, 0x12, 0x22, 0x0a,
+	0x0d, 0x73, 0x69, 0x64, 0x65, 0x5f, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x02,
+	0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x73, 0x69, 0x64, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x49,
+	0x64, 0x12, 0x16, 0x0a, 0x06, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x18, 0x03, 0x20, 0x01, 0x28,
+	0x0c, 0x52, 0x06, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x1a, 0x7f, 0x0a, 0x0c, 0x42, 0x61, 0x67,
+	0x55, 0x73, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x21, 0x0a, 0x0c, 0x74, 0x72, 0x61,
+	0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52,
+	0x0b, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x49, 0x64, 0x12, 0x22, 0x0a, 0x0d,
+	0x75, 0x73, 0x65, 0x72, 0x5f, 0x73, 0x74, 0x61, 0x74, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20,
+	0x01, 0x28, 0x09, 0x52, 0x0b, 0x75, 0x73, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x65, 0x49, 0x64,
+	0x12, 0x16, 0x0a, 0x06, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c,
+	0x52, 0x06, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18,
+	0x04, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x42, 0x06, 0x0a, 0x04, 0x74, 0x79,
+	0x70, 0x65, 0x22, 0x40, 0x0a, 0x0f, 0x53, 0x74, 0x61, 0x74, 0x65, 0x47, 0x65, 0x74, 0x52, 0x65,
+	0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x2d, 0x0a, 0x12, 0x63, 0x6f, 0x6e, 0x74, 0x69, 0x6e, 0x75,
+	0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x74, 0x6f, 0x6b, 0x65, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28,
+	0x0c, 0x52, 0x11, 0x63, 0x6f, 0x6e, 0x74, 0x69, 0x6e, 0x75, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x54,
+	0x6f, 0x6b, 0x65, 0x6e, 0x22, 0x55, 0x0a, 0x10, 0x53, 0x74, 0x61, 0x74, 0x65, 0x47, 0x65, 0x74,
+	0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x2d, 0x0a, 0x12, 0x63, 0x6f, 0x6e, 0x74,
+	0x69, 0x6e, 0x75, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x74, 0x6f, 0x6b, 0x65, 0x6e, 0x18, 0x01,
+	0x20, 0x01, 0x28, 0x0c, 0x52, 0x11, 0x63, 0x6f, 0x6e, 0x74, 0x69, 0x6e, 0x75, 0x61, 0x74, 0x69,
+	0x6f, 0x6e, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x12, 0x12, 0x0a, 0x04, 0x64, 0x61, 0x74, 0x61, 0x18,
+	0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x04, 0x64, 0x61, 0x74, 0x61, 0x22, 0x28, 0x0a, 0x12, 0x53,
+	0x74, 0x61, 0x74, 0x65, 0x41, 0x70, 0x70, 0x65, 0x6e, 0x64, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73,
+	0x74, 0x12, 0x12, 0x0a, 0x04, 0x64, 0x61, 0x74, 0x61, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52,
+	0x04, 0x64, 0x61, 0x74, 0x61, 0x22, 0x15, 0x0a, 0x13, 0x53, 0x74, 0x61, 0x74, 0x65, 0x41, 0x70,
+	0x70, 0x65, 0x6e, 0x64, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x13, 0x0a, 0x11,
+	0x53, 0x74, 0x61, 0x74, 0x65, 0x43, 0x6c, 0x65, 0x61, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73,
+	0x74, 0x22, 0x14, 0x0a, 0x12, 0x53, 0x74, 0x61, 0x74, 0x65, 0x43, 0x6c, 0x65, 0x61, 0x72, 0x52,
+	0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0xa2, 0x04, 0x0a, 0x08, 0x4c, 0x6f, 0x67, 0x45,
+	0x6e, 0x74, 0x72, 0x79, 0x12, 0x59, 0x0a, 0x08, 0x73, 0x65, 0x76, 0x65, 0x72, 0x69, 0x74, 0x79,
+	0x18, 0x01, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x3d, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61,
+	0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66,
+	0x6e, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x4c,
+	0x6f, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x53, 0x65, 0x76, 0x65, 0x72, 0x69, 0x74, 0x79,
+	0x2e, 0x45, 0x6e, 0x75, 0x6d, 0x52, 0x08, 0x73, 0x65, 0x76, 0x65, 0x72, 0x69, 0x74, 0x79, 0x12,
+	0x38, 0x0a, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, 0x02, 0x20, 0x01,
+	0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74,
+	0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x09,
+	0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x12, 0x18, 0x0a, 0x07, 0x6d, 0x65, 0x73,
+	0x73, 0x61, 0x67, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x6d, 0x65, 0x73, 0x73,
+	0x61, 0x67, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x74, 0x72, 0x61, 0x63, 0x65, 0x18, 0x04, 0x20, 0x01,
+	0x28, 0x09, 0x52, 0x05, 0x74, 0x72, 0x61, 0x63, 0x65, 0x12, 0x25, 0x0a, 0x0e, 0x69, 0x6e, 0x73,
+	0x74, 0x72, 0x75, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x05, 0x20, 0x01, 0x28,
+	0x09, 0x52, 0x0d, 0x69, 0x6e, 0x73, 0x74, 0x72, 0x75, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64,
+	0x12, 0x21, 0x0a, 0x0c, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x5f, 0x69, 0x64,
+	0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72,
+	0x6d, 0x49, 0x64, 0x12, 0x21, 0x0a, 0x0c, 0x6c, 0x6f, 0x67, 0x5f, 0x6c, 0x6f, 0x63, 0x61, 0x74,
+	0x69, 0x6f, 0x6e, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x6c, 0x6f, 0x67, 0x4c, 0x6f,
+	0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x16, 0x0a, 0x06, 0x74, 0x68, 0x72, 0x65, 0x61, 0x64,
+	0x18, 0x08, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x74, 0x68, 0x72, 0x65, 0x61, 0x64, 0x1a, 0x58,
+	0x0a, 0x04, 0x4c, 0x69, 0x73, 0x74, 0x12, 0x50, 0x0a, 0x0b, 0x6c, 0x6f, 0x67, 0x5f, 0x65, 0x6e,
+	0x74, 0x72, 0x69, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x2f, 0x2e, 0x6f, 0x72,
+	0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f,
+	0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e,
+	0x2e, 0x76, 0x31, 0x2e, 0x4c, 0x6f, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0a, 0x6c, 0x6f,
+	0x67, 0x45, 0x6e, 0x74, 0x72, 0x69, 0x65, 0x73, 0x1a, 0x72, 0x0a, 0x08, 0x53, 0x65, 0x76, 0x65,
+	0x72, 0x69, 0x74, 0x79, 0x22, 0x66, 0x0a, 0x04, 0x45, 0x6e, 0x75, 0x6d, 0x12, 0x0f, 0x0a, 0x0b,
+	0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, 0x09, 0x0a,
+	0x05, 0x54, 0x52, 0x41, 0x43, 0x45, 0x10, 0x01, 0x12, 0x09, 0x0a, 0x05, 0x44, 0x45, 0x42, 0x55,
+	0x47, 0x10, 0x02, 0x12, 0x08, 0x0a, 0x04, 0x49, 0x4e, 0x46, 0x4f, 0x10, 0x03, 0x12, 0x0a, 0x0a,
+	0x06, 0x4e, 0x4f, 0x54, 0x49, 0x43, 0x45, 0x10, 0x04, 0x12, 0x08, 0x0a, 0x04, 0x57, 0x41, 0x52,
+	0x4e, 0x10, 0x05, 0x12, 0x09, 0x0a, 0x05, 0x45, 0x52, 0x52, 0x4f, 0x52, 0x10, 0x06, 0x12, 0x0c,
+	0x0a, 0x08, 0x43, 0x52, 0x49, 0x54, 0x49, 0x43, 0x41, 0x4c, 0x10, 0x07, 0x22, 0x0c, 0x0a, 0x0a,
+	0x4c, 0x6f, 0x67, 0x43, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x22, 0xe1, 0x04, 0x0a, 0x12, 0x53,
+	0x74, 0x61, 0x72, 0x74, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73,
+	0x74, 0x12, 0x1b, 0x0a, 0x09, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x01,
+	0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x49, 0x64, 0x12, 0x62,
+	0x0a, 0x10, 0x63, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x5f, 0x65, 0x6e, 0x64, 0x70, 0x6f, 0x69,
+	0x6e, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x37, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61,
+	0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c,
+	0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x70, 0x69,
+	0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f,
+	0x72, 0x52, 0x0f, 0x63, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x45, 0x6e, 0x64, 0x70, 0x6f, 0x69,
+	0x6e, 0x74, 0x12, 0x62, 0x0a, 0x10, 0x6c, 0x6f, 0x67, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x65, 0x6e,
+	0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x37, 0x2e, 0x6f,
+	0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d,
+	0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31,
+	0x2e, 0x41, 0x70, 0x69, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x44, 0x65, 0x73, 0x63, 0x72,
+	0x69, 0x70, 0x74, 0x6f, 0x72, 0x52, 0x0f, 0x6c, 0x6f, 0x67, 0x67, 0x69, 0x6e, 0x67, 0x45, 0x6e,
+	0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x12, 0x64, 0x0a, 0x11, 0x61, 0x72, 0x74, 0x69, 0x66, 0x61,
+	0x63, 0x74, 0x5f, 0x65, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28,
+	0x0b, 0x32, 0x37, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62,
+	0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69,
+	0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x70, 0x69, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65,
+	0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x52, 0x10, 0x61, 0x72, 0x74, 0x69,
+	0x66, 0x61, 0x63, 0x74, 0x45, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x12, 0x66, 0x0a, 0x12,
+	0x70, 0x72, 0x6f, 0x76, 0x69, 0x73, 0x69, 0x6f, 0x6e, 0x5f, 0x65, 0x6e, 0x64, 0x70, 0x6f, 0x69,
+	0x6e, 0x74, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x37, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61,
+	0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c,
+	0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x70, 0x69,
+	0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f,
+	0x72, 0x52, 0x11, 0x70, 0x72, 0x6f, 0x76, 0x69, 0x73, 0x69, 0x6f, 0x6e, 0x45, 0x6e, 0x64, 0x70,
+	0x6f, 0x69, 0x6e, 0x74, 0x12, 0x5d, 0x0a, 0x06, 0x70, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x18, 0x0a,
+	0x20, 0x03, 0x28, 0x0b, 0x32, 0x45, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68,
+	0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f,
+	0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x74, 0x61,
+	0x72, 0x74, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x2e,
+	0x50, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x06, 0x70, 0x61, 0x72,
+	0x61, 0x6d, 0x73, 0x1a, 0x39, 0x0a, 0x0b, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x45, 0x6e, 0x74,
+	0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52,
+	0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20,
+	0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x2b,
+	0x0a, 0x13, 0x53, 0x74, 0x61, 0x72, 0x74, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x52, 0x65, 0x73,
+	0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x01,
+	0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x22, 0x30, 0x0a, 0x11, 0x53,
+	0x74, 0x6f, 0x70, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74,
+	0x12, 0x1b, 0x0a, 0x09, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20,
+	0x01, 0x28, 0x09, 0x52, 0x08, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x49, 0x64, 0x22, 0x2a, 0x0a,
+	0x12, 0x53, 0x74, 0x6f, 0x70, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f,
+	0x6e, 0x73, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x18, 0x01, 0x20, 0x01,
+	0x28, 0x09, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x22, 0x25, 0x0a, 0x13, 0x57, 0x6f, 0x72,
+	0x6b, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74,
+	0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64,
+	0x22, 0x5d, 0x0a, 0x14, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73,
+	0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01,
+	0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, 0x12, 0x14, 0x0a, 0x05, 0x65, 0x72, 0x72, 0x6f,
+	0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x12, 0x1f,
+	0x0a, 0x0b, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x18, 0x03, 0x20,
+	0x01, 0x28, 0x09, 0x52, 0x0a, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x49, 0x6e, 0x66, 0x6f, 0x32,
+	0xc3, 0x02, 0x0a, 0x0d, 0x42, 0x65, 0x61, 0x6d, 0x46, 0x6e, 0x43, 0x6f, 0x6e, 0x74, 0x72, 0x6f,
+	0x6c, 0x12, 0x86, 0x01, 0x0a, 0x07, 0x43, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x12, 0x3a, 0x2e,
+	0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e,
+	0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69,
+	0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x49, 0x6e, 0x73, 0x74, 0x72, 0x75, 0x63, 0x74, 0x69, 0x6f,
+	0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x1a, 0x39, 0x2e, 0x6f, 0x72, 0x67, 0x2e,
+	0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65,
+	0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76,
+	0x31, 0x2e, 0x49, 0x6e, 0x73, 0x74, 0x72, 0x75, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71,
+	0x75, 0x65, 0x73, 0x74, 0x22, 0x00, 0x28, 0x01, 0x30, 0x01, 0x12, 0xa8, 0x01, 0x0a, 0x1a, 0x47,
+	0x65, 0x74, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x44,
+	0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x12, 0x48, 0x2e, 0x6f, 0x72, 0x67, 0x2e,
+	0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65,
+	0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76,
+	0x31, 0x2e, 0x47, 0x65, 0x74, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x42, 0x75, 0x6e, 0x64,
+	0x6c, 0x65, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x52, 0x65, 0x71, 0x75,
+	0x65, 0x73, 0x74, 0x1a, 0x3e, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65,
+	0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65,
+	0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x72, 0x6f, 0x63,
+	0x65, 0x73, 0x73, 0x42, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70,
+	0x74, 0x6f, 0x72, 0x22, 0x00, 0x32, 0x7c, 0x0a, 0x0a, 0x42, 0x65, 0x61, 0x6d, 0x46, 0x6e, 0x44,
+	0x61, 0x74, 0x61, 0x12, 0x6e, 0x0a, 0x04, 0x44, 0x61, 0x74, 0x61, 0x12, 0x2f, 0x2e, 0x6f, 0x72,
+	0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f,
+	0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e,
+	0x2e, 0x76, 0x31, 0x2e, 0x45, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x1a, 0x2f, 0x2e, 0x6f,
+	0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d,
+	0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f,
+	0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x45, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x22, 0x00, 0x28,
+	0x01, 0x30, 0x01, 0x32, 0x87, 0x01, 0x0a, 0x0b, 0x42, 0x65, 0x61, 0x6d, 0x46, 0x6e, 0x53, 0x74,
+	0x61, 0x74, 0x65, 0x12, 0x78, 0x0a, 0x05, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x33, 0x2e, 0x6f,
+	0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d,
+	0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f,
+	0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73,
+	0x74, 0x1a, 0x34, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62,
+	0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65,
+	0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52,
+	0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x28, 0x01, 0x30, 0x01, 0x32, 0x89, 0x01,
+	0x0a, 0x0d, 0x42, 0x65, 0x61, 0x6d, 0x46, 0x6e, 0x4c, 0x6f, 0x67, 0x67, 0x69, 0x6e, 0x67, 0x12,
+	0x78, 0x0a, 0x07, 0x4c, 0x6f, 0x67, 0x67, 0x69, 0x6e, 0x67, 0x12, 0x34, 0x2e, 0x6f, 0x72, 0x67,
+	0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64,
+	0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e,
+	0x76, 0x31, 0x2e, 0x4c, 0x6f, 0x67, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x2e, 0x4c, 0x69, 0x73, 0x74,
+	0x1a, 0x31, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65,
+	0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65, 0x63,
+	0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x4c, 0x6f, 0x67, 0x43, 0x6f, 0x6e, 0x74,
+	0x72, 0x6f, 0x6c, 0x22, 0x00, 0x28, 0x01, 0x30, 0x01, 0x32, 0xa9, 0x02, 0x0a, 0x18, 0x42, 0x65,
+	0x61, 0x6d, 0x46, 0x6e, 0x45, 0x78, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x57, 0x6f, 0x72, 0x6b,
+	0x65, 0x72, 0x50, 0x6f, 0x6f, 0x6c, 0x12, 0x86, 0x01, 0x0a, 0x0b, 0x53, 0x74, 0x61, 0x72, 0x74,
+	0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x12, 0x39, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61,
+	0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66,
+	0x6e, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x53,
+	0x74, 0x61, 0x72, 0x74, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73,
+	0x74, 0x1a, 0x3a, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62,
+	0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65,
+	0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x74, 0x61, 0x72, 0x74, 0x57,
+	0x6f, 0x72, 0x6b, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12,
+	0x83, 0x01, 0x0a, 0x0a, 0x53, 0x74, 0x6f, 0x70, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x12, 0x38,
+	0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d,
+	0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74,
+	0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x74, 0x6f, 0x70, 0x57, 0x6f, 0x72, 0x6b, 0x65,
+	0x72, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x39, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61,
+	0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c,
+	0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31,
+	0x2e, 0x53, 0x74, 0x6f, 0x70, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x52, 0x65, 0x73, 0x70, 0x6f,
+	0x6e, 0x73, 0x65, 0x22, 0x00, 0x32, 0xa4, 0x01, 0x0a, 0x12, 0x42, 0x65, 0x61, 0x6d, 0x46, 0x6e,
+	0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x8d, 0x01, 0x0a,
+	0x0c, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x3b, 0x2e,
+	0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e,
+	0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69,
+	0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74,
+	0x75, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x1a, 0x3a, 0x2e, 0x6f, 0x72, 0x67,
+	0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64,
+	0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e,
+	0x76, 0x31, 0x2e, 0x57, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x52,
+	0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x22, 0x00, 0x28, 0x01, 0x30, 0x01, 0x42, 0x7e, 0x0a, 0x24,
+	0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e,
+	0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f,
+	0x6e, 0x2e, 0x76, 0x31, 0x42, 0x09, 0x42, 0x65, 0x61, 0x6d, 0x46, 0x6e, 0x41, 0x70, 0x69, 0x5a,
+	0x4b, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x61, 0x70, 0x61, 0x63,
+	0x68, 0x65, 0x2f, 0x62, 0x65, 0x61, 0x6d, 0x2f, 0x73, 0x64, 0x6b, 0x73, 0x2f, 0x67, 0x6f, 0x2f,
+	0x70, 0x6b, 0x67, 0x2f, 0x62, 0x65, 0x61, 0x6d, 0x2f, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2f, 0x66,
+	0x6e, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x76, 0x31, 0x3b, 0x66, 0x6e,
+	0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x76, 0x31, 0x62, 0x06, 0x70, 0x72,
+	0x6f, 0x74, 0x6f, 0x33,
 }
 
-var xxx_messageInfo_LogEntry_Severity proto.InternalMessageInfo
+var (
+	file_beam_fn_api_proto_rawDescOnce sync.Once
+	file_beam_fn_api_proto_rawDescData = file_beam_fn_api_proto_rawDesc
+)
 
-type LogControl struct {
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
+func file_beam_fn_api_proto_rawDescGZIP() []byte {
+	file_beam_fn_api_proto_rawDescOnce.Do(func() {
+		file_beam_fn_api_proto_rawDescData = protoimpl.X.CompressGZIP(file_beam_fn_api_proto_rawDescData)
+	})
+	return file_beam_fn_api_proto_rawDescData
 }
 
-func (m *LogControl) Reset()         { *m = LogControl{} }
-func (m *LogControl) String() string { return proto.CompactTextString(m) }
-func (*LogControl) ProtoMessage()    {}
-func (*LogControl) Descriptor() ([]byte, []int) {
-	return fileDescriptor_6d954c03a4758710, []int{30}
+var file_beam_fn_api_proto_enumTypes = make([]protoimpl.EnumInfo, 1)
+var file_beam_fn_api_proto_msgTypes = make([]protoimpl.MessageInfo, 62)
+var file_beam_fn_api_proto_goTypes = []interface{}{
+	(LogEntry_Severity_Enum)(0),                       // 0: org.apache.beam.model.fn_execution.v1.LogEntry.Severity.Enum
+	(*RemoteGrpcPort)(nil),                            // 1: org.apache.beam.model.fn_execution.v1.RemoteGrpcPort
+	(*GetProcessBundleDescriptorRequest)(nil),         // 2: org.apache.beam.model.fn_execution.v1.GetProcessBundleDescriptorRequest
+	(*InstructionRequest)(nil),                        // 3: org.apache.beam.model.fn_execution.v1.InstructionRequest
+	(*InstructionResponse)(nil),                       // 4: org.apache.beam.model.fn_execution.v1.InstructionResponse
+	(*RegisterRequest)(nil),                           // 5: org.apache.beam.model.fn_execution.v1.RegisterRequest
+	(*RegisterResponse)(nil),                          // 6: org.apache.beam.model.fn_execution.v1.RegisterResponse
+	(*ProcessBundleDescriptor)(nil),                   // 7: org.apache.beam.model.fn_execution.v1.ProcessBundleDescriptor
+	(*BundleApplication)(nil),                         // 8: org.apache.beam.model.fn_execution.v1.BundleApplication
+	(*DelayedBundleApplication)(nil),                  // 9: org.apache.beam.model.fn_execution.v1.DelayedBundleApplication
+	(*ProcessBundleRequest)(nil),                      // 10: org.apache.beam.model.fn_execution.v1.ProcessBundleRequest
+	(*ProcessBundleResponse)(nil),                     // 11: org.apache.beam.model.fn_execution.v1.ProcessBundleResponse
+	(*ProcessBundleProgressRequest)(nil),              // 12: org.apache.beam.model.fn_execution.v1.ProcessBundleProgressRequest
+	(*MonitoringInfosMetadataRequest)(nil),            // 13: org.apache.beam.model.fn_execution.v1.MonitoringInfosMetadataRequest
+	(*ProcessBundleProgressResponse)(nil),             // 14: org.apache.beam.model.fn_execution.v1.ProcessBundleProgressResponse
+	(*MonitoringInfosMetadataResponse)(nil),           // 15: org.apache.beam.model.fn_execution.v1.MonitoringInfosMetadataResponse
+	(*ProcessBundleSplitRequest)(nil),                 // 16: org.apache.beam.model.fn_execution.v1.ProcessBundleSplitRequest
+	(*ProcessBundleSplitResponse)(nil),                // 17: org.apache.beam.model.fn_execution.v1.ProcessBundleSplitResponse
+	(*FinalizeBundleRequest)(nil),                     // 18: org.apache.beam.model.fn_execution.v1.FinalizeBundleRequest
+	(*FinalizeBundleResponse)(nil),                    // 19: org.apache.beam.model.fn_execution.v1.FinalizeBundleResponse
+	(*Elements)(nil),                                  // 20: org.apache.beam.model.fn_execution.v1.Elements
+	(*StateRequest)(nil),                              // 21: org.apache.beam.model.fn_execution.v1.StateRequest
+	(*StateResponse)(nil),                             // 22: org.apache.beam.model.fn_execution.v1.StateResponse
+	(*StateKey)(nil),                                  // 23: org.apache.beam.model.fn_execution.v1.StateKey
+	(*StateGetRequest)(nil),                           // 24: org.apache.beam.model.fn_execution.v1.StateGetRequest
+	(*StateGetResponse)(nil),                          // 25: org.apache.beam.model.fn_execution.v1.StateGetResponse
+	(*StateAppendRequest)(nil),                        // 26: org.apache.beam.model.fn_execution.v1.StateAppendRequest
+	(*StateAppendResponse)(nil),                       // 27: org.apache.beam.model.fn_execution.v1.StateAppendResponse
+	(*StateClearRequest)(nil),                         // 28: org.apache.beam.model.fn_execution.v1.StateClearRequest
+	(*StateClearResponse)(nil),                        // 29: org.apache.beam.model.fn_execution.v1.StateClearResponse
+	(*LogEntry)(nil),                                  // 30: org.apache.beam.model.fn_execution.v1.LogEntry
+	(*LogControl)(nil),                                // 31: org.apache.beam.model.fn_execution.v1.LogControl
+	(*StartWorkerRequest)(nil),                        // 32: org.apache.beam.model.fn_execution.v1.StartWorkerRequest
+	(*StartWorkerResponse)(nil),                       // 33: org.apache.beam.model.fn_execution.v1.StartWorkerResponse
+	(*StopWorkerRequest)(nil),                         // 34: org.apache.beam.model.fn_execution.v1.StopWorkerRequest
+	(*StopWorkerResponse)(nil),                        // 35: org.apache.beam.model.fn_execution.v1.StopWorkerResponse
+	(*WorkerStatusRequest)(nil),                       // 36: org.apache.beam.model.fn_execution.v1.WorkerStatusRequest
+	(*WorkerStatusResponse)(nil),                      // 37: org.apache.beam.model.fn_execution.v1.WorkerStatusResponse
+	nil,                                               // 38: org.apache.beam.model.fn_execution.v1.ProcessBundleDescriptor.TransformsEntry
+	nil,                                               // 39: org.apache.beam.model.fn_execution.v1.ProcessBundleDescriptor.PcollectionsEntry
+	nil,                                               // 40: org.apache.beam.model.fn_execution.v1.ProcessBundleDescriptor.WindowingStrategiesEntry
+	nil,                                               // 41: org.apache.beam.model.fn_execution.v1.ProcessBundleDescriptor.CodersEntry
+	nil,                                               // 42: org.apache.beam.model.fn_execution.v1.ProcessBundleDescriptor.EnvironmentsEntry
+	nil,                                               // 43: org.apache.beam.model.fn_execution.v1.BundleApplication.OutputWatermarksEntry
+	(*ProcessBundleRequest_CacheToken)(nil),           // 44: org.apache.beam.model.fn_execution.v1.ProcessBundleRequest.CacheToken
+	(*ProcessBundleRequest_CacheToken_UserState)(nil), // 45: org.apache.beam.model.fn_execution.v1.ProcessBundleRequest.CacheToken.UserState
+	(*ProcessBundleRequest_CacheToken_SideInput)(nil), // 46: org.apache.beam.model.fn_execution.v1.ProcessBundleRequest.CacheToken.SideInput
+	nil, // 47: org.apache.beam.model.fn_execution.v1.ProcessBundleResponse.MonitoringDataEntry
+	nil, // 48: org.apache.beam.model.fn_execution.v1.ProcessBundleProgressResponse.MonitoringDataEntry
+	nil, // 49: org.apache.beam.model.fn_execution.v1.MonitoringInfosMetadataResponse.MonitoringInfoEntry
+	(*ProcessBundleSplitRequest_DesiredSplit)(nil), // 50: org.apache.beam.model.fn_execution.v1.ProcessBundleSplitRequest.DesiredSplit
+	nil, // 51: org.apache.beam.model.fn_execution.v1.ProcessBundleSplitRequest.DesiredSplitsEntry
+	(*ProcessBundleSplitResponse_ChannelSplit)(nil), // 52: org.apache.beam.model.fn_execution.v1.ProcessBundleSplitResponse.ChannelSplit
+	(*Elements_Data)(nil),                           // 53: org.apache.beam.model.fn_execution.v1.Elements.Data
+	(*Elements_Timers)(nil),                         // 54: org.apache.beam.model.fn_execution.v1.Elements.Timers
+	(*StateKey_Runner)(nil),                         // 55: org.apache.beam.model.fn_execution.v1.StateKey.Runner
+	(*StateKey_IterableSideInput)(nil),              // 56: org.apache.beam.model.fn_execution.v1.StateKey.IterableSideInput
+	(*StateKey_MultimapSideInput)(nil),              // 57: org.apache.beam.model.fn_execution.v1.StateKey.MultimapSideInput
+	(*StateKey_MultimapKeysSideInput)(nil),          // 58: org.apache.beam.model.fn_execution.v1.StateKey.MultimapKeysSideInput
+	(*StateKey_BagUserState)(nil),                   // 59: org.apache.beam.model.fn_execution.v1.StateKey.BagUserState
+	(*LogEntry_List)(nil),                           // 60: org.apache.beam.model.fn_execution.v1.LogEntry.List
+	(*LogEntry_Severity)(nil),                       // 61: org.apache.beam.model.fn_execution.v1.LogEntry.Severity
+	nil,                                             // 62: org.apache.beam.model.fn_execution.v1.StartWorkerRequest.ParamsEntry
+	(*pipeline_v1.ApiServiceDescriptor)(nil),        // 63: org.apache.beam.model.pipeline.v1.ApiServiceDescriptor
+	(pipeline_v1.IsBounded_Enum)(0),                 // 64: org.apache.beam.model.pipeline.v1.IsBounded.Enum
+	(*duration.Duration)(nil),                       // 65: google.protobuf.Duration
+	(*pipeline_v1.MonitoringInfo)(nil),              // 66: org.apache.beam.model.pipeline.v1.MonitoringInfo
+	(*timestamp.Timestamp)(nil),                     // 67: google.protobuf.Timestamp
+	(*pipeline_v1.PTransform)(nil),                  // 68: org.apache.beam.model.pipeline.v1.PTransform
+	(*pipeline_v1.PCollection)(nil),                 // 69: org.apache.beam.model.pipeline.v1.PCollection
+	(*pipeline_v1.WindowingStrategy)(nil),           // 70: org.apache.beam.model.pipeline.v1.WindowingStrategy
+	(*pipeline_v1.Coder)(nil),                       // 71: org.apache.beam.model.pipeline.v1.Coder
+	(*pipeline_v1.Environment)(nil),                 // 72: org.apache.beam.model.pipeline.v1.Environment
+}
+var file_beam_fn_api_proto_depIdxs = []int32{
+	63, // 0: org.apache.beam.model.fn_execution.v1.RemoteGrpcPort.api_service_descriptor:type_name -> org.apache.beam.model.pipeline.v1.ApiServiceDescriptor
+	10, // 1: org.apache.beam.model.fn_execution.v1.InstructionRequest.process_bundle:type_name -> org.apache.beam.model.fn_execution.v1.ProcessBundleRequest
+	12, // 2: org.apache.beam.model.fn_execution.v1.InstructionRequest.process_bundle_progress:type_name -> org.apache.beam.model.fn_execution.v1.ProcessBundleProgressRequest
+	16, // 3: org.apache.beam.model.fn_execution.v1.InstructionRequest.process_bundle_split:type_name -> org.apache.beam.model.fn_execution.v1.ProcessBundleSplitRequest
+	18, // 4: org.apache.beam.model.fn_execution.v1.InstructionRequest.finalize_bundle:type_name -> org.apache.beam.model.fn_execution.v1.FinalizeBundleRequest
+	13, // 5: org.apache.beam.model.fn_execution.v1.InstructionRequest.monitoring_infos:type_name -> org.apache.beam.model.fn_execution.v1.MonitoringInfosMetadataRequest
+	5,  // 6: org.apache.beam.model.fn_execution.v1.InstructionRequest.register:type_name -> org.apache.beam.model.fn_execution.v1.RegisterRequest
+	11, // 7: org.apache.beam.model.fn_execution.v1.InstructionResponse.process_bundle:type_name -> org.apache.beam.model.fn_execution.v1.ProcessBundleResponse
+	14, // 8: org.apache.beam.model.fn_execution.v1.InstructionResponse.process_bundle_progress:type_name -> org.apache.beam.model.fn_execution.v1.ProcessBundleProgressResponse
+	17, // 9: org.apache.beam.model.fn_execution.v1.InstructionResponse.process_bundle_split:type_name -> org.apache.beam.model.fn_execution.v1.ProcessBundleSplitResponse
+	19, // 10: org.apache.beam.model.fn_execution.v1.InstructionResponse.finalize_bundle:type_name -> org.apache.beam.model.fn_execution.v1.FinalizeBundleResponse
+	15, // 11: org.apache.beam.model.fn_execution.v1.InstructionResponse.monitoring_infos:type_name -> org.apache.beam.model.fn_execution.v1.MonitoringInfosMetadataResponse
+	6,  // 12: org.apache.beam.model.fn_execution.v1.InstructionResponse.register:type_name -> org.apache.beam.model.fn_execution.v1.RegisterResponse
+	7,  // 13: org.apache.beam.model.fn_execution.v1.RegisterRequest.process_bundle_descriptor:type_name -> org.apache.beam.model.fn_execution.v1.ProcessBundleDescriptor
+	38, // 14: org.apache.beam.model.fn_execution.v1.ProcessBundleDescriptor.transforms:type_name -> org.apache.beam.model.fn_execution.v1.ProcessBundleDescriptor.TransformsEntry
+	39, // 15: org.apache.beam.model.fn_execution.v1.ProcessBundleDescriptor.pcollections:type_name -> org.apache.beam.model.fn_execution.v1.ProcessBundleDescriptor.PcollectionsEntry
+	40, // 16: org.apache.beam.model.fn_execution.v1.ProcessBundleDescriptor.windowing_strategies:type_name -> org.apache.beam.model.fn_execution.v1.ProcessBundleDescriptor.WindowingStrategiesEntry
+	41, // 17: org.apache.beam.model.fn_execution.v1.ProcessBundleDescriptor.coders:type_name -> org.apache.beam.model.fn_execution.v1.ProcessBundleDescriptor.CodersEntry
+	42, // 18: org.apache.beam.model.fn_execution.v1.ProcessBundleDescriptor.environments:type_name -> org.apache.beam.model.fn_execution.v1.ProcessBundleDescriptor.EnvironmentsEntry
+	63, // 19: org.apache.beam.model.fn_execution.v1.ProcessBundleDescriptor.state_api_service_descriptor:type_name -> org.apache.beam.model.pipeline.v1.ApiServiceDescriptor
+	63, // 20: org.apache.beam.model.fn_execution.v1.ProcessBundleDescriptor.timer_api_service_descriptor:type_name -> org.apache.beam.model.pipeline.v1.ApiServiceDescriptor
+	43, // 21: org.apache.beam.model.fn_execution.v1.BundleApplication.output_watermarks:type_name -> org.apache.beam.model.fn_execution.v1.BundleApplication.OutputWatermarksEntry
+	64, // 22: org.apache.beam.model.fn_execution.v1.BundleApplication.is_bounded:type_name -> org.apache.beam.model.pipeline.v1.IsBounded.Enum
+	8,  // 23: org.apache.beam.model.fn_execution.v1.DelayedBundleApplication.application:type_name -> org.apache.beam.model.fn_execution.v1.BundleApplication
+	65, // 24: org.apache.beam.model.fn_execution.v1.DelayedBundleApplication.requested_time_delay:type_name -> google.protobuf.Duration
+	44, // 25: org.apache.beam.model.fn_execution.v1.ProcessBundleRequest.cache_tokens:type_name -> org.apache.beam.model.fn_execution.v1.ProcessBundleRequest.CacheToken
+	9,  // 26: org.apache.beam.model.fn_execution.v1.ProcessBundleResponse.residual_roots:type_name -> org.apache.beam.model.fn_execution.v1.DelayedBundleApplication
+	66, // 27: org.apache.beam.model.fn_execution.v1.ProcessBundleResponse.monitoring_infos:type_name -> org.apache.beam.model.pipeline.v1.MonitoringInfo
+	47, // 28: org.apache.beam.model.fn_execution.v1.ProcessBundleResponse.monitoring_data:type_name -> org.apache.beam.model.fn_execution.v1.ProcessBundleResponse.MonitoringDataEntry
+	66, // 29: org.apache.beam.model.fn_execution.v1.ProcessBundleProgressResponse.monitoring_infos:type_name -> org.apache.beam.model.pipeline.v1.MonitoringInfo
+	48, // 30: org.apache.beam.model.fn_execution.v1.ProcessBundleProgressResponse.monitoring_data:type_name -> org.apache.beam.model.fn_execution.v1.ProcessBundleProgressResponse.MonitoringDataEntry
+	49, // 31: org.apache.beam.model.fn_execution.v1.MonitoringInfosMetadataResponse.monitoring_info:type_name -> org.apache.beam.model.fn_execution.v1.MonitoringInfosMetadataResponse.MonitoringInfoEntry
+	51, // 32: org.apache.beam.model.fn_execution.v1.ProcessBundleSplitRequest.desired_splits:type_name -> org.apache.beam.model.fn_execution.v1.ProcessBundleSplitRequest.DesiredSplitsEntry
+	8,  // 33: org.apache.beam.model.fn_execution.v1.ProcessBundleSplitResponse.primary_roots:type_name -> org.apache.beam.model.fn_execution.v1.BundleApplication
+	9,  // 34: org.apache.beam.model.fn_execution.v1.ProcessBundleSplitResponse.residual_roots:type_name -> org.apache.beam.model.fn_execution.v1.DelayedBundleApplication
+	52, // 35: org.apache.beam.model.fn_execution.v1.ProcessBundleSplitResponse.channel_splits:type_name -> org.apache.beam.model.fn_execution.v1.ProcessBundleSplitResponse.ChannelSplit
+	53, // 36: org.apache.beam.model.fn_execution.v1.Elements.data:type_name -> org.apache.beam.model.fn_execution.v1.Elements.Data
+	54, // 37: org.apache.beam.model.fn_execution.v1.Elements.timers:type_name -> org.apache.beam.model.fn_execution.v1.Elements.Timers
+	23, // 38: org.apache.beam.model.fn_execution.v1.StateRequest.state_key:type_name -> org.apache.beam.model.fn_execution.v1.StateKey
+	24, // 39: org.apache.beam.model.fn_execution.v1.StateRequest.get:type_name -> org.apache.beam.model.fn_execution.v1.StateGetRequest
+	26, // 40: org.apache.beam.model.fn_execution.v1.StateRequest.append:type_name -> org.apache.beam.model.fn_execution.v1.StateAppendRequest
+	28, // 41: org.apache.beam.model.fn_execution.v1.StateRequest.clear:type_name -> org.apache.beam.model.fn_execution.v1.StateClearRequest
+	25, // 42: org.apache.beam.model.fn_execution.v1.StateResponse.get:type_name -> org.apache.beam.model.fn_execution.v1.StateGetResponse
+	27, // 43: org.apache.beam.model.fn_execution.v1.StateResponse.append:type_name -> org.apache.beam.model.fn_execution.v1.StateAppendResponse
+	29, // 44: org.apache.beam.model.fn_execution.v1.StateResponse.clear:type_name -> org.apache.beam.model.fn_execution.v1.StateClearResponse
+	55, // 45: org.apache.beam.model.fn_execution.v1.StateKey.runner:type_name -> org.apache.beam.model.fn_execution.v1.StateKey.Runner
+	57, // 46: org.apache.beam.model.fn_execution.v1.StateKey.multimap_side_input:type_name -> org.apache.beam.model.fn_execution.v1.StateKey.MultimapSideInput
+	59, // 47: org.apache.beam.model.fn_execution.v1.StateKey.bag_user_state:type_name -> org.apache.beam.model.fn_execution.v1.StateKey.BagUserState
+	56, // 48: org.apache.beam.model.fn_execution.v1.StateKey.iterable_side_input:type_name -> org.apache.beam.model.fn_execution.v1.StateKey.IterableSideInput
+	58, // 49: org.apache.beam.model.fn_execution.v1.StateKey.multimap_keys_side_input:type_name -> org.apache.beam.model.fn_execution.v1.StateKey.MultimapKeysSideInput
+	0,  // 50: org.apache.beam.model.fn_execution.v1.LogEntry.severity:type_name -> org.apache.beam.model.fn_execution.v1.LogEntry.Severity.Enum
+	67, // 51: org.apache.beam.model.fn_execution.v1.LogEntry.timestamp:type_name -> google.protobuf.Timestamp
+	63, // 52: org.apache.beam.model.fn_execution.v1.StartWorkerRequest.control_endpoint:type_name -> org.apache.beam.model.pipeline.v1.ApiServiceDescriptor
+	63, // 53: org.apache.beam.model.fn_execution.v1.StartWorkerRequest.logging_endpoint:type_name -> org.apache.beam.model.pipeline.v1.ApiServiceDescriptor
+	63, // 54: org.apache.beam.model.fn_execution.v1.StartWorkerRequest.artifact_endpoint:type_name -> org.apache.beam.model.pipeline.v1.ApiServiceDescriptor
+	63, // 55: org.apache.beam.model.fn_execution.v1.StartWorkerRequest.provision_endpoint:type_name -> org.apache.beam.model.pipeline.v1.ApiServiceDescriptor
+	62, // 56: org.apache.beam.model.fn_execution.v1.StartWorkerRequest.params:type_name -> org.apache.beam.model.fn_execution.v1.StartWorkerRequest.ParamsEntry
+	68, // 57: org.apache.beam.model.fn_execution.v1.ProcessBundleDescriptor.TransformsEntry.value:type_name -> org.apache.beam.model.pipeline.v1.PTransform
+	69, // 58: org.apache.beam.model.fn_execution.v1.ProcessBundleDescriptor.PcollectionsEntry.value:type_name -> org.apache.beam.model.pipeline.v1.PCollection
+	70, // 59: org.apache.beam.model.fn_execution.v1.ProcessBundleDescriptor.WindowingStrategiesEntry.value:type_name -> org.apache.beam.model.pipeline.v1.WindowingStrategy
+	71, // 60: org.apache.beam.model.fn_execution.v1.ProcessBundleDescriptor.CodersEntry.value:type_name -> org.apache.beam.model.pipeline.v1.Coder
+	72, // 61: org.apache.beam.model.fn_execution.v1.ProcessBundleDescriptor.EnvironmentsEntry.value:type_name -> org.apache.beam.model.pipeline.v1.Environment
+	67, // 62: org.apache.beam.model.fn_execution.v1.BundleApplication.OutputWatermarksEntry.value:type_name -> google.protobuf.Timestamp
+	45, // 63: org.apache.beam.model.fn_execution.v1.ProcessBundleRequest.CacheToken.user_state:type_name -> org.apache.beam.model.fn_execution.v1.ProcessBundleRequest.CacheToken.UserState
+	46, // 64: org.apache.beam.model.fn_execution.v1.ProcessBundleRequest.CacheToken.side_input:type_name -> org.apache.beam.model.fn_execution.v1.ProcessBundleRequest.CacheToken.SideInput
+	66, // 65: org.apache.beam.model.fn_execution.v1.MonitoringInfosMetadataResponse.MonitoringInfoEntry.value:type_name -> org.apache.beam.model.pipeline.v1.MonitoringInfo
+	50, // 66: org.apache.beam.model.fn_execution.v1.ProcessBundleSplitRequest.DesiredSplitsEntry.value:type_name -> org.apache.beam.model.fn_execution.v1.ProcessBundleSplitRequest.DesiredSplit
+	30, // 67: org.apache.beam.model.fn_execution.v1.LogEntry.List.log_entries:type_name -> org.apache.beam.model.fn_execution.v1.LogEntry
+	4,  // 68: org.apache.beam.model.fn_execution.v1.BeamFnControl.Control:input_type -> org.apache.beam.model.fn_execution.v1.InstructionResponse
+	2,  // 69: org.apache.beam.model.fn_execution.v1.BeamFnControl.GetProcessBundleDescriptor:input_type -> org.apache.beam.model.fn_execution.v1.GetProcessBundleDescriptorRequest
+	20, // 70: org.apache.beam.model.fn_execution.v1.BeamFnData.Data:input_type -> org.apache.beam.model.fn_execution.v1.Elements
+	21, // 71: org.apache.beam.model.fn_execution.v1.BeamFnState.State:input_type -> org.apache.beam.model.fn_execution.v1.StateRequest
+	60, // 72: org.apache.beam.model.fn_execution.v1.BeamFnLogging.Logging:input_type -> org.apache.beam.model.fn_execution.v1.LogEntry.List
+	32, // 73: org.apache.beam.model.fn_execution.v1.BeamFnExternalWorkerPool.StartWorker:input_type -> org.apache.beam.model.fn_execution.v1.StartWorkerRequest
+	34, // 74: org.apache.beam.model.fn_execution.v1.BeamFnExternalWorkerPool.StopWorker:input_type -> org.apache.beam.model.fn_execution.v1.StopWorkerRequest
+	37, // 75: org.apache.beam.model.fn_execution.v1.BeamFnWorkerStatus.WorkerStatus:input_type -> org.apache.beam.model.fn_execution.v1.WorkerStatusResponse
+	3,  // 76: org.apache.beam.model.fn_execution.v1.BeamFnControl.Control:output_type -> org.apache.beam.model.fn_execution.v1.InstructionRequest
+	7,  // 77: org.apache.beam.model.fn_execution.v1.BeamFnControl.GetProcessBundleDescriptor:output_type -> org.apache.beam.model.fn_execution.v1.ProcessBundleDescriptor
+	20, // 78: org.apache.beam.model.fn_execution.v1.BeamFnData.Data:output_type -> org.apache.beam.model.fn_execution.v1.Elements
+	22, // 79: org.apache.beam.model.fn_execution.v1.BeamFnState.State:output_type -> org.apache.beam.model.fn_execution.v1.StateResponse
+	31, // 80: org.apache.beam.model.fn_execution.v1.BeamFnLogging.Logging:output_type -> org.apache.beam.model.fn_execution.v1.LogControl
+	33, // 81: org.apache.beam.model.fn_execution.v1.BeamFnExternalWorkerPool.StartWorker:output_type -> org.apache.beam.model.fn_execution.v1.StartWorkerResponse
+	35, // 82: org.apache.beam.model.fn_execution.v1.BeamFnExternalWorkerPool.StopWorker:output_type -> org.apache.beam.model.fn_execution.v1.StopWorkerResponse
+	36, // 83: org.apache.beam.model.fn_execution.v1.BeamFnWorkerStatus.WorkerStatus:output_type -> org.apache.beam.model.fn_execution.v1.WorkerStatusRequest
+	76, // [76:84] is the sub-list for method output_type
+	68, // [68:76] is the sub-list for method input_type
+	68, // [68:68] is the sub-list for extension type_name
+	68, // [68:68] is the sub-list for extension extendee
+	0,  // [0:68] is the sub-list for field type_name
 }
 
-func (m *LogControl) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_LogControl.Unmarshal(m, b)
-}
-func (m *LogControl) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_LogControl.Marshal(b, m, deterministic)
-}
-func (m *LogControl) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_LogControl.Merge(m, src)
-}
-func (m *LogControl) XXX_Size() int {
-	return xxx_messageInfo_LogControl.Size(m)
-}
-func (m *LogControl) XXX_DiscardUnknown() {
-	xxx_messageInfo_LogControl.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_LogControl proto.InternalMessageInfo
-
-type StartWorkerRequest struct {
-	WorkerId             string                            `protobuf:"bytes,1,opt,name=worker_id,json=workerId,proto3" json:"worker_id,omitempty"`
-	ControlEndpoint      *pipeline_v1.ApiServiceDescriptor `protobuf:"bytes,2,opt,name=control_endpoint,json=controlEndpoint,proto3" json:"control_endpoint,omitempty"`
-	LoggingEndpoint      *pipeline_v1.ApiServiceDescriptor `protobuf:"bytes,3,opt,name=logging_endpoint,json=loggingEndpoint,proto3" json:"logging_endpoint,omitempty"`
-	ArtifactEndpoint     *pipeline_v1.ApiServiceDescriptor `protobuf:"bytes,4,opt,name=artifact_endpoint,json=artifactEndpoint,proto3" json:"artifact_endpoint,omitempty"`
-	ProvisionEndpoint    *pipeline_v1.ApiServiceDescriptor `protobuf:"bytes,5,opt,name=provision_endpoint,json=provisionEndpoint,proto3" json:"provision_endpoint,omitempty"`
-	Params               map[string]string                 `protobuf:"bytes,10,rep,name=params,proto3" json:"params,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"`
-	XXX_NoUnkeyedLiteral struct{}                          `json:"-"`
-	XXX_unrecognized     []byte                            `json:"-"`
-	XXX_sizecache        int32                             `json:"-"`
-}
-
-func (m *StartWorkerRequest) Reset()         { *m = StartWorkerRequest{} }
-func (m *StartWorkerRequest) String() string { return proto.CompactTextString(m) }
-func (*StartWorkerRequest) ProtoMessage()    {}
-func (*StartWorkerRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_6d954c03a4758710, []int{31}
-}
-
-func (m *StartWorkerRequest) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_StartWorkerRequest.Unmarshal(m, b)
-}
-func (m *StartWorkerRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_StartWorkerRequest.Marshal(b, m, deterministic)
-}
-func (m *StartWorkerRequest) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_StartWorkerRequest.Merge(m, src)
-}
-func (m *StartWorkerRequest) XXX_Size() int {
-	return xxx_messageInfo_StartWorkerRequest.Size(m)
-}
-func (m *StartWorkerRequest) XXX_DiscardUnknown() {
-	xxx_messageInfo_StartWorkerRequest.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_StartWorkerRequest proto.InternalMessageInfo
-
-func (m *StartWorkerRequest) GetWorkerId() string {
-	if m != nil {
-		return m.WorkerId
+func init() { file_beam_fn_api_proto_init() }
+func file_beam_fn_api_proto_init() {
+	if File_beam_fn_api_proto != nil {
+		return
 	}
-	return ""
-}
-
-func (m *StartWorkerRequest) GetControlEndpoint() *pipeline_v1.ApiServiceDescriptor {
-	if m != nil {
-		return m.ControlEndpoint
+	if !protoimpl.UnsafeEnabled {
+		file_beam_fn_api_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*RemoteGrpcPort); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_fn_api_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*GetProcessBundleDescriptorRequest); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_fn_api_proto_msgTypes[2].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*InstructionRequest); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_fn_api_proto_msgTypes[3].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*InstructionResponse); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_fn_api_proto_msgTypes[4].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*RegisterRequest); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_fn_api_proto_msgTypes[5].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*RegisterResponse); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_fn_api_proto_msgTypes[6].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*ProcessBundleDescriptor); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_fn_api_proto_msgTypes[7].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*BundleApplication); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_fn_api_proto_msgTypes[8].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*DelayedBundleApplication); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_fn_api_proto_msgTypes[9].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*ProcessBundleRequest); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_fn_api_proto_msgTypes[10].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*ProcessBundleResponse); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_fn_api_proto_msgTypes[11].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*ProcessBundleProgressRequest); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_fn_api_proto_msgTypes[12].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*MonitoringInfosMetadataRequest); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_fn_api_proto_msgTypes[13].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*ProcessBundleProgressResponse); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_fn_api_proto_msgTypes[14].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*MonitoringInfosMetadataResponse); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_fn_api_proto_msgTypes[15].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*ProcessBundleSplitRequest); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_fn_api_proto_msgTypes[16].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*ProcessBundleSplitResponse); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_fn_api_proto_msgTypes[17].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*FinalizeBundleRequest); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_fn_api_proto_msgTypes[18].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*FinalizeBundleResponse); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_fn_api_proto_msgTypes[19].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*Elements); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_fn_api_proto_msgTypes[20].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*StateRequest); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_fn_api_proto_msgTypes[21].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*StateResponse); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_fn_api_proto_msgTypes[22].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*StateKey); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_fn_api_proto_msgTypes[23].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*StateGetRequest); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_fn_api_proto_msgTypes[24].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*StateGetResponse); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_fn_api_proto_msgTypes[25].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*StateAppendRequest); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_fn_api_proto_msgTypes[26].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*StateAppendResponse); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_fn_api_proto_msgTypes[27].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*StateClearRequest); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_fn_api_proto_msgTypes[28].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*StateClearResponse); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_fn_api_proto_msgTypes[29].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*LogEntry); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_fn_api_proto_msgTypes[30].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*LogControl); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_fn_api_proto_msgTypes[31].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*StartWorkerRequest); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_fn_api_proto_msgTypes[32].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*StartWorkerResponse); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_fn_api_proto_msgTypes[33].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*StopWorkerRequest); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_fn_api_proto_msgTypes[34].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*StopWorkerResponse); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_fn_api_proto_msgTypes[35].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*WorkerStatusRequest); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_fn_api_proto_msgTypes[36].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*WorkerStatusResponse); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_fn_api_proto_msgTypes[43].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*ProcessBundleRequest_CacheToken); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_fn_api_proto_msgTypes[44].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*ProcessBundleRequest_CacheToken_UserState); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_fn_api_proto_msgTypes[45].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*ProcessBundleRequest_CacheToken_SideInput); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_fn_api_proto_msgTypes[49].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*ProcessBundleSplitRequest_DesiredSplit); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_fn_api_proto_msgTypes[51].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*ProcessBundleSplitResponse_ChannelSplit); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_fn_api_proto_msgTypes[52].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*Elements_Data); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_fn_api_proto_msgTypes[53].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*Elements_Timers); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_fn_api_proto_msgTypes[54].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*StateKey_Runner); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_fn_api_proto_msgTypes[55].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*StateKey_IterableSideInput); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_fn_api_proto_msgTypes[56].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*StateKey_MultimapSideInput); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_fn_api_proto_msgTypes[57].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*StateKey_MultimapKeysSideInput); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_fn_api_proto_msgTypes[58].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*StateKey_BagUserState); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_fn_api_proto_msgTypes[59].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*LogEntry_List); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_fn_api_proto_msgTypes[60].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*LogEntry_Severity); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
 	}
-	return nil
-}
-
-func (m *StartWorkerRequest) GetLoggingEndpoint() *pipeline_v1.ApiServiceDescriptor {
-	if m != nil {
-		return m.LoggingEndpoint
+	file_beam_fn_api_proto_msgTypes[2].OneofWrappers = []interface{}{
+		(*InstructionRequest_ProcessBundle)(nil),
+		(*InstructionRequest_ProcessBundleProgress)(nil),
+		(*InstructionRequest_ProcessBundleSplit)(nil),
+		(*InstructionRequest_FinalizeBundle)(nil),
+		(*InstructionRequest_MonitoringInfos)(nil),
+		(*InstructionRequest_Register)(nil),
 	}
-	return nil
-}
-
-func (m *StartWorkerRequest) GetArtifactEndpoint() *pipeline_v1.ApiServiceDescriptor {
-	if m != nil {
-		return m.ArtifactEndpoint
+	file_beam_fn_api_proto_msgTypes[3].OneofWrappers = []interface{}{
+		(*InstructionResponse_ProcessBundle)(nil),
+		(*InstructionResponse_ProcessBundleProgress)(nil),
+		(*InstructionResponse_ProcessBundleSplit)(nil),
+		(*InstructionResponse_FinalizeBundle)(nil),
+		(*InstructionResponse_MonitoringInfos)(nil),
+		(*InstructionResponse_Register)(nil),
 	}
-	return nil
-}
-
-func (m *StartWorkerRequest) GetProvisionEndpoint() *pipeline_v1.ApiServiceDescriptor {
-	if m != nil {
-		return m.ProvisionEndpoint
+	file_beam_fn_api_proto_msgTypes[20].OneofWrappers = []interface{}{
+		(*StateRequest_Get)(nil),
+		(*StateRequest_Append)(nil),
+		(*StateRequest_Clear)(nil),
 	}
-	return nil
-}
-
-func (m *StartWorkerRequest) GetParams() map[string]string {
-	if m != nil {
-		return m.Params
+	file_beam_fn_api_proto_msgTypes[21].OneofWrappers = []interface{}{
+		(*StateResponse_Get)(nil),
+		(*StateResponse_Append)(nil),
+		(*StateResponse_Clear)(nil),
 	}
-	return nil
-}
-
-type StartWorkerResponse struct {
-	Error                string   `protobuf:"bytes,1,opt,name=error,proto3" json:"error,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *StartWorkerResponse) Reset()         { *m = StartWorkerResponse{} }
-func (m *StartWorkerResponse) String() string { return proto.CompactTextString(m) }
-func (*StartWorkerResponse) ProtoMessage()    {}
-func (*StartWorkerResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_6d954c03a4758710, []int{32}
-}
-
-func (m *StartWorkerResponse) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_StartWorkerResponse.Unmarshal(m, b)
-}
-func (m *StartWorkerResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_StartWorkerResponse.Marshal(b, m, deterministic)
-}
-func (m *StartWorkerResponse) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_StartWorkerResponse.Merge(m, src)
-}
-func (m *StartWorkerResponse) XXX_Size() int {
-	return xxx_messageInfo_StartWorkerResponse.Size(m)
-}
-func (m *StartWorkerResponse) XXX_DiscardUnknown() {
-	xxx_messageInfo_StartWorkerResponse.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_StartWorkerResponse proto.InternalMessageInfo
-
-func (m *StartWorkerResponse) GetError() string {
-	if m != nil {
-		return m.Error
+	file_beam_fn_api_proto_msgTypes[22].OneofWrappers = []interface{}{
+		(*StateKey_Runner_)(nil),
+		(*StateKey_MultimapSideInput_)(nil),
+		(*StateKey_BagUserState_)(nil),
+		(*StateKey_IterableSideInput_)(nil),
+		(*StateKey_MultimapKeysSideInput_)(nil),
 	}
-	return ""
-}
-
-type StopWorkerRequest struct {
-	WorkerId             string   `protobuf:"bytes,1,opt,name=worker_id,json=workerId,proto3" json:"worker_id,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *StopWorkerRequest) Reset()         { *m = StopWorkerRequest{} }
-func (m *StopWorkerRequest) String() string { return proto.CompactTextString(m) }
-func (*StopWorkerRequest) ProtoMessage()    {}
-func (*StopWorkerRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_6d954c03a4758710, []int{33}
-}
-
-func (m *StopWorkerRequest) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_StopWorkerRequest.Unmarshal(m, b)
-}
-func (m *StopWorkerRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_StopWorkerRequest.Marshal(b, m, deterministic)
-}
-func (m *StopWorkerRequest) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_StopWorkerRequest.Merge(m, src)
-}
-func (m *StopWorkerRequest) XXX_Size() int {
-	return xxx_messageInfo_StopWorkerRequest.Size(m)
-}
-func (m *StopWorkerRequest) XXX_DiscardUnknown() {
-	xxx_messageInfo_StopWorkerRequest.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_StopWorkerRequest proto.InternalMessageInfo
-
-func (m *StopWorkerRequest) GetWorkerId() string {
-	if m != nil {
-		return m.WorkerId
+	file_beam_fn_api_proto_msgTypes[43].OneofWrappers = []interface{}{
+		(*ProcessBundleRequest_CacheToken_UserState_)(nil),
+		(*ProcessBundleRequest_CacheToken_SideInput_)(nil),
 	}
-	return ""
-}
-
-type StopWorkerResponse struct {
-	Error                string   `protobuf:"bytes,1,opt,name=error,proto3" json:"error,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *StopWorkerResponse) Reset()         { *m = StopWorkerResponse{} }
-func (m *StopWorkerResponse) String() string { return proto.CompactTextString(m) }
-func (*StopWorkerResponse) ProtoMessage()    {}
-func (*StopWorkerResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_6d954c03a4758710, []int{34}
-}
-
-func (m *StopWorkerResponse) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_StopWorkerResponse.Unmarshal(m, b)
-}
-func (m *StopWorkerResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_StopWorkerResponse.Marshal(b, m, deterministic)
-}
-func (m *StopWorkerResponse) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_StopWorkerResponse.Merge(m, src)
-}
-func (m *StopWorkerResponse) XXX_Size() int {
-	return xxx_messageInfo_StopWorkerResponse.Size(m)
-}
-func (m *StopWorkerResponse) XXX_DiscardUnknown() {
-	xxx_messageInfo_StopWorkerResponse.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_StopWorkerResponse proto.InternalMessageInfo
-
-func (m *StopWorkerResponse) GetError() string {
-	if m != nil {
-		return m.Error
-	}
-	return ""
-}
-
-// Request from runner to SDK Harness asking for its status. For more details see
-// https://s.apache.org/beam-fn-api-harness-status
-type WorkerStatusRequest struct {
-	// (Required) Unique ID identifying this request.
-	Id                   string   `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *WorkerStatusRequest) Reset()         { *m = WorkerStatusRequest{} }
-func (m *WorkerStatusRequest) String() string { return proto.CompactTextString(m) }
-func (*WorkerStatusRequest) ProtoMessage()    {}
-func (*WorkerStatusRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_6d954c03a4758710, []int{35}
-}
-
-func (m *WorkerStatusRequest) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_WorkerStatusRequest.Unmarshal(m, b)
-}
-func (m *WorkerStatusRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_WorkerStatusRequest.Marshal(b, m, deterministic)
-}
-func (m *WorkerStatusRequest) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_WorkerStatusRequest.Merge(m, src)
-}
-func (m *WorkerStatusRequest) XXX_Size() int {
-	return xxx_messageInfo_WorkerStatusRequest.Size(m)
-}
-func (m *WorkerStatusRequest) XXX_DiscardUnknown() {
-	xxx_messageInfo_WorkerStatusRequest.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_WorkerStatusRequest proto.InternalMessageInfo
-
-func (m *WorkerStatusRequest) GetId() string {
-	if m != nil {
-		return m.Id
-	}
-	return ""
-}
-
-// Response from SDK Harness to runner containing the debug related status info.
-type WorkerStatusResponse struct {
-	// (Required) Unique ID from the original request.
-	Id string `protobuf:"bytes,1,opt,name=id,proto3" json:"id,omitempty"`
-	// (Optional) Error message if exception encountered generating the status response.
-	Error string `protobuf:"bytes,2,opt,name=error,proto3" json:"error,omitempty"`
-	// (Optional) Status debugging info reported by SDK harness worker. Content and
-	// format is not strongly enforced but should be print-friendly and
-	// appropriate as an HTTP response body for end user. For details of the preferred
-	// info to include in the message see
-	// https://s.apache.org/beam-fn-api-harness-status
-	StatusInfo           string   `protobuf:"bytes,3,opt,name=status_info,json=statusInfo,proto3" json:"status_info,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *WorkerStatusResponse) Reset()         { *m = WorkerStatusResponse{} }
-func (m *WorkerStatusResponse) String() string { return proto.CompactTextString(m) }
-func (*WorkerStatusResponse) ProtoMessage()    {}
-func (*WorkerStatusResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_6d954c03a4758710, []int{36}
-}
-
-func (m *WorkerStatusResponse) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_WorkerStatusResponse.Unmarshal(m, b)
-}
-func (m *WorkerStatusResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_WorkerStatusResponse.Marshal(b, m, deterministic)
-}
-func (m *WorkerStatusResponse) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_WorkerStatusResponse.Merge(m, src)
-}
-func (m *WorkerStatusResponse) XXX_Size() int {
-	return xxx_messageInfo_WorkerStatusResponse.Size(m)
-}
-func (m *WorkerStatusResponse) XXX_DiscardUnknown() {
-	xxx_messageInfo_WorkerStatusResponse.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_WorkerStatusResponse proto.InternalMessageInfo
-
-func (m *WorkerStatusResponse) GetId() string {
-	if m != nil {
-		return m.Id
-	}
-	return ""
-}
-
-func (m *WorkerStatusResponse) GetError() string {
-	if m != nil {
-		return m.Error
-	}
-	return ""
-}
-
-func (m *WorkerStatusResponse) GetStatusInfo() string {
-	if m != nil {
-		return m.StatusInfo
-	}
-	return ""
-}
-
-func init() {
-	proto.RegisterEnum("org.apache.beam.model.fn_execution.v1.LogEntry_Severity_Enum", LogEntry_Severity_Enum_name, LogEntry_Severity_Enum_value)
-	proto.RegisterType((*RemoteGrpcPort)(nil), "org.apache.beam.model.fn_execution.v1.RemoteGrpcPort")
-	proto.RegisterType((*GetProcessBundleDescriptorRequest)(nil), "org.apache.beam.model.fn_execution.v1.GetProcessBundleDescriptorRequest")
-	proto.RegisterType((*InstructionRequest)(nil), "org.apache.beam.model.fn_execution.v1.InstructionRequest")
-	proto.RegisterType((*InstructionResponse)(nil), "org.apache.beam.model.fn_execution.v1.InstructionResponse")
-	proto.RegisterType((*RegisterRequest)(nil), "org.apache.beam.model.fn_execution.v1.RegisterRequest")
-	proto.RegisterType((*RegisterResponse)(nil), "org.apache.beam.model.fn_execution.v1.RegisterResponse")
-	proto.RegisterType((*ProcessBundleDescriptor)(nil), "org.apache.beam.model.fn_execution.v1.ProcessBundleDescriptor")
-	proto.RegisterMapType((map[string]*pipeline_v1.Coder)(nil), "org.apache.beam.model.fn_execution.v1.ProcessBundleDescriptor.CodersEntry")
-	proto.RegisterMapType((map[string]*pipeline_v1.Environment)(nil), "org.apache.beam.model.fn_execution.v1.ProcessBundleDescriptor.EnvironmentsEntry")
-	proto.RegisterMapType((map[string]*pipeline_v1.PCollection)(nil), "org.apache.beam.model.fn_execution.v1.ProcessBundleDescriptor.PcollectionsEntry")
-	proto.RegisterMapType((map[string]*pipeline_v1.PTransform)(nil), "org.apache.beam.model.fn_execution.v1.ProcessBundleDescriptor.TransformsEntry")
-	proto.RegisterMapType((map[string]*pipeline_v1.WindowingStrategy)(nil), "org.apache.beam.model.fn_execution.v1.ProcessBundleDescriptor.WindowingStrategiesEntry")
-	proto.RegisterType((*BundleApplication)(nil), "org.apache.beam.model.fn_execution.v1.BundleApplication")
-	proto.RegisterMapType((map[string]*timestamp.Timestamp)(nil), "org.apache.beam.model.fn_execution.v1.BundleApplication.OutputWatermarksEntry")
-	proto.RegisterType((*DelayedBundleApplication)(nil), "org.apache.beam.model.fn_execution.v1.DelayedBundleApplication")
-	proto.RegisterType((*ProcessBundleRequest)(nil), "org.apache.beam.model.fn_execution.v1.ProcessBundleRequest")
-	proto.RegisterType((*ProcessBundleRequest_CacheToken)(nil), "org.apache.beam.model.fn_execution.v1.ProcessBundleRequest.CacheToken")
-	proto.RegisterType((*ProcessBundleRequest_CacheToken_UserState)(nil), "org.apache.beam.model.fn_execution.v1.ProcessBundleRequest.CacheToken.UserState")
-	proto.RegisterType((*ProcessBundleRequest_CacheToken_SideInput)(nil), "org.apache.beam.model.fn_execution.v1.ProcessBundleRequest.CacheToken.SideInput")
-	proto.RegisterType((*ProcessBundleResponse)(nil), "org.apache.beam.model.fn_execution.v1.ProcessBundleResponse")
-	proto.RegisterMapType((map[string][]byte)(nil), "org.apache.beam.model.fn_execution.v1.ProcessBundleResponse.MonitoringDataEntry")
-	proto.RegisterType((*ProcessBundleProgressRequest)(nil), "org.apache.beam.model.fn_execution.v1.ProcessBundleProgressRequest")
-	proto.RegisterType((*ProcessBundleProgressMetadataRequest)(nil), "org.apache.beam.model.fn_execution.v1.ProcessBundleProgressMetadataRequest")
-	proto.RegisterType((*ProcessBundleProgressResponse)(nil), "org.apache.beam.model.fn_execution.v1.ProcessBundleProgressResponse")
-	proto.RegisterMapType((map[string][]byte)(nil), "org.apache.beam.model.fn_execution.v1.ProcessBundleProgressResponse.MonitoringDataEntry")
-	proto.RegisterType((*ProcessBundleProgressMetadataResponse)(nil), "org.apache.beam.model.fn_execution.v1.ProcessBundleProgressMetadataResponse")
-	proto.RegisterMapType((map[string]*pipeline_v1.MonitoringInfo)(nil), "org.apache.beam.model.fn_execution.v1.ProcessBundleProgressMetadataResponse.MonitoringInfoEntry")
-	proto.RegisterType((*ProcessBundleSplitRequest)(nil), "org.apache.beam.model.fn_execution.v1.ProcessBundleSplitRequest")
-	proto.RegisterMapType((map[string]*ProcessBundleSplitRequest_DesiredSplit)(nil), "org.apache.beam.model.fn_execution.v1.ProcessBundleSplitRequest.DesiredSplitsEntry")
-	proto.RegisterType((*ProcessBundleSplitRequest_DesiredSplit)(nil), "org.apache.beam.model.fn_execution.v1.ProcessBundleSplitRequest.DesiredSplit")
-	proto.RegisterType((*ProcessBundleSplitResponse)(nil), "org.apache.beam.model.fn_execution.v1.ProcessBundleSplitResponse")
-	proto.RegisterType((*ProcessBundleSplitResponse_ChannelSplit)(nil), "org.apache.beam.model.fn_execution.v1.ProcessBundleSplitResponse.ChannelSplit")
-	proto.RegisterType((*FinalizeBundleRequest)(nil), "org.apache.beam.model.fn_execution.v1.FinalizeBundleRequest")
-	proto.RegisterType((*FinalizeBundleResponse)(nil), "org.apache.beam.model.fn_execution.v1.FinalizeBundleResponse")
-	proto.RegisterType((*Elements)(nil), "org.apache.beam.model.fn_execution.v1.Elements")
-	proto.RegisterType((*Elements_Data)(nil), "org.apache.beam.model.fn_execution.v1.Elements.Data")
-	proto.RegisterType((*Elements_Timers)(nil), "org.apache.beam.model.fn_execution.v1.Elements.Timers")
-	proto.RegisterType((*StateRequest)(nil), "org.apache.beam.model.fn_execution.v1.StateRequest")
-	proto.RegisterType((*StateResponse)(nil), "org.apache.beam.model.fn_execution.v1.StateResponse")
-	proto.RegisterType((*StateKey)(nil), "org.apache.beam.model.fn_execution.v1.StateKey")
-	proto.RegisterType((*StateKey_Runner)(nil), "org.apache.beam.model.fn_execution.v1.StateKey.Runner")
-	proto.RegisterType((*StateKey_IterableSideInput)(nil), "org.apache.beam.model.fn_execution.v1.StateKey.IterableSideInput")
-	proto.RegisterType((*StateKey_MultimapSideInput)(nil), "org.apache.beam.model.fn_execution.v1.StateKey.MultimapSideInput")
-	proto.RegisterType((*StateKey_MultimapKeysSideInput)(nil), "org.apache.beam.model.fn_execution.v1.StateKey.MultimapKeysSideInput")
-	proto.RegisterType((*StateKey_BagUserState)(nil), "org.apache.beam.model.fn_execution.v1.StateKey.BagUserState")
-	proto.RegisterType((*StateGetRequest)(nil), "org.apache.beam.model.fn_execution.v1.StateGetRequest")
-	proto.RegisterType((*StateGetResponse)(nil), "org.apache.beam.model.fn_execution.v1.StateGetResponse")
-	proto.RegisterType((*StateAppendRequest)(nil), "org.apache.beam.model.fn_execution.v1.StateAppendRequest")
-	proto.RegisterType((*StateAppendResponse)(nil), "org.apache.beam.model.fn_execution.v1.StateAppendResponse")
-	proto.RegisterType((*StateClearRequest)(nil), "org.apache.beam.model.fn_execution.v1.StateClearRequest")
-	proto.RegisterType((*StateClearResponse)(nil), "org.apache.beam.model.fn_execution.v1.StateClearResponse")
-	proto.RegisterType((*LogEntry)(nil), "org.apache.beam.model.fn_execution.v1.LogEntry")
-	proto.RegisterType((*LogEntry_List)(nil), "org.apache.beam.model.fn_execution.v1.LogEntry.List")
-	proto.RegisterType((*LogEntry_Severity)(nil), "org.apache.beam.model.fn_execution.v1.LogEntry.Severity")
-	proto.RegisterType((*LogControl)(nil), "org.apache.beam.model.fn_execution.v1.LogControl")
-	proto.RegisterType((*StartWorkerRequest)(nil), "org.apache.beam.model.fn_execution.v1.StartWorkerRequest")
-	proto.RegisterMapType((map[string]string)(nil), "org.apache.beam.model.fn_execution.v1.StartWorkerRequest.ParamsEntry")
-	proto.RegisterType((*StartWorkerResponse)(nil), "org.apache.beam.model.fn_execution.v1.StartWorkerResponse")
-	proto.RegisterType((*StopWorkerRequest)(nil), "org.apache.beam.model.fn_execution.v1.StopWorkerRequest")
-	proto.RegisterType((*StopWorkerResponse)(nil), "org.apache.beam.model.fn_execution.v1.StopWorkerResponse")
-	proto.RegisterType((*WorkerStatusRequest)(nil), "org.apache.beam.model.fn_execution.v1.WorkerStatusRequest")
-	proto.RegisterType((*WorkerStatusResponse)(nil), "org.apache.beam.model.fn_execution.v1.WorkerStatusResponse")
-}
-
-func init() { proto.RegisterFile("beam_fn_api.proto", fileDescriptor_6d954c03a4758710) }
-
-var fileDescriptor_6d954c03a4758710 = []byte{
-	// 3018 bytes of a gzipped FileDescriptorProto
-	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xbc, 0x5a, 0x4d, 0x6c, 0x1b, 0xc7,
-	0xf5, 0xf7, 0xf2, 0x4b, 0xe4, 0x23, 0x25, 0x51, 0x23, 0xc9, 0xa6, 0xf7, 0x9f, 0xfc, 0xa3, 0x2c,
-	0xe2, 0x42, 0x68, 0x1b, 0xda, 0x56, 0x8c, 0xc4, 0x4e, 0xd3, 0x24, 0xfa, 0xa0, 0x6d, 0xda, 0xb2,
-	0xcc, 0xae, 0xe4, 0x3a, 0x0d, 0x1a, 0x6c, 0x57, 0xdc, 0x21, 0xbd, 0xf0, 0x72, 0x77, 0x33, 0xb3,
-	0x94, 0xac, 0x34, 0x6d, 0x1a, 0xb4, 0x68, 0x91, 0x02, 0x41, 0x2f, 0x05, 0xfa, 0x85, 0x5c, 0x8a,
-	0x1e, 0x1a, 0x14, 0xe8, 0xb1, 0x87, 0x5e, 0x73, 0xe8, 0xb9, 0x97, 0xdc, 0x7b, 0xeb, 0xe7, 0xa1,
-	0x97, 0x5e, 0x5b, 0xec, 0xcc, 0xec, 0x07, 0xc9, 0xa5, 0xbd, 0x24, 0x95, 0xdc, 0x76, 0xbe, 0x7e,
-	0xbf, 0x37, 0x6f, 0xe6, 0xbd, 0x79, 0xf3, 0x66, 0x61, 0xe9, 0x10, 0xeb, 0x3d, 0xad, 0x63, 0x6b,
-	0xba, 0x6b, 0xd6, 0x5d, 0xe2, 0x78, 0x0e, 0xba, 0xe0, 0x90, 0x6e, 0x5d, 0x77, 0xf5, 0xf6, 0x03,
-	0x5c, 0xf7, 0x5b, 0xeb, 0x3d, 0xc7, 0xc0, 0x56, 0xbd, 0x63, 0x6b, 0xf8, 0x11, 0x6e, 0xf7, 0x3d,
-	0xd3, 0xb1, 0xeb, 0x47, 0x97, 0xe5, 0x55, 0x36, 0x92, 0xf4, 0x6d, 0x1b, 0x93, 0x68, 0xb4, 0xbc,
-	0x88, 0x6d, 0xc3, 0x75, 0x4c, 0xdb, 0xa3, 0xa2, 0x62, 0xad, 0xeb, 0x38, 0x5d, 0x0b, 0x5f, 0x64,
-	0xa5, 0xc3, 0x7e, 0xe7, 0xa2, 0x81, 0x69, 0x9b, 0x98, 0xae, 0xe7, 0x10, 0xd1, 0xe3, 0x99, 0xe1,
-	0x1e, 0x9e, 0xd9, 0xc3, 0xd4, 0xd3, 0x7b, 0xae, 0xe8, 0xf0, 0xff, 0x23, 0x10, 0x7d, 0xa2, 0x33,
-	0x39, 0xc6, 0xb4, 0x1f, 0x13, 0xdd, 0x75, 0x31, 0x09, 0x44, 0x98, 0xef, 0x61, 0x8f, 0x98, 0x6d,
-	0x51, 0x54, 0x7e, 0x29, 0xc1, 0x82, 0x8a, 0x7b, 0x8e, 0x87, 0x6f, 0x10, 0xb7, 0xdd, 0x72, 0x88,
-	0x87, 0x7a, 0x70, 0x56, 0x77, 0x4d, 0x8d, 0x62, 0x72, 0x64, 0xb6, 0xb1, 0x16, 0x89, 0x58, 0x93,
-	0xd6, 0xa4, 0xf5, 0xf2, 0xc6, 0x4b, 0xf5, 0x64, 0xa5, 0xb8, 0xa6, 0x8b, 0x2d, 0xd3, 0xc6, 0xf5,
-	0xa3, 0xcb, 0xf5, 0x4d, 0xd7, 0xdc, 0xe7, 0xe3, 0x77, 0xc2, 0xe1, 0xea, 0x8a, 0x9e, 0x50, 0x8b,
-	0xce, 0x43, 0xb1, 0xed, 0x18, 0x98, 0x68, 0xa6, 0x51, 0xcb, 0xac, 0x49, 0xeb, 0x25, 0x75, 0x8e,
-	0x95, 0x9b, 0x86, 0x62, 0xc0, 0xb3, 0x37, 0xb0, 0xd7, 0x22, 0x4e, 0x1b, 0x53, 0xba, 0xd5, 0xb7,
-	0x0d, 0x2b, 0x0e, 0x87, 0xdf, 0xee, 0x63, 0xea, 0xa1, 0xd7, 0xe0, 0x29, 0x97, 0xf7, 0xd0, 0x0e,
-	0x59, 0x97, 0x98, 0xc4, 0x3e, 0xa6, 0xc4, 0x30, 0xcf, 0xbb, 0xc9, 0x28, 0x4d, 0x43, 0xf9, 0x6f,
-	0x1e, 0x50, 0xd3, 0xa6, 0x1e, 0xe9, 0xb7, 0x7d, 0x3d, 0x06, 0xb8, 0x17, 0x60, 0xc1, 0x8c, 0x6a,
-	0x23, 0xa4, 0xf9, 0x58, 0x6d, 0xd3, 0x40, 0x18, 0x16, 0x06, 0xe9, 0x6b, 0x7f, 0x9b, 0x63, 0x6a,
-	0xfa, 0x4a, 0x3d, 0xd5, 0xde, 0xa9, 0x0f, 0x4c, 0x4f, 0x90, 0xdf, 0x3c, 0xa3, 0xce, 0x0f, 0x08,
-	0x8c, 0xbe, 0x0b, 0xe7, 0x86, 0x66, 0xe9, 0x12, 0xa7, 0x4b, 0x30, 0xa5, 0xb5, 0xbf, 0x73, 0xbe,
-	0xed, 0x69, 0xf8, 0x5a, 0x02, 0x24, 0xe2, 0x5d, 0x75, 0x93, 0xda, 0x51, 0x1f, 0x56, 0x86, 0xf8,
-	0xa9, 0x6b, 0x99, 0x5e, 0xed, 0x1f, 0x9c, 0xfc, 0xf5, 0x69, 0xc8, 0xf7, 0x7d, 0x84, 0x88, 0x19,
-	0xb9, 0x23, 0x8d, 0xe8, 0x01, 0x2c, 0x76, 0x4c, 0x5b, 0xb7, 0xcc, 0x77, 0x70, 0xa0, 0xde, 0x7f,
-	0x72, 0xc6, 0x57, 0x52, 0x32, 0x5e, 0x17, 0xc3, 0x87, 0xf5, 0xbb, 0xd0, 0x19, 0x68, 0x40, 0x3f,
-	0x93, 0x60, 0x6d, 0x8c, 0x86, 0xb5, 0x1e, 0xf6, 0x74, 0x43, 0xf7, 0xf4, 0xda, 0xbf, 0x38, 0xf7,
-	0xed, 0x59, 0x54, 0x7d, 0x47, 0x80, 0x45, 0xa2, 0x3c, 0xed, 0x3e, 0xae, 0x1f, 0xba, 0x07, 0x45,
-	0x82, 0xbb, 0x26, 0xf5, 0x30, 0xa9, 0xfd, 0x95, 0x0b, 0xf0, 0x62, 0x4a, 0x01, 0x54, 0x31, 0x2e,
-	0xe2, 0x0a, 0xa1, 0xb6, 0x4a, 0x30, 0x47, 0x78, 0xb5, 0xf2, 0x51, 0x01, 0x96, 0x07, 0x2c, 0x80,
-	0xba, 0x8e, 0x4d, 0x71, 0x5a, 0x13, 0x58, 0x81, 0x3c, 0x26, 0xc4, 0x21, 0xc2, 0x7c, 0x79, 0x01,
-	0x75, 0xc6, 0x19, 0xc6, 0x2b, 0xd3, 0x19, 0x06, 0x97, 0x69, 0xd4, 0x32, 0xde, 0x7b, 0xa2, 0x65,
-	0xec, 0xcc, 0x66, 0x19, 0x21, 0xf1, 0x18, 0xd3, 0x38, 0x7a, 0xbc, 0x69, 0x6c, 0xce, 0x60, 0x1a,
-	0x21, 0x75, 0x92, 0x6d, 0x98, 0x63, 0x6d, 0xe3, 0xab, 0x53, 0xda, 0x46, 0x48, 0x37, 0x6c, 0x1c,
-	0x3f, 0x9f, 0xc0, 0x38, 0x76, 0x4f, 0xc7, 0x38, 0x42, 0x59, 0x9e, 0x60, 0x1d, 0x5f, 0x1f, 0xb5,
-	0x8e, 0x97, 0x26, 0xb6, 0x8e, 0x90, 0x2c, 0x32, 0x0f, 0xf0, 0x71, 0x79, 0xbd, 0xf2, 0xa1, 0x04,
-	0x8b, 0x43, 0xa6, 0x84, 0xde, 0x81, 0xf3, 0x63, 0x8f, 0x9d, 0x9a, 0xb4, 0x96, 0x5d, 0x2f, 0x6f,
-	0xbc, 0x3a, 0x8d, 0x26, 0x62, 0x07, 0xdc, 0xb9, 0x31, 0x67, 0x96, 0x82, 0xa0, 0x3a, 0x2c, 0xbb,
-	0xf2, 0x51, 0x19, 0xce, 0x8d, 0x01, 0x42, 0x0b, 0x90, 0x09, 0x6d, 0x37, 0x63, 0x1a, 0xc8, 0x06,
-	0xf0, 0x88, 0x6e, 0xd3, 0x8e, 0x43, 0x7a, 0xb4, 0x96, 0x61, 0xc2, 0xee, 0xcd, 0x26, 0x6c, 0xfd,
-	0x20, 0x04, 0x6c, 0xd8, 0x1e, 0x39, 0x51, 0x63, 0x0c, 0xc8, 0x83, 0x8a, 0xdb, 0x76, 0x2c, 0x0b,
-	0x33, 0x8f, 0x41, 0x6b, 0x59, 0xc6, 0xd8, 0x9a, 0x91, 0xb1, 0x15, 0x83, 0xe4, 0x9c, 0x03, 0x2c,
-	0xe8, 0xc7, 0x12, 0xac, 0x1c, 0x9b, 0xb6, 0xe1, 0x1c, 0x9b, 0x76, 0x57, 0xa3, 0x1e, 0xd1, 0x3d,
-	0xdc, 0x35, 0x31, 0xad, 0xe5, 0x18, 0xfd, 0xfd, 0x19, 0xe9, 0xef, 0x07, 0xd0, 0xfb, 0x21, 0x32,
-	0x97, 0x62, 0xf9, 0x78, 0xb4, 0x05, 0x1d, 0x42, 0x81, 0x45, 0x35, 0xb4, 0x96, 0x67, 0xec, 0xb7,
-	0x66, 0x64, 0xdf, 0x66, 0x60, 0x9c, 0x50, 0x20, 0xfb, 0x6a, 0xc6, 0xf6, 0x91, 0x49, 0x1c, 0xbb,
-	0x87, 0x6d, 0x8f, 0xd6, 0x0a, 0xa7, 0xa2, 0xe6, 0x46, 0x0c, 0x52, 0xa8, 0x39, 0xce, 0x82, 0x1e,
-	0xc1, 0x53, 0xd4, 0xd3, 0x3d, 0xac, 0x8d, 0x09, 0x1a, 0xe7, 0x66, 0x0b, 0x1a, 0xcf, 0x33, 0xf0,
-	0xa4, 0x26, 0x9f, 0xd9, 0x8f, 0x8e, 0xc9, 0x38, 0xe6, 0xe2, 0x8c, 0xcc, 0x0c, 0x3c, 0xa9, 0x49,
-	0xb6, 0x60, 0x71, 0x68, 0xbf, 0xa3, 0x2a, 0x64, 0x1f, 0xe2, 0x13, 0x61, 0x64, 0xfe, 0x27, 0xda,
-	0x86, 0xfc, 0x91, 0x6e, 0xf5, 0x31, 0x3b, 0x16, 0xcb, 0x1b, 0xcf, 0xa7, 0x90, 0xa3, 0x15, 0xa2,
-	0xaa, 0x7c, 0xec, 0xcb, 0x99, 0xab, 0x92, 0xec, 0xc0, 0xd2, 0xc8, 0x5e, 0x4f, 0xe0, 0xdb, 0x19,
-	0xe4, 0xab, 0xa7, 0xe1, 0xdb, 0x0e, 0x61, 0xe3, 0x84, 0xef, 0x42, 0x6d, 0xdc, 0xee, 0x4e, 0xe0,
-	0xbd, 0x35, 0xc8, 0x7b, 0x25, 0x05, 0xef, 0x30, 0xfa, 0x49, 0x9c, 0xbd, 0x0d, 0xe5, 0xd8, 0xee,
-	0x4e, 0x20, 0x7c, 0x75, 0x90, 0x70, 0x3d, 0x05, 0x21, 0x03, 0x1c, 0xd2, 0xe9, 0xc8, 0xc6, 0x3e,
-	0x1d, 0x9d, 0xc6, 0x60, 0x63, 0x84, 0xca, 0xaf, 0xb2, 0xb0, 0xc4, 0x6d, 0x6b, 0xd3, 0x75, 0x2d,
-	0xb3, 0xcd, 0xee, 0x6c, 0xe8, 0x59, 0xa8, 0x84, 0x7e, 0x32, 0x8a, 0xaf, 0xca, 0x61, 0x5d, 0xd3,
-	0xf0, 0xef, 0x47, 0xa6, 0xed, 0xf6, 0xbd, 0xd8, 0xfd, 0x88, 0x95, 0x9b, 0x06, 0xaa, 0xc1, 0x1c,
-	0xb6, 0xb0, 0xcf, 0x54, 0xcb, 0xae, 0x49, 0xeb, 0x15, 0x35, 0x28, 0xa2, 0x6f, 0xc3, 0x92, 0xd3,
-	0xf7, 0xfc, 0x51, 0xc7, 0xba, 0x87, 0x49, 0x4f, 0x27, 0x0f, 0x03, 0xbf, 0x97, 0xd6, 0xd1, 0x8f,
-	0x08, 0x5b, 0xbf, 0xcb, 0x10, 0xef, 0x87, 0x80, 0xdc, 0x1b, 0x54, 0x9d, 0xa1, 0x6a, 0xd4, 0x02,
-	0x30, 0xa9, 0x76, 0xe8, 0xf4, 0x6d, 0x03, 0x1b, 0xb5, 0xfc, 0x9a, 0xb4, 0xbe, 0xb0, 0x71, 0x39,
-	0x85, 0xe6, 0x9a, 0x74, 0x8b, 0x8f, 0xa9, 0x37, 0xec, 0x7e, 0x4f, 0x2d, 0x99, 0x41, 0x59, 0xd6,
-	0x60, 0x35, 0x91, 0x3c, 0x61, 0xc5, 0x2e, 0x0d, 0xae, 0x98, 0x5c, 0xe7, 0xf7, 0xe1, 0x7a, 0x70,
-	0x1f, 0xae, 0x1f, 0x04, 0x17, 0xea, 0xf8, 0xea, 0x7c, 0x22, 0x41, 0x6d, 0x07, 0x5b, 0xfa, 0x09,
-	0x36, 0x46, 0x17, 0xe9, 0x4d, 0x28, 0xeb, 0x51, 0x51, 0xdc, 0x82, 0xaf, 0x4e, 0xab, 0x46, 0x35,
-	0x0e, 0x86, 0x6e, 0xc3, 0x8a, 0x88, 0xc2, 0xb1, 0xa1, 0xf9, 0x0e, 0x47, 0x33, 0x7c, 0x31, 0x84,
-	0xf4, 0xe7, 0x47, 0xa4, 0xdf, 0x11, 0xb7, 0x7d, 0x15, 0x85, 0xc3, 0xfc, 0x09, 0x31, 0xd9, 0x95,
-	0x3f, 0xe4, 0x60, 0x25, 0xe9, 0x3a, 0x39, 0xf3, 0x1d, 0x19, 0x99, 0x50, 0x69, 0xfb, 0x33, 0xd5,
-	0x3c, 0xe7, 0x21, 0xb6, 0x83, 0x98, 0xe1, 0xfa, 0x0c, 0x57, 0xdc, 0xfa, 0xb6, 0x3f, 0xea, 0xc0,
-	0x87, 0x53, 0xcb, 0xed, 0xf0, 0x9b, 0xca, 0xff, 0xce, 0x00, 0x44, 0x6d, 0xe8, 0x6d, 0x80, 0x3e,
-	0xc5, 0x44, 0x63, 0xc7, 0x80, 0xd0, 0x7d, 0xeb, 0x74, 0x78, 0xeb, 0xf7, 0x28, 0x26, 0xfb, 0x3e,
-	0xee, 0xcd, 0x33, 0x6a, 0xa9, 0x1f, 0x14, 0x7c, 0x4a, 0x6a, 0x1a, 0x58, 0x63, 0x66, 0x26, 0x56,
-	0xe2, 0xb4, 0x28, 0xf7, 0x4d, 0x03, 0x37, 0x7d, 0x5c, 0x9f, 0x92, 0x06, 0x05, 0xff, 0x0a, 0xc5,
-	0x34, 0x5b, 0x03, 0x66, 0xc7, 0xbc, 0x20, 0x97, 0xa1, 0x14, 0x8a, 0x28, 0xab, 0x50, 0x0a, 0x07,
-	0xa7, 0xf1, 0x1b, 0x0a, 0xcc, 0x47, 0xb3, 0x88, 0x9c, 0x47, 0x39, 0x24, 0x6d, 0x1a, 0x5b, 0x05,
-	0xc8, 0x79, 0x27, 0x2e, 0x56, 0x3e, 0xcd, 0xc2, 0x6a, 0xe2, 0x75, 0xcb, 0xbf, 0xc5, 0x11, 0x4c,
-	0x4d, 0xa3, 0xaf, 0x5b, 0x1a, 0x71, 0x1c, 0x2f, 0x58, 0xfa, 0xd7, 0x52, 0xea, 0x63, 0x9c, 0x51,
-	0xa9, 0xf3, 0x01, 0xac, 0xea, 0xa3, 0xa2, 0x6f, 0x42, 0xb5, 0xe7, 0xd8, 0xa6, 0xe7, 0x10, 0x3f,
-	0x58, 0x33, 0xed, 0x8e, 0x13, 0x84, 0x89, 0x69, 0x3c, 0xc7, 0x9d, 0x70, 0x68, 0xd3, 0xee, 0x38,
-	0xea, 0x62, 0x6f, 0xa0, 0x4c, 0xd1, 0x0b, 0xb0, 0xea, 0x9b, 0x8b, 0x49, 0x30, 0xd5, 0xc4, 0xd5,
-	0x86, 0xdb, 0x72, 0x6e, 0x4d, 0x5a, 0x2f, 0xaa, 0x2b, 0x41, 0xe3, 0xf5, 0x58, 0x1b, 0x3a, 0x81,
-	0x18, 0x8e, 0xc6, 0xae, 0x38, 0xf9, 0xe9, 0x23, 0xaa, 0x40, 0xa3, 0x31, 0x59, 0x77, 0x74, 0x4f,
-	0xe7, 0x3e, 0x74, 0xa1, 0x37, 0x50, 0x29, 0x6f, 0xc2, 0x72, 0x42, 0xb7, 0x04, 0x6f, 0xb7, 0x12,
-	0xf7, 0x76, 0x95, 0x98, 0x47, 0xbb, 0x95, 0x2b, 0x4a, 0xd5, 0x8c, 0xd2, 0x80, 0xa7, 0x1e, 0x97,
-	0xef, 0x49, 0x79, 0xc3, 0x57, 0x0e, 0xe0, 0xb9, 0x34, 0xb9, 0x0c, 0xf4, 0x65, 0x40, 0x43, 0xab,
-	0xc8, 0x21, 0xb3, 0xeb, 0x25, 0xb5, 0x3a, 0xb8, 0x28, 0x4d, 0x43, 0xf9, 0x34, 0x03, 0x4f, 0x3f,
-	0xf6, 0xce, 0xfd, 0x19, 0xef, 0x8a, 0xf7, 0xa5, 0x71, 0x2b, 0xfc, 0xc6, 0x69, 0x64, 0x0c, 0x3e,
-	0xd7, 0x95, 0xbe, 0x95, 0x2b, 0x66, 0xaa, 0xd9, 0x5b, 0xb9, 0x62, 0xae, 0x9a, 0x57, 0x3e, 0xce,
-	0xc0, 0x85, 0x54, 0xd7, 0x6b, 0xf4, 0xc1, 0xa0, 0x0a, 0x7c, 0x0d, 0x8b, 0xcb, 0xeb, 0xb7, 0x4e,
-	0xf3, 0x1a, 0x3f, 0xb4, 0x14, 0x23, 0xaa, 0xf0, 0x2b, 0x65, 0x2f, 0xae, 0x8a, 0xb0, 0x5b, 0x82,
-	0x2a, 0x6e, 0x0c, 0x1e, 0xf1, 0x53, 0x6c, 0x85, 0xd8, 0xc9, 0xff, 0x7e, 0x0e, 0xce, 0x8f, 0xcd,
-	0x4a, 0xa6, 0xcd, 0x80, 0xbd, 0x03, 0x0b, 0x06, 0xa6, 0x26, 0xc1, 0x06, 0xcf, 0xfd, 0x04, 0xbb,
-	0x74, 0x7f, 0xd6, 0xb4, 0x68, 0x7d, 0x87, 0xc3, 0xb2, 0x3a, 0x11, 0x70, 0xcd, 0x1b, 0xf1, 0x3a,
-	0xf9, 0xf7, 0x12, 0x54, 0xe2, 0xbd, 0xd0, 0x06, 0xac, 0x76, 0x88, 0xce, 0x05, 0x76, 0x3a, 0x1a,
-	0xc1, 0x3d, 0xdd, 0xb4, 0x0d, 0xcc, 0xd3, 0xf7, 0x92, 0xba, 0x1c, 0x34, 0xde, 0xed, 0xa8, 0x41,
-	0x13, 0xba, 0x04, 0x2b, 0xba, 0x65, 0x39, 0xc7, 0xc1, 0x04, 0x34, 0xfe, 0x6c, 0xc1, 0xa6, 0x91,
-	0x55, 0x91, 0x68, 0x63, 0xf8, 0x2d, 0xd6, 0x82, 0xae, 0x42, 0x0d, 0x53, 0xcf, 0xec, 0xe9, 0x7e,
-	0xe0, 0xc2, 0xcf, 0x18, 0x11, 0x7c, 0x52, 0xb6, 0x2e, 0x59, 0xf5, 0x6c, 0xd8, 0xce, 0x8e, 0x9b,
-	0x86, 0x68, 0x95, 0x7f, 0x22, 0x01, 0x1a, 0x9d, 0x56, 0xc2, 0x3a, 0xb7, 0x07, 0xd7, 0xf9, 0xce,
-	0xa9, 0x2a, 0x33, 0xbe, 0x07, 0xfe, 0x93, 0x05, 0x79, 0x7c, 0xfa, 0x0d, 0xbd, 0x05, 0xf3, 0x2e,
-	0x31, 0x7b, 0x3a, 0x39, 0x11, 0x47, 0x20, 0xb7, 0x90, 0xe9, 0x23, 0xc0, 0x8a, 0x80, 0xe3, 0x47,
-	0xdf, 0xe7, 0x75, 0xc4, 0xf6, 0x61, 0xa1, 0xfd, 0x40, 0xb7, 0x6d, 0x6c, 0x0d, 0x6e, 0xd2, 0xbd,
-	0x99, 0x13, 0x94, 0xf5, 0x6d, 0x8e, 0xcb, 0x2b, 0xe7, 0xdb, 0xb1, 0x12, 0x95, 0x7f, 0x21, 0x41,
-	0x25, 0xde, 0x9e, 0x26, 0x76, 0xb9, 0x04, 0x2b, 0x96, 0x4e, 0x3d, 0x2d, 0x50, 0x7b, 0x70, 0xcb,
-	0xe1, 0x1b, 0x0b, 0xf9, 0x6d, 0x2d, 0xde, 0x24, 0x76, 0x15, 0xba, 0x02, 0x67, 0x3b, 0x26, 0xa1,
-	0x9e, 0x16, 0xaa, 0x32, 0x7e, 0x33, 0xca, 0xaa, 0x2b, 0xac, 0x55, 0x15, 0x8d, 0x62, 0x94, 0xf2,
-	0x2a, 0xac, 0x26, 0xbe, 0x0f, 0xa4, 0x3d, 0x17, 0x6b, 0x70, 0x36, 0x39, 0x87, 0xaa, 0xfc, 0x39,
-	0x0b, 0xc5, 0x60, 0xc7, 0xa3, 0x9b, 0x90, 0x63, 0x87, 0x0b, 0xdf, 0x37, 0x57, 0x52, 0xea, 0x3b,
-	0x18, 0x5e, 0xf7, 0x8f, 0x05, 0x95, 0x21, 0xa0, 0x3d, 0x28, 0xb0, 0xac, 0x44, 0xb0, 0x47, 0x5e,
-	0x9c, 0x14, 0xeb, 0x80, 0x8d, 0x56, 0x05, 0x8a, 0xfc, 0x1d, 0xc8, 0xf9, 0xe8, 0x69, 0xfd, 0xdc,
-	0xf0, 0xd2, 0x65, 0x46, 0x97, 0x0e, 0x89, 0xb9, 0xf2, 0x0b, 0x29, 0x97, 0xfa, 0x1c, 0xcc, 0x99,
-	0x54, 0xf3, 0x57, 0x4d, 0x04, 0x5c, 0x05, 0x93, 0xee, 0xea, 0xd4, 0x93, 0x7f, 0x27, 0x41, 0x81,
-	0x4b, 0x74, 0x8a, 0x12, 0x7c, 0x01, 0x16, 0x79, 0x5a, 0xa8, 0xa3, 0xf7, 0x4c, 0xeb, 0xc4, 0xef,
-	0x95, 0xe5, 0x50, 0xac, 0xfa, 0x3a, 0xab, 0x6d, 0x1a, 0xe8, 0x6c, 0xa8, 0xcb, 0x1c, 0x93, 0x55,
-	0x94, 0xe2, 0xd2, 0xe6, 0xe3, 0xd2, 0x2a, 0x3f, 0xcd, 0x42, 0x85, 0xc5, 0xe2, 0xc1, 0x2e, 0x19,
-	0xce, 0xab, 0x8e, 0xce, 0x21, 0x93, 0x34, 0x87, 0x5d, 0x28, 0xf1, 0x8c, 0x99, 0xef, 0xef, 0xb2,
-	0xcc, 0xb7, 0x5d, 0x4c, 0xb9, 0x8e, 0x8c, 0xfe, 0x36, 0x3e, 0x51, 0x8b, 0x54, 0x7c, 0xa1, 0xdb,
-	0x90, 0xed, 0x62, 0x6f, 0xd2, 0x97, 0x21, 0x06, 0x74, 0x03, 0xc7, 0x9e, 0xdf, 0x7c, 0x14, 0x74,
-	0x00, 0x05, 0xdd, 0x75, 0xb1, 0x6d, 0x04, 0x8f, 0x35, 0xd7, 0x26, 0xc1, 0xdb, 0x64, 0x43, 0x23,
-	0x48, 0x81, 0x85, 0xbe, 0x06, 0xf9, 0xb6, 0x85, 0x75, 0x12, 0x3c, 0xc8, 0x5c, 0x9d, 0x04, 0x74,
-	0xdb, 0x1f, 0x19, 0x61, 0x72, 0xa4, 0xf8, 0xeb, 0xd5, 0x1f, 0x33, 0x30, 0x2f, 0x96, 0x45, 0x38,
-	0xec, 0xe1, 0x75, 0x49, 0x7e, 0xa0, 0xda, 0x1d, 0x50, 0xdc, 0x4b, 0x13, 0x2b, 0x2e, 0x7c, 0x34,
-	0x60, 0x9a, 0xbb, 0x37, 0xac, 0xb9, 0x97, 0xa7, 0xd1, 0x5c, 0x88, 0x19, 0xa8, 0x4e, 0x1d, 0x52,
-	0xdd, 0xb5, 0x29, 0x54, 0x17, 0x82, 0x0a, 0xdd, 0xc5, 0x9f, 0x36, 0xfe, 0x54, 0x84, 0x62, 0xb0,
-	0xa9, 0x50, 0x0b, 0x0a, 0xfc, 0x1f, 0x06, 0x71, 0xcf, 0x7e, 0x71, 0xc2, 0x5d, 0x59, 0x57, 0xd9,
-	0x68, 0x5f, 0x7c, 0x8e, 0x83, 0x28, 0x2c, 0xf7, 0xfa, 0x96, 0x1f, 0x06, 0xb8, 0xda, 0xc8, 0x9d,
-	0x7a, 0x73, 0x52, 0xf8, 0x3b, 0x02, 0x2a, 0x7e, 0x89, 0x5e, 0xea, 0x0d, 0x57, 0x22, 0x03, 0x16,
-	0x0e, 0xf5, 0xae, 0x16, 0x4b, 0x1b, 0x64, 0x27, 0x7a, 0x78, 0x0c, 0xf9, 0xb6, 0xf4, 0x6e, 0x3c,
-	0x45, 0x50, 0x39, 0x8c, 0x95, 0xfd, 0xa9, 0x99, 0x1e, 0x26, 0xfa, 0xa1, 0x85, 0xe3, 0x53, 0xcb,
-	0x4d, 0x37, 0xb5, 0xa6, 0x80, 0x1a, 0x98, 0x9a, 0x39, 0x5c, 0x89, 0xbe, 0x27, 0x41, 0x2d, 0x54,
-	0xe8, 0x43, 0x7c, 0x42, 0xe3, 0xd4, 0x79, 0x46, 0xdd, 0x98, 0x56, 0xab, 0xb7, 0xf1, 0x09, 0x8d,
-	0xd3, 0xaf, 0xf6, 0x92, 0x1a, 0x64, 0x19, 0x0a, 0x7c, 0x99, 0xe3, 0x11, 0x5b, 0x85, 0x45, 0x6c,
-	0x32, 0x81, 0xa5, 0x91, 0x89, 0x9c, 0x52, 0xae, 0xc2, 0x77, 0xd7, 0xfc, 0x61, 0x45, 0x1c, 0x2d,
-	0xa2, 0x24, 0xff, 0x40, 0x82, 0xa5, 0x91, 0x8d, 0xf1, 0x19, 0x93, 0x06, 0x53, 0xcf, 0x45, 0x53,
-	0x3f, 0x82, 0xd5, 0x44, 0x45, 0x7e, 0xd6, 0xd3, 0x7f, 0x0f, 0x2a, 0xf1, 0x6d, 0x9a, 0x92, 0x2e,
-	0xb2, 0x8d, 0x18, 0x5d, 0x98, 0x01, 0x9b, 0x64, 0xe2, 0x61, 0x0e, 0xe9, 0x75, 0x58, 0x1c, 0x3a,
-	0x54, 0xd0, 0xf3, 0x80, 0xda, 0x8e, 0xed, 0x99, 0x76, 0x9f, 0x05, 0xa4, 0x3c, 0x79, 0x28, 0xf6,
-	0xcb, 0x52, 0xbc, 0x85, 0xe5, 0xc6, 0x94, 0x7b, 0x50, 0x1d, 0xf6, 0xae, 0x13, 0x42, 0x84, 0xd1,
-	0x47, 0x26, 0x8a, 0x3e, 0x94, 0x75, 0x40, 0xa3, 0xa7, 0x53, 0xd8, 0x53, 0x8a, 0xf5, 0x5c, 0x85,
-	0xe5, 0x04, 0x6f, 0xac, 0x2c, 0xc3, 0xd2, 0xc8, 0x49, 0xa4, 0xac, 0x08, 0xd4, 0x01, 0x1f, 0xab,
-	0xfc, 0x3a, 0x07, 0xc5, 0x5d, 0xa7, 0xcb, 0x6f, 0x34, 0xdf, 0x80, 0x22, 0xc5, 0x47, 0x98, 0x98,
-	0x1e, 0x37, 0x92, 0x85, 0xd4, 0x0f, 0xf3, 0x01, 0x44, 0x7d, 0x5f, 0x8c, 0xe7, 0x19, 0xf1, 0x10,
-	0x0e, 0x5d, 0x85, 0x52, 0xf8, 0x63, 0x58, 0x8a, 0x4c, 0x77, 0xd4, 0x19, 0xd5, 0x60, 0xae, 0x87,
-	0x29, 0xd5, 0xbb, 0x58, 0x44, 0x45, 0x41, 0x91, 0xe5, 0x20, 0x89, 0xde, 0xc6, 0x6c, 0x71, 0x4b,
-	0x2a, 0x2f, 0x24, 0xc4, 0x34, 0xf9, 0x34, 0x71, 0x59, 0x61, 0x74, 0xdb, 0x3d, 0x0b, 0x15, 0xcb,
-	0xe9, 0x6a, 0x96, 0x23, 0xf2, 0xe8, 0x73, 0xbc, 0x8b, 0xe5, 0x74, 0x77, 0x45, 0x15, 0x0b, 0xc9,
-	0x1e, 0x10, 0xac, 0x1b, 0xec, 0xed, 0xae, 0xa4, 0x8a, 0x92, 0xfc, 0x06, 0xe4, 0x76, 0x4d, 0xea,
-	0xa1, 0x16, 0xf8, 0xdd, 0x35, 0x6c, 0x7b, 0xc4, 0xc4, 0xc1, 0x3d, 0xec, 0xe2, 0x84, 0x4a, 0x55,
-	0xc1, 0xe2, 0x5f, 0x26, 0xa6, 0x32, 0x81, 0x62, 0xa0, 0x63, 0xa5, 0x03, 0x39, 0x5f, 0xcd, 0x68,
-	0x11, 0xca, 0xf7, 0xf6, 0xf6, 0x5b, 0x8d, 0xed, 0xe6, 0xf5, 0x66, 0x63, 0xa7, 0x7a, 0x06, 0x95,
-	0x20, 0x7f, 0xa0, 0x6e, 0x6e, 0x37, 0xaa, 0x92, 0xff, 0xb9, 0xd3, 0xd8, 0xba, 0x77, 0xa3, 0x9a,
-	0x41, 0x45, 0xc8, 0x35, 0xf7, 0xae, 0xdf, 0xad, 0x66, 0x11, 0x40, 0x61, 0xef, 0xee, 0x41, 0x73,
-	0xbb, 0x51, 0xcd, 0xf9, 0xb5, 0xf7, 0x37, 0xd5, 0xbd, 0x6a, 0xde, 0xef, 0xda, 0x50, 0xd5, 0xbb,
-	0x6a, 0xb5, 0x80, 0x2a, 0x50, 0xdc, 0x56, 0x9b, 0x07, 0xcd, 0xed, 0xcd, 0xdd, 0xea, 0x9c, 0x52,
-	0x01, 0xd8, 0x75, 0xba, 0xdb, 0x8e, 0xed, 0x11, 0xc7, 0x52, 0xfe, 0x92, 0x63, 0x3b, 0x89, 0x78,
-	0xf7, 0x1d, 0xf2, 0x30, 0xfa, 0xbf, 0xe0, 0xff, 0xa0, 0x74, 0xcc, 0x2a, 0x22, 0x23, 0x2e, 0xf2,
-	0x8a, 0xa6, 0x81, 0x0e, 0xa1, 0xda, 0xe6, 0xc3, 0xb5, 0xe0, 0x17, 0x43, 0xb1, 0x0b, 0xa6, 0x7e,
-	0xed, 0x5c, 0x14, 0x80, 0x0d, 0x81, 0xe7, 0x73, 0x58, 0x4e, 0xb7, 0x6b, 0xda, 0xdd, 0x88, 0x23,
-	0x3b, 0x23, 0x87, 0x00, 0x0c, 0x39, 0x0c, 0x58, 0xd2, 0x89, 0x67, 0x76, 0xf4, 0xb6, 0x17, 0x91,
-	0xe4, 0x66, 0x23, 0xa9, 0x06, 0x88, 0x21, 0x4b, 0x07, 0x90, 0x4b, 0x9c, 0x23, 0x93, 0xfa, 0x1b,
-	0x38, 0xa4, 0xc9, 0xcf, 0x46, 0xb3, 0x14, 0x42, 0x86, 0x3c, 0x6f, 0x41, 0xc1, 0xd5, 0x89, 0xde,
-	0xa3, 0x35, 0x60, 0x1b, 0x73, 0x82, 0x93, 0x78, 0x68, 0xf5, 0xeb, 0x2d, 0x86, 0x23, 0x9e, 0xf7,
-	0x39, 0xa8, 0x7c, 0x0d, 0xca, 0xb1, 0xea, 0x27, 0xa5, 0x08, 0x4b, 0xf1, 0x04, 0xc7, 0x97, 0x98,
-	0x63, 0x8b, 0x48, 0x84, 0x73, 0x0d, 0xe3, 0x62, 0x29, 0x16, 0x17, 0x2b, 0x97, 0x7c, 0x77, 0xe7,
-	0xb8, 0xe9, 0xb7, 0xa3, 0xf2, 0x45, 0x7f, 0x07, 0x47, 0x23, 0x1e, 0x8b, 0x7e, 0x01, 0x96, 0x79,
-	0x3f, 0xdf, 0x7b, 0xf6, 0xe9, 0x98, 0xab, 0x94, 0xf2, 0x16, 0xac, 0x0c, 0x76, 0x9b, 0x28, 0xb4,
-	0x7f, 0x06, 0xca, 0x94, 0x8d, 0xe3, 0x19, 0x4d, 0xee, 0xe8, 0x80, 0x57, 0x35, 0xed, 0x8e, 0xb3,
-	0xf1, 0x49, 0x06, 0xe6, 0xb7, 0xb0, 0xde, 0xbb, 0x6e, 0x0b, 0x33, 0x44, 0x3f, 0x94, 0x60, 0x2e,
-	0xf8, 0x4e, 0x1b, 0xba, 0x27, 0xfc, 0x33, 0x27, 0x5f, 0x9b, 0x66, 0x2c, 0x3f, 0x52, 0xce, 0xac,
-	0x4b, 0x97, 0x24, 0xf4, 0x5b, 0x09, 0xe4, 0xf1, 0x7f, 0xbd, 0xa2, 0x9b, 0x29, 0xf1, 0x9f, 0xf8,
-	0xe3, 0xac, 0x3c, 0xe3, 0xef, 0x49, 0xca, 0x99, 0x8d, 0x77, 0x01, 0xb8, 0x12, 0x59, 0x0e, 0xc1,
-	0x16, 0xb9, 0x84, 0x8b, 0x13, 0xe6, 0x24, 0xe4, 0x49, 0x07, 0x70, 0x45, 0x6d, 0xfc, 0x48, 0x82,
-	0x32, 0xa7, 0xe7, 0x91, 0xcf, 0x23, 0xc8, 0xf3, 0x8f, 0x17, 0x26, 0x89, 0x80, 0x03, 0x6d, 0x5c,
-	0x99, 0x6c, 0x90, 0x38, 0xef, 0xb9, 0x24, 0x1f, 0x48, 0xc1, 0x6e, 0xda, 0xe5, 0x0e, 0x0e, 0x3d,
-	0x82, 0xb9, 0xe0, 0xf3, 0xca, 0xa4, 0x67, 0xbe, 0x7f, 0xd2, 0xc9, 0x97, 0xd3, 0x8f, 0x0a, 0x0e,
-	0x12, 0x2e, 0xcb, 0xc7, 0x19, 0xa8, 0x71, 0x59, 0x1a, 0x8f, 0x3c, 0x4c, 0x6c, 0xdd, 0xe2, 0x76,
-	0xd4, 0x72, 0xf8, 0x26, 0x2f, 0xc7, 0x1c, 0x01, 0xba, 0x36, 0xb5, 0x87, 0x92, 0x5f, 0x9e, 0x66,
-	0x68, 0xa0, 0x35, 0xf4, 0x7d, 0x09, 0x20, 0x72, 0x19, 0x28, 0x7d, 0x42, 0x60, 0xc8, 0x2f, 0xc9,
-	0xd7, 0xa6, 0x18, 0x19, 0x48, 0xb1, 0xf1, 0x1b, 0x09, 0x10, 0xd7, 0x55, 0xdc, 0xd7, 0xa0, 0x0f,
-	0x25, 0xa8, 0x0c, 0x54, 0xa4, 0xfd, 0x95, 0x3b, 0xc9, 0x63, 0xa5, 0x56, 0x54, 0x82, 0x57, 0xe4,
-	0x4b, 0xba, 0xb5, 0x09, 0xcf, 0x8d, 0x83, 0x88, 0x23, 0x6c, 0x95, 0xf8, 0x5c, 0x36, 0x5d, 0xf3,
-	0xcd, 0x85, 0x58, 0x93, 0x76, 0x74, 0xf9, 0xb0, 0xc0, 0x82, 0xc2, 0x17, 0xfe, 0x17, 0x00, 0x00,
-	0xff, 0xff, 0xa8, 0x54, 0xe2, 0x28, 0xe3, 0x30, 0x00, 0x00,
+	type x struct{}
+	out := protoimpl.TypeBuilder{
+		File: protoimpl.DescBuilder{
+			GoPackagePath: reflect.TypeOf(x{}).PkgPath(),
+			RawDescriptor: file_beam_fn_api_proto_rawDesc,
+			NumEnums:      1,
+			NumMessages:   62,
+			NumExtensions: 0,
+			NumServices:   6,
+		},
+		GoTypes:           file_beam_fn_api_proto_goTypes,
+		DependencyIndexes: file_beam_fn_api_proto_depIdxs,
+		EnumInfos:         file_beam_fn_api_proto_enumTypes,
+		MessageInfos:      file_beam_fn_api_proto_msgTypes,
+	}.Build()
+	File_beam_fn_api_proto = out.File
+	file_beam_fn_api_proto_rawDesc = nil
+	file_beam_fn_api_proto_goTypes = nil
+	file_beam_fn_api_proto_depIdxs = nil
 }
 
 // Reference imports to suppress errors if they are not otherwise used.
 var _ context.Context
-var _ grpc.ClientConn
+var _ grpc.ClientConnInterface
 
 // 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
+const _ = grpc.SupportPackageIsVersion6
 
 // BeamFnControlClient is the client API for BeamFnControl service.
 //
@@ -3660,10 +5426,10 @@
 }
 
 type beamFnControlClient struct {
-	cc *grpc.ClientConn
+	cc grpc.ClientConnInterface
 }
 
-func NewBeamFnControlClient(cc *grpc.ClientConn) BeamFnControlClient {
+func NewBeamFnControlClient(cc grpc.ClientConnInterface) BeamFnControlClient {
 	return &beamFnControlClient{cc}
 }
 
@@ -3721,10 +5487,10 @@
 type UnimplementedBeamFnControlServer struct {
 }
 
-func (*UnimplementedBeamFnControlServer) Control(srv BeamFnControl_ControlServer) error {
+func (*UnimplementedBeamFnControlServer) Control(BeamFnControl_ControlServer) error {
 	return status.Errorf(codes.Unimplemented, "method Control not implemented")
 }
-func (*UnimplementedBeamFnControlServer) GetProcessBundleDescriptor(ctx context.Context, req *GetProcessBundleDescriptorRequest) (*ProcessBundleDescriptor, error) {
+func (*UnimplementedBeamFnControlServer) GetProcessBundleDescriptor(context.Context, *GetProcessBundleDescriptorRequest) (*ProcessBundleDescriptor, error) {
 	return nil, status.Errorf(codes.Unimplemented, "method GetProcessBundleDescriptor not implemented")
 }
 
@@ -3805,10 +5571,10 @@
 }
 
 type beamFnDataClient struct {
-	cc *grpc.ClientConn
+	cc grpc.ClientConnInterface
 }
 
-func NewBeamFnDataClient(cc *grpc.ClientConn) BeamFnDataClient {
+func NewBeamFnDataClient(cc grpc.ClientConnInterface) BeamFnDataClient {
 	return &beamFnDataClient{cc}
 }
 
@@ -3853,7 +5619,7 @@
 type UnimplementedBeamFnDataServer struct {
 }
 
-func (*UnimplementedBeamFnDataServer) Data(srv BeamFnData_DataServer) error {
+func (*UnimplementedBeamFnDataServer) Data(BeamFnData_DataServer) error {
 	return status.Errorf(codes.Unimplemented, "method Data not implemented")
 }
 
@@ -3911,10 +5677,10 @@
 }
 
 type beamFnStateClient struct {
-	cc *grpc.ClientConn
+	cc grpc.ClientConnInterface
 }
 
-func NewBeamFnStateClient(cc *grpc.ClientConn) BeamFnStateClient {
+func NewBeamFnStateClient(cc grpc.ClientConnInterface) BeamFnStateClient {
 	return &beamFnStateClient{cc}
 }
 
@@ -3959,7 +5725,7 @@
 type UnimplementedBeamFnStateServer struct {
 }
 
-func (*UnimplementedBeamFnStateServer) State(srv BeamFnState_StateServer) error {
+func (*UnimplementedBeamFnStateServer) State(BeamFnState_StateServer) error {
 	return status.Errorf(codes.Unimplemented, "method State not implemented")
 }
 
@@ -4018,10 +5784,10 @@
 }
 
 type beamFnLoggingClient struct {
-	cc *grpc.ClientConn
+	cc grpc.ClientConnInterface
 }
 
-func NewBeamFnLoggingClient(cc *grpc.ClientConn) BeamFnLoggingClient {
+func NewBeamFnLoggingClient(cc grpc.ClientConnInterface) BeamFnLoggingClient {
 	return &beamFnLoggingClient{cc}
 }
 
@@ -4067,7 +5833,7 @@
 type UnimplementedBeamFnLoggingServer struct {
 }
 
-func (*UnimplementedBeamFnLoggingServer) Logging(srv BeamFnLogging_LoggingServer) error {
+func (*UnimplementedBeamFnLoggingServer) Logging(BeamFnLogging_LoggingServer) error {
 	return status.Errorf(codes.Unimplemented, "method Logging not implemented")
 }
 
@@ -4127,10 +5893,10 @@
 }
 
 type beamFnExternalWorkerPoolClient struct {
-	cc *grpc.ClientConn
+	cc grpc.ClientConnInterface
 }
 
-func NewBeamFnExternalWorkerPoolClient(cc *grpc.ClientConn) BeamFnExternalWorkerPoolClient {
+func NewBeamFnExternalWorkerPoolClient(cc grpc.ClientConnInterface) BeamFnExternalWorkerPoolClient {
 	return &beamFnExternalWorkerPoolClient{cc}
 }
 
@@ -4164,10 +5930,10 @@
 type UnimplementedBeamFnExternalWorkerPoolServer struct {
 }
 
-func (*UnimplementedBeamFnExternalWorkerPoolServer) StartWorker(ctx context.Context, req *StartWorkerRequest) (*StartWorkerResponse, error) {
+func (*UnimplementedBeamFnExternalWorkerPoolServer) StartWorker(context.Context, *StartWorkerRequest) (*StartWorkerResponse, error) {
 	return nil, status.Errorf(codes.Unimplemented, "method StartWorker not implemented")
 }
-func (*UnimplementedBeamFnExternalWorkerPoolServer) StopWorker(ctx context.Context, req *StopWorkerRequest) (*StopWorkerResponse, error) {
+func (*UnimplementedBeamFnExternalWorkerPoolServer) StopWorker(context.Context, *StopWorkerRequest) (*StopWorkerResponse, error) {
 	return nil, status.Errorf(codes.Unimplemented, "method StopWorker not implemented")
 }
 
@@ -4236,10 +6002,10 @@
 }
 
 type beamFnWorkerStatusClient struct {
-	cc *grpc.ClientConn
+	cc grpc.ClientConnInterface
 }
 
-func NewBeamFnWorkerStatusClient(cc *grpc.ClientConn) BeamFnWorkerStatusClient {
+func NewBeamFnWorkerStatusClient(cc grpc.ClientConnInterface) BeamFnWorkerStatusClient {
 	return &beamFnWorkerStatusClient{cc}
 }
 
@@ -4283,7 +6049,7 @@
 type UnimplementedBeamFnWorkerStatusServer struct {
 }
 
-func (*UnimplementedBeamFnWorkerStatusServer) WorkerStatus(srv BeamFnWorkerStatus_WorkerStatusServer) error {
+func (*UnimplementedBeamFnWorkerStatusServer) WorkerStatus(BeamFnWorkerStatus_WorkerStatusServer) error {
 	return status.Errorf(codes.Unimplemented, "method WorkerStatus not implemented")
 }
 
diff --git a/sdks/go/pkg/beam/model/fnexecution_v1/beam_provision_api.pb.go b/sdks/go/pkg/beam/model/fnexecution_v1/beam_provision_api.pb.go
index ea561b9..ff73cd2 100644
--- a/sdks/go/pkg/beam/model/fnexecution_v1/beam_provision_api.pb.go
+++ b/sdks/go/pkg/beam/model/fnexecution_v1/beam_provision_api.pb.go
@@ -1,99 +1,135 @@
+//
+// 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.
+
+//
+// Protocol Buffers describing the Provision API, for communicating with a runner
+// for job and environment provisioning information over GRPC.
+
 // Code generated by protoc-gen-go. DO NOT EDIT.
+// versions:
+// 	protoc-gen-go v1.25.0-devel
+// 	protoc        v3.13.0
 // source: beam_provision_api.proto
 
 package fnexecution_v1
 
 import (
 	context "context"
-	fmt "fmt"
 	pipeline_v1 "github.com/apache/beam/sdks/go/pkg/beam/model/pipeline_v1"
-	proto "github.com/golang/protobuf/proto"
 	_struct "github.com/golang/protobuf/ptypes/struct"
 	grpc "google.golang.org/grpc"
 	codes "google.golang.org/grpc/codes"
 	status "google.golang.org/grpc/status"
-	math "math"
+	protoreflect "google.golang.org/protobuf/reflect/protoreflect"
+	protoimpl "google.golang.org/protobuf/runtime/protoimpl"
+	reflect "reflect"
+	sync "sync"
 )
 
-// Reference imports to suppress errors if they are not otherwise used.
-var _ = proto.Marshal
-var _ = fmt.Errorf
-var _ = math.Inf
-
-// This is a compile-time assertion to ensure that this generated file
-// is compatible with the proto package it is being compiled against.
-// A compilation error at this line likely means your copy of the
-// proto package needs to be updated.
-const _ = proto.ProtoPackageIsVersion3 // please upgrade the proto package
+const (
+	// Verify that this generated code is sufficiently up-to-date.
+	_ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion)
+	// Verify that runtime/protoimpl is sufficiently up-to-date.
+	_ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20)
+)
 
 // A request to get the provision info of a SDK harness worker instance.
 type GetProvisionInfoRequest struct {
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
 }
 
-func (m *GetProvisionInfoRequest) Reset()         { *m = GetProvisionInfoRequest{} }
-func (m *GetProvisionInfoRequest) String() string { return proto.CompactTextString(m) }
-func (*GetProvisionInfoRequest) ProtoMessage()    {}
+func (x *GetProvisionInfoRequest) Reset() {
+	*x = GetProvisionInfoRequest{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_provision_api_proto_msgTypes[0]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *GetProvisionInfoRequest) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*GetProvisionInfoRequest) ProtoMessage() {}
+
+func (x *GetProvisionInfoRequest) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_provision_api_proto_msgTypes[0]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use GetProvisionInfoRequest.ProtoReflect.Descriptor instead.
 func (*GetProvisionInfoRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_92e393e5933c7d6f, []int{0}
+	return file_beam_provision_api_proto_rawDescGZIP(), []int{0}
 }
 
-func (m *GetProvisionInfoRequest) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_GetProvisionInfoRequest.Unmarshal(m, b)
-}
-func (m *GetProvisionInfoRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_GetProvisionInfoRequest.Marshal(b, m, deterministic)
-}
-func (m *GetProvisionInfoRequest) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_GetProvisionInfoRequest.Merge(m, src)
-}
-func (m *GetProvisionInfoRequest) XXX_Size() int {
-	return xxx_messageInfo_GetProvisionInfoRequest.Size(m)
-}
-func (m *GetProvisionInfoRequest) XXX_DiscardUnknown() {
-	xxx_messageInfo_GetProvisionInfoRequest.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_GetProvisionInfoRequest proto.InternalMessageInfo
-
 // A response containing the provision info of a SDK harness worker instance.
 type GetProvisionInfoResponse struct {
-	Info                 *ProvisionInfo `protobuf:"bytes,1,opt,name=info,proto3" json:"info,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}       `json:"-"`
-	XXX_unrecognized     []byte         `json:"-"`
-	XXX_sizecache        int32          `json:"-"`
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	Info *ProvisionInfo `protobuf:"bytes,1,opt,name=info,proto3" json:"info,omitempty"`
 }
 
-func (m *GetProvisionInfoResponse) Reset()         { *m = GetProvisionInfoResponse{} }
-func (m *GetProvisionInfoResponse) String() string { return proto.CompactTextString(m) }
-func (*GetProvisionInfoResponse) ProtoMessage()    {}
+func (x *GetProvisionInfoResponse) Reset() {
+	*x = GetProvisionInfoResponse{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_provision_api_proto_msgTypes[1]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *GetProvisionInfoResponse) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*GetProvisionInfoResponse) ProtoMessage() {}
+
+func (x *GetProvisionInfoResponse) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_provision_api_proto_msgTypes[1]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use GetProvisionInfoResponse.ProtoReflect.Descriptor instead.
 func (*GetProvisionInfoResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_92e393e5933c7d6f, []int{1}
+	return file_beam_provision_api_proto_rawDescGZIP(), []int{1}
 }
 
-func (m *GetProvisionInfoResponse) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_GetProvisionInfoResponse.Unmarshal(m, b)
-}
-func (m *GetProvisionInfoResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_GetProvisionInfoResponse.Marshal(b, m, deterministic)
-}
-func (m *GetProvisionInfoResponse) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_GetProvisionInfoResponse.Merge(m, src)
-}
-func (m *GetProvisionInfoResponse) XXX_Size() int {
-	return xxx_messageInfo_GetProvisionInfoResponse.Size(m)
-}
-func (m *GetProvisionInfoResponse) XXX_DiscardUnknown() {
-	xxx_messageInfo_GetProvisionInfoResponse.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_GetProvisionInfoResponse proto.InternalMessageInfo
-
-func (m *GetProvisionInfoResponse) GetInfo() *ProvisionInfo {
-	if m != nil {
-		return m.Info
+func (x *GetProvisionInfoResponse) GetInfo() *ProvisionInfo {
+	if x != nil {
+		return x.Info
 	}
 	return nil
 }
@@ -101,6 +137,10 @@
 // Runtime provisioning information for a SDK harness worker instance,
 // such as pipeline options, resource constraints and other job metadata
 type ProvisionInfo struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
 	// (required) Pipeline options. For non-template jobs, the options are
 	// identical to what is passed to job submission.
 	PipelineOptions *_struct.Struct `protobuf:"bytes,3,opt,name=pipeline_options,json=pipelineOptions,proto3" json:"pipeline_options,omitempty"`
@@ -119,133 +159,278 @@
 	// (optional) The control endpoint this SDK should use.
 	ControlEndpoint *pipeline_v1.ApiServiceDescriptor `protobuf:"bytes,10,opt,name=control_endpoint,json=controlEndpoint,proto3" json:"control_endpoint,omitempty"`
 	// The set of dependencies that should be staged into this environment.
-	Dependencies         []*pipeline_v1.ArtifactInformation `protobuf:"bytes,11,rep,name=dependencies,proto3" json:"dependencies,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}                           `json:"-"`
-	XXX_unrecognized     []byte                             `json:"-"`
-	XXX_sizecache        int32                              `json:"-"`
+	Dependencies []*pipeline_v1.ArtifactInformation `protobuf:"bytes,11,rep,name=dependencies,proto3" json:"dependencies,omitempty"`
 }
 
-func (m *ProvisionInfo) Reset()         { *m = ProvisionInfo{} }
-func (m *ProvisionInfo) String() string { return proto.CompactTextString(m) }
-func (*ProvisionInfo) ProtoMessage()    {}
+func (x *ProvisionInfo) Reset() {
+	*x = ProvisionInfo{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_provision_api_proto_msgTypes[2]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *ProvisionInfo) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*ProvisionInfo) ProtoMessage() {}
+
+func (x *ProvisionInfo) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_provision_api_proto_msgTypes[2]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use ProvisionInfo.ProtoReflect.Descriptor instead.
 func (*ProvisionInfo) Descriptor() ([]byte, []int) {
-	return fileDescriptor_92e393e5933c7d6f, []int{2}
+	return file_beam_provision_api_proto_rawDescGZIP(), []int{2}
 }
 
-func (m *ProvisionInfo) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_ProvisionInfo.Unmarshal(m, b)
-}
-func (m *ProvisionInfo) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_ProvisionInfo.Marshal(b, m, deterministic)
-}
-func (m *ProvisionInfo) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ProvisionInfo.Merge(m, src)
-}
-func (m *ProvisionInfo) XXX_Size() int {
-	return xxx_messageInfo_ProvisionInfo.Size(m)
-}
-func (m *ProvisionInfo) XXX_DiscardUnknown() {
-	xxx_messageInfo_ProvisionInfo.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_ProvisionInfo proto.InternalMessageInfo
-
-func (m *ProvisionInfo) GetPipelineOptions() *_struct.Struct {
-	if m != nil {
-		return m.PipelineOptions
+func (x *ProvisionInfo) GetPipelineOptions() *_struct.Struct {
+	if x != nil {
+		return x.PipelineOptions
 	}
 	return nil
 }
 
-func (m *ProvisionInfo) GetRetrievalToken() string {
-	if m != nil {
-		return m.RetrievalToken
+func (x *ProvisionInfo) GetRetrievalToken() string {
+	if x != nil {
+		return x.RetrievalToken
 	}
 	return ""
 }
 
-func (m *ProvisionInfo) GetStatusEndpoint() *pipeline_v1.ApiServiceDescriptor {
-	if m != nil {
-		return m.StatusEndpoint
+func (x *ProvisionInfo) GetStatusEndpoint() *pipeline_v1.ApiServiceDescriptor {
+	if x != nil {
+		return x.StatusEndpoint
 	}
 	return nil
 }
 
-func (m *ProvisionInfo) GetLoggingEndpoint() *pipeline_v1.ApiServiceDescriptor {
-	if m != nil {
-		return m.LoggingEndpoint
+func (x *ProvisionInfo) GetLoggingEndpoint() *pipeline_v1.ApiServiceDescriptor {
+	if x != nil {
+		return x.LoggingEndpoint
 	}
 	return nil
 }
 
-func (m *ProvisionInfo) GetArtifactEndpoint() *pipeline_v1.ApiServiceDescriptor {
-	if m != nil {
-		return m.ArtifactEndpoint
+func (x *ProvisionInfo) GetArtifactEndpoint() *pipeline_v1.ApiServiceDescriptor {
+	if x != nil {
+		return x.ArtifactEndpoint
 	}
 	return nil
 }
 
-func (m *ProvisionInfo) GetControlEndpoint() *pipeline_v1.ApiServiceDescriptor {
-	if m != nil {
-		return m.ControlEndpoint
+func (x *ProvisionInfo) GetControlEndpoint() *pipeline_v1.ApiServiceDescriptor {
+	if x != nil {
+		return x.ControlEndpoint
 	}
 	return nil
 }
 
-func (m *ProvisionInfo) GetDependencies() []*pipeline_v1.ArtifactInformation {
-	if m != nil {
-		return m.Dependencies
+func (x *ProvisionInfo) GetDependencies() []*pipeline_v1.ArtifactInformation {
+	if x != nil {
+		return x.Dependencies
 	}
 	return nil
 }
 
-func init() {
-	proto.RegisterType((*GetProvisionInfoRequest)(nil), "org.apache.beam.model.fn_execution.v1.GetProvisionInfoRequest")
-	proto.RegisterType((*GetProvisionInfoResponse)(nil), "org.apache.beam.model.fn_execution.v1.GetProvisionInfoResponse")
-	proto.RegisterType((*ProvisionInfo)(nil), "org.apache.beam.model.fn_execution.v1.ProvisionInfo")
+var File_beam_provision_api_proto protoreflect.FileDescriptor
+
+var file_beam_provision_api_proto_rawDesc = []byte{
+	0x0a, 0x18, 0x62, 0x65, 0x61, 0x6d, 0x5f, 0x70, 0x72, 0x6f, 0x76, 0x69, 0x73, 0x69, 0x6f, 0x6e,
+	0x5f, 0x61, 0x70, 0x69, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x25, 0x6f, 0x72, 0x67, 0x2e,
+	0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65,
+	0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76,
+	0x31, 0x1a, 0x15, 0x62, 0x65, 0x61, 0x6d, 0x5f, 0x72, 0x75, 0x6e, 0x6e, 0x65, 0x72, 0x5f, 0x61,
+	0x70, 0x69, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x0f, 0x65, 0x6e, 0x64, 0x70, 0x6f, 0x69,
+	0x6e, 0x74, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x1c, 0x67, 0x6f, 0x6f, 0x67, 0x6c,
+	0x65, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x73, 0x74, 0x72, 0x75, 0x63,
+	0x74, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0x19, 0x0a, 0x17, 0x47, 0x65, 0x74, 0x50, 0x72,
+	0x6f, 0x76, 0x69, 0x73, 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x71, 0x75, 0x65,
+	0x73, 0x74, 0x22, 0x64, 0x0a, 0x18, 0x47, 0x65, 0x74, 0x50, 0x72, 0x6f, 0x76, 0x69, 0x73, 0x69,
+	0x6f, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x48,
+	0x0a, 0x04, 0x69, 0x6e, 0x66, 0x6f, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x34, 0x2e, 0x6f,
+	0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d,
+	0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f,
+	0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x72, 0x6f, 0x76, 0x69, 0x73, 0x69, 0x6f, 0x6e, 0x49, 0x6e,
+	0x66, 0x6f, 0x52, 0x04, 0x69, 0x6e, 0x66, 0x6f, 0x22, 0xe8, 0x04, 0x0a, 0x0d, 0x50, 0x72, 0x6f,
+	0x76, 0x69, 0x73, 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x42, 0x0a, 0x10, 0x70, 0x69,
+	0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x5f, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x03,
+	0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72,
+	0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x53, 0x74, 0x72, 0x75, 0x63, 0x74, 0x52, 0x0f, 0x70,
+	0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x27,
+	0x0a, 0x0f, 0x72, 0x65, 0x74, 0x72, 0x69, 0x65, 0x76, 0x61, 0x6c, 0x5f, 0x74, 0x6f, 0x6b, 0x65,
+	0x6e, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x72, 0x65, 0x74, 0x72, 0x69, 0x65, 0x76,
+	0x61, 0x6c, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x12, 0x60, 0x0a, 0x0f, 0x73, 0x74, 0x61, 0x74, 0x75,
+	0x73, 0x5f, 0x65, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0b,
+	0x32, 0x37, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65,
+	0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e,
+	0x65, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x70, 0x69, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x44,
+	0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x52, 0x0e, 0x73, 0x74, 0x61, 0x74, 0x75,
+	0x73, 0x45, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x12, 0x62, 0x0a, 0x10, 0x6c, 0x6f, 0x67,
+	0x67, 0x69, 0x6e, 0x67, 0x5f, 0x65, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x18, 0x08, 0x20,
+	0x01, 0x28, 0x0b, 0x32, 0x37, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65,
+	0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65,
+	0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x70, 0x69, 0x53, 0x65, 0x72, 0x76, 0x69,
+	0x63, 0x65, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x52, 0x0f, 0x6c, 0x6f,
+	0x67, 0x67, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x12, 0x64, 0x0a,
+	0x11, 0x61, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x5f, 0x65, 0x6e, 0x64, 0x70, 0x6f, 0x69,
+	0x6e, 0x74, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x37, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61,
+	0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c,
+	0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x70, 0x69,
+	0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f,
+	0x72, 0x52, 0x10, 0x61, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x45, 0x6e, 0x64, 0x70, 0x6f,
+	0x69, 0x6e, 0x74, 0x12, 0x62, 0x0a, 0x10, 0x63, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x5f, 0x65,
+	0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x37, 0x2e,
+	0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e,
+	0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76,
+	0x31, 0x2e, 0x41, 0x70, 0x69, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x44, 0x65, 0x73, 0x63,
+	0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x52, 0x0f, 0x63, 0x6f, 0x6e, 0x74, 0x72, 0x6f, 0x6c, 0x45,
+	0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x12, 0x5a, 0x0a, 0x0c, 0x64, 0x65, 0x70, 0x65, 0x6e,
+	0x64, 0x65, 0x6e, 0x63, 0x69, 0x65, 0x73, 0x18, 0x0b, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x36, 0x2e,
+	0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e,
+	0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76,
+	0x31, 0x2e, 0x41, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x72, 0x6d,
+	0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0c, 0x64, 0x65, 0x70, 0x65, 0x6e, 0x64, 0x65, 0x6e, 0x63,
+	0x69, 0x65, 0x73, 0x32, 0xa8, 0x01, 0x0a, 0x10, 0x50, 0x72, 0x6f, 0x76, 0x69, 0x73, 0x69, 0x6f,
+	0x6e, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x93, 0x01, 0x0a, 0x10, 0x47, 0x65, 0x74,
+	0x50, 0x72, 0x6f, 0x76, 0x69, 0x73, 0x69, 0x6f, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x3e, 0x2e,
+	0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e,
+	0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69,
+	0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, 0x50, 0x72, 0x6f, 0x76, 0x69, 0x73, 0x69,
+	0x6f, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x3f, 0x2e,
+	0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e,
+	0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69,
+	0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, 0x50, 0x72, 0x6f, 0x76, 0x69, 0x73, 0x69,
+	0x6f, 0x6e, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x42, 0x81,
+	0x01, 0x0a, 0x24, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65,
+	0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x66, 0x6e, 0x65, 0x78, 0x65, 0x63, 0x75,
+	0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x42, 0x0c, 0x50, 0x72, 0x6f, 0x76, 0x69, 0x73, 0x69,
+	0x6f, 0x6e, 0x41, 0x70, 0x69, 0x5a, 0x4b, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f,
+	0x6d, 0x2f, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2f, 0x62, 0x65, 0x61, 0x6d, 0x2f, 0x73, 0x64,
+	0x6b, 0x73, 0x2f, 0x67, 0x6f, 0x2f, 0x70, 0x6b, 0x67, 0x2f, 0x62, 0x65, 0x61, 0x6d, 0x2f, 0x6d,
+	0x6f, 0x64, 0x65, 0x6c, 0x2f, 0x66, 0x6e, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e,
+	0x5f, 0x76, 0x31, 0x3b, 0x66, 0x6e, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f,
+	0x76, 0x31, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
 }
 
-func init() { proto.RegisterFile("beam_provision_api.proto", fileDescriptor_92e393e5933c7d6f) }
+var (
+	file_beam_provision_api_proto_rawDescOnce sync.Once
+	file_beam_provision_api_proto_rawDescData = file_beam_provision_api_proto_rawDesc
+)
 
-var fileDescriptor_92e393e5933c7d6f = []byte{
-	// 446 bytes of a gzipped FileDescriptorProto
-	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xa4, 0x93, 0xcf, 0x6e, 0xd3, 0x40,
-	0x10, 0xc6, 0x65, 0xb5, 0x2a, 0x74, 0x53, 0x62, 0x63, 0x09, 0xd5, 0x44, 0x1c, 0xa2, 0x08, 0x44,
-	0x4e, 0x5b, 0x35, 0x20, 0xb8, 0x81, 0x1a, 0x15, 0x01, 0x27, 0x90, 0xcb, 0xa9, 0x17, 0xb3, 0xb1,
-	0xc7, 0x66, 0x85, 0x33, 0xb3, 0xec, 0xae, 0x2d, 0xde, 0x83, 0x97, 0xe0, 0xd1, 0x78, 0x0c, 0xe4,
-	0xff, 0x09, 0x28, 0x52, 0xd4, 0x1c, 0x3d, 0xb3, 0xf3, 0xfb, 0xe6, 0xb3, 0xbe, 0x61, 0xc1, 0x0a,
-	0xc4, 0x3a, 0x52, 0x9a, 0x4a, 0x69, 0x24, 0x61, 0x24, 0x94, 0xe4, 0x4a, 0x93, 0x25, 0xff, 0x19,
-	0xe9, 0x8c, 0x0b, 0x25, 0xe2, 0x6f, 0xc0, 0xab, 0x47, 0x7c, 0x4d, 0x09, 0xe4, 0x3c, 0xc5, 0x08,
-	0x7e, 0x42, 0x5c, 0x58, 0x49, 0xc8, 0xcb, 0xcb, 0xc9, 0xa3, 0x1a, 0xa0, 0x0b, 0x44, 0xd0, 0xc3,
-	0xf4, 0xc4, 0x05, 0x4c, 0x14, 0x49, 0xb4, 0xa6, 0x2d, 0x3c, 0xc9, 0x88, 0xb2, 0x1c, 0x2e, 0xea,
-	0xaf, 0x55, 0x91, 0x5e, 0x18, 0xab, 0x8b, 0xd8, 0x36, 0xdd, 0xd9, 0x63, 0x76, 0xfe, 0x1e, 0xec,
-	0xe7, 0x6e, 0x8d, 0x8f, 0x98, 0x52, 0x08, 0x3f, 0x0a, 0x30, 0x76, 0x96, 0xb0, 0xe0, 0xff, 0x96,
-	0x51, 0x84, 0x06, 0xfc, 0x0f, 0xec, 0x58, 0x62, 0x4a, 0x81, 0x33, 0x75, 0xe6, 0xa3, 0xc5, 0x4b,
-	0xbe, 0xd7, 0xca, 0x7c, 0x9b, 0x55, 0x13, 0x66, 0x7f, 0x8e, 0xd9, 0x83, 0xad, 0xba, 0xbf, 0x64,
-	0x9e, 0x92, 0x0a, 0x72, 0x89, 0x10, 0x91, 0xaa, 0x66, 0x4d, 0x70, 0x54, 0xeb, 0x9c, 0xf3, 0xc6,
-	0x0b, 0xef, 0xbc, 0xf0, 0x9b, 0xda, 0x4b, 0xe8, 0x76, 0x03, 0x9f, 0x9a, 0xf7, 0xfe, 0x73, 0xe6,
-	0x6a, 0xb0, 0x5a, 0x42, 0x29, 0xf2, 0xc8, 0xd2, 0x77, 0xc0, 0xe0, 0x64, 0xea, 0xcc, 0x4f, 0xc3,
-	0x71, 0x5f, 0xfe, 0x52, 0x55, 0xfd, 0xaf, 0xcc, 0x35, 0x56, 0xd8, 0xc2, 0x44, 0xdd, 0x7f, 0x0b,
-	0xee, 0xd5, 0x5a, 0xaf, 0x77, 0x78, 0xea, 0x94, 0x2a, 0x3f, 0x57, 0x4a, 0xde, 0x80, 0x2e, 0x65,
-	0x0c, 0xd7, 0x60, 0x62, 0x2d, 0x95, 0x25, 0x1d, 0x8e, 0x1b, 0xde, 0xbb, 0x16, 0xe7, 0xaf, 0x98,
-	0x97, 0x53, 0x96, 0x49, 0xcc, 0x06, 0x89, 0xfb, 0x87, 0x49, 0xb8, 0x2d, 0xb0, 0xd7, 0x48, 0xd8,
-	0x43, 0xa1, 0xad, 0x4c, 0x45, 0x6c, 0x07, 0x91, 0xd3, 0xc3, 0x44, 0xbc, 0x8e, 0xb8, 0xe9, 0x24,
-	0x26, 0xb4, 0x9a, 0xf2, 0x41, 0x84, 0x1d, 0xe8, 0xa4, 0x05, 0xf6, 0x1a, 0xb7, 0xec, 0x2c, 0x01,
-	0x05, 0x98, 0x00, 0xc6, 0x12, 0x4c, 0x30, 0x9a, 0x1e, 0xcd, 0x47, 0x8b, 0x57, 0xfb, 0xf0, 0xdb,
-	0x75, 0xab, 0x0c, 0xe9, 0xb5, 0xa8, 0x72, 0x10, 0x6e, 0xb1, 0x16, 0xbf, 0x1d, 0xe6, 0xf5, 0x51,
-	0x6b, 0x77, 0xf1, 0x7f, 0x39, 0xcc, 0xfb, 0x37, 0xe6, 0xfe, 0x9b, 0x3d, 0x03, 0xbd, 0xe3, 0x74,
-	0x26, 0x6f, 0xef, 0x3c, 0xdf, 0xdc, 0xd7, 0xf2, 0x9a, 0x3d, 0xdd, 0x45, 0xd8, 0x04, 0x2c, 0xcf,
-	0xfa, 0xf1, 0x2b, 0x25, 0x6f, 0xc7, 0x1b, 0xdd, 0xa8, 0xbc, 0x5c, 0x9d, 0xd4, 0x77, 0xf2, 0xe2,
-	0x6f, 0x00, 0x00, 0x00, 0xff, 0xff, 0xca, 0x2c, 0x91, 0x7e, 0x6c, 0x04, 0x00, 0x00,
+func file_beam_provision_api_proto_rawDescGZIP() []byte {
+	file_beam_provision_api_proto_rawDescOnce.Do(func() {
+		file_beam_provision_api_proto_rawDescData = protoimpl.X.CompressGZIP(file_beam_provision_api_proto_rawDescData)
+	})
+	return file_beam_provision_api_proto_rawDescData
+}
+
+var file_beam_provision_api_proto_msgTypes = make([]protoimpl.MessageInfo, 3)
+var file_beam_provision_api_proto_goTypes = []interface{}{
+	(*GetProvisionInfoRequest)(nil),          // 0: org.apache.beam.model.fn_execution.v1.GetProvisionInfoRequest
+	(*GetProvisionInfoResponse)(nil),         // 1: org.apache.beam.model.fn_execution.v1.GetProvisionInfoResponse
+	(*ProvisionInfo)(nil),                    // 2: org.apache.beam.model.fn_execution.v1.ProvisionInfo
+	(*_struct.Struct)(nil),                   // 3: google.protobuf.Struct
+	(*pipeline_v1.ApiServiceDescriptor)(nil), // 4: org.apache.beam.model.pipeline.v1.ApiServiceDescriptor
+	(*pipeline_v1.ArtifactInformation)(nil),  // 5: org.apache.beam.model.pipeline.v1.ArtifactInformation
+}
+var file_beam_provision_api_proto_depIdxs = []int32{
+	2, // 0: org.apache.beam.model.fn_execution.v1.GetProvisionInfoResponse.info:type_name -> org.apache.beam.model.fn_execution.v1.ProvisionInfo
+	3, // 1: org.apache.beam.model.fn_execution.v1.ProvisionInfo.pipeline_options:type_name -> google.protobuf.Struct
+	4, // 2: org.apache.beam.model.fn_execution.v1.ProvisionInfo.status_endpoint:type_name -> org.apache.beam.model.pipeline.v1.ApiServiceDescriptor
+	4, // 3: org.apache.beam.model.fn_execution.v1.ProvisionInfo.logging_endpoint:type_name -> org.apache.beam.model.pipeline.v1.ApiServiceDescriptor
+	4, // 4: org.apache.beam.model.fn_execution.v1.ProvisionInfo.artifact_endpoint:type_name -> org.apache.beam.model.pipeline.v1.ApiServiceDescriptor
+	4, // 5: org.apache.beam.model.fn_execution.v1.ProvisionInfo.control_endpoint:type_name -> org.apache.beam.model.pipeline.v1.ApiServiceDescriptor
+	5, // 6: org.apache.beam.model.fn_execution.v1.ProvisionInfo.dependencies:type_name -> org.apache.beam.model.pipeline.v1.ArtifactInformation
+	0, // 7: org.apache.beam.model.fn_execution.v1.ProvisionService.GetProvisionInfo:input_type -> org.apache.beam.model.fn_execution.v1.GetProvisionInfoRequest
+	1, // 8: org.apache.beam.model.fn_execution.v1.ProvisionService.GetProvisionInfo:output_type -> org.apache.beam.model.fn_execution.v1.GetProvisionInfoResponse
+	8, // [8:9] is the sub-list for method output_type
+	7, // [7:8] is the sub-list for method input_type
+	7, // [7:7] is the sub-list for extension type_name
+	7, // [7:7] is the sub-list for extension extendee
+	0, // [0:7] is the sub-list for field type_name
+}
+
+func init() { file_beam_provision_api_proto_init() }
+func file_beam_provision_api_proto_init() {
+	if File_beam_provision_api_proto != nil {
+		return
+	}
+	if !protoimpl.UnsafeEnabled {
+		file_beam_provision_api_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*GetProvisionInfoRequest); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_provision_api_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*GetProvisionInfoResponse); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_provision_api_proto_msgTypes[2].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*ProvisionInfo); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+	}
+	type x struct{}
+	out := protoimpl.TypeBuilder{
+		File: protoimpl.DescBuilder{
+			GoPackagePath: reflect.TypeOf(x{}).PkgPath(),
+			RawDescriptor: file_beam_provision_api_proto_rawDesc,
+			NumEnums:      0,
+			NumMessages:   3,
+			NumExtensions: 0,
+			NumServices:   1,
+		},
+		GoTypes:           file_beam_provision_api_proto_goTypes,
+		DependencyIndexes: file_beam_provision_api_proto_depIdxs,
+		MessageInfos:      file_beam_provision_api_proto_msgTypes,
+	}.Build()
+	File_beam_provision_api_proto = out.File
+	file_beam_provision_api_proto_rawDesc = nil
+	file_beam_provision_api_proto_goTypes = nil
+	file_beam_provision_api_proto_depIdxs = nil
 }
 
 // Reference imports to suppress errors if they are not otherwise used.
 var _ context.Context
-var _ grpc.ClientConn
+var _ grpc.ClientConnInterface
 
 // 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
+const _ = grpc.SupportPackageIsVersion6
 
 // ProvisionServiceClient is the client API for ProvisionService service.
 //
@@ -256,10 +441,10 @@
 }
 
 type provisionServiceClient struct {
-	cc *grpc.ClientConn
+	cc grpc.ClientConnInterface
 }
 
-func NewProvisionServiceClient(cc *grpc.ClientConn) ProvisionServiceClient {
+func NewProvisionServiceClient(cc grpc.ClientConnInterface) ProvisionServiceClient {
 	return &provisionServiceClient{cc}
 }
 
@@ -282,7 +467,7 @@
 type UnimplementedProvisionServiceServer struct {
 }
 
-func (*UnimplementedProvisionServiceServer) GetProvisionInfo(ctx context.Context, req *GetProvisionInfoRequest) (*GetProvisionInfoResponse, error) {
+func (*UnimplementedProvisionServiceServer) GetProvisionInfo(context.Context, *GetProvisionInfoRequest) (*GetProvisionInfoResponse, error) {
 	return nil, status.Errorf(codes.Unimplemented, "method GetProvisionInfo not implemented")
 }
 
diff --git a/sdks/go/pkg/beam/model/gen.go b/sdks/go/pkg/beam/model/gen.go
index 435ad4a..dc706a2 100644
--- a/sdks/go/pkg/beam/model/gen.go
+++ b/sdks/go/pkg/beam/model/gen.go
@@ -22,6 +22,6 @@
 
 // Until file is automatically generated, keep the listed proto files in alphabetical order.
 
-//go:generate protoc -I../../../../../model/pipeline/src/main/proto ../../../../../model/pipeline/src/main/proto/beam_runner_api.proto ../../../../../model/pipeline/src/main/proto/endpoints.proto ../../../../../model/pipeline/src/main/proto/external_transforms.proto ../../../../../model/pipeline/src/main/proto/metrics.proto ../../../../../model/pipeline/src/main/proto/schema.proto  ../../../../../model/pipeline/src/main/proto/standard_window_fns.proto --go_out=Mpipeline_v1,plugins=grpc:pipeline_v1
-//go:generate protoc -I../../../../../model/pipeline/src/main/proto -I../../../../../model/job-management/src/main/proto ../../../../../model/job-management/src/main/proto/beam_artifact_api.proto ../../../../../model/job-management/src/main/proto/beam_expansion_api.proto  ../../../../../model/job-management/src/main/proto/beam_job_api.proto  --go_out=Mbeam_runner_api.proto=github.com/apache/beam/sdks/go/pkg/beam/model/pipeline_v1,Mendpoints.proto=github.com/apache/beam/sdks/go/pkg/beam/model/pipeline_v1,Mmetrics.proto=github.com/apache/beam/sdks/go/pkg/beam/model/pipeline_v1,jobmanagement_v1,plugins=grpc:jobmanagement_v1
-//go:generate protoc -I../../../../../model/pipeline/src/main/proto -I../../../../../model/fn-execution/src/main/proto ../../../../../model/fn-execution/src/main/proto/beam_fn_api.proto ../../../../../model/fn-execution/src/main/proto/beam_provision_api.proto --go_out=Mbeam_runner_api.proto=github.com/apache/beam/sdks/go/pkg/beam/model/pipeline_v1,Mendpoints.proto=github.com/apache/beam/sdks/go/pkg/beam/model/pipeline_v1,Mmetrics.proto=github.com/apache/beam/sdks/go/pkg/beam/model/pipeline_v1,fnexecution_v1,plugins=grpc:fnexecution_v1
+//go:generate protoc -I../../../../../model/pipeline/src/main/proto ../../../../../model/pipeline/src/main/proto/beam_runner_api.proto ../../../../../model/pipeline/src/main/proto/endpoints.proto ../../../../../model/pipeline/src/main/proto/external_transforms.proto ../../../../../model/pipeline/src/main/proto/metrics.proto ../../../../../model/pipeline/src/main/proto/schema.proto  ../../../../../model/pipeline/src/main/proto/standard_window_fns.proto --go_out=M../../../../../../../../,plugins=grpc:../../../../../../../..
+//go:generate protoc -I../../../../../model/pipeline/src/main/proto -I../../../../../model/job-management/src/main/proto ../../../../../model/job-management/src/main/proto/beam_artifact_api.proto ../../../../../model/job-management/src/main/proto/beam_expansion_api.proto  ../../../../../model/job-management/src/main/proto/beam_job_api.proto --go_out=M../../../../../../../../,plugins=grpc:../../../../../../../..
+//go:generate protoc -I../../../../../model/pipeline/src/main/proto -I../../../../../model/fn-execution/src/main/proto ../../../../../model/fn-execution/src/main/proto/beam_fn_api.proto ../../../../../model/fn-execution/src/main/proto/beam_provision_api.proto --go_out=M../../../../../../../../,plugins=grpc:../../../../../../../..
diff --git a/sdks/go/pkg/beam/model/jobmanagement_v1/beam_artifact_api.pb.go b/sdks/go/pkg/beam/model/jobmanagement_v1/beam_artifact_api.pb.go
index d4a786f..2cb93ba 100644
--- a/sdks/go/pkg/beam/model/jobmanagement_v1/beam_artifact_api.pb.go
+++ b/sdks/go/pkg/beam/model/jobmanagement_v1/beam_artifact_api.pb.go
@@ -1,29 +1,50 @@
+//
+// 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.
+
+//
+// Protocol Buffers describing the Artifact API, for communicating with a runner
+// for artifact staging and retrieval over GRPC.
+
 // Code generated by protoc-gen-go. DO NOT EDIT.
+// versions:
+// 	protoc-gen-go v1.25.0-devel
+// 	protoc        v3.13.0
 // source: beam_artifact_api.proto
 
 package jobmanagement_v1
 
 import (
 	context "context"
-	fmt "fmt"
 	pipeline_v1 "github.com/apache/beam/sdks/go/pkg/beam/model/pipeline_v1"
-	proto "github.com/golang/protobuf/proto"
 	grpc "google.golang.org/grpc"
 	codes "google.golang.org/grpc/codes"
 	status "google.golang.org/grpc/status"
-	math "math"
+	protoreflect "google.golang.org/protobuf/reflect/protoreflect"
+	protoimpl "google.golang.org/protobuf/runtime/protoimpl"
+	reflect "reflect"
+	sync "sync"
 )
 
-// Reference imports to suppress errors if they are not otherwise used.
-var _ = proto.Marshal
-var _ = fmt.Errorf
-var _ = math.Inf
-
-// This is a compile-time assertion to ensure that this generated file
-// is compatible with the proto package it is being compiled against.
-// A compilation error at this line likely means your copy of the
-// proto package needs to be updated.
-const _ = proto.ProtoPackageIsVersion3 // please upgrade the proto package
+const (
+	// Verify that this generated code is sufficiently up-to-date.
+	_ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion)
+	// Verify that runtime/protoimpl is sufficiently up-to-date.
+	_ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20)
+)
 
 type CommitManifestResponse_Constants int32
 
@@ -32,233 +53,315 @@
 	CommitManifestResponse_NO_ARTIFACTS_STAGED_TOKEN CommitManifestResponse_Constants = 0
 )
 
-var CommitManifestResponse_Constants_name = map[int32]string{
-	0: "NO_ARTIFACTS_STAGED_TOKEN",
-}
+// Enum value maps for CommitManifestResponse_Constants.
+var (
+	CommitManifestResponse_Constants_name = map[int32]string{
+		0: "NO_ARTIFACTS_STAGED_TOKEN",
+	}
+	CommitManifestResponse_Constants_value = map[string]int32{
+		"NO_ARTIFACTS_STAGED_TOKEN": 0,
+	}
+)
 
-var CommitManifestResponse_Constants_value = map[string]int32{
-	"NO_ARTIFACTS_STAGED_TOKEN": 0,
+func (x CommitManifestResponse_Constants) Enum() *CommitManifestResponse_Constants {
+	p := new(CommitManifestResponse_Constants)
+	*p = x
+	return p
 }
 
 func (x CommitManifestResponse_Constants) String() string {
-	return proto.EnumName(CommitManifestResponse_Constants_name, int32(x))
+	return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x))
 }
 
+func (CommitManifestResponse_Constants) Descriptor() protoreflect.EnumDescriptor {
+	return file_beam_artifact_api_proto_enumTypes[0].Descriptor()
+}
+
+func (CommitManifestResponse_Constants) Type() protoreflect.EnumType {
+	return &file_beam_artifact_api_proto_enumTypes[0]
+}
+
+func (x CommitManifestResponse_Constants) Number() protoreflect.EnumNumber {
+	return protoreflect.EnumNumber(x)
+}
+
+// Deprecated: Use CommitManifestResponse_Constants.Descriptor instead.
 func (CommitManifestResponse_Constants) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_8ef4db42c81e3972, []int{17, 0}
+	return file_beam_artifact_api_proto_rawDescGZIP(), []int{17, 0}
 }
 
 // A request for artifact resolution.
 type ResolveArtifactsRequest struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
 	// An (ordered) set of artifacts to (jointly) resolve.
 	Artifacts []*pipeline_v1.ArtifactInformation `protobuf:"bytes,1,rep,name=artifacts,proto3" json:"artifacts,omitempty"`
 	// A set of artifact type urns that are understood by the requester.
 	// An attempt should be made to resolve the artifacts in terms of these URNs,
 	// but other URNs may be used as well with the understanding that they must
 	// be fetch-able as bytes via GetArtifact.
-	PreferredUrns        []string `protobuf:"bytes,2,rep,name=preferred_urns,json=preferredUrns,proto3" json:"preferred_urns,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
+	PreferredUrns []string `protobuf:"bytes,2,rep,name=preferred_urns,json=preferredUrns,proto3" json:"preferred_urns,omitempty"`
 }
 
-func (m *ResolveArtifactsRequest) Reset()         { *m = ResolveArtifactsRequest{} }
-func (m *ResolveArtifactsRequest) String() string { return proto.CompactTextString(m) }
-func (*ResolveArtifactsRequest) ProtoMessage()    {}
+func (x *ResolveArtifactsRequest) Reset() {
+	*x = ResolveArtifactsRequest{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_artifact_api_proto_msgTypes[0]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *ResolveArtifactsRequest) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*ResolveArtifactsRequest) ProtoMessage() {}
+
+func (x *ResolveArtifactsRequest) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_artifact_api_proto_msgTypes[0]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use ResolveArtifactsRequest.ProtoReflect.Descriptor instead.
 func (*ResolveArtifactsRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_8ef4db42c81e3972, []int{0}
+	return file_beam_artifact_api_proto_rawDescGZIP(), []int{0}
 }
 
-func (m *ResolveArtifactsRequest) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_ResolveArtifactsRequest.Unmarshal(m, b)
-}
-func (m *ResolveArtifactsRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_ResolveArtifactsRequest.Marshal(b, m, deterministic)
-}
-func (m *ResolveArtifactsRequest) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ResolveArtifactsRequest.Merge(m, src)
-}
-func (m *ResolveArtifactsRequest) XXX_Size() int {
-	return xxx_messageInfo_ResolveArtifactsRequest.Size(m)
-}
-func (m *ResolveArtifactsRequest) XXX_DiscardUnknown() {
-	xxx_messageInfo_ResolveArtifactsRequest.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_ResolveArtifactsRequest proto.InternalMessageInfo
-
-func (m *ResolveArtifactsRequest) GetArtifacts() []*pipeline_v1.ArtifactInformation {
-	if m != nil {
-		return m.Artifacts
+func (x *ResolveArtifactsRequest) GetArtifacts() []*pipeline_v1.ArtifactInformation {
+	if x != nil {
+		return x.Artifacts
 	}
 	return nil
 }
 
-func (m *ResolveArtifactsRequest) GetPreferredUrns() []string {
-	if m != nil {
-		return m.PreferredUrns
+func (x *ResolveArtifactsRequest) GetPreferredUrns() []string {
+	if x != nil {
+		return x.PreferredUrns
 	}
 	return nil
 }
 
 // A response for artifact resolution.
 type ResolveArtifactsResponse struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
 	// A full (ordered) set of replacements for the set of requested artifacts,
 	// preferably in terms of the requested type URNs.  If there is no better
 	// resolution, the original list is returned.
-	Replacements         []*pipeline_v1.ArtifactInformation `protobuf:"bytes,1,rep,name=replacements,proto3" json:"replacements,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}                           `json:"-"`
-	XXX_unrecognized     []byte                             `json:"-"`
-	XXX_sizecache        int32                              `json:"-"`
+	Replacements []*pipeline_v1.ArtifactInformation `protobuf:"bytes,1,rep,name=replacements,proto3" json:"replacements,omitempty"`
 }
 
-func (m *ResolveArtifactsResponse) Reset()         { *m = ResolveArtifactsResponse{} }
-func (m *ResolveArtifactsResponse) String() string { return proto.CompactTextString(m) }
-func (*ResolveArtifactsResponse) ProtoMessage()    {}
+func (x *ResolveArtifactsResponse) Reset() {
+	*x = ResolveArtifactsResponse{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_artifact_api_proto_msgTypes[1]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *ResolveArtifactsResponse) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*ResolveArtifactsResponse) ProtoMessage() {}
+
+func (x *ResolveArtifactsResponse) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_artifact_api_proto_msgTypes[1]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use ResolveArtifactsResponse.ProtoReflect.Descriptor instead.
 func (*ResolveArtifactsResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_8ef4db42c81e3972, []int{1}
+	return file_beam_artifact_api_proto_rawDescGZIP(), []int{1}
 }
 
-func (m *ResolveArtifactsResponse) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_ResolveArtifactsResponse.Unmarshal(m, b)
-}
-func (m *ResolveArtifactsResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_ResolveArtifactsResponse.Marshal(b, m, deterministic)
-}
-func (m *ResolveArtifactsResponse) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ResolveArtifactsResponse.Merge(m, src)
-}
-func (m *ResolveArtifactsResponse) XXX_Size() int {
-	return xxx_messageInfo_ResolveArtifactsResponse.Size(m)
-}
-func (m *ResolveArtifactsResponse) XXX_DiscardUnknown() {
-	xxx_messageInfo_ResolveArtifactsResponse.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_ResolveArtifactsResponse proto.InternalMessageInfo
-
-func (m *ResolveArtifactsResponse) GetReplacements() []*pipeline_v1.ArtifactInformation {
-	if m != nil {
-		return m.Replacements
+func (x *ResolveArtifactsResponse) GetReplacements() []*pipeline_v1.ArtifactInformation {
+	if x != nil {
+		return x.Replacements
 	}
 	return nil
 }
 
 // A request to get an artifact.
 type GetArtifactRequest struct {
-	Artifact             *pipeline_v1.ArtifactInformation `protobuf:"bytes,1,opt,name=artifact,proto3" json:"artifact,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}                         `json:"-"`
-	XXX_unrecognized     []byte                           `json:"-"`
-	XXX_sizecache        int32                            `json:"-"`
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	Artifact *pipeline_v1.ArtifactInformation `protobuf:"bytes,1,opt,name=artifact,proto3" json:"artifact,omitempty"`
 }
 
-func (m *GetArtifactRequest) Reset()         { *m = GetArtifactRequest{} }
-func (m *GetArtifactRequest) String() string { return proto.CompactTextString(m) }
-func (*GetArtifactRequest) ProtoMessage()    {}
+func (x *GetArtifactRequest) Reset() {
+	*x = GetArtifactRequest{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_artifact_api_proto_msgTypes[2]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *GetArtifactRequest) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*GetArtifactRequest) ProtoMessage() {}
+
+func (x *GetArtifactRequest) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_artifact_api_proto_msgTypes[2]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use GetArtifactRequest.ProtoReflect.Descriptor instead.
 func (*GetArtifactRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_8ef4db42c81e3972, []int{2}
+	return file_beam_artifact_api_proto_rawDescGZIP(), []int{2}
 }
 
-func (m *GetArtifactRequest) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_GetArtifactRequest.Unmarshal(m, b)
-}
-func (m *GetArtifactRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_GetArtifactRequest.Marshal(b, m, deterministic)
-}
-func (m *GetArtifactRequest) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_GetArtifactRequest.Merge(m, src)
-}
-func (m *GetArtifactRequest) XXX_Size() int {
-	return xxx_messageInfo_GetArtifactRequest.Size(m)
-}
-func (m *GetArtifactRequest) XXX_DiscardUnknown() {
-	xxx_messageInfo_GetArtifactRequest.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_GetArtifactRequest proto.InternalMessageInfo
-
-func (m *GetArtifactRequest) GetArtifact() *pipeline_v1.ArtifactInformation {
-	if m != nil {
-		return m.Artifact
+func (x *GetArtifactRequest) GetArtifact() *pipeline_v1.ArtifactInformation {
+	if x != nil {
+		return x.Artifact
 	}
 	return nil
 }
 
 // Part of a response to getting an artifact.
 type GetArtifactResponse struct {
-	Data                 []byte   `protobuf:"bytes,1,opt,name=data,proto3" json:"data,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	Data []byte `protobuf:"bytes,1,opt,name=data,proto3" json:"data,omitempty"`
 }
 
-func (m *GetArtifactResponse) Reset()         { *m = GetArtifactResponse{} }
-func (m *GetArtifactResponse) String() string { return proto.CompactTextString(m) }
-func (*GetArtifactResponse) ProtoMessage()    {}
+func (x *GetArtifactResponse) Reset() {
+	*x = GetArtifactResponse{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_artifact_api_proto_msgTypes[3]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *GetArtifactResponse) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*GetArtifactResponse) ProtoMessage() {}
+
+func (x *GetArtifactResponse) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_artifact_api_proto_msgTypes[3]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use GetArtifactResponse.ProtoReflect.Descriptor instead.
 func (*GetArtifactResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_8ef4db42c81e3972, []int{3}
+	return file_beam_artifact_api_proto_rawDescGZIP(), []int{3}
 }
 
-func (m *GetArtifactResponse) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_GetArtifactResponse.Unmarshal(m, b)
-}
-func (m *GetArtifactResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_GetArtifactResponse.Marshal(b, m, deterministic)
-}
-func (m *GetArtifactResponse) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_GetArtifactResponse.Merge(m, src)
-}
-func (m *GetArtifactResponse) XXX_Size() int {
-	return xxx_messageInfo_GetArtifactResponse.Size(m)
-}
-func (m *GetArtifactResponse) XXX_DiscardUnknown() {
-	xxx_messageInfo_GetArtifactResponse.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_GetArtifactResponse proto.InternalMessageInfo
-
-func (m *GetArtifactResponse) GetData() []byte {
-	if m != nil {
-		return m.Data
+func (x *GetArtifactResponse) GetData() []byte {
+	if x != nil {
+		return x.Data
 	}
 	return nil
 }
 
 // Wraps an ArtifactRetrievalService request for use in ReverseArtifactRetrievalService.
 type ArtifactRequestWrapper struct {
-	// Types that are valid to be assigned to Request:
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	// Types that are assignable to Request:
 	//	*ArtifactRequestWrapper_ResolveArtifact
 	//	*ArtifactRequestWrapper_GetArtifact
-	Request              isArtifactRequestWrapper_Request `protobuf_oneof:"request"`
-	XXX_NoUnkeyedLiteral struct{}                         `json:"-"`
-	XXX_unrecognized     []byte                           `json:"-"`
-	XXX_sizecache        int32                            `json:"-"`
+	Request isArtifactRequestWrapper_Request `protobuf_oneof:"request"`
 }
 
-func (m *ArtifactRequestWrapper) Reset()         { *m = ArtifactRequestWrapper{} }
-func (m *ArtifactRequestWrapper) String() string { return proto.CompactTextString(m) }
-func (*ArtifactRequestWrapper) ProtoMessage()    {}
+func (x *ArtifactRequestWrapper) Reset() {
+	*x = ArtifactRequestWrapper{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_artifact_api_proto_msgTypes[4]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *ArtifactRequestWrapper) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*ArtifactRequestWrapper) ProtoMessage() {}
+
+func (x *ArtifactRequestWrapper) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_artifact_api_proto_msgTypes[4]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use ArtifactRequestWrapper.ProtoReflect.Descriptor instead.
 func (*ArtifactRequestWrapper) Descriptor() ([]byte, []int) {
-	return fileDescriptor_8ef4db42c81e3972, []int{4}
+	return file_beam_artifact_api_proto_rawDescGZIP(), []int{4}
 }
 
-func (m *ArtifactRequestWrapper) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_ArtifactRequestWrapper.Unmarshal(m, b)
-}
-func (m *ArtifactRequestWrapper) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_ArtifactRequestWrapper.Marshal(b, m, deterministic)
-}
-func (m *ArtifactRequestWrapper) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ArtifactRequestWrapper.Merge(m, src)
-}
-func (m *ArtifactRequestWrapper) XXX_Size() int {
-	return xxx_messageInfo_ArtifactRequestWrapper.Size(m)
-}
-func (m *ArtifactRequestWrapper) XXX_DiscardUnknown() {
-	xxx_messageInfo_ArtifactRequestWrapper.DiscardUnknown(m)
+func (m *ArtifactRequestWrapper) GetRequest() isArtifactRequestWrapper_Request {
+	if m != nil {
+		return m.Request
+	}
+	return nil
 }
 
-var xxx_messageInfo_ArtifactRequestWrapper proto.InternalMessageInfo
+func (x *ArtifactRequestWrapper) GetResolveArtifact() *ResolveArtifactsRequest {
+	if x, ok := x.GetRequest().(*ArtifactRequestWrapper_ResolveArtifact); ok {
+		return x.ResolveArtifact
+	}
+	return nil
+}
+
+func (x *ArtifactRequestWrapper) GetGetArtifact() *GetArtifactRequest {
+	if x, ok := x.GetRequest().(*ArtifactRequestWrapper_GetArtifact); ok {
+		return x.GetArtifact
+	}
+	return nil
+}
 
 type isArtifactRequestWrapper_Request interface {
 	isArtifactRequestWrapper_Request()
@@ -276,37 +379,12 @@
 
 func (*ArtifactRequestWrapper_GetArtifact) isArtifactRequestWrapper_Request() {}
 
-func (m *ArtifactRequestWrapper) GetRequest() isArtifactRequestWrapper_Request {
-	if m != nil {
-		return m.Request
-	}
-	return nil
-}
-
-func (m *ArtifactRequestWrapper) GetResolveArtifact() *ResolveArtifactsRequest {
-	if x, ok := m.GetRequest().(*ArtifactRequestWrapper_ResolveArtifact); ok {
-		return x.ResolveArtifact
-	}
-	return nil
-}
-
-func (m *ArtifactRequestWrapper) GetGetArtifact() *GetArtifactRequest {
-	if x, ok := m.GetRequest().(*ArtifactRequestWrapper_GetArtifact); ok {
-		return x.GetArtifact
-	}
-	return nil
-}
-
-// XXX_OneofWrappers is for the internal use of the proto package.
-func (*ArtifactRequestWrapper) XXX_OneofWrappers() []interface{} {
-	return []interface{}{
-		(*ArtifactRequestWrapper_ResolveArtifact)(nil),
-		(*ArtifactRequestWrapper_GetArtifact)(nil),
-	}
-}
-
 // Wraps an ArtifactRetrievalService response for use in ReverseArtifactRetrievalService.
 type ArtifactResponseWrapper struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
 	// A token indicating which job these artifacts are being staged for.
 	StagingToken string `protobuf:"bytes,1,opt,name=staging_token,json=stagingToken,proto3" json:"staging_token,omitempty"`
 	// Whether this is the last response for this request (for those responses that
@@ -314,54 +392,79 @@
 	IsLast bool `protobuf:"varint,2,opt,name=is_last,json=isLast,proto3" json:"is_last,omitempty"`
 	// The response itself.
 	//
-	// Types that are valid to be assigned to Response:
+	// Types that are assignable to Response:
 	//	*ArtifactResponseWrapper_ResolveArtifactResponse
 	//	*ArtifactResponseWrapper_GetArtifactResponse
-	Response             isArtifactResponseWrapper_Response `protobuf_oneof:"response"`
-	XXX_NoUnkeyedLiteral struct{}                           `json:"-"`
-	XXX_unrecognized     []byte                             `json:"-"`
-	XXX_sizecache        int32                              `json:"-"`
+	Response isArtifactResponseWrapper_Response `protobuf_oneof:"response"`
 }
 
-func (m *ArtifactResponseWrapper) Reset()         { *m = ArtifactResponseWrapper{} }
-func (m *ArtifactResponseWrapper) String() string { return proto.CompactTextString(m) }
-func (*ArtifactResponseWrapper) ProtoMessage()    {}
+func (x *ArtifactResponseWrapper) Reset() {
+	*x = ArtifactResponseWrapper{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_artifact_api_proto_msgTypes[5]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *ArtifactResponseWrapper) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*ArtifactResponseWrapper) ProtoMessage() {}
+
+func (x *ArtifactResponseWrapper) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_artifact_api_proto_msgTypes[5]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use ArtifactResponseWrapper.ProtoReflect.Descriptor instead.
 func (*ArtifactResponseWrapper) Descriptor() ([]byte, []int) {
-	return fileDescriptor_8ef4db42c81e3972, []int{5}
+	return file_beam_artifact_api_proto_rawDescGZIP(), []int{5}
 }
 
-func (m *ArtifactResponseWrapper) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_ArtifactResponseWrapper.Unmarshal(m, b)
-}
-func (m *ArtifactResponseWrapper) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_ArtifactResponseWrapper.Marshal(b, m, deterministic)
-}
-func (m *ArtifactResponseWrapper) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ArtifactResponseWrapper.Merge(m, src)
-}
-func (m *ArtifactResponseWrapper) XXX_Size() int {
-	return xxx_messageInfo_ArtifactResponseWrapper.Size(m)
-}
-func (m *ArtifactResponseWrapper) XXX_DiscardUnknown() {
-	xxx_messageInfo_ArtifactResponseWrapper.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_ArtifactResponseWrapper proto.InternalMessageInfo
-
-func (m *ArtifactResponseWrapper) GetStagingToken() string {
-	if m != nil {
-		return m.StagingToken
+func (x *ArtifactResponseWrapper) GetStagingToken() string {
+	if x != nil {
+		return x.StagingToken
 	}
 	return ""
 }
 
-func (m *ArtifactResponseWrapper) GetIsLast() bool {
-	if m != nil {
-		return m.IsLast
+func (x *ArtifactResponseWrapper) GetIsLast() bool {
+	if x != nil {
+		return x.IsLast
 	}
 	return false
 }
 
+func (m *ArtifactResponseWrapper) GetResponse() isArtifactResponseWrapper_Response {
+	if m != nil {
+		return m.Response
+	}
+	return nil
+}
+
+func (x *ArtifactResponseWrapper) GetResolveArtifactResponse() *ResolveArtifactsResponse {
+	if x, ok := x.GetResponse().(*ArtifactResponseWrapper_ResolveArtifactResponse); ok {
+		return x.ResolveArtifactResponse
+	}
+	return nil
+}
+
+func (x *ArtifactResponseWrapper) GetGetArtifactResponse() *GetArtifactResponse {
+	if x, ok := x.GetResponse().(*ArtifactResponseWrapper_GetArtifactResponse); ok {
+		return x.GetArtifactResponse
+	}
+	return nil
+}
+
 type isArtifactResponseWrapper_Response interface {
 	isArtifactResponseWrapper_Response()
 }
@@ -378,507 +481,487 @@
 
 func (*ArtifactResponseWrapper_GetArtifactResponse) isArtifactResponseWrapper_Response() {}
 
-func (m *ArtifactResponseWrapper) GetResponse() isArtifactResponseWrapper_Response {
-	if m != nil {
-		return m.Response
-	}
-	return nil
-}
-
-func (m *ArtifactResponseWrapper) GetResolveArtifactResponse() *ResolveArtifactsResponse {
-	if x, ok := m.GetResponse().(*ArtifactResponseWrapper_ResolveArtifactResponse); ok {
-		return x.ResolveArtifactResponse
-	}
-	return nil
-}
-
-func (m *ArtifactResponseWrapper) GetGetArtifactResponse() *GetArtifactResponse {
-	if x, ok := m.GetResponse().(*ArtifactResponseWrapper_GetArtifactResponse); ok {
-		return x.GetArtifactResponse
-	}
-	return nil
-}
-
-// XXX_OneofWrappers is for the internal use of the proto package.
-func (*ArtifactResponseWrapper) XXX_OneofWrappers() []interface{} {
-	return []interface{}{
-		(*ArtifactResponseWrapper_ResolveArtifactResponse)(nil),
-		(*ArtifactResponseWrapper_GetArtifactResponse)(nil),
-	}
-}
-
 // An artifact identifier and associated metadata.
 type ArtifactMetadata struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
 	// (Required) The name of the artifact.
 	Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"`
 	// (Optional) The Unix-like permissions of the artifact
 	Permissions uint32 `protobuf:"varint,2,opt,name=permissions,proto3" json:"permissions,omitempty"`
 	// (Optional) The hex-encoded sha256 checksum of the artifact. Used, among other things, by
 	// harness boot code to validate the integrity of the artifact.
-	Sha256               string   `protobuf:"bytes,4,opt,name=sha256,proto3" json:"sha256,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
+	Sha256 string `protobuf:"bytes,4,opt,name=sha256,proto3" json:"sha256,omitempty"`
 }
 
-func (m *ArtifactMetadata) Reset()         { *m = ArtifactMetadata{} }
-func (m *ArtifactMetadata) String() string { return proto.CompactTextString(m) }
-func (*ArtifactMetadata) ProtoMessage()    {}
+func (x *ArtifactMetadata) Reset() {
+	*x = ArtifactMetadata{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_artifact_api_proto_msgTypes[6]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *ArtifactMetadata) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*ArtifactMetadata) ProtoMessage() {}
+
+func (x *ArtifactMetadata) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_artifact_api_proto_msgTypes[6]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use ArtifactMetadata.ProtoReflect.Descriptor instead.
 func (*ArtifactMetadata) Descriptor() ([]byte, []int) {
-	return fileDescriptor_8ef4db42c81e3972, []int{6}
+	return file_beam_artifact_api_proto_rawDescGZIP(), []int{6}
 }
 
-func (m *ArtifactMetadata) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_ArtifactMetadata.Unmarshal(m, b)
-}
-func (m *ArtifactMetadata) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_ArtifactMetadata.Marshal(b, m, deterministic)
-}
-func (m *ArtifactMetadata) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ArtifactMetadata.Merge(m, src)
-}
-func (m *ArtifactMetadata) XXX_Size() int {
-	return xxx_messageInfo_ArtifactMetadata.Size(m)
-}
-func (m *ArtifactMetadata) XXX_DiscardUnknown() {
-	xxx_messageInfo_ArtifactMetadata.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_ArtifactMetadata proto.InternalMessageInfo
-
-func (m *ArtifactMetadata) GetName() string {
-	if m != nil {
-		return m.Name
+func (x *ArtifactMetadata) GetName() string {
+	if x != nil {
+		return x.Name
 	}
 	return ""
 }
 
-func (m *ArtifactMetadata) GetPermissions() uint32 {
-	if m != nil {
-		return m.Permissions
+func (x *ArtifactMetadata) GetPermissions() uint32 {
+	if x != nil {
+		return x.Permissions
 	}
 	return 0
 }
 
-func (m *ArtifactMetadata) GetSha256() string {
-	if m != nil {
-		return m.Sha256
+func (x *ArtifactMetadata) GetSha256() string {
+	if x != nil {
+		return x.Sha256
 	}
 	return ""
 }
 
 // A collection of artifacts.
 type Manifest struct {
-	Artifact             []*ArtifactMetadata `protobuf:"bytes,1,rep,name=artifact,proto3" json:"artifact,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}            `json:"-"`
-	XXX_unrecognized     []byte              `json:"-"`
-	XXX_sizecache        int32               `json:"-"`
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	Artifact []*ArtifactMetadata `protobuf:"bytes,1,rep,name=artifact,proto3" json:"artifact,omitempty"`
 }
 
-func (m *Manifest) Reset()         { *m = Manifest{} }
-func (m *Manifest) String() string { return proto.CompactTextString(m) }
-func (*Manifest) ProtoMessage()    {}
+func (x *Manifest) Reset() {
+	*x = Manifest{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_artifact_api_proto_msgTypes[7]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *Manifest) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*Manifest) ProtoMessage() {}
+
+func (x *Manifest) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_artifact_api_proto_msgTypes[7]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use Manifest.ProtoReflect.Descriptor instead.
 func (*Manifest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_8ef4db42c81e3972, []int{7}
+	return file_beam_artifact_api_proto_rawDescGZIP(), []int{7}
 }
 
-func (m *Manifest) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_Manifest.Unmarshal(m, b)
-}
-func (m *Manifest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_Manifest.Marshal(b, m, deterministic)
-}
-func (m *Manifest) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Manifest.Merge(m, src)
-}
-func (m *Manifest) XXX_Size() int {
-	return xxx_messageInfo_Manifest.Size(m)
-}
-func (m *Manifest) XXX_DiscardUnknown() {
-	xxx_messageInfo_Manifest.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_Manifest proto.InternalMessageInfo
-
-func (m *Manifest) GetArtifact() []*ArtifactMetadata {
-	if m != nil {
-		return m.Artifact
+func (x *Manifest) GetArtifact() []*ArtifactMetadata {
+	if x != nil {
+		return x.Artifact
 	}
 	return nil
 }
 
 // A manifest with location information.
 type ProxyManifest struct {
-	Manifest             *Manifest                 `protobuf:"bytes,1,opt,name=manifest,proto3" json:"manifest,omitempty"`
-	Location             []*ProxyManifest_Location `protobuf:"bytes,2,rep,name=location,proto3" json:"location,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}                  `json:"-"`
-	XXX_unrecognized     []byte                    `json:"-"`
-	XXX_sizecache        int32                     `json:"-"`
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	Manifest *Manifest                 `protobuf:"bytes,1,opt,name=manifest,proto3" json:"manifest,omitempty"`
+	Location []*ProxyManifest_Location `protobuf:"bytes,2,rep,name=location,proto3" json:"location,omitempty"`
 }
 
-func (m *ProxyManifest) Reset()         { *m = ProxyManifest{} }
-func (m *ProxyManifest) String() string { return proto.CompactTextString(m) }
-func (*ProxyManifest) ProtoMessage()    {}
+func (x *ProxyManifest) Reset() {
+	*x = ProxyManifest{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_artifact_api_proto_msgTypes[8]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *ProxyManifest) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*ProxyManifest) ProtoMessage() {}
+
+func (x *ProxyManifest) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_artifact_api_proto_msgTypes[8]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use ProxyManifest.ProtoReflect.Descriptor instead.
 func (*ProxyManifest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_8ef4db42c81e3972, []int{8}
+	return file_beam_artifact_api_proto_rawDescGZIP(), []int{8}
 }
 
-func (m *ProxyManifest) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_ProxyManifest.Unmarshal(m, b)
-}
-func (m *ProxyManifest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_ProxyManifest.Marshal(b, m, deterministic)
-}
-func (m *ProxyManifest) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ProxyManifest.Merge(m, src)
-}
-func (m *ProxyManifest) XXX_Size() int {
-	return xxx_messageInfo_ProxyManifest.Size(m)
-}
-func (m *ProxyManifest) XXX_DiscardUnknown() {
-	xxx_messageInfo_ProxyManifest.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_ProxyManifest proto.InternalMessageInfo
-
-func (m *ProxyManifest) GetManifest() *Manifest {
-	if m != nil {
-		return m.Manifest
+func (x *ProxyManifest) GetManifest() *Manifest {
+	if x != nil {
+		return x.Manifest
 	}
 	return nil
 }
 
-func (m *ProxyManifest) GetLocation() []*ProxyManifest_Location {
-	if m != nil {
-		return m.Location
+func (x *ProxyManifest) GetLocation() []*ProxyManifest_Location {
+	if x != nil {
+		return x.Location
 	}
 	return nil
 }
 
-type ProxyManifest_Location struct {
-	Name                 string   `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"`
-	Uri                  string   `protobuf:"bytes,2,opt,name=uri,proto3" json:"uri,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *ProxyManifest_Location) Reset()         { *m = ProxyManifest_Location{} }
-func (m *ProxyManifest_Location) String() string { return proto.CompactTextString(m) }
-func (*ProxyManifest_Location) ProtoMessage()    {}
-func (*ProxyManifest_Location) Descriptor() ([]byte, []int) {
-	return fileDescriptor_8ef4db42c81e3972, []int{8, 0}
-}
-
-func (m *ProxyManifest_Location) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_ProxyManifest_Location.Unmarshal(m, b)
-}
-func (m *ProxyManifest_Location) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_ProxyManifest_Location.Marshal(b, m, deterministic)
-}
-func (m *ProxyManifest_Location) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ProxyManifest_Location.Merge(m, src)
-}
-func (m *ProxyManifest_Location) XXX_Size() int {
-	return xxx_messageInfo_ProxyManifest_Location.Size(m)
-}
-func (m *ProxyManifest_Location) XXX_DiscardUnknown() {
-	xxx_messageInfo_ProxyManifest_Location.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_ProxyManifest_Location proto.InternalMessageInfo
-
-func (m *ProxyManifest_Location) GetName() string {
-	if m != nil {
-		return m.Name
-	}
-	return ""
-}
-
-func (m *ProxyManifest_Location) GetUri() string {
-	if m != nil {
-		return m.Uri
-	}
-	return ""
-}
-
 // A request to get the manifest of a Job.
 type GetManifestRequest struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
 	// (Required) An opaque token representing the entirety of the staged artifacts.
 	// Returned in CommitManifestResponse.
-	RetrievalToken       string   `protobuf:"bytes,1,opt,name=retrieval_token,json=retrievalToken,proto3" json:"retrieval_token,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
+	RetrievalToken string `protobuf:"bytes,1,opt,name=retrieval_token,json=retrievalToken,proto3" json:"retrieval_token,omitempty"`
 }
 
-func (m *GetManifestRequest) Reset()         { *m = GetManifestRequest{} }
-func (m *GetManifestRequest) String() string { return proto.CompactTextString(m) }
-func (*GetManifestRequest) ProtoMessage()    {}
+func (x *GetManifestRequest) Reset() {
+	*x = GetManifestRequest{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_artifact_api_proto_msgTypes[9]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *GetManifestRequest) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*GetManifestRequest) ProtoMessage() {}
+
+func (x *GetManifestRequest) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_artifact_api_proto_msgTypes[9]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use GetManifestRequest.ProtoReflect.Descriptor instead.
 func (*GetManifestRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_8ef4db42c81e3972, []int{9}
+	return file_beam_artifact_api_proto_rawDescGZIP(), []int{9}
 }
 
-func (m *GetManifestRequest) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_GetManifestRequest.Unmarshal(m, b)
-}
-func (m *GetManifestRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_GetManifestRequest.Marshal(b, m, deterministic)
-}
-func (m *GetManifestRequest) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_GetManifestRequest.Merge(m, src)
-}
-func (m *GetManifestRequest) XXX_Size() int {
-	return xxx_messageInfo_GetManifestRequest.Size(m)
-}
-func (m *GetManifestRequest) XXX_DiscardUnknown() {
-	xxx_messageInfo_GetManifestRequest.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_GetManifestRequest proto.InternalMessageInfo
-
-func (m *GetManifestRequest) GetRetrievalToken() string {
-	if m != nil {
-		return m.RetrievalToken
+func (x *GetManifestRequest) GetRetrievalToken() string {
+	if x != nil {
+		return x.RetrievalToken
 	}
 	return ""
 }
 
 // A response containing a job manifest.
 type GetManifestResponse struct {
-	Manifest             *Manifest `protobuf:"bytes,1,opt,name=manifest,proto3" json:"manifest,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}  `json:"-"`
-	XXX_unrecognized     []byte    `json:"-"`
-	XXX_sizecache        int32     `json:"-"`
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	Manifest *Manifest `protobuf:"bytes,1,opt,name=manifest,proto3" json:"manifest,omitempty"`
 }
 
-func (m *GetManifestResponse) Reset()         { *m = GetManifestResponse{} }
-func (m *GetManifestResponse) String() string { return proto.CompactTextString(m) }
-func (*GetManifestResponse) ProtoMessage()    {}
+func (x *GetManifestResponse) Reset() {
+	*x = GetManifestResponse{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_artifact_api_proto_msgTypes[10]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *GetManifestResponse) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*GetManifestResponse) ProtoMessage() {}
+
+func (x *GetManifestResponse) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_artifact_api_proto_msgTypes[10]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use GetManifestResponse.ProtoReflect.Descriptor instead.
 func (*GetManifestResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_8ef4db42c81e3972, []int{10}
+	return file_beam_artifact_api_proto_rawDescGZIP(), []int{10}
 }
 
-func (m *GetManifestResponse) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_GetManifestResponse.Unmarshal(m, b)
-}
-func (m *GetManifestResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_GetManifestResponse.Marshal(b, m, deterministic)
-}
-func (m *GetManifestResponse) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_GetManifestResponse.Merge(m, src)
-}
-func (m *GetManifestResponse) XXX_Size() int {
-	return xxx_messageInfo_GetManifestResponse.Size(m)
-}
-func (m *GetManifestResponse) XXX_DiscardUnknown() {
-	xxx_messageInfo_GetManifestResponse.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_GetManifestResponse proto.InternalMessageInfo
-
-func (m *GetManifestResponse) GetManifest() *Manifest {
-	if m != nil {
-		return m.Manifest
+func (x *GetManifestResponse) GetManifest() *Manifest {
+	if x != nil {
+		return x.Manifest
 	}
 	return nil
 }
 
 // A request to get an artifact. The artifact must be present in the manifest for the job.
 type LegacyGetArtifactRequest struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
 	// (Required) The name of the artifact to retrieve.
 	Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"`
 	// (Required) An opaque token representing the entirety of the staged artifacts.
 	// Returned in CommitManifestResponse.
-	RetrievalToken       string   `protobuf:"bytes,2,opt,name=retrieval_token,json=retrievalToken,proto3" json:"retrieval_token,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
+	RetrievalToken string `protobuf:"bytes,2,opt,name=retrieval_token,json=retrievalToken,proto3" json:"retrieval_token,omitempty"`
 }
 
-func (m *LegacyGetArtifactRequest) Reset()         { *m = LegacyGetArtifactRequest{} }
-func (m *LegacyGetArtifactRequest) String() string { return proto.CompactTextString(m) }
-func (*LegacyGetArtifactRequest) ProtoMessage()    {}
+func (x *LegacyGetArtifactRequest) Reset() {
+	*x = LegacyGetArtifactRequest{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_artifact_api_proto_msgTypes[11]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *LegacyGetArtifactRequest) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*LegacyGetArtifactRequest) ProtoMessage() {}
+
+func (x *LegacyGetArtifactRequest) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_artifact_api_proto_msgTypes[11]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use LegacyGetArtifactRequest.ProtoReflect.Descriptor instead.
 func (*LegacyGetArtifactRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_8ef4db42c81e3972, []int{11}
+	return file_beam_artifact_api_proto_rawDescGZIP(), []int{11}
 }
 
-func (m *LegacyGetArtifactRequest) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_LegacyGetArtifactRequest.Unmarshal(m, b)
-}
-func (m *LegacyGetArtifactRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_LegacyGetArtifactRequest.Marshal(b, m, deterministic)
-}
-func (m *LegacyGetArtifactRequest) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_LegacyGetArtifactRequest.Merge(m, src)
-}
-func (m *LegacyGetArtifactRequest) XXX_Size() int {
-	return xxx_messageInfo_LegacyGetArtifactRequest.Size(m)
-}
-func (m *LegacyGetArtifactRequest) XXX_DiscardUnknown() {
-	xxx_messageInfo_LegacyGetArtifactRequest.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_LegacyGetArtifactRequest proto.InternalMessageInfo
-
-func (m *LegacyGetArtifactRequest) GetName() string {
-	if m != nil {
-		return m.Name
+func (x *LegacyGetArtifactRequest) GetName() string {
+	if x != nil {
+		return x.Name
 	}
 	return ""
 }
 
-func (m *LegacyGetArtifactRequest) GetRetrievalToken() string {
-	if m != nil {
-		return m.RetrievalToken
+func (x *LegacyGetArtifactRequest) GetRetrievalToken() string {
+	if x != nil {
+		return x.RetrievalToken
 	}
 	return ""
 }
 
 // Part of an artifact.
 type ArtifactChunk struct {
-	Data                 []byte   `protobuf:"bytes,1,opt,name=data,proto3" json:"data,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	Data []byte `protobuf:"bytes,1,opt,name=data,proto3" json:"data,omitempty"`
 }
 
-func (m *ArtifactChunk) Reset()         { *m = ArtifactChunk{} }
-func (m *ArtifactChunk) String() string { return proto.CompactTextString(m) }
-func (*ArtifactChunk) ProtoMessage()    {}
+func (x *ArtifactChunk) Reset() {
+	*x = ArtifactChunk{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_artifact_api_proto_msgTypes[12]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *ArtifactChunk) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*ArtifactChunk) ProtoMessage() {}
+
+func (x *ArtifactChunk) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_artifact_api_proto_msgTypes[12]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use ArtifactChunk.ProtoReflect.Descriptor instead.
 func (*ArtifactChunk) Descriptor() ([]byte, []int) {
-	return fileDescriptor_8ef4db42c81e3972, []int{12}
+	return file_beam_artifact_api_proto_rawDescGZIP(), []int{12}
 }
 
-func (m *ArtifactChunk) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_ArtifactChunk.Unmarshal(m, b)
-}
-func (m *ArtifactChunk) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_ArtifactChunk.Marshal(b, m, deterministic)
-}
-func (m *ArtifactChunk) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ArtifactChunk.Merge(m, src)
-}
-func (m *ArtifactChunk) XXX_Size() int {
-	return xxx_messageInfo_ArtifactChunk.Size(m)
-}
-func (m *ArtifactChunk) XXX_DiscardUnknown() {
-	xxx_messageInfo_ArtifactChunk.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_ArtifactChunk proto.InternalMessageInfo
-
-func (m *ArtifactChunk) GetData() []byte {
-	if m != nil {
-		return m.Data
+func (x *ArtifactChunk) GetData() []byte {
+	if x != nil {
+		return x.Data
 	}
 	return nil
 }
 
 type PutArtifactMetadata struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
 	// (Required) A token for artifact staging session. This token can be obtained
 	// from PrepareJob request in JobService
 	StagingSessionToken string `protobuf:"bytes,1,opt,name=staging_session_token,json=stagingSessionToken,proto3" json:"staging_session_token,omitempty"`
 	// (Required) The Artifact metadata.
-	Metadata             *ArtifactMetadata `protobuf:"bytes,2,opt,name=metadata,proto3" json:"metadata,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}          `json:"-"`
-	XXX_unrecognized     []byte            `json:"-"`
-	XXX_sizecache        int32             `json:"-"`
+	Metadata *ArtifactMetadata `protobuf:"bytes,2,opt,name=metadata,proto3" json:"metadata,omitempty"`
 }
 
-func (m *PutArtifactMetadata) Reset()         { *m = PutArtifactMetadata{} }
-func (m *PutArtifactMetadata) String() string { return proto.CompactTextString(m) }
-func (*PutArtifactMetadata) ProtoMessage()    {}
+func (x *PutArtifactMetadata) Reset() {
+	*x = PutArtifactMetadata{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_artifact_api_proto_msgTypes[13]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *PutArtifactMetadata) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*PutArtifactMetadata) ProtoMessage() {}
+
+func (x *PutArtifactMetadata) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_artifact_api_proto_msgTypes[13]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use PutArtifactMetadata.ProtoReflect.Descriptor instead.
 func (*PutArtifactMetadata) Descriptor() ([]byte, []int) {
-	return fileDescriptor_8ef4db42c81e3972, []int{13}
+	return file_beam_artifact_api_proto_rawDescGZIP(), []int{13}
 }
 
-func (m *PutArtifactMetadata) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_PutArtifactMetadata.Unmarshal(m, b)
-}
-func (m *PutArtifactMetadata) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_PutArtifactMetadata.Marshal(b, m, deterministic)
-}
-func (m *PutArtifactMetadata) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_PutArtifactMetadata.Merge(m, src)
-}
-func (m *PutArtifactMetadata) XXX_Size() int {
-	return xxx_messageInfo_PutArtifactMetadata.Size(m)
-}
-func (m *PutArtifactMetadata) XXX_DiscardUnknown() {
-	xxx_messageInfo_PutArtifactMetadata.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_PutArtifactMetadata proto.InternalMessageInfo
-
-func (m *PutArtifactMetadata) GetStagingSessionToken() string {
-	if m != nil {
-		return m.StagingSessionToken
+func (x *PutArtifactMetadata) GetStagingSessionToken() string {
+	if x != nil {
+		return x.StagingSessionToken
 	}
 	return ""
 }
 
-func (m *PutArtifactMetadata) GetMetadata() *ArtifactMetadata {
-	if m != nil {
-		return m.Metadata
+func (x *PutArtifactMetadata) GetMetadata() *ArtifactMetadata {
+	if x != nil {
+		return x.Metadata
 	}
 	return nil
 }
 
 // A request to stage an artifact.
 type PutArtifactRequest struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
 	// (Required)
 	//
-	// Types that are valid to be assigned to Content:
+	// Types that are assignable to Content:
 	//	*PutArtifactRequest_Metadata
 	//	*PutArtifactRequest_Data
-	Content              isPutArtifactRequest_Content `protobuf_oneof:"content"`
-	XXX_NoUnkeyedLiteral struct{}                     `json:"-"`
-	XXX_unrecognized     []byte                       `json:"-"`
-	XXX_sizecache        int32                        `json:"-"`
+	Content isPutArtifactRequest_Content `protobuf_oneof:"content"`
 }
 
-func (m *PutArtifactRequest) Reset()         { *m = PutArtifactRequest{} }
-func (m *PutArtifactRequest) String() string { return proto.CompactTextString(m) }
-func (*PutArtifactRequest) ProtoMessage()    {}
+func (x *PutArtifactRequest) Reset() {
+	*x = PutArtifactRequest{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_artifact_api_proto_msgTypes[14]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *PutArtifactRequest) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*PutArtifactRequest) ProtoMessage() {}
+
+func (x *PutArtifactRequest) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_artifact_api_proto_msgTypes[14]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use PutArtifactRequest.ProtoReflect.Descriptor instead.
 func (*PutArtifactRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_8ef4db42c81e3972, []int{14}
+	return file_beam_artifact_api_proto_rawDescGZIP(), []int{14}
 }
 
-func (m *PutArtifactRequest) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_PutArtifactRequest.Unmarshal(m, b)
-}
-func (m *PutArtifactRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_PutArtifactRequest.Marshal(b, m, deterministic)
-}
-func (m *PutArtifactRequest) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_PutArtifactRequest.Merge(m, src)
-}
-func (m *PutArtifactRequest) XXX_Size() int {
-	return xxx_messageInfo_PutArtifactRequest.Size(m)
-}
-func (m *PutArtifactRequest) XXX_DiscardUnknown() {
-	xxx_messageInfo_PutArtifactRequest.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_PutArtifactRequest proto.InternalMessageInfo
-
-type isPutArtifactRequest_Content interface {
-	isPutArtifactRequest_Content()
-}
-
-type PutArtifactRequest_Metadata struct {
-	Metadata *PutArtifactMetadata `protobuf:"bytes,1,opt,name=metadata,proto3,oneof"`
-}
-
-type PutArtifactRequest_Data struct {
-	Data *ArtifactChunk `protobuf:"bytes,2,opt,name=data,proto3,oneof"`
-}
-
-func (*PutArtifactRequest_Metadata) isPutArtifactRequest_Content() {}
-
-func (*PutArtifactRequest_Data) isPutArtifactRequest_Content() {}
-
 func (m *PutArtifactRequest) GetContent() isPutArtifactRequest_Content {
 	if m != nil {
 		return m.Content
@@ -886,256 +969,837 @@
 	return nil
 }
 
-func (m *PutArtifactRequest) GetMetadata() *PutArtifactMetadata {
-	if x, ok := m.GetContent().(*PutArtifactRequest_Metadata); ok {
+func (x *PutArtifactRequest) GetMetadata() *PutArtifactMetadata {
+	if x, ok := x.GetContent().(*PutArtifactRequest_Metadata); ok {
 		return x.Metadata
 	}
 	return nil
 }
 
-func (m *PutArtifactRequest) GetData() *ArtifactChunk {
-	if x, ok := m.GetContent().(*PutArtifactRequest_Data); ok {
+func (x *PutArtifactRequest) GetData() *ArtifactChunk {
+	if x, ok := x.GetContent().(*PutArtifactRequest_Data); ok {
 		return x.Data
 	}
 	return nil
 }
 
-// XXX_OneofWrappers is for the internal use of the proto package.
-func (*PutArtifactRequest) XXX_OneofWrappers() []interface{} {
-	return []interface{}{
-		(*PutArtifactRequest_Metadata)(nil),
-		(*PutArtifactRequest_Data)(nil),
+type isPutArtifactRequest_Content interface {
+	isPutArtifactRequest_Content()
+}
+
+type PutArtifactRequest_Metadata struct {
+	// The first message in a PutArtifact call must contain this field.
+	Metadata *PutArtifactMetadata `protobuf:"bytes,1,opt,name=metadata,proto3,oneof"`
+}
+
+type PutArtifactRequest_Data struct {
+	// A chunk of the artifact. All messages after the first in a PutArtifact call must contain a
+	// chunk.
+	Data *ArtifactChunk `protobuf:"bytes,2,opt,name=data,proto3,oneof"`
+}
+
+func (*PutArtifactRequest_Metadata) isPutArtifactRequest_Content() {}
+
+func (*PutArtifactRequest_Data) isPutArtifactRequest_Content() {}
+
+type PutArtifactResponse struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+}
+
+func (x *PutArtifactResponse) Reset() {
+	*x = PutArtifactResponse{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_artifact_api_proto_msgTypes[15]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
 	}
 }
 
-type PutArtifactResponse struct {
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
+func (x *PutArtifactResponse) String() string {
+	return protoimpl.X.MessageStringOf(x)
 }
 
-func (m *PutArtifactResponse) Reset()         { *m = PutArtifactResponse{} }
-func (m *PutArtifactResponse) String() string { return proto.CompactTextString(m) }
-func (*PutArtifactResponse) ProtoMessage()    {}
+func (*PutArtifactResponse) ProtoMessage() {}
+
+func (x *PutArtifactResponse) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_artifact_api_proto_msgTypes[15]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use PutArtifactResponse.ProtoReflect.Descriptor instead.
 func (*PutArtifactResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_8ef4db42c81e3972, []int{15}
+	return file_beam_artifact_api_proto_rawDescGZIP(), []int{15}
 }
 
-func (m *PutArtifactResponse) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_PutArtifactResponse.Unmarshal(m, b)
-}
-func (m *PutArtifactResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_PutArtifactResponse.Marshal(b, m, deterministic)
-}
-func (m *PutArtifactResponse) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_PutArtifactResponse.Merge(m, src)
-}
-func (m *PutArtifactResponse) XXX_Size() int {
-	return xxx_messageInfo_PutArtifactResponse.Size(m)
-}
-func (m *PutArtifactResponse) XXX_DiscardUnknown() {
-	xxx_messageInfo_PutArtifactResponse.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_PutArtifactResponse proto.InternalMessageInfo
-
 // A request to commit the manifest for a Job. All artifacts must have been successfully uploaded
 // before this call is made.
 type CommitManifestRequest struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
 	// (Required) The manifest to commit.
 	Manifest *Manifest `protobuf:"bytes,1,opt,name=manifest,proto3" json:"manifest,omitempty"`
 	// (Required) A token for artifact staging session. This token can be obtained
 	// from PrepareJob request in JobService
-	StagingSessionToken  string   `protobuf:"bytes,2,opt,name=staging_session_token,json=stagingSessionToken,proto3" json:"staging_session_token,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
+	StagingSessionToken string `protobuf:"bytes,2,opt,name=staging_session_token,json=stagingSessionToken,proto3" json:"staging_session_token,omitempty"`
 }
 
-func (m *CommitManifestRequest) Reset()         { *m = CommitManifestRequest{} }
-func (m *CommitManifestRequest) String() string { return proto.CompactTextString(m) }
-func (*CommitManifestRequest) ProtoMessage()    {}
+func (x *CommitManifestRequest) Reset() {
+	*x = CommitManifestRequest{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_artifact_api_proto_msgTypes[16]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *CommitManifestRequest) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*CommitManifestRequest) ProtoMessage() {}
+
+func (x *CommitManifestRequest) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_artifact_api_proto_msgTypes[16]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use CommitManifestRequest.ProtoReflect.Descriptor instead.
 func (*CommitManifestRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_8ef4db42c81e3972, []int{16}
+	return file_beam_artifact_api_proto_rawDescGZIP(), []int{16}
 }
 
-func (m *CommitManifestRequest) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_CommitManifestRequest.Unmarshal(m, b)
-}
-func (m *CommitManifestRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_CommitManifestRequest.Marshal(b, m, deterministic)
-}
-func (m *CommitManifestRequest) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_CommitManifestRequest.Merge(m, src)
-}
-func (m *CommitManifestRequest) XXX_Size() int {
-	return xxx_messageInfo_CommitManifestRequest.Size(m)
-}
-func (m *CommitManifestRequest) XXX_DiscardUnknown() {
-	xxx_messageInfo_CommitManifestRequest.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_CommitManifestRequest proto.InternalMessageInfo
-
-func (m *CommitManifestRequest) GetManifest() *Manifest {
-	if m != nil {
-		return m.Manifest
+func (x *CommitManifestRequest) GetManifest() *Manifest {
+	if x != nil {
+		return x.Manifest
 	}
 	return nil
 }
 
-func (m *CommitManifestRequest) GetStagingSessionToken() string {
-	if m != nil {
-		return m.StagingSessionToken
+func (x *CommitManifestRequest) GetStagingSessionToken() string {
+	if x != nil {
+		return x.StagingSessionToken
 	}
 	return ""
 }
 
 // The result of committing a manifest.
 type CommitManifestResponse struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
 	// (Required) An opaque token representing the entirety of the staged artifacts.
 	// This can be used to retrieve the manifest and artifacts from an associated
 	// LegacyArtifactRetrievalService.
-	RetrievalToken       string   `protobuf:"bytes,1,opt,name=retrieval_token,json=retrievalToken,proto3" json:"retrieval_token,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
+	RetrievalToken string `protobuf:"bytes,1,opt,name=retrieval_token,json=retrievalToken,proto3" json:"retrieval_token,omitempty"`
 }
 
-func (m *CommitManifestResponse) Reset()         { *m = CommitManifestResponse{} }
-func (m *CommitManifestResponse) String() string { return proto.CompactTextString(m) }
-func (*CommitManifestResponse) ProtoMessage()    {}
+func (x *CommitManifestResponse) Reset() {
+	*x = CommitManifestResponse{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_artifact_api_proto_msgTypes[17]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *CommitManifestResponse) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*CommitManifestResponse) ProtoMessage() {}
+
+func (x *CommitManifestResponse) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_artifact_api_proto_msgTypes[17]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use CommitManifestResponse.ProtoReflect.Descriptor instead.
 func (*CommitManifestResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_8ef4db42c81e3972, []int{17}
+	return file_beam_artifact_api_proto_rawDescGZIP(), []int{17}
 }
 
-func (m *CommitManifestResponse) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_CommitManifestResponse.Unmarshal(m, b)
-}
-func (m *CommitManifestResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_CommitManifestResponse.Marshal(b, m, deterministic)
-}
-func (m *CommitManifestResponse) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_CommitManifestResponse.Merge(m, src)
-}
-func (m *CommitManifestResponse) XXX_Size() int {
-	return xxx_messageInfo_CommitManifestResponse.Size(m)
-}
-func (m *CommitManifestResponse) XXX_DiscardUnknown() {
-	xxx_messageInfo_CommitManifestResponse.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_CommitManifestResponse proto.InternalMessageInfo
-
-func (m *CommitManifestResponse) GetRetrievalToken() string {
-	if m != nil {
-		return m.RetrievalToken
+func (x *CommitManifestResponse) GetRetrievalToken() string {
+	if x != nil {
+		return x.RetrievalToken
 	}
 	return ""
 }
 
-func init() {
-	proto.RegisterEnum("org.apache.beam.model.job_management.v1.CommitManifestResponse_Constants", CommitManifestResponse_Constants_name, CommitManifestResponse_Constants_value)
-	proto.RegisterType((*ResolveArtifactsRequest)(nil), "org.apache.beam.model.job_management.v1.ResolveArtifactsRequest")
-	proto.RegisterType((*ResolveArtifactsResponse)(nil), "org.apache.beam.model.job_management.v1.ResolveArtifactsResponse")
-	proto.RegisterType((*GetArtifactRequest)(nil), "org.apache.beam.model.job_management.v1.GetArtifactRequest")
-	proto.RegisterType((*GetArtifactResponse)(nil), "org.apache.beam.model.job_management.v1.GetArtifactResponse")
-	proto.RegisterType((*ArtifactRequestWrapper)(nil), "org.apache.beam.model.job_management.v1.ArtifactRequestWrapper")
-	proto.RegisterType((*ArtifactResponseWrapper)(nil), "org.apache.beam.model.job_management.v1.ArtifactResponseWrapper")
-	proto.RegisterType((*ArtifactMetadata)(nil), "org.apache.beam.model.job_management.v1.ArtifactMetadata")
-	proto.RegisterType((*Manifest)(nil), "org.apache.beam.model.job_management.v1.Manifest")
-	proto.RegisterType((*ProxyManifest)(nil), "org.apache.beam.model.job_management.v1.ProxyManifest")
-	proto.RegisterType((*ProxyManifest_Location)(nil), "org.apache.beam.model.job_management.v1.ProxyManifest.Location")
-	proto.RegisterType((*GetManifestRequest)(nil), "org.apache.beam.model.job_management.v1.GetManifestRequest")
-	proto.RegisterType((*GetManifestResponse)(nil), "org.apache.beam.model.job_management.v1.GetManifestResponse")
-	proto.RegisterType((*LegacyGetArtifactRequest)(nil), "org.apache.beam.model.job_management.v1.LegacyGetArtifactRequest")
-	proto.RegisterType((*ArtifactChunk)(nil), "org.apache.beam.model.job_management.v1.ArtifactChunk")
-	proto.RegisterType((*PutArtifactMetadata)(nil), "org.apache.beam.model.job_management.v1.PutArtifactMetadata")
-	proto.RegisterType((*PutArtifactRequest)(nil), "org.apache.beam.model.job_management.v1.PutArtifactRequest")
-	proto.RegisterType((*PutArtifactResponse)(nil), "org.apache.beam.model.job_management.v1.PutArtifactResponse")
-	proto.RegisterType((*CommitManifestRequest)(nil), "org.apache.beam.model.job_management.v1.CommitManifestRequest")
-	proto.RegisterType((*CommitManifestResponse)(nil), "org.apache.beam.model.job_management.v1.CommitManifestResponse")
+type ProxyManifest_Location struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"`
+	Uri  string `protobuf:"bytes,2,opt,name=uri,proto3" json:"uri,omitempty"`
 }
 
-func init() { proto.RegisterFile("beam_artifact_api.proto", fileDescriptor_8ef4db42c81e3972) }
+func (x *ProxyManifest_Location) Reset() {
+	*x = ProxyManifest_Location{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_artifact_api_proto_msgTypes[18]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
 
-var fileDescriptor_8ef4db42c81e3972 = []byte{
-	// 1044 bytes of a gzipped FileDescriptorProto
-	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xac, 0x57, 0xbf, 0x6f, 0x23, 0x45,
-	0x14, 0xf6, 0x38, 0x47, 0x62, 0x3f, 0x27, 0x39, 0x6b, 0x22, 0xc7, 0x3e, 0x8b, 0x1f, 0xd6, 0x9e,
-	0xe0, 0x4c, 0xb3, 0x4a, 0x8c, 0x38, 0x09, 0x11, 0xb8, 0x73, 0xc2, 0x11, 0x47, 0x24, 0x77, 0xa7,
-	0xb5, 0xa3, 0x93, 0x42, 0xb1, 0x4c, 0xec, 0x89, 0xb3, 0x9c, 0x77, 0x76, 0x99, 0x19, 0x5b, 0x5c,
-	0x47, 0x89, 0x10, 0x0d, 0x0d, 0x48, 0x54, 0x94, 0x88, 0x9a, 0x82, 0x0e, 0x01, 0x15, 0x7f, 0x03,
-	0xff, 0x00, 0x14, 0xb4, 0x14, 0x54, 0x68, 0x67, 0x7f, 0xd8, 0x6b, 0xef, 0x22, 0xff, 0xb8, 0x6e,
-	0x33, 0xe3, 0xf7, 0x7d, 0xef, 0x7d, 0xf3, 0xbe, 0x37, 0x13, 0x28, 0x5f, 0x52, 0x62, 0x9b, 0x84,
-	0x4b, 0xeb, 0x8a, 0x74, 0xa5, 0x49, 0x5c, 0x4b, 0x77, 0xb9, 0x23, 0x1d, 0x7c, 0xc7, 0xe1, 0x7d,
-	0x9d, 0xb8, 0xa4, 0x7b, 0x4d, 0x75, 0xef, 0x37, 0xba, 0xed, 0xf4, 0xe8, 0x40, 0xff, 0xd8, 0xb9,
-	0x34, 0x6d, 0xc2, 0x48, 0x9f, 0xda, 0x94, 0x49, 0x7d, 0xb4, 0x5f, 0x2d, 0x29, 0x04, 0x3e, 0x64,
-	0x8c, 0xf2, 0x71, 0xbc, 0xf6, 0x35, 0x82, 0xb2, 0x41, 0x85, 0x33, 0x18, 0xd1, 0x66, 0x80, 0x2e,
-	0x0c, 0xfa, 0xc9, 0x90, 0x0a, 0x89, 0x3b, 0x90, 0x0f, 0x19, 0x45, 0x05, 0xd5, 0xd6, 0xea, 0x85,
-	0xc6, 0x5d, 0x3d, 0x99, 0xcf, 0xb5, 0x5c, 0x3a, 0xb0, 0x18, 0xd5, 0x47, 0xfb, 0x7a, 0x88, 0x73,
-	0xc2, 0xae, 0x1c, 0x6e, 0x13, 0x69, 0x39, 0xcc, 0x18, 0x03, 0xe1, 0x57, 0x61, 0xdb, 0xe5, 0xf4,
-	0x8a, 0x72, 0x4e, 0x7b, 0xe6, 0x90, 0x33, 0x51, 0xc9, 0xd6, 0xd6, 0xea, 0x79, 0x63, 0x2b, 0x5a,
-	0x3d, 0xe7, 0x4c, 0x68, 0x23, 0xa8, 0xcc, 0xe6, 0x25, 0x5c, 0x87, 0x09, 0x8a, 0x2f, 0x60, 0x93,
-	0x53, 0x77, 0x40, 0xba, 0xaa, 0xba, 0x55, 0x73, 0x8b, 0x61, 0x69, 0xd7, 0x80, 0x8f, 0xa9, 0x0c,
-	0x7f, 0x17, 0x4a, 0x61, 0x40, 0x2e, 0xac, 0xa0, 0x82, 0x6a, 0x68, 0x05, 0xb6, 0x08, 0x47, 0x7b,
-	0x1d, 0x76, 0x62, 0x4c, 0x41, 0x71, 0x18, 0x6e, 0xf4, 0x88, 0x24, 0x8a, 0x66, 0xd3, 0x50, 0xdf,
-	0xda, 0x3f, 0x08, 0x76, 0xa7, 0x52, 0x7a, 0xc2, 0x89, 0xeb, 0x52, 0x8e, 0x19, 0x14, 0xb9, 0xaf,
-	0x53, 0xd4, 0x1e, 0x95, 0x3f, 0x37, 0x54, 0x8a, 0xf7, 0xf5, 0x39, 0x9b, 0x43, 0x4f, 0xe9, 0x80,
-	0x56, 0xc6, 0xb8, 0xc9, 0xe3, 0x5b, 0x98, 0xc0, 0x66, 0x9f, 0xca, 0x31, 0xd7, 0x5f, 0x3e, 0xd7,
-	0xdb, 0x73, 0x73, 0xcd, 0xaa, 0xdb, 0xca, 0x18, 0x85, 0xfe, 0x78, 0xf5, 0x30, 0x0f, 0x1b, 0xdc,
-	0xdf, 0xd1, 0xfe, 0xc8, 0x42, 0x79, 0x5a, 0xa1, 0xb0, 0xf2, 0xdb, 0xb0, 0x25, 0x24, 0xe9, 0x5b,
-	0xac, 0x6f, 0x4a, 0xe7, 0x29, 0x65, 0x4a, 0xb1, 0xbc, 0xb1, 0x19, 0x2c, 0x76, 0xbc, 0x35, 0x5c,
-	0x86, 0x0d, 0x4b, 0x98, 0x03, 0x22, 0x64, 0x25, 0x5b, 0x43, 0xf5, 0x9c, 0xb1, 0x6e, 0x89, 0x53,
-	0x22, 0x24, 0xfe, 0x0c, 0xc1, 0xad, 0x69, 0xe1, 0x4c, 0x1e, 0x50, 0x84, 0x0a, 0x36, 0x57, 0x50,
-	0xd0, 0x47, 0x6a, 0x65, 0x8c, 0xf2, 0x94, 0x84, 0xd1, 0x49, 0x0b, 0x28, 0x4d, 0x4a, 0x39, 0x66,
-	0x0f, 0x34, 0x3d, 0x58, 0x4e, 0xd3, 0x88, 0x78, 0xa7, 0x3f, 0xbb, 0x7c, 0x08, 0x90, 0x0b, 0x79,
-	0xb4, 0x8f, 0xa0, 0x18, 0xee, 0x9f, 0x51, 0x49, 0xbc, 0x56, 0xf3, 0xda, 0x8f, 0x11, 0x9b, 0x06,
-	0x62, 0xaa, 0x6f, 0x5c, 0x83, 0x82, 0x4b, 0xb9, 0x6d, 0x09, 0x61, 0x39, 0xca, 0xaf, 0xa8, 0xbe,
-	0x65, 0x4c, 0x2e, 0xe1, 0x5d, 0x58, 0x17, 0xd7, 0xa4, 0xf1, 0xe6, 0xdd, 0xca, 0x0d, 0x15, 0x17,
-	0xfc, 0xa5, 0x11, 0xc8, 0x9d, 0x11, 0x66, 0x5d, 0x79, 0x1e, 0x3a, 0x8f, 0x79, 0xc8, 0x73, 0xec,
-	0x5b, 0x73, 0x17, 0x38, 0x9d, 0xe6, 0x84, 0x8d, 0xfe, 0x46, 0xb0, 0xf5, 0x98, 0x3b, 0x9f, 0x3e,
-	0x8b, 0x88, 0xce, 0x20, 0x67, 0x07, 0xdf, 0x81, 0x59, 0xf7, 0xe7, 0x26, 0x0a, 0x41, 0x8c, 0x08,
-	0x02, 0x7f, 0x08, 0xb9, 0x81, 0xd3, 0x55, 0xee, 0x55, 0xa3, 0xaa, 0xd0, 0xb8, 0x37, 0x37, 0x5c,
-	0x2c, 0x31, 0xfd, 0x34, 0x80, 0x31, 0x22, 0xc0, 0xea, 0x1e, 0xe4, 0xc2, 0xd5, 0x44, 0xe9, 0x8b,
-	0xb0, 0x36, 0xe4, 0x96, 0x92, 0x3c, 0x6f, 0x78, 0x9f, 0xda, 0x3b, 0x6a, 0x40, 0x45, 0x79, 0x06,
-	0x03, 0xea, 0x0e, 0xdc, 0xe4, 0x54, 0x72, 0x8b, 0x8e, 0xc8, 0x20, 0x66, 0x87, 0xed, 0x68, 0x59,
-	0x19, 0x42, 0xeb, 0xa9, 0xa9, 0x33, 0x0e, 0x0f, 0x7a, 0xf1, 0xf9, 0x6a, 0xa6, 0x3d, 0x81, 0xca,
-	0x29, 0xed, 0x93, 0xee, 0xb3, 0x84, 0x59, 0x9a, 0x54, 0x66, 0x42, 0xfa, 0xd9, 0xc4, 0xf4, 0x6f,
-	0xc3, 0x56, 0x88, 0x77, 0x74, 0x3d, 0x64, 0x4f, 0x13, 0xc7, 0xe5, 0x77, 0x08, 0x76, 0x1e, 0x0f,
-	0xe5, 0x4c, 0x6f, 0x37, 0xa0, 0x14, 0x4e, 0x0c, 0x41, 0x55, 0xe7, 0xc6, 0xa4, 0xda, 0x09, 0x36,
-	0xdb, 0xfe, 0x9e, 0x3f, 0x40, 0xce, 0x21, 0x67, 0x07, 0xf1, 0x2a, 0xa5, 0xd5, 0xba, 0x36, 0x84,
-	0xd2, 0x7e, 0x47, 0x80, 0x27, 0x52, 0x0c, 0xb5, 0xb9, 0x98, 0x60, 0x43, 0x0b, 0x0e, 0x81, 0x84,
-	0x8a, 0x5b, 0x99, 0x31, 0x25, 0x3e, 0x0d, 0x94, 0xca, 0xfe, 0xef, 0xfd, 0x95, 0x5e, 0x85, 0xd2,
-	0xbb, 0x95, 0xf1, 0x35, 0xf6, 0x86, 0x74, 0xd7, 0x61, 0x92, 0x32, 0xa9, 0x95, 0x62, 0x6a, 0x87,
-	0x2d, 0xa5, 0x7d, 0x8b, 0xa0, 0x74, 0xe4, 0xd8, 0xb6, 0x35, 0xd3, 0xac, 0xcf, 0xd9, 0xa0, 0xa9,
-	0xc7, 0x9a, 0x4d, 0x3d, 0x56, 0xed, 0x4b, 0x04, 0xbb, 0xd3, 0xc9, 0x05, 0x56, 0x98, 0xdb, 0x4a,
-	0x27, 0x90, 0x3f, 0x72, 0x98, 0x90, 0x84, 0x49, 0x81, 0x0f, 0xe0, 0xd6, 0xc3, 0x47, 0x66, 0xd3,
-	0xe8, 0x9c, 0xbc, 0xdf, 0x3c, 0xea, 0xb4, 0xcd, 0x76, 0xa7, 0x79, 0xfc, 0xe0, 0x3d, 0xb3, 0xf3,
-	0xe8, 0x83, 0x07, 0x0f, 0x8b, 0x99, 0xea, 0x4b, 0x3f, 0xfc, 0xf8, 0xef, 0xaf, 0x2f, 0x94, 0x4d,
-	0x93, 0x39, 0xd1, 0xc0, 0x17, 0xa6, 0x97, 0x19, 0xed, 0x99, 0x66, 0xe3, 0x97, 0x2c, 0x54, 0xc6,
-	0x02, 0x06, 0x2c, 0x6d, 0xca, 0x47, 0x56, 0x97, 0xe2, 0x6f, 0x10, 0x14, 0xa7, 0xef, 0x17, 0xbc,
-	0xf2, 0xe5, 0x5e, 0x5d, 0xfd, 0x72, 0xc3, 0x5f, 0x20, 0x28, 0x4c, 0x38, 0x1c, 0xaf, 0xf2, 0x0a,
-	0xa8, 0xae, 0x74, 0xdd, 0xed, 0xa1, 0xc6, 0x6f, 0x13, 0x8f, 0xa4, 0x76, 0x78, 0xe4, 0xbe, 0x82,
-	0xdf, 0x23, 0x78, 0xc5, 0xa0, 0x23, 0xca, 0x05, 0x4d, 0x55, 0xf9, 0xfe, 0xc2, 0x86, 0x98, 0x7a,
-	0x90, 0x54, 0xef, 0x2d, 0x81, 0x30, 0xf9, 0x96, 0xab, 0xa3, 0x3d, 0xd4, 0xf8, 0x39, 0x0b, 0x2f,
-	0xfa, 0xa3, 0x33, 0xa5, 0x16, 0x4f, 0xf3, 0x09, 0xbb, 0x2d, 0xa0, 0xf9, 0xec, 0xbc, 0xa9, 0x1e,
-	0x2c, 0x17, 0xec, 0xd7, 0x5d, 0x47, 0xf8, 0x2b, 0x04, 0xdb, 0x71, 0x1b, 0xe1, 0x77, 0xe7, 0x86,
-	0x4c, 0x1c, 0x0e, 0x0b, 0xa8, 0x98, 0xec, 0xdf, 0xc6, 0x4f, 0x59, 0x78, 0x39, 0xae, 0xe0, 0xcc,
-	0x59, 0x7f, 0xee, 0xf7, 0x6d, 0x94, 0xf3, 0x42, 0x7d, 0x3b, 0x9d, 0xf0, 0xc1, 0x72, 0xc1, 0x29,
-	0x16, 0x9a, 0xdf, 0x95, 0x69, 0x17, 0x6c, 0x75, 0xc9, 0xd1, 0xbe, 0x87, 0x0e, 0x8f, 0xe1, 0xb5,
-	0xd4, 0xd0, 0x58, 0xe4, 0x61, 0x21, 0x0c, 0x6d, 0xba, 0xd6, 0x45, 0x31, 0xb6, 0x6d, 0x8e, 0xf6,
-	0x2f, 0xd7, 0xd5, 0x7f, 0x97, 0x6f, 0xfc, 0x17, 0x00, 0x00, 0xff, 0xff, 0x77, 0x41, 0xcb, 0x1a,
-	0xb8, 0x0e, 0x00, 0x00,
+func (x *ProxyManifest_Location) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*ProxyManifest_Location) ProtoMessage() {}
+
+func (x *ProxyManifest_Location) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_artifact_api_proto_msgTypes[18]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use ProxyManifest_Location.ProtoReflect.Descriptor instead.
+func (*ProxyManifest_Location) Descriptor() ([]byte, []int) {
+	return file_beam_artifact_api_proto_rawDescGZIP(), []int{8, 0}
+}
+
+func (x *ProxyManifest_Location) GetName() string {
+	if x != nil {
+		return x.Name
+	}
+	return ""
+}
+
+func (x *ProxyManifest_Location) GetUri() string {
+	if x != nil {
+		return x.Uri
+	}
+	return ""
+}
+
+var File_beam_artifact_api_proto protoreflect.FileDescriptor
+
+var file_beam_artifact_api_proto_rawDesc = []byte{
+	0x0a, 0x17, 0x62, 0x65, 0x61, 0x6d, 0x5f, 0x61, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x5f,
+	0x61, 0x70, 0x69, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x27, 0x6f, 0x72, 0x67, 0x2e, 0x61,
+	0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c,
+	0x2e, 0x6a, 0x6f, 0x62, 0x5f, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e,
+	0x76, 0x31, 0x1a, 0x15, 0x62, 0x65, 0x61, 0x6d, 0x5f, 0x72, 0x75, 0x6e, 0x6e, 0x65, 0x72, 0x5f,
+	0x61, 0x70, 0x69, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0x96, 0x01, 0x0a, 0x17, 0x52, 0x65,
+	0x73, 0x6f, 0x6c, 0x76, 0x65, 0x41, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x73, 0x52, 0x65,
+	0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x54, 0x0a, 0x09, 0x61, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63,
+	0x74, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x36, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61,
+	0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c,
+	0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x72, 0x74,
+	0x69, 0x66, 0x61, 0x63, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x72, 0x6d, 0x61, 0x74, 0x69, 0x6f, 0x6e,
+	0x52, 0x09, 0x61, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x73, 0x12, 0x25, 0x0a, 0x0e, 0x70,
+	0x72, 0x65, 0x66, 0x65, 0x72, 0x72, 0x65, 0x64, 0x5f, 0x75, 0x72, 0x6e, 0x73, 0x18, 0x02, 0x20,
+	0x03, 0x28, 0x09, 0x52, 0x0d, 0x70, 0x72, 0x65, 0x66, 0x65, 0x72, 0x72, 0x65, 0x64, 0x55, 0x72,
+	0x6e, 0x73, 0x22, 0x76, 0x0a, 0x18, 0x52, 0x65, 0x73, 0x6f, 0x6c, 0x76, 0x65, 0x41, 0x72, 0x74,
+	0x69, 0x66, 0x61, 0x63, 0x74, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x5a,
+	0x0a, 0x0c, 0x72, 0x65, 0x70, 0x6c, 0x61, 0x63, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x01,
+	0x20, 0x03, 0x28, 0x0b, 0x32, 0x36, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68,
+	0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70,
+	0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63,
+	0x74, 0x49, 0x6e, 0x66, 0x6f, 0x72, 0x6d, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0c, 0x72, 0x65,
+	0x70, 0x6c, 0x61, 0x63, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x22, 0x68, 0x0a, 0x12, 0x47, 0x65,
+	0x74, 0x41, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74,
+	0x12, 0x52, 0x0a, 0x08, 0x61, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x18, 0x01, 0x20, 0x01,
+	0x28, 0x0b, 0x32, 0x36, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e,
+	0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c,
+	0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x49,
+	0x6e, 0x66, 0x6f, 0x72, 0x6d, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x08, 0x61, 0x72, 0x74, 0x69,
+	0x66, 0x61, 0x63, 0x74, 0x22, 0x29, 0x0a, 0x13, 0x47, 0x65, 0x74, 0x41, 0x72, 0x74, 0x69, 0x66,
+	0x61, 0x63, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x12, 0x0a, 0x04, 0x64,
+	0x61, 0x74, 0x61, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x04, 0x64, 0x61, 0x74, 0x61, 0x22,
+	0xf6, 0x01, 0x0a, 0x16, 0x41, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x52, 0x65, 0x71, 0x75,
+	0x65, 0x73, 0x74, 0x57, 0x72, 0x61, 0x70, 0x70, 0x65, 0x72, 0x12, 0x6e, 0x0a, 0x10, 0x72, 0x65,
+	0x73, 0x6f, 0x6c, 0x76, 0x65, 0x5f, 0x61, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x18, 0xe8,
+	0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x40, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63,
+	0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x6a, 0x6f,
+	0x62, 0x5f, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x76, 0x31, 0x2e,
+	0x52, 0x65, 0x73, 0x6f, 0x6c, 0x76, 0x65, 0x41, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x73,
+	0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, 0x52, 0x0f, 0x72, 0x65, 0x73, 0x6f, 0x6c,
+	0x76, 0x65, 0x41, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x12, 0x61, 0x0a, 0x0c, 0x67, 0x65,
+	0x74, 0x5f, 0x61, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x18, 0xe9, 0x07, 0x20, 0x01, 0x28,
+	0x0b, 0x32, 0x3b, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62,
+	0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x6a, 0x6f, 0x62, 0x5f, 0x6d, 0x61,
+	0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, 0x41,
+	0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00,
+	0x52, 0x0b, 0x67, 0x65, 0x74, 0x41, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x42, 0x09, 0x0a,
+	0x07, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x22, 0xdb, 0x02, 0x0a, 0x17, 0x41, 0x72, 0x74,
+	0x69, 0x66, 0x61, 0x63, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x57, 0x72, 0x61,
+	0x70, 0x70, 0x65, 0x72, 0x12, 0x23, 0x0a, 0x0d, 0x73, 0x74, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f,
+	0x74, 0x6f, 0x6b, 0x65, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x73, 0x74, 0x61,
+	0x67, 0x69, 0x6e, 0x67, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x12, 0x17, 0x0a, 0x07, 0x69, 0x73, 0x5f,
+	0x6c, 0x61, 0x73, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x08, 0x52, 0x06, 0x69, 0x73, 0x4c, 0x61,
+	0x73, 0x74, 0x12, 0x80, 0x01, 0x0a, 0x19, 0x72, 0x65, 0x73, 0x6f, 0x6c, 0x76, 0x65, 0x5f, 0x61,
+	0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x5f, 0x72, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65,
+	0x18, 0xe8, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x41, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70,
+	0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e,
+	0x6a, 0x6f, 0x62, 0x5f, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x76,
+	0x31, 0x2e, 0x52, 0x65, 0x73, 0x6f, 0x6c, 0x76, 0x65, 0x41, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63,
+	0x74, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x48, 0x00, 0x52, 0x17, 0x72, 0x65,
+	0x73, 0x6f, 0x6c, 0x76, 0x65, 0x41, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x52, 0x65, 0x73,
+	0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x73, 0x0a, 0x15, 0x67, 0x65, 0x74, 0x5f, 0x61, 0x72, 0x74,
+	0x69, 0x66, 0x61, 0x63, 0x74, 0x5f, 0x72, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x18, 0xe9,
+	0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3c, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63,
+	0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x6a, 0x6f,
+	0x62, 0x5f, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x76, 0x31, 0x2e,
+	0x47, 0x65, 0x74, 0x41, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f,
+	0x6e, 0x73, 0x65, 0x48, 0x00, 0x52, 0x13, 0x67, 0x65, 0x74, 0x41, 0x72, 0x74, 0x69, 0x66, 0x61,
+	0x63, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x42, 0x0a, 0x0a, 0x08, 0x72, 0x65,
+	0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x60, 0x0a, 0x10, 0x41, 0x72, 0x74, 0x69, 0x66, 0x61,
+	0x63, 0x74, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61,
+	0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x20,
+	0x0a, 0x0b, 0x70, 0x65, 0x72, 0x6d, 0x69, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x02, 0x20,
+	0x01, 0x28, 0x0d, 0x52, 0x0b, 0x70, 0x65, 0x72, 0x6d, 0x69, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x73,
+	0x12, 0x16, 0x0a, 0x06, 0x73, 0x68, 0x61, 0x32, 0x35, 0x36, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09,
+	0x52, 0x06, 0x73, 0x68, 0x61, 0x32, 0x35, 0x36, 0x22, 0x61, 0x0a, 0x08, 0x4d, 0x61, 0x6e, 0x69,
+	0x66, 0x65, 0x73, 0x74, 0x12, 0x55, 0x0a, 0x08, 0x61, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74,
+	0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x39, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61,
+	0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x6a,
+	0x6f, 0x62, 0x5f, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x76, 0x31,
+	0x2e, 0x41, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74,
+	0x61, 0x52, 0x08, 0x61, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x22, 0xed, 0x01, 0x0a, 0x0d,
+	0x50, 0x72, 0x6f, 0x78, 0x79, 0x4d, 0x61, 0x6e, 0x69, 0x66, 0x65, 0x73, 0x74, 0x12, 0x4d, 0x0a,
+	0x08, 0x6d, 0x61, 0x6e, 0x69, 0x66, 0x65, 0x73, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32,
+	0x31, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61,
+	0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x6a, 0x6f, 0x62, 0x5f, 0x6d, 0x61, 0x6e, 0x61,
+	0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x76, 0x31, 0x2e, 0x4d, 0x61, 0x6e, 0x69, 0x66, 0x65,
+	0x73, 0x74, 0x52, 0x08, 0x6d, 0x61, 0x6e, 0x69, 0x66, 0x65, 0x73, 0x74, 0x12, 0x5b, 0x0a, 0x08,
+	0x6c, 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x3f,
+	0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d,
+	0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x6a, 0x6f, 0x62, 0x5f, 0x6d, 0x61, 0x6e, 0x61, 0x67,
+	0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x72, 0x6f, 0x78, 0x79, 0x4d, 0x61,
+	0x6e, 0x69, 0x66, 0x65, 0x73, 0x74, 0x2e, 0x4c, 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52,
+	0x08, 0x6c, 0x6f, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x1a, 0x30, 0x0a, 0x08, 0x4c, 0x6f, 0x63,
+	0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20,
+	0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x10, 0x0a, 0x03, 0x75, 0x72, 0x69,
+	0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x75, 0x72, 0x69, 0x22, 0x3d, 0x0a, 0x12, 0x47,
+	0x65, 0x74, 0x4d, 0x61, 0x6e, 0x69, 0x66, 0x65, 0x73, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73,
+	0x74, 0x12, 0x27, 0x0a, 0x0f, 0x72, 0x65, 0x74, 0x72, 0x69, 0x65, 0x76, 0x61, 0x6c, 0x5f, 0x74,
+	0x6f, 0x6b, 0x65, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x72, 0x65, 0x74, 0x72,
+	0x69, 0x65, 0x76, 0x61, 0x6c, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x22, 0x64, 0x0a, 0x13, 0x47, 0x65,
+	0x74, 0x4d, 0x61, 0x6e, 0x69, 0x66, 0x65, 0x73, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73,
+	0x65, 0x12, 0x4d, 0x0a, 0x08, 0x6d, 0x61, 0x6e, 0x69, 0x66, 0x65, 0x73, 0x74, 0x18, 0x01, 0x20,
+	0x01, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65,
+	0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x6a, 0x6f, 0x62, 0x5f,
+	0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x76, 0x31, 0x2e, 0x4d, 0x61,
+	0x6e, 0x69, 0x66, 0x65, 0x73, 0x74, 0x52, 0x08, 0x6d, 0x61, 0x6e, 0x69, 0x66, 0x65, 0x73, 0x74,
+	0x22, 0x57, 0x0a, 0x18, 0x4c, 0x65, 0x67, 0x61, 0x63, 0x79, 0x47, 0x65, 0x74, 0x41, 0x72, 0x74,
+	0x69, 0x66, 0x61, 0x63, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x12, 0x0a, 0x04,
+	0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65,
+	0x12, 0x27, 0x0a, 0x0f, 0x72, 0x65, 0x74, 0x72, 0x69, 0x65, 0x76, 0x61, 0x6c, 0x5f, 0x74, 0x6f,
+	0x6b, 0x65, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x72, 0x65, 0x74, 0x72, 0x69,
+	0x65, 0x76, 0x61, 0x6c, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x22, 0x23, 0x0a, 0x0d, 0x41, 0x72, 0x74,
+	0x69, 0x66, 0x61, 0x63, 0x74, 0x43, 0x68, 0x75, 0x6e, 0x6b, 0x12, 0x12, 0x0a, 0x04, 0x64, 0x61,
+	0x74, 0x61, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x04, 0x64, 0x61, 0x74, 0x61, 0x22, 0xa0,
+	0x01, 0x0a, 0x13, 0x50, 0x75, 0x74, 0x41, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x4d, 0x65,
+	0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x12, 0x32, 0x0a, 0x15, 0x73, 0x74, 0x61, 0x67, 0x69, 0x6e,
+	0x67, 0x5f, 0x73, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x5f, 0x74, 0x6f, 0x6b, 0x65, 0x6e, 0x18,
+	0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x13, 0x73, 0x74, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x53, 0x65,
+	0x73, 0x73, 0x69, 0x6f, 0x6e, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x12, 0x55, 0x0a, 0x08, 0x6d, 0x65,
+	0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x39, 0x2e, 0x6f,
+	0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d,
+	0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x6a, 0x6f, 0x62, 0x5f, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d,
+	0x65, 0x6e, 0x74, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x4d,
+	0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x52, 0x08, 0x6d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74,
+	0x61, 0x22, 0xc9, 0x01, 0x0a, 0x12, 0x50, 0x75, 0x74, 0x41, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63,
+	0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x5a, 0x0a, 0x08, 0x6d, 0x65, 0x74, 0x61,
+	0x64, 0x61, 0x74, 0x61, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3c, 0x2e, 0x6f, 0x72, 0x67,
+	0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64,
+	0x65, 0x6c, 0x2e, 0x6a, 0x6f, 0x62, 0x5f, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e,
+	0x74, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x75, 0x74, 0x41, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74,
+	0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0x48, 0x00, 0x52, 0x08, 0x6d, 0x65, 0x74, 0x61,
+	0x64, 0x61, 0x74, 0x61, 0x12, 0x4c, 0x0a, 0x04, 0x64, 0x61, 0x74, 0x61, 0x18, 0x02, 0x20, 0x01,
+	0x28, 0x0b, 0x32, 0x36, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e,
+	0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x6a, 0x6f, 0x62, 0x5f, 0x6d,
+	0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x72, 0x74,
+	0x69, 0x66, 0x61, 0x63, 0x74, 0x43, 0x68, 0x75, 0x6e, 0x6b, 0x48, 0x00, 0x52, 0x04, 0x64, 0x61,
+	0x74, 0x61, 0x42, 0x09, 0x0a, 0x07, 0x63, 0x6f, 0x6e, 0x74, 0x65, 0x6e, 0x74, 0x22, 0x15, 0x0a,
+	0x13, 0x50, 0x75, 0x74, 0x41, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x52, 0x65, 0x73, 0x70,
+	0x6f, 0x6e, 0x73, 0x65, 0x22, 0x9a, 0x01, 0x0a, 0x15, 0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x4d,
+	0x61, 0x6e, 0x69, 0x66, 0x65, 0x73, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x4d,
+	0x0a, 0x08, 0x6d, 0x61, 0x6e, 0x69, 0x66, 0x65, 0x73, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b,
+	0x32, 0x31, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65,
+	0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x6a, 0x6f, 0x62, 0x5f, 0x6d, 0x61, 0x6e,
+	0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x76, 0x31, 0x2e, 0x4d, 0x61, 0x6e, 0x69, 0x66,
+	0x65, 0x73, 0x74, 0x52, 0x08, 0x6d, 0x61, 0x6e, 0x69, 0x66, 0x65, 0x73, 0x74, 0x12, 0x32, 0x0a,
+	0x15, 0x73, 0x74, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f, 0x73, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e,
+	0x5f, 0x74, 0x6f, 0x6b, 0x65, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x13, 0x73, 0x74,
+	0x61, 0x67, 0x69, 0x6e, 0x67, 0x53, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x54, 0x6f, 0x6b, 0x65,
+	0x6e, 0x22, 0x8c, 0x01, 0x0a, 0x16, 0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x4d, 0x61, 0x6e, 0x69,
+	0x66, 0x65, 0x73, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x27, 0x0a, 0x0f,
+	0x72, 0x65, 0x74, 0x72, 0x69, 0x65, 0x76, 0x61, 0x6c, 0x5f, 0x74, 0x6f, 0x6b, 0x65, 0x6e, 0x18,
+	0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x72, 0x65, 0x74, 0x72, 0x69, 0x65, 0x76, 0x61, 0x6c,
+	0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x22, 0x49, 0x0a, 0x09, 0x43, 0x6f, 0x6e, 0x73, 0x74, 0x61, 0x6e,
+	0x74, 0x73, 0x12, 0x3c, 0x0a, 0x19, 0x4e, 0x4f, 0x5f, 0x41, 0x52, 0x54, 0x49, 0x46, 0x41, 0x43,
+	0x54, 0x53, 0x5f, 0x53, 0x54, 0x41, 0x47, 0x45, 0x44, 0x5f, 0x54, 0x4f, 0x4b, 0x45, 0x4e, 0x10,
+	0x00, 0x1a, 0x1d, 0xaa, 0xb4, 0xfa, 0xc2, 0x05, 0x17, 0x5f, 0x5f, 0x6e, 0x6f, 0x5f, 0x61, 0x72,
+	0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x73, 0x5f, 0x73, 0x74, 0x61, 0x67, 0x65, 0x64, 0x5f, 0x5f,
+	0x32, 0xc1, 0x02, 0x0a, 0x18, 0x41, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x52, 0x65, 0x74,
+	0x72, 0x69, 0x65, 0x76, 0x61, 0x6c, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x97, 0x01,
+	0x0a, 0x10, 0x52, 0x65, 0x73, 0x6f, 0x6c, 0x76, 0x65, 0x41, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63,
+	0x74, 0x73, 0x12, 0x40, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e,
+	0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x6a, 0x6f, 0x62, 0x5f, 0x6d,
+	0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, 0x73,
+	0x6f, 0x6c, 0x76, 0x65, 0x41, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x73, 0x52, 0x65, 0x71,
+	0x75, 0x65, 0x73, 0x74, 0x1a, 0x41, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68,
+	0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x6a, 0x6f, 0x62,
+	0x5f, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x76, 0x31, 0x2e, 0x52,
+	0x65, 0x73, 0x6f, 0x6c, 0x76, 0x65, 0x41, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x73, 0x52,
+	0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x8a, 0x01, 0x0a, 0x0b, 0x47, 0x65, 0x74, 0x41,
+	0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x12, 0x3b, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70,
+	0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e,
+	0x6a, 0x6f, 0x62, 0x5f, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x76,
+	0x31, 0x2e, 0x47, 0x65, 0x74, 0x41, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x52, 0x65, 0x71,
+	0x75, 0x65, 0x73, 0x74, 0x1a, 0x3c, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68,
+	0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x6a, 0x6f, 0x62,
+	0x5f, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x76, 0x31, 0x2e, 0x47,
+	0x65, 0x74, 0x41, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e,
+	0x73, 0x65, 0x30, 0x01, 0x32, 0xc3, 0x01, 0x0a, 0x16, 0x41, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63,
+	0x74, 0x53, 0x74, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12,
+	0xa8, 0x01, 0x0a, 0x1f, 0x52, 0x65, 0x76, 0x65, 0x72, 0x73, 0x65, 0x41, 0x72, 0x74, 0x69, 0x66,
+	0x61, 0x63, 0x74, 0x52, 0x65, 0x74, 0x72, 0x69, 0x65, 0x76, 0x61, 0x6c, 0x53, 0x65, 0x72, 0x76,
+	0x69, 0x63, 0x65, 0x12, 0x40, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65,
+	0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x6a, 0x6f, 0x62, 0x5f,
+	0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x72,
+	0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x57, 0x72,
+	0x61, 0x70, 0x70, 0x65, 0x72, 0x1a, 0x3f, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63,
+	0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x6a, 0x6f,
+	0x62, 0x5f, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x76, 0x31, 0x2e,
+	0x41, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x57,
+	0x72, 0x61, 0x70, 0x70, 0x65, 0x72, 0x28, 0x01, 0x30, 0x01, 0x32, 0xbf, 0x02, 0x0a, 0x1c, 0x4c,
+	0x65, 0x67, 0x61, 0x63, 0x79, 0x41, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x53, 0x74, 0x61,
+	0x67, 0x69, 0x6e, 0x67, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x8a, 0x01, 0x0a, 0x0b,
+	0x50, 0x75, 0x74, 0x41, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x12, 0x3b, 0x2e, 0x6f, 0x72,
+	0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f,
+	0x64, 0x65, 0x6c, 0x2e, 0x6a, 0x6f, 0x62, 0x5f, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65,
+	0x6e, 0x74, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x75, 0x74, 0x41, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63,
+	0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x3c, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61,
+	0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c,
+	0x2e, 0x6a, 0x6f, 0x62, 0x5f, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e,
+	0x76, 0x31, 0x2e, 0x50, 0x75, 0x74, 0x41, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x52, 0x65,
+	0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x28, 0x01, 0x12, 0x91, 0x01, 0x0a, 0x0e, 0x43, 0x6f, 0x6d,
+	0x6d, 0x69, 0x74, 0x4d, 0x61, 0x6e, 0x69, 0x66, 0x65, 0x73, 0x74, 0x12, 0x3e, 0x2e, 0x6f, 0x72,
+	0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f,
+	0x64, 0x65, 0x6c, 0x2e, 0x6a, 0x6f, 0x62, 0x5f, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65,
+	0x6e, 0x74, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x4d, 0x61, 0x6e, 0x69,
+	0x66, 0x65, 0x73, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x3f, 0x2e, 0x6f, 0x72,
+	0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f,
+	0x64, 0x65, 0x6c, 0x2e, 0x6a, 0x6f, 0x62, 0x5f, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65,
+	0x6e, 0x74, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x4d, 0x61, 0x6e, 0x69,
+	0x66, 0x65, 0x73, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x32, 0xb8, 0x02, 0x0a,
+	0x1e, 0x4c, 0x65, 0x67, 0x61, 0x63, 0x79, 0x41, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x52,
+	0x65, 0x74, 0x72, 0x69, 0x65, 0x76, 0x61, 0x6c, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12,
+	0x88, 0x01, 0x0a, 0x0b, 0x47, 0x65, 0x74, 0x4d, 0x61, 0x6e, 0x69, 0x66, 0x65, 0x73, 0x74, 0x12,
+	0x3b, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61,
+	0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x6a, 0x6f, 0x62, 0x5f, 0x6d, 0x61, 0x6e, 0x61,
+	0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, 0x4d, 0x61, 0x6e,
+	0x69, 0x66, 0x65, 0x73, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x3c, 0x2e, 0x6f,
+	0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d,
+	0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x6a, 0x6f, 0x62, 0x5f, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d,
+	0x65, 0x6e, 0x74, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, 0x4d, 0x61, 0x6e, 0x69, 0x66, 0x65,
+	0x73, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x8a, 0x01, 0x0a, 0x0b, 0x47,
+	0x65, 0x74, 0x41, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x12, 0x41, 0x2e, 0x6f, 0x72, 0x67,
+	0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64,
+	0x65, 0x6c, 0x2e, 0x6a, 0x6f, 0x62, 0x5f, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e,
+	0x74, 0x2e, 0x76, 0x31, 0x2e, 0x4c, 0x65, 0x67, 0x61, 0x63, 0x79, 0x47, 0x65, 0x74, 0x41, 0x72,
+	0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x36, 0x2e,
+	0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e,
+	0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x6a, 0x6f, 0x62, 0x5f, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65,
+	0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74,
+	0x43, 0x68, 0x75, 0x6e, 0x6b, 0x30, 0x01, 0x42, 0x86, 0x01, 0x0a, 0x26, 0x6f, 0x72, 0x67, 0x2e,
+	0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65,
+	0x6c, 0x2e, 0x6a, 0x6f, 0x62, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e,
+	0x76, 0x31, 0x42, 0x0b, 0x41, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x41, 0x70, 0x69, 0x5a,
+	0x4f, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x61, 0x70, 0x61, 0x63,
+	0x68, 0x65, 0x2f, 0x62, 0x65, 0x61, 0x6d, 0x2f, 0x73, 0x64, 0x6b, 0x73, 0x2f, 0x67, 0x6f, 0x2f,
+	0x70, 0x6b, 0x67, 0x2f, 0x62, 0x65, 0x61, 0x6d, 0x2f, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2f, 0x6a,
+	0x6f, 0x62, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x76, 0x31, 0x3b,
+	0x6a, 0x6f, 0x62, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x76, 0x31,
+	0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
+}
+
+var (
+	file_beam_artifact_api_proto_rawDescOnce sync.Once
+	file_beam_artifact_api_proto_rawDescData = file_beam_artifact_api_proto_rawDesc
+)
+
+func file_beam_artifact_api_proto_rawDescGZIP() []byte {
+	file_beam_artifact_api_proto_rawDescOnce.Do(func() {
+		file_beam_artifact_api_proto_rawDescData = protoimpl.X.CompressGZIP(file_beam_artifact_api_proto_rawDescData)
+	})
+	return file_beam_artifact_api_proto_rawDescData
+}
+
+var file_beam_artifact_api_proto_enumTypes = make([]protoimpl.EnumInfo, 1)
+var file_beam_artifact_api_proto_msgTypes = make([]protoimpl.MessageInfo, 19)
+var file_beam_artifact_api_proto_goTypes = []interface{}{
+	(CommitManifestResponse_Constants)(0),   // 0: org.apache.beam.model.job_management.v1.CommitManifestResponse.Constants
+	(*ResolveArtifactsRequest)(nil),         // 1: org.apache.beam.model.job_management.v1.ResolveArtifactsRequest
+	(*ResolveArtifactsResponse)(nil),        // 2: org.apache.beam.model.job_management.v1.ResolveArtifactsResponse
+	(*GetArtifactRequest)(nil),              // 3: org.apache.beam.model.job_management.v1.GetArtifactRequest
+	(*GetArtifactResponse)(nil),             // 4: org.apache.beam.model.job_management.v1.GetArtifactResponse
+	(*ArtifactRequestWrapper)(nil),          // 5: org.apache.beam.model.job_management.v1.ArtifactRequestWrapper
+	(*ArtifactResponseWrapper)(nil),         // 6: org.apache.beam.model.job_management.v1.ArtifactResponseWrapper
+	(*ArtifactMetadata)(nil),                // 7: org.apache.beam.model.job_management.v1.ArtifactMetadata
+	(*Manifest)(nil),                        // 8: org.apache.beam.model.job_management.v1.Manifest
+	(*ProxyManifest)(nil),                   // 9: org.apache.beam.model.job_management.v1.ProxyManifest
+	(*GetManifestRequest)(nil),              // 10: org.apache.beam.model.job_management.v1.GetManifestRequest
+	(*GetManifestResponse)(nil),             // 11: org.apache.beam.model.job_management.v1.GetManifestResponse
+	(*LegacyGetArtifactRequest)(nil),        // 12: org.apache.beam.model.job_management.v1.LegacyGetArtifactRequest
+	(*ArtifactChunk)(nil),                   // 13: org.apache.beam.model.job_management.v1.ArtifactChunk
+	(*PutArtifactMetadata)(nil),             // 14: org.apache.beam.model.job_management.v1.PutArtifactMetadata
+	(*PutArtifactRequest)(nil),              // 15: org.apache.beam.model.job_management.v1.PutArtifactRequest
+	(*PutArtifactResponse)(nil),             // 16: org.apache.beam.model.job_management.v1.PutArtifactResponse
+	(*CommitManifestRequest)(nil),           // 17: org.apache.beam.model.job_management.v1.CommitManifestRequest
+	(*CommitManifestResponse)(nil),          // 18: org.apache.beam.model.job_management.v1.CommitManifestResponse
+	(*ProxyManifest_Location)(nil),          // 19: org.apache.beam.model.job_management.v1.ProxyManifest.Location
+	(*pipeline_v1.ArtifactInformation)(nil), // 20: org.apache.beam.model.pipeline.v1.ArtifactInformation
+}
+var file_beam_artifact_api_proto_depIdxs = []int32{
+	20, // 0: org.apache.beam.model.job_management.v1.ResolveArtifactsRequest.artifacts:type_name -> org.apache.beam.model.pipeline.v1.ArtifactInformation
+	20, // 1: org.apache.beam.model.job_management.v1.ResolveArtifactsResponse.replacements:type_name -> org.apache.beam.model.pipeline.v1.ArtifactInformation
+	20, // 2: org.apache.beam.model.job_management.v1.GetArtifactRequest.artifact:type_name -> org.apache.beam.model.pipeline.v1.ArtifactInformation
+	1,  // 3: org.apache.beam.model.job_management.v1.ArtifactRequestWrapper.resolve_artifact:type_name -> org.apache.beam.model.job_management.v1.ResolveArtifactsRequest
+	3,  // 4: org.apache.beam.model.job_management.v1.ArtifactRequestWrapper.get_artifact:type_name -> org.apache.beam.model.job_management.v1.GetArtifactRequest
+	2,  // 5: org.apache.beam.model.job_management.v1.ArtifactResponseWrapper.resolve_artifact_response:type_name -> org.apache.beam.model.job_management.v1.ResolveArtifactsResponse
+	4,  // 6: org.apache.beam.model.job_management.v1.ArtifactResponseWrapper.get_artifact_response:type_name -> org.apache.beam.model.job_management.v1.GetArtifactResponse
+	7,  // 7: org.apache.beam.model.job_management.v1.Manifest.artifact:type_name -> org.apache.beam.model.job_management.v1.ArtifactMetadata
+	8,  // 8: org.apache.beam.model.job_management.v1.ProxyManifest.manifest:type_name -> org.apache.beam.model.job_management.v1.Manifest
+	19, // 9: org.apache.beam.model.job_management.v1.ProxyManifest.location:type_name -> org.apache.beam.model.job_management.v1.ProxyManifest.Location
+	8,  // 10: org.apache.beam.model.job_management.v1.GetManifestResponse.manifest:type_name -> org.apache.beam.model.job_management.v1.Manifest
+	7,  // 11: org.apache.beam.model.job_management.v1.PutArtifactMetadata.metadata:type_name -> org.apache.beam.model.job_management.v1.ArtifactMetadata
+	14, // 12: org.apache.beam.model.job_management.v1.PutArtifactRequest.metadata:type_name -> org.apache.beam.model.job_management.v1.PutArtifactMetadata
+	13, // 13: org.apache.beam.model.job_management.v1.PutArtifactRequest.data:type_name -> org.apache.beam.model.job_management.v1.ArtifactChunk
+	8,  // 14: org.apache.beam.model.job_management.v1.CommitManifestRequest.manifest:type_name -> org.apache.beam.model.job_management.v1.Manifest
+	1,  // 15: org.apache.beam.model.job_management.v1.ArtifactRetrievalService.ResolveArtifacts:input_type -> org.apache.beam.model.job_management.v1.ResolveArtifactsRequest
+	3,  // 16: org.apache.beam.model.job_management.v1.ArtifactRetrievalService.GetArtifact:input_type -> org.apache.beam.model.job_management.v1.GetArtifactRequest
+	6,  // 17: org.apache.beam.model.job_management.v1.ArtifactStagingService.ReverseArtifactRetrievalService:input_type -> org.apache.beam.model.job_management.v1.ArtifactResponseWrapper
+	15, // 18: org.apache.beam.model.job_management.v1.LegacyArtifactStagingService.PutArtifact:input_type -> org.apache.beam.model.job_management.v1.PutArtifactRequest
+	17, // 19: org.apache.beam.model.job_management.v1.LegacyArtifactStagingService.CommitManifest:input_type -> org.apache.beam.model.job_management.v1.CommitManifestRequest
+	10, // 20: org.apache.beam.model.job_management.v1.LegacyArtifactRetrievalService.GetManifest:input_type -> org.apache.beam.model.job_management.v1.GetManifestRequest
+	12, // 21: org.apache.beam.model.job_management.v1.LegacyArtifactRetrievalService.GetArtifact:input_type -> org.apache.beam.model.job_management.v1.LegacyGetArtifactRequest
+	2,  // 22: org.apache.beam.model.job_management.v1.ArtifactRetrievalService.ResolveArtifacts:output_type -> org.apache.beam.model.job_management.v1.ResolveArtifactsResponse
+	4,  // 23: org.apache.beam.model.job_management.v1.ArtifactRetrievalService.GetArtifact:output_type -> org.apache.beam.model.job_management.v1.GetArtifactResponse
+	5,  // 24: org.apache.beam.model.job_management.v1.ArtifactStagingService.ReverseArtifactRetrievalService:output_type -> org.apache.beam.model.job_management.v1.ArtifactRequestWrapper
+	16, // 25: org.apache.beam.model.job_management.v1.LegacyArtifactStagingService.PutArtifact:output_type -> org.apache.beam.model.job_management.v1.PutArtifactResponse
+	18, // 26: org.apache.beam.model.job_management.v1.LegacyArtifactStagingService.CommitManifest:output_type -> org.apache.beam.model.job_management.v1.CommitManifestResponse
+	11, // 27: org.apache.beam.model.job_management.v1.LegacyArtifactRetrievalService.GetManifest:output_type -> org.apache.beam.model.job_management.v1.GetManifestResponse
+	13, // 28: org.apache.beam.model.job_management.v1.LegacyArtifactRetrievalService.GetArtifact:output_type -> org.apache.beam.model.job_management.v1.ArtifactChunk
+	22, // [22:29] is the sub-list for method output_type
+	15, // [15:22] is the sub-list for method input_type
+	15, // [15:15] is the sub-list for extension type_name
+	15, // [15:15] is the sub-list for extension extendee
+	0,  // [0:15] is the sub-list for field type_name
+}
+
+func init() { file_beam_artifact_api_proto_init() }
+func file_beam_artifact_api_proto_init() {
+	if File_beam_artifact_api_proto != nil {
+		return
+	}
+	if !protoimpl.UnsafeEnabled {
+		file_beam_artifact_api_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*ResolveArtifactsRequest); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_artifact_api_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*ResolveArtifactsResponse); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_artifact_api_proto_msgTypes[2].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*GetArtifactRequest); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_artifact_api_proto_msgTypes[3].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*GetArtifactResponse); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_artifact_api_proto_msgTypes[4].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*ArtifactRequestWrapper); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_artifact_api_proto_msgTypes[5].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*ArtifactResponseWrapper); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_artifact_api_proto_msgTypes[6].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*ArtifactMetadata); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_artifact_api_proto_msgTypes[7].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*Manifest); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_artifact_api_proto_msgTypes[8].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*ProxyManifest); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_artifact_api_proto_msgTypes[9].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*GetManifestRequest); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_artifact_api_proto_msgTypes[10].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*GetManifestResponse); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_artifact_api_proto_msgTypes[11].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*LegacyGetArtifactRequest); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_artifact_api_proto_msgTypes[12].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*ArtifactChunk); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_artifact_api_proto_msgTypes[13].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*PutArtifactMetadata); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_artifact_api_proto_msgTypes[14].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*PutArtifactRequest); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_artifact_api_proto_msgTypes[15].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*PutArtifactResponse); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_artifact_api_proto_msgTypes[16].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*CommitManifestRequest); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_artifact_api_proto_msgTypes[17].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*CommitManifestResponse); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_artifact_api_proto_msgTypes[18].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*ProxyManifest_Location); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+	}
+	file_beam_artifact_api_proto_msgTypes[4].OneofWrappers = []interface{}{
+		(*ArtifactRequestWrapper_ResolveArtifact)(nil),
+		(*ArtifactRequestWrapper_GetArtifact)(nil),
+	}
+	file_beam_artifact_api_proto_msgTypes[5].OneofWrappers = []interface{}{
+		(*ArtifactResponseWrapper_ResolveArtifactResponse)(nil),
+		(*ArtifactResponseWrapper_GetArtifactResponse)(nil),
+	}
+	file_beam_artifact_api_proto_msgTypes[14].OneofWrappers = []interface{}{
+		(*PutArtifactRequest_Metadata)(nil),
+		(*PutArtifactRequest_Data)(nil),
+	}
+	type x struct{}
+	out := protoimpl.TypeBuilder{
+		File: protoimpl.DescBuilder{
+			GoPackagePath: reflect.TypeOf(x{}).PkgPath(),
+			RawDescriptor: file_beam_artifact_api_proto_rawDesc,
+			NumEnums:      1,
+			NumMessages:   19,
+			NumExtensions: 0,
+			NumServices:   4,
+		},
+		GoTypes:           file_beam_artifact_api_proto_goTypes,
+		DependencyIndexes: file_beam_artifact_api_proto_depIdxs,
+		EnumInfos:         file_beam_artifact_api_proto_enumTypes,
+		MessageInfos:      file_beam_artifact_api_proto_msgTypes,
+	}.Build()
+	File_beam_artifact_api_proto = out.File
+	file_beam_artifact_api_proto_rawDesc = nil
+	file_beam_artifact_api_proto_goTypes = nil
+	file_beam_artifact_api_proto_depIdxs = nil
 }
 
 // Reference imports to suppress errors if they are not otherwise used.
 var _ context.Context
-var _ grpc.ClientConn
+var _ grpc.ClientConnInterface
 
 // 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
+const _ = grpc.SupportPackageIsVersion6
 
 // ArtifactRetrievalServiceClient is the client API for ArtifactRetrievalService service.
 //
@@ -1150,10 +1814,10 @@
 }
 
 type artifactRetrievalServiceClient struct {
-	cc *grpc.ClientConn
+	cc grpc.ClientConnInterface
 }
 
-func NewArtifactRetrievalServiceClient(cc *grpc.ClientConn) ArtifactRetrievalServiceClient {
+func NewArtifactRetrievalServiceClient(cc grpc.ClientConnInterface) ArtifactRetrievalServiceClient {
 	return &artifactRetrievalServiceClient{cc}
 }
 
@@ -1212,10 +1876,10 @@
 type UnimplementedArtifactRetrievalServiceServer struct {
 }
 
-func (*UnimplementedArtifactRetrievalServiceServer) ResolveArtifacts(ctx context.Context, req *ResolveArtifactsRequest) (*ResolveArtifactsResponse, error) {
+func (*UnimplementedArtifactRetrievalServiceServer) ResolveArtifacts(context.Context, *ResolveArtifactsRequest) (*ResolveArtifactsResponse, error) {
 	return nil, status.Errorf(codes.Unimplemented, "method ResolveArtifacts not implemented")
 }
-func (*UnimplementedArtifactRetrievalServiceServer) GetArtifact(req *GetArtifactRequest, srv ArtifactRetrievalService_GetArtifactServer) error {
+func (*UnimplementedArtifactRetrievalServiceServer) GetArtifact(*GetArtifactRequest, ArtifactRetrievalService_GetArtifactServer) error {
 	return status.Errorf(codes.Unimplemented, "method GetArtifact not implemented")
 }
 
@@ -1289,10 +1953,10 @@
 }
 
 type artifactStagingServiceClient struct {
-	cc *grpc.ClientConn
+	cc grpc.ClientConnInterface
 }
 
-func NewArtifactStagingServiceClient(cc *grpc.ClientConn) ArtifactStagingServiceClient {
+func NewArtifactStagingServiceClient(cc grpc.ClientConnInterface) ArtifactStagingServiceClient {
 	return &artifactStagingServiceClient{cc}
 }
 
@@ -1336,7 +2000,7 @@
 type UnimplementedArtifactStagingServiceServer struct {
 }
 
-func (*UnimplementedArtifactStagingServiceServer) ReverseArtifactRetrievalService(srv ArtifactStagingService_ReverseArtifactRetrievalServiceServer) error {
+func (*UnimplementedArtifactStagingServiceServer) ReverseArtifactRetrievalService(ArtifactStagingService_ReverseArtifactRetrievalServiceServer) error {
 	return status.Errorf(codes.Unimplemented, "method ReverseArtifactRetrievalService not implemented")
 }
 
@@ -1401,10 +2065,10 @@
 }
 
 type legacyArtifactStagingServiceClient struct {
-	cc *grpc.ClientConn
+	cc grpc.ClientConnInterface
 }
 
-func NewLegacyArtifactStagingServiceClient(cc *grpc.ClientConn) LegacyArtifactStagingServiceClient {
+func NewLegacyArtifactStagingServiceClient(cc grpc.ClientConnInterface) LegacyArtifactStagingServiceClient {
 	return &legacyArtifactStagingServiceClient{cc}
 }
 
@@ -1468,10 +2132,10 @@
 type UnimplementedLegacyArtifactStagingServiceServer struct {
 }
 
-func (*UnimplementedLegacyArtifactStagingServiceServer) PutArtifact(srv LegacyArtifactStagingService_PutArtifactServer) error {
+func (*UnimplementedLegacyArtifactStagingServiceServer) PutArtifact(LegacyArtifactStagingService_PutArtifactServer) error {
 	return status.Errorf(codes.Unimplemented, "method PutArtifact not implemented")
 }
-func (*UnimplementedLegacyArtifactStagingServiceServer) CommitManifest(ctx context.Context, req *CommitManifestRequest) (*CommitManifestResponse, error) {
+func (*UnimplementedLegacyArtifactStagingServiceServer) CommitManifest(context.Context, *CommitManifestRequest) (*CommitManifestResponse, error) {
 	return nil, status.Errorf(codes.Unimplemented, "method CommitManifest not implemented")
 }
 
@@ -1553,10 +2217,10 @@
 }
 
 type legacyArtifactRetrievalServiceClient struct {
-	cc *grpc.ClientConn
+	cc grpc.ClientConnInterface
 }
 
-func NewLegacyArtifactRetrievalServiceClient(cc *grpc.ClientConn) LegacyArtifactRetrievalServiceClient {
+func NewLegacyArtifactRetrievalServiceClient(cc grpc.ClientConnInterface) LegacyArtifactRetrievalServiceClient {
 	return &legacyArtifactRetrievalServiceClient{cc}
 }
 
@@ -1613,10 +2277,10 @@
 type UnimplementedLegacyArtifactRetrievalServiceServer struct {
 }
 
-func (*UnimplementedLegacyArtifactRetrievalServiceServer) GetManifest(ctx context.Context, req *GetManifestRequest) (*GetManifestResponse, error) {
+func (*UnimplementedLegacyArtifactRetrievalServiceServer) GetManifest(context.Context, *GetManifestRequest) (*GetManifestResponse, error) {
 	return nil, status.Errorf(codes.Unimplemented, "method GetManifest not implemented")
 }
-func (*UnimplementedLegacyArtifactRetrievalServiceServer) GetArtifact(req *LegacyGetArtifactRequest, srv LegacyArtifactRetrievalService_GetArtifactServer) error {
+func (*UnimplementedLegacyArtifactRetrievalServiceServer) GetArtifact(*LegacyGetArtifactRequest, LegacyArtifactRetrievalService_GetArtifactServer) error {
 	return status.Errorf(codes.Unimplemented, "method GetArtifact not implemented")
 }
 
diff --git a/sdks/go/pkg/beam/model/jobmanagement_v1/beam_expansion_api.pb.go b/sdks/go/pkg/beam/model/jobmanagement_v1/beam_expansion_api.pb.go
index 33e34dc..e9ec1d0 100644
--- a/sdks/go/pkg/beam/model/jobmanagement_v1/beam_expansion_api.pb.go
+++ b/sdks/go/pkg/beam/model/jobmanagement_v1/beam_expansion_api.pb.go
@@ -1,31 +1,56 @@
+//
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+//
+// Protocol Buffers describing the Expansion API, an api for expanding
+// transforms in a remote SDK.
+
 // Code generated by protoc-gen-go. DO NOT EDIT.
+// versions:
+// 	protoc-gen-go v1.25.0-devel
+// 	protoc        v3.13.0
 // source: beam_expansion_api.proto
 
 package jobmanagement_v1
 
 import (
 	context "context"
-	fmt "fmt"
 	pipeline_v1 "github.com/apache/beam/sdks/go/pkg/beam/model/pipeline_v1"
-	proto "github.com/golang/protobuf/proto"
 	grpc "google.golang.org/grpc"
 	codes "google.golang.org/grpc/codes"
 	status "google.golang.org/grpc/status"
-	math "math"
+	protoreflect "google.golang.org/protobuf/reflect/protoreflect"
+	protoimpl "google.golang.org/protobuf/runtime/protoimpl"
+	reflect "reflect"
+	sync "sync"
 )
 
-// Reference imports to suppress errors if they are not otherwise used.
-var _ = proto.Marshal
-var _ = fmt.Errorf
-var _ = math.Inf
-
-// This is a compile-time assertion to ensure that this generated file
-// is compatible with the proto package it is being compiled against.
-// A compilation error at this line likely means your copy of the
-// proto package needs to be updated.
-const _ = proto.ProtoPackageIsVersion3 // please upgrade the proto package
+const (
+	// Verify that this generated code is sufficiently up-to-date.
+	_ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion)
+	// Verify that runtime/protoimpl is sufficiently up-to-date.
+	_ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20)
+)
 
 type ExpansionRequest struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
 	// Set of components needed to interpret the transform, or which
 	// may be useful for its expansion.  This includes the input
 	// PCollections (if any) to the to-be-expanded transform, along
@@ -37,59 +62,67 @@
 	Transform *pipeline_v1.PTransform `protobuf:"bytes,2,opt,name=transform,proto3" json:"transform,omitempty"`
 	// A namespace (prefix) to use for the id of any newly created
 	// components.
-	Namespace            string   `protobuf:"bytes,3,opt,name=namespace,proto3" json:"namespace,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
+	Namespace string `protobuf:"bytes,3,opt,name=namespace,proto3" json:"namespace,omitempty"`
 }
 
-func (m *ExpansionRequest) Reset()         { *m = ExpansionRequest{} }
-func (m *ExpansionRequest) String() string { return proto.CompactTextString(m) }
-func (*ExpansionRequest) ProtoMessage()    {}
+func (x *ExpansionRequest) Reset() {
+	*x = ExpansionRequest{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_expansion_api_proto_msgTypes[0]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *ExpansionRequest) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*ExpansionRequest) ProtoMessage() {}
+
+func (x *ExpansionRequest) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_expansion_api_proto_msgTypes[0]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use ExpansionRequest.ProtoReflect.Descriptor instead.
 func (*ExpansionRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_0877284f21c25569, []int{0}
+	return file_beam_expansion_api_proto_rawDescGZIP(), []int{0}
 }
 
-func (m *ExpansionRequest) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_ExpansionRequest.Unmarshal(m, b)
-}
-func (m *ExpansionRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_ExpansionRequest.Marshal(b, m, deterministic)
-}
-func (m *ExpansionRequest) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ExpansionRequest.Merge(m, src)
-}
-func (m *ExpansionRequest) XXX_Size() int {
-	return xxx_messageInfo_ExpansionRequest.Size(m)
-}
-func (m *ExpansionRequest) XXX_DiscardUnknown() {
-	xxx_messageInfo_ExpansionRequest.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_ExpansionRequest proto.InternalMessageInfo
-
-func (m *ExpansionRequest) GetComponents() *pipeline_v1.Components {
-	if m != nil {
-		return m.Components
+func (x *ExpansionRequest) GetComponents() *pipeline_v1.Components {
+	if x != nil {
+		return x.Components
 	}
 	return nil
 }
 
-func (m *ExpansionRequest) GetTransform() *pipeline_v1.PTransform {
-	if m != nil {
-		return m.Transform
+func (x *ExpansionRequest) GetTransform() *pipeline_v1.PTransform {
+	if x != nil {
+		return x.Transform
 	}
 	return nil
 }
 
-func (m *ExpansionRequest) GetNamespace() string {
-	if m != nil {
-		return m.Namespace
+func (x *ExpansionRequest) GetNamespace() string {
+	if x != nil {
+		return x.Namespace
 	}
 	return ""
 }
 
 type ExpansionResponse struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
 	// Set of components needed to execute the expanded transform,
 	// including the (original) inputs, outputs, and subtransforms.
 	Components *pipeline_v1.Components `protobuf:"bytes,1,opt,name=components,proto3" json:"components,omitempty"`
@@ -101,102 +134,216 @@
 	Requirements []string `protobuf:"bytes,3,rep,name=requirements,proto3" json:"requirements,omitempty"`
 	// (Optional) An string representation of any error encountered while
 	// attempting to expand this transform.
-	Error                string   `protobuf:"bytes,10,opt,name=error,proto3" json:"error,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
+	Error string `protobuf:"bytes,10,opt,name=error,proto3" json:"error,omitempty"`
 }
 
-func (m *ExpansionResponse) Reset()         { *m = ExpansionResponse{} }
-func (m *ExpansionResponse) String() string { return proto.CompactTextString(m) }
-func (*ExpansionResponse) ProtoMessage()    {}
+func (x *ExpansionResponse) Reset() {
+	*x = ExpansionResponse{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_expansion_api_proto_msgTypes[1]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *ExpansionResponse) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*ExpansionResponse) ProtoMessage() {}
+
+func (x *ExpansionResponse) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_expansion_api_proto_msgTypes[1]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use ExpansionResponse.ProtoReflect.Descriptor instead.
 func (*ExpansionResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_0877284f21c25569, []int{1}
+	return file_beam_expansion_api_proto_rawDescGZIP(), []int{1}
 }
 
-func (m *ExpansionResponse) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_ExpansionResponse.Unmarshal(m, b)
-}
-func (m *ExpansionResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_ExpansionResponse.Marshal(b, m, deterministic)
-}
-func (m *ExpansionResponse) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ExpansionResponse.Merge(m, src)
-}
-func (m *ExpansionResponse) XXX_Size() int {
-	return xxx_messageInfo_ExpansionResponse.Size(m)
-}
-func (m *ExpansionResponse) XXX_DiscardUnknown() {
-	xxx_messageInfo_ExpansionResponse.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_ExpansionResponse proto.InternalMessageInfo
-
-func (m *ExpansionResponse) GetComponents() *pipeline_v1.Components {
-	if m != nil {
-		return m.Components
+func (x *ExpansionResponse) GetComponents() *pipeline_v1.Components {
+	if x != nil {
+		return x.Components
 	}
 	return nil
 }
 
-func (m *ExpansionResponse) GetTransform() *pipeline_v1.PTransform {
-	if m != nil {
-		return m.Transform
+func (x *ExpansionResponse) GetTransform() *pipeline_v1.PTransform {
+	if x != nil {
+		return x.Transform
 	}
 	return nil
 }
 
-func (m *ExpansionResponse) GetRequirements() []string {
-	if m != nil {
-		return m.Requirements
+func (x *ExpansionResponse) GetRequirements() []string {
+	if x != nil {
+		return x.Requirements
 	}
 	return nil
 }
 
-func (m *ExpansionResponse) GetError() string {
-	if m != nil {
-		return m.Error
+func (x *ExpansionResponse) GetError() string {
+	if x != nil {
+		return x.Error
 	}
 	return ""
 }
 
-func init() {
-	proto.RegisterType((*ExpansionRequest)(nil), "org.apache.beam.model.expansion.v1.ExpansionRequest")
-	proto.RegisterType((*ExpansionResponse)(nil), "org.apache.beam.model.expansion.v1.ExpansionResponse")
+var File_beam_expansion_api_proto protoreflect.FileDescriptor
+
+var file_beam_expansion_api_proto_rawDesc = []byte{
+	0x0a, 0x18, 0x62, 0x65, 0x61, 0x6d, 0x5f, 0x65, 0x78, 0x70, 0x61, 0x6e, 0x73, 0x69, 0x6f, 0x6e,
+	0x5f, 0x61, 0x70, 0x69, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x22, 0x6f, 0x72, 0x67, 0x2e,
+	0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65,
+	0x6c, 0x2e, 0x65, 0x78, 0x70, 0x61, 0x6e, 0x73, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x1a, 0x15,
+	0x62, 0x65, 0x61, 0x6d, 0x5f, 0x72, 0x75, 0x6e, 0x6e, 0x65, 0x72, 0x5f, 0x61, 0x70, 0x69, 0x2e,
+	0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0xcc, 0x01, 0x0a, 0x10, 0x45, 0x78, 0x70, 0x61, 0x6e, 0x73,
+	0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x4d, 0x0a, 0x0a, 0x63, 0x6f,
+	0x6d, 0x70, 0x6f, 0x6e, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2d,
+	0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d,
+	0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e,
+	0x76, 0x31, 0x2e, 0x43, 0x6f, 0x6d, 0x70, 0x6f, 0x6e, 0x65, 0x6e, 0x74, 0x73, 0x52, 0x0a, 0x63,
+	0x6f, 0x6d, 0x70, 0x6f, 0x6e, 0x65, 0x6e, 0x74, 0x73, 0x12, 0x4b, 0x0a, 0x09, 0x74, 0x72, 0x61,
+	0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2d, 0x2e, 0x6f,
+	0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d,
+	0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31,
+	0x2e, 0x50, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x52, 0x09, 0x74, 0x72, 0x61,
+	0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x12, 0x1c, 0x0a, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73, 0x70,
+	0x61, 0x63, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6e, 0x61, 0x6d, 0x65, 0x73,
+	0x70, 0x61, 0x63, 0x65, 0x22, 0xe9, 0x01, 0x0a, 0x11, 0x45, 0x78, 0x70, 0x61, 0x6e, 0x73, 0x69,
+	0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x4d, 0x0a, 0x0a, 0x63, 0x6f,
+	0x6d, 0x70, 0x6f, 0x6e, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2d,
+	0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d,
+	0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e,
+	0x76, 0x31, 0x2e, 0x43, 0x6f, 0x6d, 0x70, 0x6f, 0x6e, 0x65, 0x6e, 0x74, 0x73, 0x52, 0x0a, 0x63,
+	0x6f, 0x6d, 0x70, 0x6f, 0x6e, 0x65, 0x6e, 0x74, 0x73, 0x12, 0x4b, 0x0a, 0x09, 0x74, 0x72, 0x61,
+	0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2d, 0x2e, 0x6f,
+	0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d,
+	0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31,
+	0x2e, 0x50, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x52, 0x09, 0x74, 0x72, 0x61,
+	0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x12, 0x22, 0x0a, 0x0c, 0x72, 0x65, 0x71, 0x75, 0x69, 0x72,
+	0x65, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0c, 0x72, 0x65,
+	0x71, 0x75, 0x69, 0x72, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x12, 0x14, 0x0a, 0x05, 0x65, 0x72,
+	0x72, 0x6f, 0x72, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x65, 0x72, 0x72, 0x6f, 0x72,
+	0x32, 0x89, 0x01, 0x0a, 0x10, 0x45, 0x78, 0x70, 0x61, 0x6e, 0x73, 0x69, 0x6f, 0x6e, 0x53, 0x65,
+	0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x75, 0x0a, 0x06, 0x45, 0x78, 0x70, 0x61, 0x6e, 0x64, 0x12,
+	0x34, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61,
+	0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x65, 0x78, 0x70, 0x61, 0x6e, 0x73, 0x69, 0x6f,
+	0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x45, 0x78, 0x70, 0x61, 0x6e, 0x73, 0x69, 0x6f, 0x6e, 0x52, 0x65,
+	0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x35, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63,
+	0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x65, 0x78,
+	0x70, 0x61, 0x6e, 0x73, 0x69, 0x6f, 0x6e, 0x2e, 0x76, 0x31, 0x2e, 0x45, 0x78, 0x70, 0x61, 0x6e,
+	0x73, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x42, 0x83, 0x01, 0x0a,
+	0x22, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d,
+	0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x65, 0x78, 0x70, 0x61, 0x6e, 0x73, 0x69, 0x6f, 0x6e,
+	0x2e, 0x76, 0x31, 0x42, 0x0c, 0x45, 0x78, 0x70, 0x61, 0x6e, 0x73, 0x69, 0x6f, 0x6e, 0x41, 0x70,
+	0x69, 0x5a, 0x4f, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x61, 0x70,
+	0x61, 0x63, 0x68, 0x65, 0x2f, 0x62, 0x65, 0x61, 0x6d, 0x2f, 0x73, 0x64, 0x6b, 0x73, 0x2f, 0x67,
+	0x6f, 0x2f, 0x70, 0x6b, 0x67, 0x2f, 0x62, 0x65, 0x61, 0x6d, 0x2f, 0x6d, 0x6f, 0x64, 0x65, 0x6c,
+	0x2f, 0x6a, 0x6f, 0x62, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x76,
+	0x31, 0x3b, 0x6a, 0x6f, 0x62, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x5f,
+	0x76, 0x31, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
 }
 
-func init() { proto.RegisterFile("beam_expansion_api.proto", fileDescriptor_0877284f21c25569) }
+var (
+	file_beam_expansion_api_proto_rawDescOnce sync.Once
+	file_beam_expansion_api_proto_rawDescData = file_beam_expansion_api_proto_rawDesc
+)
 
-var fileDescriptor_0877284f21c25569 = []byte{
-	// 302 bytes of a gzipped FileDescriptorProto
-	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xcc, 0x92, 0xcd, 0x4a, 0xc3, 0x40,
-	0x14, 0x85, 0x89, 0xc5, 0x42, 0xae, 0x5d, 0xd4, 0x41, 0x21, 0x14, 0x17, 0x25, 0xab, 0x6e, 0x1c,
-	0x48, 0xd5, 0x07, 0xb0, 0xea, 0x4a, 0x04, 0x89, 0xae, 0xdc, 0x84, 0x49, 0x7a, 0xad, 0x23, 0x9d,
-	0x9f, 0xde, 0x49, 0x82, 0xaf, 0xe0, 0xbb, 0xf9, 0x22, 0xbe, 0x85, 0x38, 0xa5, 0x49, 0x14, 0xc1,
-	0xba, 0x73, 0x39, 0x87, 0xf3, 0xdd, 0x99, 0x33, 0xf7, 0x40, 0x94, 0xa3, 0x50, 0x19, 0xbe, 0x58,
-	0xa1, 0x9d, 0x34, 0x3a, 0x13, 0x56, 0x72, 0x4b, 0xa6, 0x34, 0x2c, 0x36, 0xb4, 0xe0, 0xc2, 0x8a,
-	0xe2, 0x09, 0xf9, 0xa7, 0x89, 0x2b, 0x33, 0xc7, 0x25, 0x6f, 0xac, 0xbc, 0x4e, 0x46, 0x87, 0x9e,
-	0xa6, 0x4a, 0x6b, 0xa4, 0x16, 0x8d, 0xdf, 0x02, 0x18, 0x5e, 0x6d, 0x7c, 0x29, 0xae, 0x2a, 0x74,
-	0x25, 0xbb, 0x01, 0x28, 0x8c, 0xb2, 0x46, 0xa3, 0x2e, 0x5d, 0x14, 0x8c, 0x83, 0xc9, 0xde, 0xf4,
-	0x98, 0xff, 0x7c, 0x89, 0x95, 0x16, 0x97, 0x52, 0x23, 0xaf, 0x13, 0x7e, 0xd1, 0x40, 0x69, 0x67,
-	0x00, 0xbb, 0x86, 0xb0, 0x24, 0xa1, 0xdd, 0xa3, 0x21, 0x15, 0xed, 0x6c, 0x3d, 0xed, 0xf6, 0x7e,
-	0x03, 0xa5, 0x2d, 0xcf, 0x8e, 0x20, 0xd4, 0x42, 0xa1, 0xb3, 0xa2, 0xc0, 0xa8, 0x37, 0x0e, 0x26,
-	0x61, 0xda, 0x0a, 0xf1, 0x7b, 0x00, 0xfb, 0x9d, 0x38, 0xce, 0x1a, 0xed, 0xf0, 0x5f, 0xe7, 0x89,
-	0x61, 0x40, 0xb8, 0xaa, 0x24, 0xa1, 0xf2, 0xaf, 0xeb, 0x8d, 0x7b, 0x93, 0x30, 0xfd, 0xa2, 0xb1,
-	0x03, 0xd8, 0x45, 0x22, 0x43, 0x11, 0xf8, 0xbc, 0xeb, 0xc3, 0xf4, 0xb5, 0xbb, 0xba, 0x3b, 0xa4,
-	0x5a, 0x16, 0xc8, 0x2a, 0xe8, 0x7b, 0x6d, 0xce, 0x4e, 0xf9, 0xef, 0xad, 0xe0, 0xdf, 0x57, 0x3f,
-	0x3a, 0xfb, 0x23, 0xb5, 0xfe, 0xe1, 0xd9, 0x25, 0x6c, 0xd1, 0xc1, 0xd9, 0xa0, 0x01, 0xcf, 0xad,
-	0x7c, 0x18, 0x3e, 0x9b, 0x5c, 0x09, 0x2d, 0x16, 0x3e, 0x65, 0x56, 0x27, 0x79, 0xdf, 0x77, 0xf2,
-	0xe4, 0x23, 0x00, 0x00, 0xff, 0xff, 0xa9, 0x06, 0xca, 0x44, 0xea, 0x02, 0x00, 0x00,
+func file_beam_expansion_api_proto_rawDescGZIP() []byte {
+	file_beam_expansion_api_proto_rawDescOnce.Do(func() {
+		file_beam_expansion_api_proto_rawDescData = protoimpl.X.CompressGZIP(file_beam_expansion_api_proto_rawDescData)
+	})
+	return file_beam_expansion_api_proto_rawDescData
+}
+
+var file_beam_expansion_api_proto_msgTypes = make([]protoimpl.MessageInfo, 2)
+var file_beam_expansion_api_proto_goTypes = []interface{}{
+	(*ExpansionRequest)(nil),       // 0: org.apache.beam.model.expansion.v1.ExpansionRequest
+	(*ExpansionResponse)(nil),      // 1: org.apache.beam.model.expansion.v1.ExpansionResponse
+	(*pipeline_v1.Components)(nil), // 2: org.apache.beam.model.pipeline.v1.Components
+	(*pipeline_v1.PTransform)(nil), // 3: org.apache.beam.model.pipeline.v1.PTransform
+}
+var file_beam_expansion_api_proto_depIdxs = []int32{
+	2, // 0: org.apache.beam.model.expansion.v1.ExpansionRequest.components:type_name -> org.apache.beam.model.pipeline.v1.Components
+	3, // 1: org.apache.beam.model.expansion.v1.ExpansionRequest.transform:type_name -> org.apache.beam.model.pipeline.v1.PTransform
+	2, // 2: org.apache.beam.model.expansion.v1.ExpansionResponse.components:type_name -> org.apache.beam.model.pipeline.v1.Components
+	3, // 3: org.apache.beam.model.expansion.v1.ExpansionResponse.transform:type_name -> org.apache.beam.model.pipeline.v1.PTransform
+	0, // 4: org.apache.beam.model.expansion.v1.ExpansionService.Expand:input_type -> org.apache.beam.model.expansion.v1.ExpansionRequest
+	1, // 5: org.apache.beam.model.expansion.v1.ExpansionService.Expand:output_type -> org.apache.beam.model.expansion.v1.ExpansionResponse
+	5, // [5:6] is the sub-list for method output_type
+	4, // [4:5] is the sub-list for method input_type
+	4, // [4:4] is the sub-list for extension type_name
+	4, // [4:4] is the sub-list for extension extendee
+	0, // [0:4] is the sub-list for field type_name
+}
+
+func init() { file_beam_expansion_api_proto_init() }
+func file_beam_expansion_api_proto_init() {
+	if File_beam_expansion_api_proto != nil {
+		return
+	}
+	if !protoimpl.UnsafeEnabled {
+		file_beam_expansion_api_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*ExpansionRequest); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_expansion_api_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*ExpansionResponse); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+	}
+	type x struct{}
+	out := protoimpl.TypeBuilder{
+		File: protoimpl.DescBuilder{
+			GoPackagePath: reflect.TypeOf(x{}).PkgPath(),
+			RawDescriptor: file_beam_expansion_api_proto_rawDesc,
+			NumEnums:      0,
+			NumMessages:   2,
+			NumExtensions: 0,
+			NumServices:   1,
+		},
+		GoTypes:           file_beam_expansion_api_proto_goTypes,
+		DependencyIndexes: file_beam_expansion_api_proto_depIdxs,
+		MessageInfos:      file_beam_expansion_api_proto_msgTypes,
+	}.Build()
+	File_beam_expansion_api_proto = out.File
+	file_beam_expansion_api_proto_rawDesc = nil
+	file_beam_expansion_api_proto_goTypes = nil
+	file_beam_expansion_api_proto_depIdxs = nil
 }
 
 // Reference imports to suppress errors if they are not otherwise used.
 var _ context.Context
-var _ grpc.ClientConn
+var _ grpc.ClientConnInterface
 
 // 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
+const _ = grpc.SupportPackageIsVersion6
 
 // ExpansionServiceClient is the client API for ExpansionService service.
 //
@@ -206,10 +353,10 @@
 }
 
 type expansionServiceClient struct {
-	cc *grpc.ClientConn
+	cc grpc.ClientConnInterface
 }
 
-func NewExpansionServiceClient(cc *grpc.ClientConn) ExpansionServiceClient {
+func NewExpansionServiceClient(cc grpc.ClientConnInterface) ExpansionServiceClient {
 	return &expansionServiceClient{cc}
 }
 
@@ -231,7 +378,7 @@
 type UnimplementedExpansionServiceServer struct {
 }
 
-func (*UnimplementedExpansionServiceServer) Expand(ctx context.Context, req *ExpansionRequest) (*ExpansionResponse, error) {
+func (*UnimplementedExpansionServiceServer) Expand(context.Context, *ExpansionRequest) (*ExpansionResponse, error) {
 	return nil, status.Errorf(codes.Unimplemented, "method Expand not implemented")
 }
 
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 18f6aaa..a4423f1 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
@@ -1,31 +1,52 @@
+//
+// 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.
+
+//
+// Protocol Buffers describing the Job API, api for communicating with a runner
+// for job submission over GRPC.
+
 // Code generated by protoc-gen-go. DO NOT EDIT.
+// versions:
+// 	protoc-gen-go v1.25.0-devel
+// 	protoc        v3.13.0
 // source: beam_job_api.proto
 
 package jobmanagement_v1
 
 import (
 	context "context"
-	fmt "fmt"
 	pipeline_v1 "github.com/apache/beam/sdks/go/pkg/beam/model/pipeline_v1"
-	proto "github.com/golang/protobuf/proto"
 	_struct "github.com/golang/protobuf/ptypes/struct"
 	timestamp "github.com/golang/protobuf/ptypes/timestamp"
 	grpc "google.golang.org/grpc"
 	codes "google.golang.org/grpc/codes"
 	status "google.golang.org/grpc/status"
-	math "math"
+	protoreflect "google.golang.org/protobuf/reflect/protoreflect"
+	protoimpl "google.golang.org/protobuf/runtime/protoimpl"
+	reflect "reflect"
+	sync "sync"
 )
 
-// Reference imports to suppress errors if they are not otherwise used.
-var _ = proto.Marshal
-var _ = fmt.Errorf
-var _ = math.Inf
-
-// This is a compile-time assertion to ensure that this generated file
-// is compatible with the proto package it is being compiled against.
-// A compilation error at this line likely means your copy of the
-// proto package needs to be updated.
-const _ = proto.ProtoPackageIsVersion3 // please upgrade the proto package
+const (
+	// Verify that this generated code is sufficiently up-to-date.
+	_ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion)
+	// Verify that runtime/protoimpl is sufficiently up-to-date.
+	_ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20)
+)
 
 type JobMessage_MessageImportance int32
 
@@ -38,30 +59,51 @@
 	JobMessage_JOB_MESSAGE_ERROR              JobMessage_MessageImportance = 5
 )
 
-var JobMessage_MessageImportance_name = map[int32]string{
-	0: "MESSAGE_IMPORTANCE_UNSPECIFIED",
-	1: "JOB_MESSAGE_DEBUG",
-	2: "JOB_MESSAGE_DETAILED",
-	3: "JOB_MESSAGE_BASIC",
-	4: "JOB_MESSAGE_WARNING",
-	5: "JOB_MESSAGE_ERROR",
-}
+// Enum value maps for JobMessage_MessageImportance.
+var (
+	JobMessage_MessageImportance_name = map[int32]string{
+		0: "MESSAGE_IMPORTANCE_UNSPECIFIED",
+		1: "JOB_MESSAGE_DEBUG",
+		2: "JOB_MESSAGE_DETAILED",
+		3: "JOB_MESSAGE_BASIC",
+		4: "JOB_MESSAGE_WARNING",
+		5: "JOB_MESSAGE_ERROR",
+	}
+	JobMessage_MessageImportance_value = map[string]int32{
+		"MESSAGE_IMPORTANCE_UNSPECIFIED": 0,
+		"JOB_MESSAGE_DEBUG":              1,
+		"JOB_MESSAGE_DETAILED":           2,
+		"JOB_MESSAGE_BASIC":              3,
+		"JOB_MESSAGE_WARNING":            4,
+		"JOB_MESSAGE_ERROR":              5,
+	}
+)
 
-var JobMessage_MessageImportance_value = map[string]int32{
-	"MESSAGE_IMPORTANCE_UNSPECIFIED": 0,
-	"JOB_MESSAGE_DEBUG":              1,
-	"JOB_MESSAGE_DETAILED":           2,
-	"JOB_MESSAGE_BASIC":              3,
-	"JOB_MESSAGE_WARNING":            4,
-	"JOB_MESSAGE_ERROR":              5,
+func (x JobMessage_MessageImportance) Enum() *JobMessage_MessageImportance {
+	p := new(JobMessage_MessageImportance)
+	*p = x
+	return p
 }
 
 func (x JobMessage_MessageImportance) String() string {
-	return proto.EnumName(JobMessage_MessageImportance_name, int32(x))
+	return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x))
 }
 
+func (JobMessage_MessageImportance) Descriptor() protoreflect.EnumDescriptor {
+	return file_beam_job_api_proto_enumTypes[0].Descriptor()
+}
+
+func (JobMessage_MessageImportance) Type() protoreflect.EnumType {
+	return &file_beam_job_api_proto_enumTypes[0]
+}
+
+func (x JobMessage_MessageImportance) Number() protoreflect.EnumNumber {
+	return protoreflect.EnumNumber(x)
+}
+
+// Deprecated: Use JobMessage_MessageImportance.Descriptor instead.
 func (JobMessage_MessageImportance) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_97c7b84f742157ae, []int{14, 0}
+	return file_beam_job_api_proto_rawDescGZIP(), []int{14, 0}
 }
 
 type JobState_Enum int32
@@ -93,42 +135,63 @@
 	JobState_UPDATING JobState_Enum = 11
 )
 
-var JobState_Enum_name = map[int32]string{
-	0:  "UNSPECIFIED",
-	1:  "STOPPED",
-	2:  "RUNNING",
-	3:  "DONE",
-	4:  "FAILED",
-	5:  "CANCELLED",
-	6:  "UPDATED",
-	7:  "DRAINING",
-	8:  "DRAINED",
-	9:  "STARTING",
-	10: "CANCELLING",
-	11: "UPDATING",
-}
+// Enum value maps for JobState_Enum.
+var (
+	JobState_Enum_name = map[int32]string{
+		0:  "UNSPECIFIED",
+		1:  "STOPPED",
+		2:  "RUNNING",
+		3:  "DONE",
+		4:  "FAILED",
+		5:  "CANCELLED",
+		6:  "UPDATED",
+		7:  "DRAINING",
+		8:  "DRAINED",
+		9:  "STARTING",
+		10: "CANCELLING",
+		11: "UPDATING",
+	}
+	JobState_Enum_value = map[string]int32{
+		"UNSPECIFIED": 0,
+		"STOPPED":     1,
+		"RUNNING":     2,
+		"DONE":        3,
+		"FAILED":      4,
+		"CANCELLED":   5,
+		"UPDATED":     6,
+		"DRAINING":    7,
+		"DRAINED":     8,
+		"STARTING":    9,
+		"CANCELLING":  10,
+		"UPDATING":    11,
+	}
+)
 
-var JobState_Enum_value = map[string]int32{
-	"UNSPECIFIED": 0,
-	"STOPPED":     1,
-	"RUNNING":     2,
-	"DONE":        3,
-	"FAILED":      4,
-	"CANCELLED":   5,
-	"UPDATED":     6,
-	"DRAINING":    7,
-	"DRAINED":     8,
-	"STARTING":    9,
-	"CANCELLING":  10,
-	"UPDATING":    11,
+func (x JobState_Enum) Enum() *JobState_Enum {
+	p := new(JobState_Enum)
+	*p = x
+	return p
 }
 
 func (x JobState_Enum) String() string {
-	return proto.EnumName(JobState_Enum_name, int32(x))
+	return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x))
 }
 
+func (JobState_Enum) Descriptor() protoreflect.EnumDescriptor {
+	return file_beam_job_api_proto_enumTypes[1].Descriptor()
+}
+
+func (JobState_Enum) Type() protoreflect.EnumType {
+	return &file_beam_job_api_proto_enumTypes[1]
+}
+
+func (x JobState_Enum) Number() protoreflect.EnumNumber {
+	return protoreflect.EnumNumber(x)
+}
+
+// Deprecated: Use JobState_Enum.Descriptor instead.
 func (JobState_Enum) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_97c7b84f742157ae, []int{16, 0}
+	return file_beam_job_api_proto_rawDescGZIP(), []int{16, 0}
 }
 
 type PipelineOptionType_Enum int32
@@ -143,30 +206,51 @@
 	PipelineOptionType_OBJECT  PipelineOptionType_Enum = 5
 )
 
-var PipelineOptionType_Enum_name = map[int32]string{
-	0: "STRING",
-	1: "BOOLEAN",
-	2: "INTEGER",
-	3: "NUMBER",
-	4: "ARRAY",
-	5: "OBJECT",
-}
+// Enum value maps for PipelineOptionType_Enum.
+var (
+	PipelineOptionType_Enum_name = map[int32]string{
+		0: "STRING",
+		1: "BOOLEAN",
+		2: "INTEGER",
+		3: "NUMBER",
+		4: "ARRAY",
+		5: "OBJECT",
+	}
+	PipelineOptionType_Enum_value = map[string]int32{
+		"STRING":  0,
+		"BOOLEAN": 1,
+		"INTEGER": 2,
+		"NUMBER":  3,
+		"ARRAY":   4,
+		"OBJECT":  5,
+	}
+)
 
-var PipelineOptionType_Enum_value = map[string]int32{
-	"STRING":  0,
-	"BOOLEAN": 1,
-	"INTEGER": 2,
-	"NUMBER":  3,
-	"ARRAY":   4,
-	"OBJECT":  5,
+func (x PipelineOptionType_Enum) Enum() *PipelineOptionType_Enum {
+	p := new(PipelineOptionType_Enum)
+	*p = x
+	return p
 }
 
 func (x PipelineOptionType_Enum) String() string {
-	return proto.EnumName(PipelineOptionType_Enum_name, int32(x))
+	return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x))
 }
 
+func (PipelineOptionType_Enum) Descriptor() protoreflect.EnumDescriptor {
+	return file_beam_job_api_proto_enumTypes[2].Descriptor()
+}
+
+func (PipelineOptionType_Enum) Type() protoreflect.EnumType {
+	return &file_beam_job_api_proto_enumTypes[2]
+}
+
+func (x PipelineOptionType_Enum) Number() protoreflect.EnumNumber {
+	return protoreflect.EnumNumber(x)
+}
+
+// Deprecated: Use PipelineOptionType_Enum.Descriptor instead.
 func (PipelineOptionType_Enum) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_97c7b84f742157ae, []int{21, 0}
+	return file_beam_job_api_proto_rawDescGZIP(), []int{21, 0}
 }
 
 // Prepare is a synchronous request that returns a preparationId back
@@ -174,61 +258,73 @@
 // Throws error ALREADY_EXISTS if the jobName is reused. Runners are permitted to deduplicate based on the name of the job.
 // Throws error UNKNOWN for all other issues
 type PrepareJobRequest struct {
-	Pipeline             *pipeline_v1.Pipeline `protobuf:"bytes,1,opt,name=pipeline,proto3" json:"pipeline,omitempty"`
-	PipelineOptions      *_struct.Struct       `protobuf:"bytes,2,opt,name=pipeline_options,json=pipelineOptions,proto3" json:"pipeline_options,omitempty"`
-	JobName              string                `protobuf:"bytes,3,opt,name=job_name,json=jobName,proto3" json:"job_name,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}              `json:"-"`
-	XXX_unrecognized     []byte                `json:"-"`
-	XXX_sizecache        int32                 `json:"-"`
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	Pipeline        *pipeline_v1.Pipeline `protobuf:"bytes,1,opt,name=pipeline,proto3" json:"pipeline,omitempty"`                                      // (required)
+	PipelineOptions *_struct.Struct       `protobuf:"bytes,2,opt,name=pipeline_options,json=pipelineOptions,proto3" json:"pipeline_options,omitempty"` // (required)
+	JobName         string                `protobuf:"bytes,3,opt,name=job_name,json=jobName,proto3" json:"job_name,omitempty"`                         // (required)
 }
 
-func (m *PrepareJobRequest) Reset()         { *m = PrepareJobRequest{} }
-func (m *PrepareJobRequest) String() string { return proto.CompactTextString(m) }
-func (*PrepareJobRequest) ProtoMessage()    {}
+func (x *PrepareJobRequest) Reset() {
+	*x = PrepareJobRequest{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_job_api_proto_msgTypes[0]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *PrepareJobRequest) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*PrepareJobRequest) ProtoMessage() {}
+
+func (x *PrepareJobRequest) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_job_api_proto_msgTypes[0]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use PrepareJobRequest.ProtoReflect.Descriptor instead.
 func (*PrepareJobRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_97c7b84f742157ae, []int{0}
+	return file_beam_job_api_proto_rawDescGZIP(), []int{0}
 }
 
-func (m *PrepareJobRequest) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_PrepareJobRequest.Unmarshal(m, b)
-}
-func (m *PrepareJobRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_PrepareJobRequest.Marshal(b, m, deterministic)
-}
-func (m *PrepareJobRequest) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_PrepareJobRequest.Merge(m, src)
-}
-func (m *PrepareJobRequest) XXX_Size() int {
-	return xxx_messageInfo_PrepareJobRequest.Size(m)
-}
-func (m *PrepareJobRequest) XXX_DiscardUnknown() {
-	xxx_messageInfo_PrepareJobRequest.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_PrepareJobRequest proto.InternalMessageInfo
-
-func (m *PrepareJobRequest) GetPipeline() *pipeline_v1.Pipeline {
-	if m != nil {
-		return m.Pipeline
+func (x *PrepareJobRequest) GetPipeline() *pipeline_v1.Pipeline {
+	if x != nil {
+		return x.Pipeline
 	}
 	return nil
 }
 
-func (m *PrepareJobRequest) GetPipelineOptions() *_struct.Struct {
-	if m != nil {
-		return m.PipelineOptions
+func (x *PrepareJobRequest) GetPipelineOptions() *_struct.Struct {
+	if x != nil {
+		return x.PipelineOptions
 	}
 	return nil
 }
 
-func (m *PrepareJobRequest) GetJobName() string {
-	if m != nil {
-		return m.JobName
+func (x *PrepareJobRequest) GetJobName() string {
+	if x != nil {
+		return x.JobName
 	}
 	return ""
 }
 
 type PrepareJobResponse struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
 	// (required) The ID used to associate calls made while preparing the job. preparationId is used
 	// to run the job.
 	PreparationId string `protobuf:"bytes,1,opt,name=preparation_id,json=preparationId,proto3" json:"preparation_id,omitempty"`
@@ -237,54 +333,58 @@
 	ArtifactStagingEndpoint *pipeline_v1.ApiServiceDescriptor `protobuf:"bytes,2,opt,name=artifact_staging_endpoint,json=artifactStagingEndpoint,proto3" json:"artifact_staging_endpoint,omitempty"`
 	// (required) Token for the artifact staging. This token also represent an artifact
 	// staging session with the artifact staging service.
-	StagingSessionToken  string   `protobuf:"bytes,3,opt,name=staging_session_token,json=stagingSessionToken,proto3" json:"staging_session_token,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
+	StagingSessionToken string `protobuf:"bytes,3,opt,name=staging_session_token,json=stagingSessionToken,proto3" json:"staging_session_token,omitempty"`
 }
 
-func (m *PrepareJobResponse) Reset()         { *m = PrepareJobResponse{} }
-func (m *PrepareJobResponse) String() string { return proto.CompactTextString(m) }
-func (*PrepareJobResponse) ProtoMessage()    {}
+func (x *PrepareJobResponse) Reset() {
+	*x = PrepareJobResponse{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_job_api_proto_msgTypes[1]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *PrepareJobResponse) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*PrepareJobResponse) ProtoMessage() {}
+
+func (x *PrepareJobResponse) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_job_api_proto_msgTypes[1]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use PrepareJobResponse.ProtoReflect.Descriptor instead.
 func (*PrepareJobResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_97c7b84f742157ae, []int{1}
+	return file_beam_job_api_proto_rawDescGZIP(), []int{1}
 }
 
-func (m *PrepareJobResponse) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_PrepareJobResponse.Unmarshal(m, b)
-}
-func (m *PrepareJobResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_PrepareJobResponse.Marshal(b, m, deterministic)
-}
-func (m *PrepareJobResponse) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_PrepareJobResponse.Merge(m, src)
-}
-func (m *PrepareJobResponse) XXX_Size() int {
-	return xxx_messageInfo_PrepareJobResponse.Size(m)
-}
-func (m *PrepareJobResponse) XXX_DiscardUnknown() {
-	xxx_messageInfo_PrepareJobResponse.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_PrepareJobResponse proto.InternalMessageInfo
-
-func (m *PrepareJobResponse) GetPreparationId() string {
-	if m != nil {
-		return m.PreparationId
+func (x *PrepareJobResponse) GetPreparationId() string {
+	if x != nil {
+		return x.PreparationId
 	}
 	return ""
 }
 
-func (m *PrepareJobResponse) GetArtifactStagingEndpoint() *pipeline_v1.ApiServiceDescriptor {
-	if m != nil {
-		return m.ArtifactStagingEndpoint
+func (x *PrepareJobResponse) GetArtifactStagingEndpoint() *pipeline_v1.ApiServiceDescriptor {
+	if x != nil {
+		return x.ArtifactStagingEndpoint
 	}
 	return nil
 }
 
-func (m *PrepareJobResponse) GetStagingSessionToken() string {
-	if m != nil {
-		return m.StagingSessionToken
+func (x *PrepareJobResponse) GetStagingSessionToken() string {
+	if x != nil {
+		return x.StagingSessionToken
 	}
 	return ""
 }
@@ -294,91 +394,107 @@
 // Throws error NOT_FOUND if the preparation ID does not exist
 // Throws error UNKNOWN for all other issues
 type RunJobRequest struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
 	// (required) The ID provided by an earlier call to prepare. Runs the job. All prerequisite tasks
 	// must have been completed.
 	PreparationId string `protobuf:"bytes,1,opt,name=preparation_id,json=preparationId,proto3" json:"preparation_id,omitempty"`
 	// (optional) If any artifacts have been staged for this job, contains the retrieval_token returned
 	// from the CommitManifestResponse.
-	RetrievalToken       string   `protobuf:"bytes,2,opt,name=retrieval_token,json=retrievalToken,proto3" json:"retrieval_token,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
+	RetrievalToken string `protobuf:"bytes,2,opt,name=retrieval_token,json=retrievalToken,proto3" json:"retrieval_token,omitempty"`
 }
 
-func (m *RunJobRequest) Reset()         { *m = RunJobRequest{} }
-func (m *RunJobRequest) String() string { return proto.CompactTextString(m) }
-func (*RunJobRequest) ProtoMessage()    {}
+func (x *RunJobRequest) Reset() {
+	*x = RunJobRequest{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_job_api_proto_msgTypes[2]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *RunJobRequest) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*RunJobRequest) ProtoMessage() {}
+
+func (x *RunJobRequest) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_job_api_proto_msgTypes[2]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use RunJobRequest.ProtoReflect.Descriptor instead.
 func (*RunJobRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_97c7b84f742157ae, []int{2}
+	return file_beam_job_api_proto_rawDescGZIP(), []int{2}
 }
 
-func (m *RunJobRequest) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_RunJobRequest.Unmarshal(m, b)
-}
-func (m *RunJobRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_RunJobRequest.Marshal(b, m, deterministic)
-}
-func (m *RunJobRequest) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_RunJobRequest.Merge(m, src)
-}
-func (m *RunJobRequest) XXX_Size() int {
-	return xxx_messageInfo_RunJobRequest.Size(m)
-}
-func (m *RunJobRequest) XXX_DiscardUnknown() {
-	xxx_messageInfo_RunJobRequest.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_RunJobRequest proto.InternalMessageInfo
-
-func (m *RunJobRequest) GetPreparationId() string {
-	if m != nil {
-		return m.PreparationId
+func (x *RunJobRequest) GetPreparationId() string {
+	if x != nil {
+		return x.PreparationId
 	}
 	return ""
 }
 
-func (m *RunJobRequest) GetRetrievalToken() string {
-	if m != nil {
-		return m.RetrievalToken
+func (x *RunJobRequest) GetRetrievalToken() string {
+	if x != nil {
+		return x.RetrievalToken
 	}
 	return ""
 }
 
 type RunJobResponse struct {
-	JobId                string   `protobuf:"bytes,1,opt,name=job_id,json=jobId,proto3" json:"job_id,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	JobId string `protobuf:"bytes,1,opt,name=job_id,json=jobId,proto3" json:"job_id,omitempty"` // (required) The ID for the executing job
 }
 
-func (m *RunJobResponse) Reset()         { *m = RunJobResponse{} }
-func (m *RunJobResponse) String() string { return proto.CompactTextString(m) }
-func (*RunJobResponse) ProtoMessage()    {}
+func (x *RunJobResponse) Reset() {
+	*x = RunJobResponse{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_job_api_proto_msgTypes[3]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *RunJobResponse) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*RunJobResponse) ProtoMessage() {}
+
+func (x *RunJobResponse) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_job_api_proto_msgTypes[3]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use RunJobResponse.ProtoReflect.Descriptor instead.
 func (*RunJobResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_97c7b84f742157ae, []int{3}
+	return file_beam_job_api_proto_rawDescGZIP(), []int{3}
 }
 
-func (m *RunJobResponse) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_RunJobResponse.Unmarshal(m, b)
-}
-func (m *RunJobResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_RunJobResponse.Marshal(b, m, deterministic)
-}
-func (m *RunJobResponse) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_RunJobResponse.Merge(m, src)
-}
-func (m *RunJobResponse) XXX_Size() int {
-	return xxx_messageInfo_RunJobResponse.Size(m)
-}
-func (m *RunJobResponse) XXX_DiscardUnknown() {
-	xxx_messageInfo_RunJobResponse.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_RunJobResponse proto.InternalMessageInfo
-
-func (m *RunJobResponse) GetJobId() string {
-	if m != nil {
-		return m.JobId
+func (x *RunJobResponse) GetJobId() string {
+	if x != nil {
+		return x.JobId
 	}
 	return ""
 }
@@ -387,144 +503,168 @@
 // Throws error GRPC_STATUS_UNAVAILABLE if server is down
 // Throws error NOT_FOUND if the jobId is not found
 type CancelJobRequest struct {
-	JobId                string   `protobuf:"bytes,1,opt,name=job_id,json=jobId,proto3" json:"job_id,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	JobId string `protobuf:"bytes,1,opt,name=job_id,json=jobId,proto3" json:"job_id,omitempty"` // (required)
 }
 
-func (m *CancelJobRequest) Reset()         { *m = CancelJobRequest{} }
-func (m *CancelJobRequest) String() string { return proto.CompactTextString(m) }
-func (*CancelJobRequest) ProtoMessage()    {}
+func (x *CancelJobRequest) Reset() {
+	*x = CancelJobRequest{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_job_api_proto_msgTypes[4]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *CancelJobRequest) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*CancelJobRequest) ProtoMessage() {}
+
+func (x *CancelJobRequest) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_job_api_proto_msgTypes[4]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use CancelJobRequest.ProtoReflect.Descriptor instead.
 func (*CancelJobRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_97c7b84f742157ae, []int{4}
+	return file_beam_job_api_proto_rawDescGZIP(), []int{4}
 }
 
-func (m *CancelJobRequest) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_CancelJobRequest.Unmarshal(m, b)
-}
-func (m *CancelJobRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_CancelJobRequest.Marshal(b, m, deterministic)
-}
-func (m *CancelJobRequest) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_CancelJobRequest.Merge(m, src)
-}
-func (m *CancelJobRequest) XXX_Size() int {
-	return xxx_messageInfo_CancelJobRequest.Size(m)
-}
-func (m *CancelJobRequest) XXX_DiscardUnknown() {
-	xxx_messageInfo_CancelJobRequest.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_CancelJobRequest proto.InternalMessageInfo
-
-func (m *CancelJobRequest) GetJobId() string {
-	if m != nil {
-		return m.JobId
+func (x *CancelJobRequest) GetJobId() string {
+	if x != nil {
+		return x.JobId
 	}
 	return ""
 }
 
 // Valid responses include any terminal state or CANCELLING
 type CancelJobResponse 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         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	State JobState_Enum `protobuf:"varint,1,opt,name=state,proto3,enum=org.apache.beam.model.job_management.v1.JobState_Enum" json:"state,omitempty"` // (required)
 }
 
-func (m *CancelJobResponse) Reset()         { *m = CancelJobResponse{} }
-func (m *CancelJobResponse) String() string { return proto.CompactTextString(m) }
-func (*CancelJobResponse) ProtoMessage()    {}
+func (x *CancelJobResponse) Reset() {
+	*x = CancelJobResponse{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_job_api_proto_msgTypes[5]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *CancelJobResponse) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*CancelJobResponse) ProtoMessage() {}
+
+func (x *CancelJobResponse) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_job_api_proto_msgTypes[5]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use CancelJobResponse.ProtoReflect.Descriptor instead.
 func (*CancelJobResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_97c7b84f742157ae, []int{5}
+	return file_beam_job_api_proto_rawDescGZIP(), []int{5}
 }
 
-func (m *CancelJobResponse) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_CancelJobResponse.Unmarshal(m, b)
-}
-func (m *CancelJobResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_CancelJobResponse.Marshal(b, m, deterministic)
-}
-func (m *CancelJobResponse) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_CancelJobResponse.Merge(m, src)
-}
-func (m *CancelJobResponse) XXX_Size() int {
-	return xxx_messageInfo_CancelJobResponse.Size(m)
-}
-func (m *CancelJobResponse) XXX_DiscardUnknown() {
-	xxx_messageInfo_CancelJobResponse.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_CancelJobResponse proto.InternalMessageInfo
-
-func (m *CancelJobResponse) GetState() JobState_Enum {
-	if m != nil {
-		return m.State
+func (x *CancelJobResponse) GetState() JobState_Enum {
+	if x != nil {
+		return x.State
 	}
 	return JobState_UNSPECIFIED
 }
 
 // A subset of info provided by ProvisionApi.ProvisionInfo
 type JobInfo struct {
-	JobId                string          `protobuf:"bytes,1,opt,name=job_id,json=jobId,proto3" json:"job_id,omitempty"`
-	JobName              string          `protobuf:"bytes,2,opt,name=job_name,json=jobName,proto3" json:"job_name,omitempty"`
-	PipelineOptions      *_struct.Struct `protobuf:"bytes,3,opt,name=pipeline_options,json=pipelineOptions,proto3" json:"pipeline_options,omitempty"`
-	State                JobState_Enum   `protobuf:"varint,4,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         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	JobId           string          `protobuf:"bytes,1,opt,name=job_id,json=jobId,proto3" json:"job_id,omitempty"`                                                // (required)
+	JobName         string          `protobuf:"bytes,2,opt,name=job_name,json=jobName,proto3" json:"job_name,omitempty"`                                          // (required)
+	PipelineOptions *_struct.Struct `protobuf:"bytes,3,opt,name=pipeline_options,json=pipelineOptions,proto3" json:"pipeline_options,omitempty"`                  // (required)
+	State           JobState_Enum   `protobuf:"varint,4,opt,name=state,proto3,enum=org.apache.beam.model.job_management.v1.JobState_Enum" json:"state,omitempty"` // (required)
 }
 
-func (m *JobInfo) Reset()         { *m = JobInfo{} }
-func (m *JobInfo) String() string { return proto.CompactTextString(m) }
-func (*JobInfo) ProtoMessage()    {}
+func (x *JobInfo) Reset() {
+	*x = JobInfo{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_job_api_proto_msgTypes[6]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *JobInfo) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*JobInfo) ProtoMessage() {}
+
+func (x *JobInfo) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_job_api_proto_msgTypes[6]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use JobInfo.ProtoReflect.Descriptor instead.
 func (*JobInfo) Descriptor() ([]byte, []int) {
-	return fileDescriptor_97c7b84f742157ae, []int{6}
+	return file_beam_job_api_proto_rawDescGZIP(), []int{6}
 }
 
-func (m *JobInfo) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_JobInfo.Unmarshal(m, b)
-}
-func (m *JobInfo) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_JobInfo.Marshal(b, m, deterministic)
-}
-func (m *JobInfo) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_JobInfo.Merge(m, src)
-}
-func (m *JobInfo) XXX_Size() int {
-	return xxx_messageInfo_JobInfo.Size(m)
-}
-func (m *JobInfo) XXX_DiscardUnknown() {
-	xxx_messageInfo_JobInfo.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_JobInfo proto.InternalMessageInfo
-
-func (m *JobInfo) GetJobId() string {
-	if m != nil {
-		return m.JobId
+func (x *JobInfo) GetJobId() string {
+	if x != nil {
+		return x.JobId
 	}
 	return ""
 }
 
-func (m *JobInfo) GetJobName() string {
-	if m != nil {
-		return m.JobName
+func (x *JobInfo) GetJobName() string {
+	if x != nil {
+		return x.JobName
 	}
 	return ""
 }
 
-func (m *JobInfo) GetPipelineOptions() *_struct.Struct {
-	if m != nil {
-		return m.PipelineOptions
+func (x *JobInfo) GetPipelineOptions() *_struct.Struct {
+	if x != nil {
+		return x.PipelineOptions
 	}
 	return nil
 }
 
-func (m *JobInfo) GetState() JobState_Enum {
-	if m != nil {
-		return m.State
+func (x *JobInfo) GetState() JobState_Enum {
+	if x != nil {
+		return x.State
 	}
 	return JobState_UNSPECIFIED
 }
@@ -532,71 +672,86 @@
 // GetJobs is a synchronus request that returns a list of invoked jobs back
 // Throws error GRPC_STATUS_UNAVAILABLE if server is down
 type GetJobsRequest struct {
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
 }
 
-func (m *GetJobsRequest) Reset()         { *m = GetJobsRequest{} }
-func (m *GetJobsRequest) String() string { return proto.CompactTextString(m) }
-func (*GetJobsRequest) ProtoMessage()    {}
+func (x *GetJobsRequest) Reset() {
+	*x = GetJobsRequest{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_job_api_proto_msgTypes[7]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *GetJobsRequest) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*GetJobsRequest) ProtoMessage() {}
+
+func (x *GetJobsRequest) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_job_api_proto_msgTypes[7]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use GetJobsRequest.ProtoReflect.Descriptor instead.
 func (*GetJobsRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_97c7b84f742157ae, []int{7}
+	return file_beam_job_api_proto_rawDescGZIP(), []int{7}
 }
 
-func (m *GetJobsRequest) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_GetJobsRequest.Unmarshal(m, b)
-}
-func (m *GetJobsRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_GetJobsRequest.Marshal(b, m, deterministic)
-}
-func (m *GetJobsRequest) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_GetJobsRequest.Merge(m, src)
-}
-func (m *GetJobsRequest) XXX_Size() int {
-	return xxx_messageInfo_GetJobsRequest.Size(m)
-}
-func (m *GetJobsRequest) XXX_DiscardUnknown() {
-	xxx_messageInfo_GetJobsRequest.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_GetJobsRequest proto.InternalMessageInfo
-
 type GetJobsResponse struct {
-	JobInfo              []*JobInfo `protobuf:"bytes,1,rep,name=job_info,json=jobInfo,proto3" json:"job_info,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}   `json:"-"`
-	XXX_unrecognized     []byte     `json:"-"`
-	XXX_sizecache        int32      `json:"-"`
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	JobInfo []*JobInfo `protobuf:"bytes,1,rep,name=job_info,json=jobInfo,proto3" json:"job_info,omitempty"` // (required)
 }
 
-func (m *GetJobsResponse) Reset()         { *m = GetJobsResponse{} }
-func (m *GetJobsResponse) String() string { return proto.CompactTextString(m) }
-func (*GetJobsResponse) ProtoMessage()    {}
+func (x *GetJobsResponse) Reset() {
+	*x = GetJobsResponse{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_job_api_proto_msgTypes[8]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *GetJobsResponse) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*GetJobsResponse) ProtoMessage() {}
+
+func (x *GetJobsResponse) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_job_api_proto_msgTypes[8]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use GetJobsResponse.ProtoReflect.Descriptor instead.
 func (*GetJobsResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_97c7b84f742157ae, []int{8}
+	return file_beam_job_api_proto_rawDescGZIP(), []int{8}
 }
 
-func (m *GetJobsResponse) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_GetJobsResponse.Unmarshal(m, b)
-}
-func (m *GetJobsResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_GetJobsResponse.Marshal(b, m, deterministic)
-}
-func (m *GetJobsResponse) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_GetJobsResponse.Merge(m, src)
-}
-func (m *GetJobsResponse) XXX_Size() int {
-	return xxx_messageInfo_GetJobsResponse.Size(m)
-}
-func (m *GetJobsResponse) XXX_DiscardUnknown() {
-	xxx_messageInfo_GetJobsResponse.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_GetJobsResponse proto.InternalMessageInfo
-
-func (m *GetJobsResponse) GetJobInfo() []*JobInfo {
-	if m != nil {
-		return m.JobInfo
+func (x *GetJobsResponse) GetJobInfo() []*JobInfo {
+	if x != nil {
+		return x.JobInfo
 	}
 	return nil
 }
@@ -605,87 +760,103 @@
 // Throws error GRPC_STATUS_UNAVAILABLE if server is down
 // Throws error NOT_FOUND if the jobId is not found
 type GetJobStateRequest struct {
-	JobId                string   `protobuf:"bytes,1,opt,name=job_id,json=jobId,proto3" json:"job_id,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	JobId string `protobuf:"bytes,1,opt,name=job_id,json=jobId,proto3" json:"job_id,omitempty"` // (required)
 }
 
-func (m *GetJobStateRequest) Reset()         { *m = GetJobStateRequest{} }
-func (m *GetJobStateRequest) String() string { return proto.CompactTextString(m) }
-func (*GetJobStateRequest) ProtoMessage()    {}
+func (x *GetJobStateRequest) Reset() {
+	*x = GetJobStateRequest{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_job_api_proto_msgTypes[9]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *GetJobStateRequest) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*GetJobStateRequest) ProtoMessage() {}
+
+func (x *GetJobStateRequest) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_job_api_proto_msgTypes[9]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use GetJobStateRequest.ProtoReflect.Descriptor instead.
 func (*GetJobStateRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_97c7b84f742157ae, []int{9}
+	return file_beam_job_api_proto_rawDescGZIP(), []int{9}
 }
 
-func (m *GetJobStateRequest) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_GetJobStateRequest.Unmarshal(m, b)
-}
-func (m *GetJobStateRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_GetJobStateRequest.Marshal(b, m, deterministic)
-}
-func (m *GetJobStateRequest) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_GetJobStateRequest.Merge(m, src)
-}
-func (m *GetJobStateRequest) XXX_Size() int {
-	return xxx_messageInfo_GetJobStateRequest.Size(m)
-}
-func (m *GetJobStateRequest) XXX_DiscardUnknown() {
-	xxx_messageInfo_GetJobStateRequest.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_GetJobStateRequest proto.InternalMessageInfo
-
-func (m *GetJobStateRequest) GetJobId() string {
-	if m != nil {
-		return m.JobId
+func (x *GetJobStateRequest) GetJobId() string {
+	if x != nil {
+		return x.JobId
 	}
 	return ""
 }
 
 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"`
-	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:"-"`
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	State     JobState_Enum        `protobuf:"varint,1,opt,name=state,proto3,enum=org.apache.beam.model.job_management.v1.JobState_Enum" json:"state,omitempty"` // (required)
+	Timestamp *timestamp.Timestamp `protobuf:"bytes,2,opt,name=timestamp,proto3" json:"timestamp,omitempty"`                                                     // (required)
 }
 
-func (m *JobStateEvent) Reset()         { *m = JobStateEvent{} }
-func (m *JobStateEvent) String() string { return proto.CompactTextString(m) }
-func (*JobStateEvent) ProtoMessage()    {}
+func (x *JobStateEvent) Reset() {
+	*x = JobStateEvent{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_job_api_proto_msgTypes[10]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *JobStateEvent) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*JobStateEvent) ProtoMessage() {}
+
+func (x *JobStateEvent) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_job_api_proto_msgTypes[10]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use JobStateEvent.ProtoReflect.Descriptor instead.
 func (*JobStateEvent) Descriptor() ([]byte, []int) {
-	return fileDescriptor_97c7b84f742157ae, []int{10}
+	return file_beam_job_api_proto_rawDescGZIP(), []int{10}
 }
 
-func (m *JobStateEvent) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_JobStateEvent.Unmarshal(m, b)
-}
-func (m *JobStateEvent) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_JobStateEvent.Marshal(b, m, deterministic)
-}
-func (m *JobStateEvent) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_JobStateEvent.Merge(m, src)
-}
-func (m *JobStateEvent) XXX_Size() int {
-	return xxx_messageInfo_JobStateEvent.Size(m)
-}
-func (m *JobStateEvent) XXX_DiscardUnknown() {
-	xxx_messageInfo_JobStateEvent.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_JobStateEvent proto.InternalMessageInfo
-
-func (m *JobStateEvent) GetState() JobState_Enum {
-	if m != nil {
-		return m.State
+func (x *JobStateEvent) GetState() JobState_Enum {
+	if x != nil {
+		return x.State
 	}
 	return JobState_UNSPECIFIED
 }
 
-func (m *JobStateEvent) GetTimestamp() *timestamp.Timestamp {
-	if m != nil {
-		return m.Timestamp
+func (x *JobStateEvent) GetTimestamp() *timestamp.Timestamp {
+	if x != nil {
+		return x.Timestamp
 	}
 	return nil
 }
@@ -694,79 +865,95 @@
 // Throws error GRPC_STATUS_UNAVAILABLE if server is down
 // Throws error NOT_FOUND if the jobId is not found
 type GetJobPipelineRequest struct {
-	JobId                string   `protobuf:"bytes,1,opt,name=job_id,json=jobId,proto3" json:"job_id,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	JobId string `protobuf:"bytes,1,opt,name=job_id,json=jobId,proto3" json:"job_id,omitempty"` // (required)
 }
 
-func (m *GetJobPipelineRequest) Reset()         { *m = GetJobPipelineRequest{} }
-func (m *GetJobPipelineRequest) String() string { return proto.CompactTextString(m) }
-func (*GetJobPipelineRequest) ProtoMessage()    {}
+func (x *GetJobPipelineRequest) Reset() {
+	*x = GetJobPipelineRequest{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_job_api_proto_msgTypes[11]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *GetJobPipelineRequest) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*GetJobPipelineRequest) ProtoMessage() {}
+
+func (x *GetJobPipelineRequest) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_job_api_proto_msgTypes[11]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use GetJobPipelineRequest.ProtoReflect.Descriptor instead.
 func (*GetJobPipelineRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_97c7b84f742157ae, []int{11}
+	return file_beam_job_api_proto_rawDescGZIP(), []int{11}
 }
 
-func (m *GetJobPipelineRequest) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_GetJobPipelineRequest.Unmarshal(m, b)
-}
-func (m *GetJobPipelineRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_GetJobPipelineRequest.Marshal(b, m, deterministic)
-}
-func (m *GetJobPipelineRequest) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_GetJobPipelineRequest.Merge(m, src)
-}
-func (m *GetJobPipelineRequest) XXX_Size() int {
-	return xxx_messageInfo_GetJobPipelineRequest.Size(m)
-}
-func (m *GetJobPipelineRequest) XXX_DiscardUnknown() {
-	xxx_messageInfo_GetJobPipelineRequest.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_GetJobPipelineRequest proto.InternalMessageInfo
-
-func (m *GetJobPipelineRequest) GetJobId() string {
-	if m != nil {
-		return m.JobId
+func (x *GetJobPipelineRequest) GetJobId() string {
+	if x != nil {
+		return x.JobId
 	}
 	return ""
 }
 
 type GetJobPipelineResponse struct {
-	Pipeline             *pipeline_v1.Pipeline `protobuf:"bytes,1,opt,name=pipeline,proto3" json:"pipeline,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}              `json:"-"`
-	XXX_unrecognized     []byte                `json:"-"`
-	XXX_sizecache        int32                 `json:"-"`
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	Pipeline *pipeline_v1.Pipeline `protobuf:"bytes,1,opt,name=pipeline,proto3" json:"pipeline,omitempty"` // (required)
 }
 
-func (m *GetJobPipelineResponse) Reset()         { *m = GetJobPipelineResponse{} }
-func (m *GetJobPipelineResponse) String() string { return proto.CompactTextString(m) }
-func (*GetJobPipelineResponse) ProtoMessage()    {}
+func (x *GetJobPipelineResponse) Reset() {
+	*x = GetJobPipelineResponse{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_job_api_proto_msgTypes[12]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *GetJobPipelineResponse) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*GetJobPipelineResponse) ProtoMessage() {}
+
+func (x *GetJobPipelineResponse) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_job_api_proto_msgTypes[12]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use GetJobPipelineResponse.ProtoReflect.Descriptor instead.
 func (*GetJobPipelineResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_97c7b84f742157ae, []int{12}
+	return file_beam_job_api_proto_rawDescGZIP(), []int{12}
 }
 
-func (m *GetJobPipelineResponse) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_GetJobPipelineResponse.Unmarshal(m, b)
-}
-func (m *GetJobPipelineResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_GetJobPipelineResponse.Marshal(b, m, deterministic)
-}
-func (m *GetJobPipelineResponse) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_GetJobPipelineResponse.Merge(m, src)
-}
-func (m *GetJobPipelineResponse) XXX_Size() int {
-	return xxx_messageInfo_GetJobPipelineResponse.Size(m)
-}
-func (m *GetJobPipelineResponse) XXX_DiscardUnknown() {
-	xxx_messageInfo_GetJobPipelineResponse.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_GetJobPipelineResponse proto.InternalMessageInfo
-
-func (m *GetJobPipelineResponse) GetPipeline() *pipeline_v1.Pipeline {
-	if m != nil {
-		return m.Pipeline
+func (x *GetJobPipelineResponse) GetPipeline() *pipeline_v1.Pipeline {
+	if x != nil {
+		return x.Pipeline
 	}
 	return nil
 }
@@ -776,141 +963,186 @@
 // and job messages back; one is used for logging and the other for detecting
 // the job ended.
 type JobMessagesRequest struct {
-	JobId                string   `protobuf:"bytes,1,opt,name=job_id,json=jobId,proto3" json:"job_id,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	JobId string `protobuf:"bytes,1,opt,name=job_id,json=jobId,proto3" json:"job_id,omitempty"` // (required)
 }
 
-func (m *JobMessagesRequest) Reset()         { *m = JobMessagesRequest{} }
-func (m *JobMessagesRequest) String() string { return proto.CompactTextString(m) }
-func (*JobMessagesRequest) ProtoMessage()    {}
+func (x *JobMessagesRequest) Reset() {
+	*x = JobMessagesRequest{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_job_api_proto_msgTypes[13]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *JobMessagesRequest) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*JobMessagesRequest) ProtoMessage() {}
+
+func (x *JobMessagesRequest) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_job_api_proto_msgTypes[13]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use JobMessagesRequest.ProtoReflect.Descriptor instead.
 func (*JobMessagesRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_97c7b84f742157ae, []int{13}
+	return file_beam_job_api_proto_rawDescGZIP(), []int{13}
 }
 
-func (m *JobMessagesRequest) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_JobMessagesRequest.Unmarshal(m, b)
-}
-func (m *JobMessagesRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_JobMessagesRequest.Marshal(b, m, deterministic)
-}
-func (m *JobMessagesRequest) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_JobMessagesRequest.Merge(m, src)
-}
-func (m *JobMessagesRequest) XXX_Size() int {
-	return xxx_messageInfo_JobMessagesRequest.Size(m)
-}
-func (m *JobMessagesRequest) XXX_DiscardUnknown() {
-	xxx_messageInfo_JobMessagesRequest.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_JobMessagesRequest proto.InternalMessageInfo
-
-func (m *JobMessagesRequest) GetJobId() string {
-	if m != nil {
-		return m.JobId
+func (x *JobMessagesRequest) GetJobId() string {
+	if x != nil {
+		return x.JobId
 	}
 	return ""
 }
 
 type JobMessage struct {
-	MessageId            string                       `protobuf:"bytes,1,opt,name=message_id,json=messageId,proto3" json:"message_id,omitempty"`
-	Time                 string                       `protobuf:"bytes,2,opt,name=time,proto3" json:"time,omitempty"`
-	Importance           JobMessage_MessageImportance `protobuf:"varint,3,opt,name=importance,proto3,enum=org.apache.beam.model.job_management.v1.JobMessage_MessageImportance" json:"importance,omitempty"`
-	MessageText          string                       `protobuf:"bytes,4,opt,name=message_text,json=messageText,proto3" json:"message_text,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}                     `json:"-"`
-	XXX_unrecognized     []byte                       `json:"-"`
-	XXX_sizecache        int32                        `json:"-"`
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	MessageId   string                       `protobuf:"bytes,1,opt,name=message_id,json=messageId,proto3" json:"message_id,omitempty"`
+	Time        string                       `protobuf:"bytes,2,opt,name=time,proto3" json:"time,omitempty"`
+	Importance  JobMessage_MessageImportance `protobuf:"varint,3,opt,name=importance,proto3,enum=org.apache.beam.model.job_management.v1.JobMessage_MessageImportance" json:"importance,omitempty"`
+	MessageText string                       `protobuf:"bytes,4,opt,name=message_text,json=messageText,proto3" json:"message_text,omitempty"`
 }
 
-func (m *JobMessage) Reset()         { *m = JobMessage{} }
-func (m *JobMessage) String() string { return proto.CompactTextString(m) }
-func (*JobMessage) ProtoMessage()    {}
+func (x *JobMessage) Reset() {
+	*x = JobMessage{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_job_api_proto_msgTypes[14]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *JobMessage) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*JobMessage) ProtoMessage() {}
+
+func (x *JobMessage) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_job_api_proto_msgTypes[14]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use JobMessage.ProtoReflect.Descriptor instead.
 func (*JobMessage) Descriptor() ([]byte, []int) {
-	return fileDescriptor_97c7b84f742157ae, []int{14}
+	return file_beam_job_api_proto_rawDescGZIP(), []int{14}
 }
 
-func (m *JobMessage) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_JobMessage.Unmarshal(m, b)
-}
-func (m *JobMessage) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_JobMessage.Marshal(b, m, deterministic)
-}
-func (m *JobMessage) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_JobMessage.Merge(m, src)
-}
-func (m *JobMessage) XXX_Size() int {
-	return xxx_messageInfo_JobMessage.Size(m)
-}
-func (m *JobMessage) XXX_DiscardUnknown() {
-	xxx_messageInfo_JobMessage.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_JobMessage proto.InternalMessageInfo
-
-func (m *JobMessage) GetMessageId() string {
-	if m != nil {
-		return m.MessageId
+func (x *JobMessage) GetMessageId() string {
+	if x != nil {
+		return x.MessageId
 	}
 	return ""
 }
 
-func (m *JobMessage) GetTime() string {
-	if m != nil {
-		return m.Time
+func (x *JobMessage) GetTime() string {
+	if x != nil {
+		return x.Time
 	}
 	return ""
 }
 
-func (m *JobMessage) GetImportance() JobMessage_MessageImportance {
-	if m != nil {
-		return m.Importance
+func (x *JobMessage) GetImportance() JobMessage_MessageImportance {
+	if x != nil {
+		return x.Importance
 	}
 	return JobMessage_MESSAGE_IMPORTANCE_UNSPECIFIED
 }
 
-func (m *JobMessage) GetMessageText() string {
-	if m != nil {
-		return m.MessageText
+func (x *JobMessage) GetMessageText() string {
+	if x != nil {
+		return x.MessageText
 	}
 	return ""
 }
 
 type JobMessagesResponse struct {
-	// Types that are valid to be assigned to Response:
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	// Types that are assignable to Response:
 	//	*JobMessagesResponse_MessageResponse
 	//	*JobMessagesResponse_StateResponse
-	Response             isJobMessagesResponse_Response `protobuf_oneof:"response"`
-	XXX_NoUnkeyedLiteral struct{}                       `json:"-"`
-	XXX_unrecognized     []byte                         `json:"-"`
-	XXX_sizecache        int32                          `json:"-"`
+	Response isJobMessagesResponse_Response `protobuf_oneof:"response"`
 }
 
-func (m *JobMessagesResponse) Reset()         { *m = JobMessagesResponse{} }
-func (m *JobMessagesResponse) String() string { return proto.CompactTextString(m) }
-func (*JobMessagesResponse) ProtoMessage()    {}
+func (x *JobMessagesResponse) Reset() {
+	*x = JobMessagesResponse{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_job_api_proto_msgTypes[15]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *JobMessagesResponse) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*JobMessagesResponse) ProtoMessage() {}
+
+func (x *JobMessagesResponse) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_job_api_proto_msgTypes[15]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use JobMessagesResponse.ProtoReflect.Descriptor instead.
 func (*JobMessagesResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_97c7b84f742157ae, []int{15}
+	return file_beam_job_api_proto_rawDescGZIP(), []int{15}
 }
 
-func (m *JobMessagesResponse) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_JobMessagesResponse.Unmarshal(m, b)
-}
-func (m *JobMessagesResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_JobMessagesResponse.Marshal(b, m, deterministic)
-}
-func (m *JobMessagesResponse) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_JobMessagesResponse.Merge(m, src)
-}
-func (m *JobMessagesResponse) XXX_Size() int {
-	return xxx_messageInfo_JobMessagesResponse.Size(m)
-}
-func (m *JobMessagesResponse) XXX_DiscardUnknown() {
-	xxx_messageInfo_JobMessagesResponse.DiscardUnknown(m)
+func (m *JobMessagesResponse) GetResponse() isJobMessagesResponse_Response {
+	if m != nil {
+		return m.Response
+	}
+	return nil
 }
 
-var xxx_messageInfo_JobMessagesResponse proto.InternalMessageInfo
+func (x *JobMessagesResponse) GetMessageResponse() *JobMessage {
+	if x, ok := x.GetResponse().(*JobMessagesResponse_MessageResponse); ok {
+		return x.MessageResponse
+	}
+	return nil
+}
+
+func (x *JobMessagesResponse) GetStateResponse() *JobStateEvent {
+	if x, ok := x.GetResponse().(*JobMessagesResponse_StateResponse); ok {
+		return x.StateResponse
+	}
+	return nil
+}
 
 type isJobMessagesResponse_Response interface {
 	isJobMessagesResponse_Response()
@@ -928,35 +1160,6 @@
 
 func (*JobMessagesResponse_StateResponse) isJobMessagesResponse_Response() {}
 
-func (m *JobMessagesResponse) GetResponse() isJobMessagesResponse_Response {
-	if m != nil {
-		return m.Response
-	}
-	return nil
-}
-
-func (m *JobMessagesResponse) GetMessageResponse() *JobMessage {
-	if x, ok := m.GetResponse().(*JobMessagesResponse_MessageResponse); ok {
-		return x.MessageResponse
-	}
-	return nil
-}
-
-func (m *JobMessagesResponse) GetStateResponse() *JobStateEvent {
-	if x, ok := m.GetResponse().(*JobMessagesResponse_StateResponse); ok {
-		return x.StateResponse
-	}
-	return nil
-}
-
-// XXX_OneofWrappers is for the internal use of the proto package.
-func (*JobMessagesResponse) XXX_OneofWrappers() []interface{} {
-	return []interface{}{
-		(*JobMessagesResponse_MessageResponse)(nil),
-		(*JobMessagesResponse_StateResponse)(nil),
-	}
-}
-
 // Enumeration of all JobStates
 //
 // The state transition diagram is:
@@ -969,158 +1172,189 @@
 // Transitions are optional such that a job may go from STOPPED to RUNNING
 // without needing to pass through STARTING.
 type JobState struct {
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
 }
 
-func (m *JobState) Reset()         { *m = JobState{} }
-func (m *JobState) String() string { return proto.CompactTextString(m) }
-func (*JobState) ProtoMessage()    {}
+func (x *JobState) Reset() {
+	*x = JobState{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_job_api_proto_msgTypes[16]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *JobState) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*JobState) ProtoMessage() {}
+
+func (x *JobState) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_job_api_proto_msgTypes[16]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use JobState.ProtoReflect.Descriptor instead.
 func (*JobState) Descriptor() ([]byte, []int) {
-	return fileDescriptor_97c7b84f742157ae, []int{16}
+	return file_beam_job_api_proto_rawDescGZIP(), []int{16}
 }
 
-func (m *JobState) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_JobState.Unmarshal(m, b)
-}
-func (m *JobState) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_JobState.Marshal(b, m, deterministic)
-}
-func (m *JobState) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_JobState.Merge(m, src)
-}
-func (m *JobState) XXX_Size() int {
-	return xxx_messageInfo_JobState.Size(m)
-}
-func (m *JobState) XXX_DiscardUnknown() {
-	xxx_messageInfo_JobState.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_JobState proto.InternalMessageInfo
-
 type GetJobMetricsRequest struct {
-	JobId                string   `protobuf:"bytes,1,opt,name=job_id,json=jobId,proto3" json:"job_id,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	JobId string `protobuf:"bytes,1,opt,name=job_id,json=jobId,proto3" json:"job_id,omitempty"` // (required)
 }
 
-func (m *GetJobMetricsRequest) Reset()         { *m = GetJobMetricsRequest{} }
-func (m *GetJobMetricsRequest) String() string { return proto.CompactTextString(m) }
-func (*GetJobMetricsRequest) ProtoMessage()    {}
+func (x *GetJobMetricsRequest) Reset() {
+	*x = GetJobMetricsRequest{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_job_api_proto_msgTypes[17]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *GetJobMetricsRequest) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*GetJobMetricsRequest) ProtoMessage() {}
+
+func (x *GetJobMetricsRequest) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_job_api_proto_msgTypes[17]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use GetJobMetricsRequest.ProtoReflect.Descriptor instead.
 func (*GetJobMetricsRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_97c7b84f742157ae, []int{17}
+	return file_beam_job_api_proto_rawDescGZIP(), []int{17}
 }
 
-func (m *GetJobMetricsRequest) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_GetJobMetricsRequest.Unmarshal(m, b)
-}
-func (m *GetJobMetricsRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_GetJobMetricsRequest.Marshal(b, m, deterministic)
-}
-func (m *GetJobMetricsRequest) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_GetJobMetricsRequest.Merge(m, src)
-}
-func (m *GetJobMetricsRequest) XXX_Size() int {
-	return xxx_messageInfo_GetJobMetricsRequest.Size(m)
-}
-func (m *GetJobMetricsRequest) XXX_DiscardUnknown() {
-	xxx_messageInfo_GetJobMetricsRequest.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_GetJobMetricsRequest proto.InternalMessageInfo
-
-func (m *GetJobMetricsRequest) GetJobId() string {
-	if m != nil {
-		return m.JobId
+func (x *GetJobMetricsRequest) GetJobId() string {
+	if x != nil {
+		return x.JobId
 	}
 	return ""
 }
 
 type GetJobMetricsResponse struct {
-	Metrics              *MetricResults `protobuf:"bytes,1,opt,name=metrics,proto3" json:"metrics,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}       `json:"-"`
-	XXX_unrecognized     []byte         `json:"-"`
-	XXX_sizecache        int32          `json:"-"`
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	Metrics *MetricResults `protobuf:"bytes,1,opt,name=metrics,proto3" json:"metrics,omitempty"`
 }
 
-func (m *GetJobMetricsResponse) Reset()         { *m = GetJobMetricsResponse{} }
-func (m *GetJobMetricsResponse) String() string { return proto.CompactTextString(m) }
-func (*GetJobMetricsResponse) ProtoMessage()    {}
+func (x *GetJobMetricsResponse) Reset() {
+	*x = GetJobMetricsResponse{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_job_api_proto_msgTypes[18]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *GetJobMetricsResponse) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*GetJobMetricsResponse) ProtoMessage() {}
+
+func (x *GetJobMetricsResponse) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_job_api_proto_msgTypes[18]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use GetJobMetricsResponse.ProtoReflect.Descriptor instead.
 func (*GetJobMetricsResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_97c7b84f742157ae, []int{18}
+	return file_beam_job_api_proto_rawDescGZIP(), []int{18}
 }
 
-func (m *GetJobMetricsResponse) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_GetJobMetricsResponse.Unmarshal(m, b)
-}
-func (m *GetJobMetricsResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_GetJobMetricsResponse.Marshal(b, m, deterministic)
-}
-func (m *GetJobMetricsResponse) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_GetJobMetricsResponse.Merge(m, src)
-}
-func (m *GetJobMetricsResponse) XXX_Size() int {
-	return xxx_messageInfo_GetJobMetricsResponse.Size(m)
-}
-func (m *GetJobMetricsResponse) XXX_DiscardUnknown() {
-	xxx_messageInfo_GetJobMetricsResponse.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_GetJobMetricsResponse proto.InternalMessageInfo
-
-func (m *GetJobMetricsResponse) GetMetrics() *MetricResults {
-	if m != nil {
-		return m.Metrics
+func (x *GetJobMetricsResponse) GetMetrics() *MetricResults {
+	if x != nil {
+		return x.Metrics
 	}
 	return nil
 }
 
 // All metrics for a given job.  Runners may support one or the other or both.
 type MetricResults struct {
-	Attempted            []*pipeline_v1.MonitoringInfo `protobuf:"bytes,1,rep,name=attempted,proto3" json:"attempted,omitempty"`
-	Committed            []*pipeline_v1.MonitoringInfo `protobuf:"bytes,2,rep,name=committed,proto3" json:"committed,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}                      `json:"-"`
-	XXX_unrecognized     []byte                        `json:"-"`
-	XXX_sizecache        int32                         `json:"-"`
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	Attempted []*pipeline_v1.MonitoringInfo `protobuf:"bytes,1,rep,name=attempted,proto3" json:"attempted,omitempty"`
+	Committed []*pipeline_v1.MonitoringInfo `protobuf:"bytes,2,rep,name=committed,proto3" json:"committed,omitempty"`
 }
 
-func (m *MetricResults) Reset()         { *m = MetricResults{} }
-func (m *MetricResults) String() string { return proto.CompactTextString(m) }
-func (*MetricResults) ProtoMessage()    {}
+func (x *MetricResults) Reset() {
+	*x = MetricResults{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_job_api_proto_msgTypes[19]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *MetricResults) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*MetricResults) ProtoMessage() {}
+
+func (x *MetricResults) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_job_api_proto_msgTypes[19]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use MetricResults.ProtoReflect.Descriptor instead.
 func (*MetricResults) Descriptor() ([]byte, []int) {
-	return fileDescriptor_97c7b84f742157ae, []int{19}
+	return file_beam_job_api_proto_rawDescGZIP(), []int{19}
 }
 
-func (m *MetricResults) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_MetricResults.Unmarshal(m, b)
-}
-func (m *MetricResults) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_MetricResults.Marshal(b, m, deterministic)
-}
-func (m *MetricResults) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_MetricResults.Merge(m, src)
-}
-func (m *MetricResults) XXX_Size() int {
-	return xxx_messageInfo_MetricResults.Size(m)
-}
-func (m *MetricResults) XXX_DiscardUnknown() {
-	xxx_messageInfo_MetricResults.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_MetricResults proto.InternalMessageInfo
-
-func (m *MetricResults) GetAttempted() []*pipeline_v1.MonitoringInfo {
-	if m != nil {
-		return m.Attempted
+func (x *MetricResults) GetAttempted() []*pipeline_v1.MonitoringInfo {
+	if x != nil {
+		return x.Attempted
 	}
 	return nil
 }
 
-func (m *MetricResults) GetCommitted() []*pipeline_v1.MonitoringInfo {
-	if m != nil {
-		return m.Committed
+func (x *MetricResults) GetCommitted() []*pipeline_v1.MonitoringInfo {
+	if x != nil {
+		return x.Committed
 	}
 	return nil
 }
@@ -1130,71 +1364,89 @@
 // list available options to the user.
 // Throws error GRPC_STATUS_UNAVAILABLE if server is down
 type DescribePipelineOptionsRequest struct {
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
 }
 
-func (m *DescribePipelineOptionsRequest) Reset()         { *m = DescribePipelineOptionsRequest{} }
-func (m *DescribePipelineOptionsRequest) String() string { return proto.CompactTextString(m) }
-func (*DescribePipelineOptionsRequest) ProtoMessage()    {}
+func (x *DescribePipelineOptionsRequest) Reset() {
+	*x = DescribePipelineOptionsRequest{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_job_api_proto_msgTypes[20]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *DescribePipelineOptionsRequest) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*DescribePipelineOptionsRequest) ProtoMessage() {}
+
+func (x *DescribePipelineOptionsRequest) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_job_api_proto_msgTypes[20]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use DescribePipelineOptionsRequest.ProtoReflect.Descriptor instead.
 func (*DescribePipelineOptionsRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_97c7b84f742157ae, []int{20}
+	return file_beam_job_api_proto_rawDescGZIP(), []int{20}
 }
 
-func (m *DescribePipelineOptionsRequest) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_DescribePipelineOptionsRequest.Unmarshal(m, b)
-}
-func (m *DescribePipelineOptionsRequest) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_DescribePipelineOptionsRequest.Marshal(b, m, deterministic)
-}
-func (m *DescribePipelineOptionsRequest) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_DescribePipelineOptionsRequest.Merge(m, src)
-}
-func (m *DescribePipelineOptionsRequest) XXX_Size() int {
-	return xxx_messageInfo_DescribePipelineOptionsRequest.Size(m)
-}
-func (m *DescribePipelineOptionsRequest) XXX_DiscardUnknown() {
-	xxx_messageInfo_DescribePipelineOptionsRequest.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_DescribePipelineOptionsRequest proto.InternalMessageInfo
-
 // Type for pipeline options.
 // Types mirror those of JSON, since that's how pipeline options are serialized.
 type PipelineOptionType struct {
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
 }
 
-func (m *PipelineOptionType) Reset()         { *m = PipelineOptionType{} }
-func (m *PipelineOptionType) String() string { return proto.CompactTextString(m) }
-func (*PipelineOptionType) ProtoMessage()    {}
+func (x *PipelineOptionType) Reset() {
+	*x = PipelineOptionType{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_job_api_proto_msgTypes[21]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *PipelineOptionType) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*PipelineOptionType) ProtoMessage() {}
+
+func (x *PipelineOptionType) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_job_api_proto_msgTypes[21]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use PipelineOptionType.ProtoReflect.Descriptor instead.
 func (*PipelineOptionType) Descriptor() ([]byte, []int) {
-	return fileDescriptor_97c7b84f742157ae, []int{21}
+	return file_beam_job_api_proto_rawDescGZIP(), []int{21}
 }
 
-func (m *PipelineOptionType) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_PipelineOptionType.Unmarshal(m, b)
-}
-func (m *PipelineOptionType) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_PipelineOptionType.Marshal(b, m, deterministic)
-}
-func (m *PipelineOptionType) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_PipelineOptionType.Merge(m, src)
-}
-func (m *PipelineOptionType) XXX_Size() int {
-	return xxx_messageInfo_PipelineOptionType.Size(m)
-}
-func (m *PipelineOptionType) XXX_DiscardUnknown() {
-	xxx_messageInfo_PipelineOptionType.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_PipelineOptionType proto.InternalMessageInfo
-
 // Metadata for a pipeline option.
 type PipelineOptionDescriptor struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
 	// (Required) The option name.
 	Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"`
 	// (Required) Type of option.
@@ -1204,246 +1456,848 @@
 	// (Optional) Default value.
 	DefaultValue string `protobuf:"bytes,4,opt,name=default_value,json=defaultValue,proto3" json:"default_value,omitempty"`
 	// (Required) The group this option belongs to.
-	Group                string   `protobuf:"bytes,5,opt,name=group,proto3" json:"group,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
+	Group string `protobuf:"bytes,5,opt,name=group,proto3" json:"group,omitempty"`
 }
 
-func (m *PipelineOptionDescriptor) Reset()         { *m = PipelineOptionDescriptor{} }
-func (m *PipelineOptionDescriptor) String() string { return proto.CompactTextString(m) }
-func (*PipelineOptionDescriptor) ProtoMessage()    {}
+func (x *PipelineOptionDescriptor) Reset() {
+	*x = PipelineOptionDescriptor{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_job_api_proto_msgTypes[22]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *PipelineOptionDescriptor) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*PipelineOptionDescriptor) ProtoMessage() {}
+
+func (x *PipelineOptionDescriptor) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_job_api_proto_msgTypes[22]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use PipelineOptionDescriptor.ProtoReflect.Descriptor instead.
 func (*PipelineOptionDescriptor) Descriptor() ([]byte, []int) {
-	return fileDescriptor_97c7b84f742157ae, []int{22}
+	return file_beam_job_api_proto_rawDescGZIP(), []int{22}
 }
 
-func (m *PipelineOptionDescriptor) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_PipelineOptionDescriptor.Unmarshal(m, b)
-}
-func (m *PipelineOptionDescriptor) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_PipelineOptionDescriptor.Marshal(b, m, deterministic)
-}
-func (m *PipelineOptionDescriptor) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_PipelineOptionDescriptor.Merge(m, src)
-}
-func (m *PipelineOptionDescriptor) XXX_Size() int {
-	return xxx_messageInfo_PipelineOptionDescriptor.Size(m)
-}
-func (m *PipelineOptionDescriptor) XXX_DiscardUnknown() {
-	xxx_messageInfo_PipelineOptionDescriptor.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_PipelineOptionDescriptor proto.InternalMessageInfo
-
-func (m *PipelineOptionDescriptor) GetName() string {
-	if m != nil {
-		return m.Name
+func (x *PipelineOptionDescriptor) GetName() string {
+	if x != nil {
+		return x.Name
 	}
 	return ""
 }
 
-func (m *PipelineOptionDescriptor) GetType() PipelineOptionType_Enum {
-	if m != nil {
-		return m.Type
+func (x *PipelineOptionDescriptor) GetType() PipelineOptionType_Enum {
+	if x != nil {
+		return x.Type
 	}
 	return PipelineOptionType_STRING
 }
 
-func (m *PipelineOptionDescriptor) GetDescription() string {
-	if m != nil {
-		return m.Description
+func (x *PipelineOptionDescriptor) GetDescription() string {
+	if x != nil {
+		return x.Description
 	}
 	return ""
 }
 
-func (m *PipelineOptionDescriptor) GetDefaultValue() string {
-	if m != nil {
-		return m.DefaultValue
+func (x *PipelineOptionDescriptor) GetDefaultValue() string {
+	if x != nil {
+		return x.DefaultValue
 	}
 	return ""
 }
 
-func (m *PipelineOptionDescriptor) GetGroup() string {
-	if m != nil {
-		return m.Group
+func (x *PipelineOptionDescriptor) GetGroup() string {
+	if x != nil {
+		return x.Group
 	}
 	return ""
 }
 
 type DescribePipelineOptionsResponse struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
 	// List of pipeline option descriptors.
-	Options              []*PipelineOptionDescriptor `protobuf:"bytes,1,rep,name=options,proto3" json:"options,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}                    `json:"-"`
-	XXX_unrecognized     []byte                      `json:"-"`
-	XXX_sizecache        int32                       `json:"-"`
+	Options []*PipelineOptionDescriptor `protobuf:"bytes,1,rep,name=options,proto3" json:"options,omitempty"`
 }
 
-func (m *DescribePipelineOptionsResponse) Reset()         { *m = DescribePipelineOptionsResponse{} }
-func (m *DescribePipelineOptionsResponse) String() string { return proto.CompactTextString(m) }
-func (*DescribePipelineOptionsResponse) ProtoMessage()    {}
+func (x *DescribePipelineOptionsResponse) Reset() {
+	*x = DescribePipelineOptionsResponse{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_job_api_proto_msgTypes[23]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *DescribePipelineOptionsResponse) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*DescribePipelineOptionsResponse) ProtoMessage() {}
+
+func (x *DescribePipelineOptionsResponse) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_job_api_proto_msgTypes[23]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use DescribePipelineOptionsResponse.ProtoReflect.Descriptor instead.
 func (*DescribePipelineOptionsResponse) Descriptor() ([]byte, []int) {
-	return fileDescriptor_97c7b84f742157ae, []int{23}
+	return file_beam_job_api_proto_rawDescGZIP(), []int{23}
 }
 
-func (m *DescribePipelineOptionsResponse) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_DescribePipelineOptionsResponse.Unmarshal(m, b)
-}
-func (m *DescribePipelineOptionsResponse) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_DescribePipelineOptionsResponse.Marshal(b, m, deterministic)
-}
-func (m *DescribePipelineOptionsResponse) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_DescribePipelineOptionsResponse.Merge(m, src)
-}
-func (m *DescribePipelineOptionsResponse) XXX_Size() int {
-	return xxx_messageInfo_DescribePipelineOptionsResponse.Size(m)
-}
-func (m *DescribePipelineOptionsResponse) XXX_DiscardUnknown() {
-	xxx_messageInfo_DescribePipelineOptionsResponse.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_DescribePipelineOptionsResponse proto.InternalMessageInfo
-
-func (m *DescribePipelineOptionsResponse) GetOptions() []*PipelineOptionDescriptor {
-	if m != nil {
-		return m.Options
+func (x *DescribePipelineOptionsResponse) GetOptions() []*PipelineOptionDescriptor {
+	if x != nil {
+		return x.Options
 	}
 	return nil
 }
 
-func init() {
-	proto.RegisterEnum("org.apache.beam.model.job_management.v1.JobMessage_MessageImportance", JobMessage_MessageImportance_name, JobMessage_MessageImportance_value)
-	proto.RegisterEnum("org.apache.beam.model.job_management.v1.JobState_Enum", JobState_Enum_name, JobState_Enum_value)
-	proto.RegisterEnum("org.apache.beam.model.job_management.v1.PipelineOptionType_Enum", PipelineOptionType_Enum_name, PipelineOptionType_Enum_value)
-	proto.RegisterType((*PrepareJobRequest)(nil), "org.apache.beam.model.job_management.v1.PrepareJobRequest")
-	proto.RegisterType((*PrepareJobResponse)(nil), "org.apache.beam.model.job_management.v1.PrepareJobResponse")
-	proto.RegisterType((*RunJobRequest)(nil), "org.apache.beam.model.job_management.v1.RunJobRequest")
-	proto.RegisterType((*RunJobResponse)(nil), "org.apache.beam.model.job_management.v1.RunJobResponse")
-	proto.RegisterType((*CancelJobRequest)(nil), "org.apache.beam.model.job_management.v1.CancelJobRequest")
-	proto.RegisterType((*CancelJobResponse)(nil), "org.apache.beam.model.job_management.v1.CancelJobResponse")
-	proto.RegisterType((*JobInfo)(nil), "org.apache.beam.model.job_management.v1.JobInfo")
-	proto.RegisterType((*GetJobsRequest)(nil), "org.apache.beam.model.job_management.v1.GetJobsRequest")
-	proto.RegisterType((*GetJobsResponse)(nil), "org.apache.beam.model.job_management.v1.GetJobsResponse")
-	proto.RegisterType((*GetJobStateRequest)(nil), "org.apache.beam.model.job_management.v1.GetJobStateRequest")
-	proto.RegisterType((*JobStateEvent)(nil), "org.apache.beam.model.job_management.v1.JobStateEvent")
-	proto.RegisterType((*GetJobPipelineRequest)(nil), "org.apache.beam.model.job_management.v1.GetJobPipelineRequest")
-	proto.RegisterType((*GetJobPipelineResponse)(nil), "org.apache.beam.model.job_management.v1.GetJobPipelineResponse")
-	proto.RegisterType((*JobMessagesRequest)(nil), "org.apache.beam.model.job_management.v1.JobMessagesRequest")
-	proto.RegisterType((*JobMessage)(nil), "org.apache.beam.model.job_management.v1.JobMessage")
-	proto.RegisterType((*JobMessagesResponse)(nil), "org.apache.beam.model.job_management.v1.JobMessagesResponse")
-	proto.RegisterType((*JobState)(nil), "org.apache.beam.model.job_management.v1.JobState")
-	proto.RegisterType((*GetJobMetricsRequest)(nil), "org.apache.beam.model.job_management.v1.GetJobMetricsRequest")
-	proto.RegisterType((*GetJobMetricsResponse)(nil), "org.apache.beam.model.job_management.v1.GetJobMetricsResponse")
-	proto.RegisterType((*MetricResults)(nil), "org.apache.beam.model.job_management.v1.MetricResults")
-	proto.RegisterType((*DescribePipelineOptionsRequest)(nil), "org.apache.beam.model.job_management.v1.DescribePipelineOptionsRequest")
-	proto.RegisterType((*PipelineOptionType)(nil), "org.apache.beam.model.job_management.v1.PipelineOptionType")
-	proto.RegisterType((*PipelineOptionDescriptor)(nil), "org.apache.beam.model.job_management.v1.PipelineOptionDescriptor")
-	proto.RegisterType((*DescribePipelineOptionsResponse)(nil), "org.apache.beam.model.job_management.v1.DescribePipelineOptionsResponse")
+var File_beam_job_api_proto protoreflect.FileDescriptor
+
+var file_beam_job_api_proto_rawDesc = []byte{
+	0x0a, 0x12, 0x62, 0x65, 0x61, 0x6d, 0x5f, 0x6a, 0x6f, 0x62, 0x5f, 0x61, 0x70, 0x69, 0x2e, 0x70,
+	0x72, 0x6f, 0x74, 0x6f, 0x12, 0x27, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65,
+	0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x6a, 0x6f, 0x62, 0x5f,
+	0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x76, 0x31, 0x1a, 0x15, 0x62,
+	0x65, 0x61, 0x6d, 0x5f, 0x72, 0x75, 0x6e, 0x6e, 0x65, 0x72, 0x5f, 0x61, 0x70, 0x69, 0x2e, 0x70,
+	0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x0f, 0x65, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x73, 0x2e,
+	0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x1c, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2f, 0x70, 0x72,
+	0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x73, 0x74, 0x72, 0x75, 0x63, 0x74, 0x2e, 0x70, 0x72,
+	0x6f, 0x74, 0x6f, 0x1a, 0x1f, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2f, 0x70, 0x72, 0x6f, 0x74,
+	0x6f, 0x62, 0x75, 0x66, 0x2f, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x2e, 0x70,
+	0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x0d, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x2e, 0x70, 0x72,
+	0x6f, 0x74, 0x6f, 0x22, 0xbb, 0x01, 0x0a, 0x11, 0x50, 0x72, 0x65, 0x70, 0x61, 0x72, 0x65, 0x4a,
+	0x6f, 0x62, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x47, 0x0a, 0x08, 0x70, 0x69, 0x70,
+	0x65, 0x6c, 0x69, 0x6e, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2b, 0x2e, 0x6f, 0x72,
+	0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f,
+	0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e,
+	0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x52, 0x08, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69,
+	0x6e, 0x65, 0x12, 0x42, 0x0a, 0x10, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x5f, 0x6f,
+	0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x67,
+	0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x53,
+	0x74, 0x72, 0x75, 0x63, 0x74, 0x52, 0x0f, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x4f,
+	0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x19, 0x0a, 0x08, 0x6a, 0x6f, 0x62, 0x5f, 0x6e, 0x61,
+	0x6d, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x6a, 0x6f, 0x62, 0x4e, 0x61, 0x6d,
+	0x65, 0x22, 0xe4, 0x01, 0x0a, 0x12, 0x50, 0x72, 0x65, 0x70, 0x61, 0x72, 0x65, 0x4a, 0x6f, 0x62,
+	0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x25, 0x0a, 0x0e, 0x70, 0x72, 0x65, 0x70,
+	0x61, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09,
+	0x52, 0x0d, 0x70, 0x72, 0x65, 0x70, 0x61, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64, 0x12,
+	0x73, 0x0a, 0x19, 0x61, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x5f, 0x73, 0x74, 0x61, 0x67,
+	0x69, 0x6e, 0x67, 0x5f, 0x65, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x18, 0x02, 0x20, 0x01,
+	0x28, 0x0b, 0x32, 0x37, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e,
+	0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c,
+	0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x70, 0x69, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63,
+	0x65, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x52, 0x17, 0x61, 0x72, 0x74,
+	0x69, 0x66, 0x61, 0x63, 0x74, 0x53, 0x74, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x45, 0x6e, 0x64, 0x70,
+	0x6f, 0x69, 0x6e, 0x74, 0x12, 0x32, 0x0a, 0x15, 0x73, 0x74, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x5f,
+	0x73, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x5f, 0x74, 0x6f, 0x6b, 0x65, 0x6e, 0x18, 0x03, 0x20,
+	0x01, 0x28, 0x09, 0x52, 0x13, 0x73, 0x74, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x53, 0x65, 0x73, 0x73,
+	0x69, 0x6f, 0x6e, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x22, 0x5f, 0x0a, 0x0d, 0x52, 0x75, 0x6e, 0x4a,
+	0x6f, 0x62, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x25, 0x0a, 0x0e, 0x70, 0x72, 0x65,
+	0x70, 0x61, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28,
+	0x09, 0x52, 0x0d, 0x70, 0x72, 0x65, 0x70, 0x61, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x49, 0x64,
+	0x12, 0x27, 0x0a, 0x0f, 0x72, 0x65, 0x74, 0x72, 0x69, 0x65, 0x76, 0x61, 0x6c, 0x5f, 0x74, 0x6f,
+	0x6b, 0x65, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x72, 0x65, 0x74, 0x72, 0x69,
+	0x65, 0x76, 0x61, 0x6c, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x22, 0x27, 0x0a, 0x0e, 0x52, 0x75, 0x6e,
+	0x4a, 0x6f, 0x62, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x15, 0x0a, 0x06, 0x6a,
+	0x6f, 0x62, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x6a, 0x6f, 0x62,
+	0x49, 0x64, 0x22, 0x29, 0x0a, 0x10, 0x43, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x4a, 0x6f, 0x62, 0x52,
+	0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x15, 0x0a, 0x06, 0x6a, 0x6f, 0x62, 0x5f, 0x69, 0x64,
+	0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x6a, 0x6f, 0x62, 0x49, 0x64, 0x22, 0x61, 0x0a,
+	0x11, 0x43, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x4a, 0x6f, 0x62, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e,
+	0x73, 0x65, 0x12, 0x4c, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28,
+	0x0e, 0x32, 0x36, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62,
+	0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x6a, 0x6f, 0x62, 0x5f, 0x6d, 0x61,
+	0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x76, 0x31, 0x2e, 0x4a, 0x6f, 0x62, 0x53,
+	0x74, 0x61, 0x74, 0x65, 0x2e, 0x45, 0x6e, 0x75, 0x6d, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65,
+	0x22, 0xcd, 0x01, 0x0a, 0x07, 0x4a, 0x6f, 0x62, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x15, 0x0a, 0x06,
+	0x6a, 0x6f, 0x62, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x6a, 0x6f,
+	0x62, 0x49, 0x64, 0x12, 0x19, 0x0a, 0x08, 0x6a, 0x6f, 0x62, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18,
+	0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x6a, 0x6f, 0x62, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x42,
+	0x0a, 0x10, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x5f, 0x6f, 0x70, 0x74, 0x69, 0x6f,
+	0x6e, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x17, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c,
+	0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x53, 0x74, 0x72, 0x75, 0x63,
+	0x74, 0x52, 0x0f, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x4f, 0x70, 0x74, 0x69, 0x6f,
+	0x6e, 0x73, 0x12, 0x4c, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28,
+	0x0e, 0x32, 0x36, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62,
+	0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x6a, 0x6f, 0x62, 0x5f, 0x6d, 0x61,
+	0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x76, 0x31, 0x2e, 0x4a, 0x6f, 0x62, 0x53,
+	0x74, 0x61, 0x74, 0x65, 0x2e, 0x45, 0x6e, 0x75, 0x6d, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65,
+	0x22, 0x10, 0x0a, 0x0e, 0x47, 0x65, 0x74, 0x4a, 0x6f, 0x62, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65,
+	0x73, 0x74, 0x22, 0x5e, 0x0a, 0x0f, 0x47, 0x65, 0x74, 0x4a, 0x6f, 0x62, 0x73, 0x52, 0x65, 0x73,
+	0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x4b, 0x0a, 0x08, 0x6a, 0x6f, 0x62, 0x5f, 0x69, 0x6e, 0x66,
+	0x6f, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x30, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70,
+	0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e,
+	0x6a, 0x6f, 0x62, 0x5f, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x76,
+	0x31, 0x2e, 0x4a, 0x6f, 0x62, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x07, 0x6a, 0x6f, 0x62, 0x49, 0x6e,
+	0x66, 0x6f, 0x22, 0x2b, 0x0a, 0x12, 0x47, 0x65, 0x74, 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, 0x74,
+	0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x15, 0x0a, 0x06, 0x6a, 0x6f, 0x62, 0x5f,
+	0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x6a, 0x6f, 0x62, 0x49, 0x64, 0x22,
+	0x97, 0x01, 0x0a, 0x0d, 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, 0x74, 0x65, 0x45, 0x76, 0x65, 0x6e,
+	0x74, 0x12, 0x4c, 0x0a, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0e,
+	0x32, 0x36, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65,
+	0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x6a, 0x6f, 0x62, 0x5f, 0x6d, 0x61, 0x6e,
+	0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x76, 0x31, 0x2e, 0x4a, 0x6f, 0x62, 0x53, 0x74,
+	0x61, 0x74, 0x65, 0x2e, 0x45, 0x6e, 0x75, 0x6d, 0x52, 0x05, 0x73, 0x74, 0x61, 0x74, 0x65, 0x12,
+	0x38, 0x0a, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, 0x02, 0x20, 0x01,
+	0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74,
+	0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x09,
+	0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x22, 0x2e, 0x0a, 0x15, 0x47, 0x65, 0x74,
+	0x4a, 0x6f, 0x62, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65,
+	0x73, 0x74, 0x12, 0x15, 0x0a, 0x06, 0x6a, 0x6f, 0x62, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01,
+	0x28, 0x09, 0x52, 0x05, 0x6a, 0x6f, 0x62, 0x49, 0x64, 0x22, 0x61, 0x0a, 0x16, 0x47, 0x65, 0x74,
+	0x4a, 0x6f, 0x62, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f,
+	0x6e, 0x73, 0x65, 0x12, 0x47, 0x0a, 0x08, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x18,
+	0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2b, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63,
+	0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69,
+	0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69,
+	0x6e, 0x65, 0x52, 0x08, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x22, 0x2b, 0x0a, 0x12,
+	0x4a, 0x6f, 0x62, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65,
+	0x73, 0x74, 0x12, 0x15, 0x0a, 0x06, 0x6a, 0x6f, 0x62, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01,
+	0x28, 0x09, 0x52, 0x05, 0x6a, 0x6f, 0x62, 0x49, 0x64, 0x22, 0xfb, 0x02, 0x0a, 0x0a, 0x4a, 0x6f,
+	0x62, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x1d, 0x0a, 0x0a, 0x6d, 0x65, 0x73, 0x73,
+	0x61, 0x67, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6d, 0x65,
+	0x73, 0x73, 0x61, 0x67, 0x65, 0x49, 0x64, 0x12, 0x12, 0x0a, 0x04, 0x74, 0x69, 0x6d, 0x65, 0x18,
+	0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x74, 0x69, 0x6d, 0x65, 0x12, 0x65, 0x0a, 0x0a, 0x69,
+	0x6d, 0x70, 0x6f, 0x72, 0x74, 0x61, 0x6e, 0x63, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0e, 0x32,
+	0x45, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61,
+	0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x6a, 0x6f, 0x62, 0x5f, 0x6d, 0x61, 0x6e, 0x61,
+	0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x76, 0x31, 0x2e, 0x4a, 0x6f, 0x62, 0x4d, 0x65, 0x73,
+	0x73, 0x61, 0x67, 0x65, 0x2e, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x49, 0x6d, 0x70, 0x6f,
+	0x72, 0x74, 0x61, 0x6e, 0x63, 0x65, 0x52, 0x0a, 0x69, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x61, 0x6e,
+	0x63, 0x65, 0x12, 0x21, 0x0a, 0x0c, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x5f, 0x74, 0x65,
+	0x78, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67,
+	0x65, 0x54, 0x65, 0x78, 0x74, 0x22, 0xaf, 0x01, 0x0a, 0x11, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67,
+	0x65, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x61, 0x6e, 0x63, 0x65, 0x12, 0x22, 0x0a, 0x1e, 0x4d,
+	0x45, 0x53, 0x53, 0x41, 0x47, 0x45, 0x5f, 0x49, 0x4d, 0x50, 0x4f, 0x52, 0x54, 0x41, 0x4e, 0x43,
+	0x45, 0x5f, 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12,
+	0x15, 0x0a, 0x11, 0x4a, 0x4f, 0x42, 0x5f, 0x4d, 0x45, 0x53, 0x53, 0x41, 0x47, 0x45, 0x5f, 0x44,
+	0x45, 0x42, 0x55, 0x47, 0x10, 0x01, 0x12, 0x18, 0x0a, 0x14, 0x4a, 0x4f, 0x42, 0x5f, 0x4d, 0x45,
+	0x53, 0x53, 0x41, 0x47, 0x45, 0x5f, 0x44, 0x45, 0x54, 0x41, 0x49, 0x4c, 0x45, 0x44, 0x10, 0x02,
+	0x12, 0x15, 0x0a, 0x11, 0x4a, 0x4f, 0x42, 0x5f, 0x4d, 0x45, 0x53, 0x53, 0x41, 0x47, 0x45, 0x5f,
+	0x42, 0x41, 0x53, 0x49, 0x43, 0x10, 0x03, 0x12, 0x17, 0x0a, 0x13, 0x4a, 0x4f, 0x42, 0x5f, 0x4d,
+	0x45, 0x53, 0x53, 0x41, 0x47, 0x45, 0x5f, 0x57, 0x41, 0x52, 0x4e, 0x49, 0x4e, 0x47, 0x10, 0x04,
+	0x12, 0x15, 0x0a, 0x11, 0x4a, 0x4f, 0x42, 0x5f, 0x4d, 0x45, 0x53, 0x53, 0x41, 0x47, 0x45, 0x5f,
+	0x45, 0x52, 0x52, 0x4f, 0x52, 0x10, 0x05, 0x22, 0xe4, 0x01, 0x0a, 0x13, 0x4a, 0x6f, 0x62, 0x4d,
+	0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12,
+	0x60, 0x0a, 0x10, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x5f, 0x72, 0x65, 0x73, 0x70, 0x6f,
+	0x6e, 0x73, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x33, 0x2e, 0x6f, 0x72, 0x67, 0x2e,
+	0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65,
+	0x6c, 0x2e, 0x6a, 0x6f, 0x62, 0x5f, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74,
+	0x2e, 0x76, 0x31, 0x2e, 0x4a, 0x6f, 0x62, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x48, 0x00,
+	0x52, 0x0f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73,
+	0x65, 0x12, 0x5f, 0x0a, 0x0e, 0x73, 0x74, 0x61, 0x74, 0x65, 0x5f, 0x72, 0x65, 0x73, 0x70, 0x6f,
+	0x6e, 0x73, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x36, 0x2e, 0x6f, 0x72, 0x67, 0x2e,
+	0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65,
+	0x6c, 0x2e, 0x6a, 0x6f, 0x62, 0x5f, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74,
+	0x2e, 0x76, 0x31, 0x2e, 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, 0x74, 0x65, 0x45, 0x76, 0x65, 0x6e,
+	0x74, 0x48, 0x00, 0x52, 0x0d, 0x73, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e,
+	0x73, 0x65, 0x42, 0x0a, 0x0a, 0x08, 0x72, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0xb7,
+	0x01, 0x0a, 0x08, 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, 0x74, 0x65, 0x22, 0xaa, 0x01, 0x0a, 0x04,
+	0x45, 0x6e, 0x75, 0x6d, 0x12, 0x0f, 0x0a, 0x0b, 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46,
+	0x49, 0x45, 0x44, 0x10, 0x00, 0x12, 0x0b, 0x0a, 0x07, 0x53, 0x54, 0x4f, 0x50, 0x50, 0x45, 0x44,
+	0x10, 0x01, 0x12, 0x0b, 0x0a, 0x07, 0x52, 0x55, 0x4e, 0x4e, 0x49, 0x4e, 0x47, 0x10, 0x02, 0x12,
+	0x08, 0x0a, 0x04, 0x44, 0x4f, 0x4e, 0x45, 0x10, 0x03, 0x12, 0x0a, 0x0a, 0x06, 0x46, 0x41, 0x49,
+	0x4c, 0x45, 0x44, 0x10, 0x04, 0x12, 0x0d, 0x0a, 0x09, 0x43, 0x41, 0x4e, 0x43, 0x45, 0x4c, 0x4c,
+	0x45, 0x44, 0x10, 0x05, 0x12, 0x0b, 0x0a, 0x07, 0x55, 0x50, 0x44, 0x41, 0x54, 0x45, 0x44, 0x10,
+	0x06, 0x12, 0x0c, 0x0a, 0x08, 0x44, 0x52, 0x41, 0x49, 0x4e, 0x49, 0x4e, 0x47, 0x10, 0x07, 0x12,
+	0x0b, 0x0a, 0x07, 0x44, 0x52, 0x41, 0x49, 0x4e, 0x45, 0x44, 0x10, 0x08, 0x12, 0x0c, 0x0a, 0x08,
+	0x53, 0x54, 0x41, 0x52, 0x54, 0x49, 0x4e, 0x47, 0x10, 0x09, 0x12, 0x0e, 0x0a, 0x0a, 0x43, 0x41,
+	0x4e, 0x43, 0x45, 0x4c, 0x4c, 0x49, 0x4e, 0x47, 0x10, 0x0a, 0x12, 0x0c, 0x0a, 0x08, 0x55, 0x50,
+	0x44, 0x41, 0x54, 0x49, 0x4e, 0x47, 0x10, 0x0b, 0x22, 0x2d, 0x0a, 0x14, 0x47, 0x65, 0x74, 0x4a,
+	0x6f, 0x62, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74,
+	0x12, 0x15, 0x0a, 0x06, 0x6a, 0x6f, 0x62, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09,
+	0x52, 0x05, 0x6a, 0x6f, 0x62, 0x49, 0x64, 0x22, 0x69, 0x0a, 0x15, 0x47, 0x65, 0x74, 0x4a, 0x6f,
+	0x62, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65,
+	0x12, 0x50, 0x0a, 0x07, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28,
+	0x0b, 0x32, 0x36, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62,
+	0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x6a, 0x6f, 0x62, 0x5f, 0x6d, 0x61,
+	0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x76, 0x31, 0x2e, 0x4d, 0x65, 0x74, 0x72,
+	0x69, 0x63, 0x52, 0x65, 0x73, 0x75, 0x6c, 0x74, 0x73, 0x52, 0x07, 0x6d, 0x65, 0x74, 0x72, 0x69,
+	0x63, 0x73, 0x22, 0xb1, 0x01, 0x0a, 0x0d, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x52, 0x65, 0x73,
+	0x75, 0x6c, 0x74, 0x73, 0x12, 0x4f, 0x0a, 0x09, 0x61, 0x74, 0x74, 0x65, 0x6d, 0x70, 0x74, 0x65,
+	0x64, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70,
+	0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e,
+	0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x4d, 0x6f, 0x6e, 0x69,
+	0x74, 0x6f, 0x72, 0x69, 0x6e, 0x67, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x09, 0x61, 0x74, 0x74, 0x65,
+	0x6d, 0x70, 0x74, 0x65, 0x64, 0x12, 0x4f, 0x0a, 0x09, 0x63, 0x6f, 0x6d, 0x6d, 0x69, 0x74, 0x74,
+	0x65, 0x64, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x31, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61,
+	0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c,
+	0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x4d, 0x6f, 0x6e,
+	0x69, 0x74, 0x6f, 0x72, 0x69, 0x6e, 0x67, 0x49, 0x6e, 0x66, 0x6f, 0x52, 0x09, 0x63, 0x6f, 0x6d,
+	0x6d, 0x69, 0x74, 0x74, 0x65, 0x64, 0x22, 0x20, 0x0a, 0x1e, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69,
+	0x62, 0x65, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e,
+	0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x22, 0x65, 0x0a, 0x12, 0x50, 0x69, 0x70, 0x65,
+	0x6c, 0x69, 0x6e, 0x65, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x79, 0x70, 0x65, 0x22, 0x4f,
+	0x0a, 0x04, 0x45, 0x6e, 0x75, 0x6d, 0x12, 0x0a, 0x0a, 0x06, 0x53, 0x54, 0x52, 0x49, 0x4e, 0x47,
+	0x10, 0x00, 0x12, 0x0b, 0x0a, 0x07, 0x42, 0x4f, 0x4f, 0x4c, 0x45, 0x41, 0x4e, 0x10, 0x01, 0x12,
+	0x0b, 0x0a, 0x07, 0x49, 0x4e, 0x54, 0x45, 0x47, 0x45, 0x52, 0x10, 0x02, 0x12, 0x0a, 0x0a, 0x06,
+	0x4e, 0x55, 0x4d, 0x42, 0x45, 0x52, 0x10, 0x03, 0x12, 0x09, 0x0a, 0x05, 0x41, 0x52, 0x52, 0x41,
+	0x59, 0x10, 0x04, 0x12, 0x0a, 0x0a, 0x06, 0x4f, 0x42, 0x4a, 0x45, 0x43, 0x54, 0x10, 0x05, 0x22,
+	0xe1, 0x01, 0x0a, 0x18, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x4f, 0x70, 0x74, 0x69,
+	0x6f, 0x6e, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x12, 0x12, 0x0a, 0x04,
+	0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65,
+	0x12, 0x54, 0x0a, 0x04, 0x74, 0x79, 0x70, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x40,
+	0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d,
+	0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x6a, 0x6f, 0x62, 0x5f, 0x6d, 0x61, 0x6e, 0x61, 0x67,
+	0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e,
+	0x65, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x54, 0x79, 0x70, 0x65, 0x2e, 0x45, 0x6e, 0x75, 0x6d,
+	0x52, 0x04, 0x74, 0x79, 0x70, 0x65, 0x12, 0x20, 0x0a, 0x0b, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69,
+	0x70, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x64, 0x65, 0x73,
+	0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x23, 0x0a, 0x0d, 0x64, 0x65, 0x66, 0x61,
+	0x75, 0x6c, 0x74, 0x5f, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52,
+	0x0c, 0x64, 0x65, 0x66, 0x61, 0x75, 0x6c, 0x74, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x14, 0x0a,
+	0x05, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x67, 0x72,
+	0x6f, 0x75, 0x70, 0x22, 0x7e, 0x0a, 0x1f, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x50,
+	0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65,
+	0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x5b, 0x0a, 0x07, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e,
+	0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x41, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70,
+	0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e,
+	0x6a, 0x6f, 0x62, 0x5f, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x76,
+	0x31, 0x2e, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e,
+	0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x52, 0x07, 0x6f, 0x70, 0x74, 0x69,
+	0x6f, 0x6e, 0x73, 0x32, 0xf6, 0x0a, 0x0a, 0x0a, 0x4a, 0x6f, 0x62, 0x53, 0x65, 0x72, 0x76, 0x69,
+	0x63, 0x65, 0x12, 0x82, 0x01, 0x0a, 0x07, 0x50, 0x72, 0x65, 0x70, 0x61, 0x72, 0x65, 0x12, 0x3a,
+	0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d,
+	0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x6a, 0x6f, 0x62, 0x5f, 0x6d, 0x61, 0x6e, 0x61, 0x67,
+	0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x72, 0x65, 0x70, 0x61, 0x72, 0x65,
+	0x4a, 0x6f, 0x62, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x3b, 0x2e, 0x6f, 0x72, 0x67,
+	0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64,
+	0x65, 0x6c, 0x2e, 0x6a, 0x6f, 0x62, 0x5f, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e,
+	0x74, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x72, 0x65, 0x70, 0x61, 0x72, 0x65, 0x4a, 0x6f, 0x62, 0x52,
+	0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x76, 0x0a, 0x03, 0x52, 0x75, 0x6e, 0x12, 0x36,
+	0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d,
+	0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x6a, 0x6f, 0x62, 0x5f, 0x6d, 0x61, 0x6e, 0x61, 0x67,
+	0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x75, 0x6e, 0x4a, 0x6f, 0x62, 0x52,
+	0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x37, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61,
+	0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x6a,
+	0x6f, 0x62, 0x5f, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x76, 0x31,
+	0x2e, 0x52, 0x75, 0x6e, 0x4a, 0x6f, 0x62, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12,
+	0x7c, 0x0a, 0x07, 0x47, 0x65, 0x74, 0x4a, 0x6f, 0x62, 0x73, 0x12, 0x37, 0x2e, 0x6f, 0x72, 0x67,
+	0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64,
+	0x65, 0x6c, 0x2e, 0x6a, 0x6f, 0x62, 0x5f, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e,
+	0x74, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, 0x4a, 0x6f, 0x62, 0x73, 0x52, 0x65, 0x71, 0x75,
+	0x65, 0x73, 0x74, 0x1a, 0x38, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65,
+	0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x6a, 0x6f, 0x62, 0x5f,
+	0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65,
+	0x74, 0x4a, 0x6f, 0x62, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x7f, 0x0a,
+	0x08, 0x47, 0x65, 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, 0x12, 0x3b, 0x2e, 0x6f, 0x72, 0x67, 0x2e,
+	0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65,
+	0x6c, 0x2e, 0x6a, 0x6f, 0x62, 0x5f, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74,
+	0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52,
+	0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x36, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61,
+	0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x6a,
+	0x6f, 0x62, 0x5f, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x76, 0x31,
+	0x2e, 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, 0x74, 0x65, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x12, 0x8e,
+	0x01, 0x0a, 0x0b, 0x47, 0x65, 0x74, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x12, 0x3e,
+	0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d,
+	0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x6a, 0x6f, 0x62, 0x5f, 0x6d, 0x61, 0x6e, 0x61, 0x67,
+	0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, 0x4a, 0x6f, 0x62, 0x50,
+	0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x3f,
+	0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d,
+	0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x6a, 0x6f, 0x62, 0x5f, 0x6d, 0x61, 0x6e, 0x61, 0x67,
+	0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, 0x4a, 0x6f, 0x62, 0x50,
+	0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12,
+	0x7f, 0x0a, 0x06, 0x43, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x12, 0x39, 0x2e, 0x6f, 0x72, 0x67, 0x2e,
+	0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65,
+	0x6c, 0x2e, 0x6a, 0x6f, 0x62, 0x5f, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74,
+	0x2e, 0x76, 0x31, 0x2e, 0x43, 0x61, 0x6e, 0x63, 0x65, 0x6c, 0x4a, 0x6f, 0x62, 0x52, 0x65, 0x71,
+	0x75, 0x65, 0x73, 0x74, 0x1a, 0x3a, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68,
+	0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x6a, 0x6f, 0x62,
+	0x5f, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x76, 0x31, 0x2e, 0x43,
+	0x61, 0x6e, 0x63, 0x65, 0x6c, 0x4a, 0x6f, 0x62, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65,
+	0x12, 0x87, 0x01, 0x0a, 0x0e, 0x47, 0x65, 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, 0x53, 0x74, 0x72,
+	0x65, 0x61, 0x6d, 0x12, 0x3b, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65,
+	0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x6a, 0x6f, 0x62, 0x5f,
+	0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65,
+	0x74, 0x4a, 0x6f, 0x62, 0x53, 0x74, 0x61, 0x74, 0x65, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74,
+	0x1a, 0x36, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65,
+	0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x6a, 0x6f, 0x62, 0x5f, 0x6d, 0x61, 0x6e,
+	0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x76, 0x31, 0x2e, 0x4a, 0x6f, 0x62, 0x53, 0x74,
+	0x61, 0x74, 0x65, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x30, 0x01, 0x12, 0x8f, 0x01, 0x0a, 0x10, 0x47,
+	0x65, 0x74, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x12,
+	0x3b, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61,
+	0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x6a, 0x6f, 0x62, 0x5f, 0x6d, 0x61, 0x6e, 0x61,
+	0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x76, 0x31, 0x2e, 0x4a, 0x6f, 0x62, 0x4d, 0x65, 0x73,
+	0x73, 0x61, 0x67, 0x65, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x3c, 0x2e, 0x6f,
+	0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d,
+	0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x6a, 0x6f, 0x62, 0x5f, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d,
+	0x65, 0x6e, 0x74, 0x2e, 0x76, 0x31, 0x2e, 0x4a, 0x6f, 0x62, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67,
+	0x65, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x30, 0x01, 0x12, 0x8e, 0x01, 0x0a,
+	0x0d, 0x47, 0x65, 0x74, 0x4a, 0x6f, 0x62, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x12, 0x3d,
+	0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d,
+	0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x6a, 0x6f, 0x62, 0x5f, 0x6d, 0x61, 0x6e, 0x61, 0x67,
+	0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, 0x4a, 0x6f, 0x62, 0x4d,
+	0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x3e, 0x2e,
+	0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e,
+	0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x6a, 0x6f, 0x62, 0x5f, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65,
+	0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x76, 0x31, 0x2e, 0x47, 0x65, 0x74, 0x4a, 0x6f, 0x62, 0x4d, 0x65,
+	0x74, 0x72, 0x69, 0x63, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0xac, 0x01,
+	0x0a, 0x17, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69,
+	0x6e, 0x65, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x47, 0x2e, 0x6f, 0x72, 0x67, 0x2e,
+	0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65,
+	0x6c, 0x2e, 0x6a, 0x6f, 0x62, 0x5f, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74,
+	0x2e, 0x76, 0x31, 0x2e, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x62, 0x65, 0x50, 0x69, 0x70, 0x65,
+	0x6c, 0x69, 0x6e, 0x65, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65,
+	0x73, 0x74, 0x1a, 0x48, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e,
+	0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x6a, 0x6f, 0x62, 0x5f, 0x6d,
+	0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x76, 0x31, 0x2e, 0x44, 0x65, 0x73,
+	0x63, 0x72, 0x69, 0x62, 0x65, 0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x4f, 0x70, 0x74,
+	0x69, 0x6f, 0x6e, 0x73, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x42, 0x81, 0x01, 0x0a,
+	0x26, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d,
+	0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x6a, 0x6f, 0x62, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65,
+	0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x76, 0x31, 0x42, 0x06, 0x4a, 0x6f, 0x62, 0x41, 0x70, 0x69, 0x5a,
+	0x4f, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x61, 0x70, 0x61, 0x63,
+	0x68, 0x65, 0x2f, 0x62, 0x65, 0x61, 0x6d, 0x2f, 0x73, 0x64, 0x6b, 0x73, 0x2f, 0x67, 0x6f, 0x2f,
+	0x70, 0x6b, 0x67, 0x2f, 0x62, 0x65, 0x61, 0x6d, 0x2f, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2f, 0x6a,
+	0x6f, 0x62, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x76, 0x31, 0x3b,
+	0x6a, 0x6f, 0x62, 0x6d, 0x61, 0x6e, 0x61, 0x67, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x76, 0x31,
+	0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
 }
 
-func init() { proto.RegisterFile("beam_job_api.proto", fileDescriptor_97c7b84f742157ae) }
+var (
+	file_beam_job_api_proto_rawDescOnce sync.Once
+	file_beam_job_api_proto_rawDescData = file_beam_job_api_proto_rawDesc
+)
 
-var fileDescriptor_97c7b84f742157ae = []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,
+func file_beam_job_api_proto_rawDescGZIP() []byte {
+	file_beam_job_api_proto_rawDescOnce.Do(func() {
+		file_beam_job_api_proto_rawDescData = protoimpl.X.CompressGZIP(file_beam_job_api_proto_rawDescData)
+	})
+	return file_beam_job_api_proto_rawDescData
+}
+
+var file_beam_job_api_proto_enumTypes = make([]protoimpl.EnumInfo, 3)
+var file_beam_job_api_proto_msgTypes = make([]protoimpl.MessageInfo, 24)
+var file_beam_job_api_proto_goTypes = []interface{}{
+	(JobMessage_MessageImportance)(0),        // 0: org.apache.beam.model.job_management.v1.JobMessage.MessageImportance
+	(JobState_Enum)(0),                       // 1: org.apache.beam.model.job_management.v1.JobState.Enum
+	(PipelineOptionType_Enum)(0),             // 2: org.apache.beam.model.job_management.v1.PipelineOptionType.Enum
+	(*PrepareJobRequest)(nil),                // 3: org.apache.beam.model.job_management.v1.PrepareJobRequest
+	(*PrepareJobResponse)(nil),               // 4: org.apache.beam.model.job_management.v1.PrepareJobResponse
+	(*RunJobRequest)(nil),                    // 5: org.apache.beam.model.job_management.v1.RunJobRequest
+	(*RunJobResponse)(nil),                   // 6: org.apache.beam.model.job_management.v1.RunJobResponse
+	(*CancelJobRequest)(nil),                 // 7: org.apache.beam.model.job_management.v1.CancelJobRequest
+	(*CancelJobResponse)(nil),                // 8: org.apache.beam.model.job_management.v1.CancelJobResponse
+	(*JobInfo)(nil),                          // 9: org.apache.beam.model.job_management.v1.JobInfo
+	(*GetJobsRequest)(nil),                   // 10: org.apache.beam.model.job_management.v1.GetJobsRequest
+	(*GetJobsResponse)(nil),                  // 11: org.apache.beam.model.job_management.v1.GetJobsResponse
+	(*GetJobStateRequest)(nil),               // 12: org.apache.beam.model.job_management.v1.GetJobStateRequest
+	(*JobStateEvent)(nil),                    // 13: org.apache.beam.model.job_management.v1.JobStateEvent
+	(*GetJobPipelineRequest)(nil),            // 14: org.apache.beam.model.job_management.v1.GetJobPipelineRequest
+	(*GetJobPipelineResponse)(nil),           // 15: org.apache.beam.model.job_management.v1.GetJobPipelineResponse
+	(*JobMessagesRequest)(nil),               // 16: org.apache.beam.model.job_management.v1.JobMessagesRequest
+	(*JobMessage)(nil),                       // 17: org.apache.beam.model.job_management.v1.JobMessage
+	(*JobMessagesResponse)(nil),              // 18: org.apache.beam.model.job_management.v1.JobMessagesResponse
+	(*JobState)(nil),                         // 19: org.apache.beam.model.job_management.v1.JobState
+	(*GetJobMetricsRequest)(nil),             // 20: org.apache.beam.model.job_management.v1.GetJobMetricsRequest
+	(*GetJobMetricsResponse)(nil),            // 21: org.apache.beam.model.job_management.v1.GetJobMetricsResponse
+	(*MetricResults)(nil),                    // 22: org.apache.beam.model.job_management.v1.MetricResults
+	(*DescribePipelineOptionsRequest)(nil),   // 23: org.apache.beam.model.job_management.v1.DescribePipelineOptionsRequest
+	(*PipelineOptionType)(nil),               // 24: org.apache.beam.model.job_management.v1.PipelineOptionType
+	(*PipelineOptionDescriptor)(nil),         // 25: org.apache.beam.model.job_management.v1.PipelineOptionDescriptor
+	(*DescribePipelineOptionsResponse)(nil),  // 26: org.apache.beam.model.job_management.v1.DescribePipelineOptionsResponse
+	(*pipeline_v1.Pipeline)(nil),             // 27: org.apache.beam.model.pipeline.v1.Pipeline
+	(*_struct.Struct)(nil),                   // 28: google.protobuf.Struct
+	(*pipeline_v1.ApiServiceDescriptor)(nil), // 29: org.apache.beam.model.pipeline.v1.ApiServiceDescriptor
+	(*timestamp.Timestamp)(nil),              // 30: google.protobuf.Timestamp
+	(*pipeline_v1.MonitoringInfo)(nil),       // 31: org.apache.beam.model.pipeline.v1.MonitoringInfo
+}
+var file_beam_job_api_proto_depIdxs = []int32{
+	27, // 0: org.apache.beam.model.job_management.v1.PrepareJobRequest.pipeline:type_name -> org.apache.beam.model.pipeline.v1.Pipeline
+	28, // 1: org.apache.beam.model.job_management.v1.PrepareJobRequest.pipeline_options:type_name -> google.protobuf.Struct
+	29, // 2: org.apache.beam.model.job_management.v1.PrepareJobResponse.artifact_staging_endpoint:type_name -> org.apache.beam.model.pipeline.v1.ApiServiceDescriptor
+	1,  // 3: org.apache.beam.model.job_management.v1.CancelJobResponse.state:type_name -> org.apache.beam.model.job_management.v1.JobState.Enum
+	28, // 4: org.apache.beam.model.job_management.v1.JobInfo.pipeline_options:type_name -> google.protobuf.Struct
+	1,  // 5: org.apache.beam.model.job_management.v1.JobInfo.state:type_name -> org.apache.beam.model.job_management.v1.JobState.Enum
+	9,  // 6: org.apache.beam.model.job_management.v1.GetJobsResponse.job_info:type_name -> org.apache.beam.model.job_management.v1.JobInfo
+	1,  // 7: org.apache.beam.model.job_management.v1.JobStateEvent.state:type_name -> org.apache.beam.model.job_management.v1.JobState.Enum
+	30, // 8: org.apache.beam.model.job_management.v1.JobStateEvent.timestamp:type_name -> google.protobuf.Timestamp
+	27, // 9: org.apache.beam.model.job_management.v1.GetJobPipelineResponse.pipeline:type_name -> org.apache.beam.model.pipeline.v1.Pipeline
+	0,  // 10: org.apache.beam.model.job_management.v1.JobMessage.importance:type_name -> org.apache.beam.model.job_management.v1.JobMessage.MessageImportance
+	17, // 11: org.apache.beam.model.job_management.v1.JobMessagesResponse.message_response:type_name -> org.apache.beam.model.job_management.v1.JobMessage
+	13, // 12: org.apache.beam.model.job_management.v1.JobMessagesResponse.state_response:type_name -> org.apache.beam.model.job_management.v1.JobStateEvent
+	22, // 13: org.apache.beam.model.job_management.v1.GetJobMetricsResponse.metrics:type_name -> org.apache.beam.model.job_management.v1.MetricResults
+	31, // 14: org.apache.beam.model.job_management.v1.MetricResults.attempted:type_name -> org.apache.beam.model.pipeline.v1.MonitoringInfo
+	31, // 15: org.apache.beam.model.job_management.v1.MetricResults.committed:type_name -> org.apache.beam.model.pipeline.v1.MonitoringInfo
+	2,  // 16: org.apache.beam.model.job_management.v1.PipelineOptionDescriptor.type:type_name -> org.apache.beam.model.job_management.v1.PipelineOptionType.Enum
+	25, // 17: org.apache.beam.model.job_management.v1.DescribePipelineOptionsResponse.options:type_name -> org.apache.beam.model.job_management.v1.PipelineOptionDescriptor
+	3,  // 18: org.apache.beam.model.job_management.v1.JobService.Prepare:input_type -> org.apache.beam.model.job_management.v1.PrepareJobRequest
+	5,  // 19: org.apache.beam.model.job_management.v1.JobService.Run:input_type -> org.apache.beam.model.job_management.v1.RunJobRequest
+	10, // 20: org.apache.beam.model.job_management.v1.JobService.GetJobs:input_type -> org.apache.beam.model.job_management.v1.GetJobsRequest
+	12, // 21: org.apache.beam.model.job_management.v1.JobService.GetState:input_type -> org.apache.beam.model.job_management.v1.GetJobStateRequest
+	14, // 22: org.apache.beam.model.job_management.v1.JobService.GetPipeline:input_type -> org.apache.beam.model.job_management.v1.GetJobPipelineRequest
+	7,  // 23: org.apache.beam.model.job_management.v1.JobService.Cancel:input_type -> org.apache.beam.model.job_management.v1.CancelJobRequest
+	12, // 24: org.apache.beam.model.job_management.v1.JobService.GetStateStream:input_type -> org.apache.beam.model.job_management.v1.GetJobStateRequest
+	16, // 25: org.apache.beam.model.job_management.v1.JobService.GetMessageStream:input_type -> org.apache.beam.model.job_management.v1.JobMessagesRequest
+	20, // 26: org.apache.beam.model.job_management.v1.JobService.GetJobMetrics:input_type -> org.apache.beam.model.job_management.v1.GetJobMetricsRequest
+	23, // 27: org.apache.beam.model.job_management.v1.JobService.DescribePipelineOptions:input_type -> org.apache.beam.model.job_management.v1.DescribePipelineOptionsRequest
+	4,  // 28: org.apache.beam.model.job_management.v1.JobService.Prepare:output_type -> org.apache.beam.model.job_management.v1.PrepareJobResponse
+	6,  // 29: org.apache.beam.model.job_management.v1.JobService.Run:output_type -> org.apache.beam.model.job_management.v1.RunJobResponse
+	11, // 30: org.apache.beam.model.job_management.v1.JobService.GetJobs:output_type -> org.apache.beam.model.job_management.v1.GetJobsResponse
+	13, // 31: org.apache.beam.model.job_management.v1.JobService.GetState:output_type -> org.apache.beam.model.job_management.v1.JobStateEvent
+	15, // 32: org.apache.beam.model.job_management.v1.JobService.GetPipeline:output_type -> org.apache.beam.model.job_management.v1.GetJobPipelineResponse
+	8,  // 33: org.apache.beam.model.job_management.v1.JobService.Cancel:output_type -> org.apache.beam.model.job_management.v1.CancelJobResponse
+	13, // 34: org.apache.beam.model.job_management.v1.JobService.GetStateStream:output_type -> org.apache.beam.model.job_management.v1.JobStateEvent
+	18, // 35: org.apache.beam.model.job_management.v1.JobService.GetMessageStream:output_type -> org.apache.beam.model.job_management.v1.JobMessagesResponse
+	21, // 36: org.apache.beam.model.job_management.v1.JobService.GetJobMetrics:output_type -> org.apache.beam.model.job_management.v1.GetJobMetricsResponse
+	26, // 37: org.apache.beam.model.job_management.v1.JobService.DescribePipelineOptions:output_type -> org.apache.beam.model.job_management.v1.DescribePipelineOptionsResponse
+	28, // [28:38] is the sub-list for method output_type
+	18, // [18:28] is the sub-list for method input_type
+	18, // [18:18] is the sub-list for extension type_name
+	18, // [18:18] is the sub-list for extension extendee
+	0,  // [0:18] is the sub-list for field type_name
+}
+
+func init() { file_beam_job_api_proto_init() }
+func file_beam_job_api_proto_init() {
+	if File_beam_job_api_proto != nil {
+		return
+	}
+	if !protoimpl.UnsafeEnabled {
+		file_beam_job_api_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*PrepareJobRequest); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_job_api_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*PrepareJobResponse); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_job_api_proto_msgTypes[2].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*RunJobRequest); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_job_api_proto_msgTypes[3].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*RunJobResponse); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_job_api_proto_msgTypes[4].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*CancelJobRequest); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_job_api_proto_msgTypes[5].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*CancelJobResponse); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_job_api_proto_msgTypes[6].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*JobInfo); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_job_api_proto_msgTypes[7].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*GetJobsRequest); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_job_api_proto_msgTypes[8].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*GetJobsResponse); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_job_api_proto_msgTypes[9].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*GetJobStateRequest); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_job_api_proto_msgTypes[10].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*JobStateEvent); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_job_api_proto_msgTypes[11].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*GetJobPipelineRequest); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_job_api_proto_msgTypes[12].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*GetJobPipelineResponse); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_job_api_proto_msgTypes[13].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*JobMessagesRequest); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_job_api_proto_msgTypes[14].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*JobMessage); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_job_api_proto_msgTypes[15].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*JobMessagesResponse); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_job_api_proto_msgTypes[16].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*JobState); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_job_api_proto_msgTypes[17].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*GetJobMetricsRequest); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_job_api_proto_msgTypes[18].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*GetJobMetricsResponse); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_job_api_proto_msgTypes[19].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*MetricResults); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_job_api_proto_msgTypes[20].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*DescribePipelineOptionsRequest); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_job_api_proto_msgTypes[21].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*PipelineOptionType); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_job_api_proto_msgTypes[22].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*PipelineOptionDescriptor); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_job_api_proto_msgTypes[23].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*DescribePipelineOptionsResponse); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+	}
+	file_beam_job_api_proto_msgTypes[15].OneofWrappers = []interface{}{
+		(*JobMessagesResponse_MessageResponse)(nil),
+		(*JobMessagesResponse_StateResponse)(nil),
+	}
+	type x struct{}
+	out := protoimpl.TypeBuilder{
+		File: protoimpl.DescBuilder{
+			GoPackagePath: reflect.TypeOf(x{}).PkgPath(),
+			RawDescriptor: file_beam_job_api_proto_rawDesc,
+			NumEnums:      3,
+			NumMessages:   24,
+			NumExtensions: 0,
+			NumServices:   1,
+		},
+		GoTypes:           file_beam_job_api_proto_goTypes,
+		DependencyIndexes: file_beam_job_api_proto_depIdxs,
+		EnumInfos:         file_beam_job_api_proto_enumTypes,
+		MessageInfos:      file_beam_job_api_proto_msgTypes,
+	}.Build()
+	File_beam_job_api_proto = out.File
+	file_beam_job_api_proto_rawDesc = nil
+	file_beam_job_api_proto_goTypes = nil
+	file_beam_job_api_proto_depIdxs = nil
 }
 
 // Reference imports to suppress errors if they are not otherwise used.
 var _ context.Context
-var _ grpc.ClientConn
+var _ grpc.ClientConnInterface
 
 // 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
+const _ = grpc.SupportPackageIsVersion6
 
 // JobServiceClient is the client API for JobService service.
 //
@@ -1473,10 +2327,10 @@
 }
 
 type jobServiceClient struct {
-	cc *grpc.ClientConn
+	cc grpc.ClientConnInterface
 }
 
-func NewJobServiceClient(cc *grpc.ClientConn) JobServiceClient {
+func NewJobServiceClient(cc grpc.ClientConnInterface) JobServiceClient {
 	return &jobServiceClient{cc}
 }
 
@@ -1645,34 +2499,34 @@
 type UnimplementedJobServiceServer struct {
 }
 
-func (*UnimplementedJobServiceServer) Prepare(ctx context.Context, req *PrepareJobRequest) (*PrepareJobResponse, error) {
+func (*UnimplementedJobServiceServer) Prepare(context.Context, *PrepareJobRequest) (*PrepareJobResponse, error) {
 	return nil, status.Errorf(codes.Unimplemented, "method Prepare not implemented")
 }
-func (*UnimplementedJobServiceServer) Run(ctx context.Context, req *RunJobRequest) (*RunJobResponse, error) {
+func (*UnimplementedJobServiceServer) Run(context.Context, *RunJobRequest) (*RunJobResponse, error) {
 	return nil, status.Errorf(codes.Unimplemented, "method Run not implemented")
 }
-func (*UnimplementedJobServiceServer) GetJobs(ctx context.Context, req *GetJobsRequest) (*GetJobsResponse, error) {
+func (*UnimplementedJobServiceServer) GetJobs(context.Context, *GetJobsRequest) (*GetJobsResponse, error) {
 	return nil, status.Errorf(codes.Unimplemented, "method GetJobs not implemented")
 }
-func (*UnimplementedJobServiceServer) GetState(ctx context.Context, req *GetJobStateRequest) (*JobStateEvent, error) {
+func (*UnimplementedJobServiceServer) GetState(context.Context, *GetJobStateRequest) (*JobStateEvent, error) {
 	return nil, status.Errorf(codes.Unimplemented, "method GetState not implemented")
 }
-func (*UnimplementedJobServiceServer) GetPipeline(ctx context.Context, req *GetJobPipelineRequest) (*GetJobPipelineResponse, error) {
+func (*UnimplementedJobServiceServer) GetPipeline(context.Context, *GetJobPipelineRequest) (*GetJobPipelineResponse, error) {
 	return nil, status.Errorf(codes.Unimplemented, "method GetPipeline not implemented")
 }
-func (*UnimplementedJobServiceServer) Cancel(ctx context.Context, req *CancelJobRequest) (*CancelJobResponse, error) {
+func (*UnimplementedJobServiceServer) Cancel(context.Context, *CancelJobRequest) (*CancelJobResponse, error) {
 	return nil, status.Errorf(codes.Unimplemented, "method Cancel not implemented")
 }
-func (*UnimplementedJobServiceServer) GetStateStream(req *GetJobStateRequest, srv JobService_GetStateStreamServer) error {
+func (*UnimplementedJobServiceServer) GetStateStream(*GetJobStateRequest, JobService_GetStateStreamServer) error {
 	return status.Errorf(codes.Unimplemented, "method GetStateStream not implemented")
 }
-func (*UnimplementedJobServiceServer) GetMessageStream(req *JobMessagesRequest, srv JobService_GetMessageStreamServer) error {
+func (*UnimplementedJobServiceServer) GetMessageStream(*JobMessagesRequest, JobService_GetMessageStreamServer) error {
 	return status.Errorf(codes.Unimplemented, "method GetMessageStream not implemented")
 }
-func (*UnimplementedJobServiceServer) GetJobMetrics(ctx context.Context, req *GetJobMetricsRequest) (*GetJobMetricsResponse, error) {
+func (*UnimplementedJobServiceServer) GetJobMetrics(context.Context, *GetJobMetricsRequest) (*GetJobMetricsResponse, error) {
 	return nil, status.Errorf(codes.Unimplemented, "method GetJobMetrics not implemented")
 }
-func (*UnimplementedJobServiceServer) DescribePipelineOptions(ctx context.Context, req *DescribePipelineOptionsRequest) (*DescribePipelineOptionsResponse, error) {
+func (*UnimplementedJobServiceServer) DescribePipelineOptions(context.Context, *DescribePipelineOptionsRequest) (*DescribePipelineOptionsResponse, error) {
 	return nil, status.Errorf(codes.Unimplemented, "method DescribePipelineOptions not implemented")
 }
 
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 5f8f250..68c777b 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
@@ -1,32 +1,50 @@
+//
+// 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.
+
+//
+// Protocol Buffers describing the Runner API, which is the runner-independent,
+// SDK-independent definition of the Beam model.
+
 // Code generated by protoc-gen-go. DO NOT EDIT.
+// versions:
+// 	protoc-gen-go v1.25.0-devel
+// 	protoc        v3.13.0
 // source: beam_runner_api.proto
 
 package pipeline_v1
 
 import (
 	context "context"
-	fmt "fmt"
-	math "math"
-
-	proto "github.com/golang/protobuf/proto"
 	descriptor "github.com/golang/protobuf/protoc-gen-go/descriptor"
-	_ "github.com/golang/protobuf/ptypes/any"
-	_ "github.com/golang/protobuf/ptypes/timestamp"
 	grpc "google.golang.org/grpc"
 	codes "google.golang.org/grpc/codes"
 	status "google.golang.org/grpc/status"
+	protoreflect "google.golang.org/protobuf/reflect/protoreflect"
+	protoimpl "google.golang.org/protobuf/runtime/protoimpl"
+	reflect "reflect"
+	sync "sync"
 )
 
-// Reference imports to suppress errors if they are not otherwise used.
-var _ = proto.Marshal
-var _ = fmt.Errorf
-var _ = math.Inf
-
-// This is a compile-time assertion to ensure that this generated file
-// is compatible with the proto package it is being compiled against.
-// A compilation error at this line likely means your copy of the
-// proto package needs to be updated.
-const _ = proto.ProtoPackageIsVersion3 // please upgrade the proto package
+const (
+	// Verify that this generated code is sufficiently up-to-date.
+	_ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion)
+	// Verify that runtime/protoimpl is sufficiently up-to-date.
+	_ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20)
+)
 
 type BeamConstants_Constants int32
 
@@ -47,24 +65,45 @@
 	BeamConstants_GLOBAL_WINDOW_MAX_TIMESTAMP_MILLIS BeamConstants_Constants = 2
 )
 
-var BeamConstants_Constants_name = map[int32]string{
-	0: "MIN_TIMESTAMP_MILLIS",
-	1: "MAX_TIMESTAMP_MILLIS",
-	2: "GLOBAL_WINDOW_MAX_TIMESTAMP_MILLIS",
-}
+// Enum value maps for BeamConstants_Constants.
+var (
+	BeamConstants_Constants_name = map[int32]string{
+		0: "MIN_TIMESTAMP_MILLIS",
+		1: "MAX_TIMESTAMP_MILLIS",
+		2: "GLOBAL_WINDOW_MAX_TIMESTAMP_MILLIS",
+	}
+	BeamConstants_Constants_value = map[string]int32{
+		"MIN_TIMESTAMP_MILLIS":               0,
+		"MAX_TIMESTAMP_MILLIS":               1,
+		"GLOBAL_WINDOW_MAX_TIMESTAMP_MILLIS": 2,
+	}
+)
 
-var BeamConstants_Constants_value = map[string]int32{
-	"MIN_TIMESTAMP_MILLIS":               0,
-	"MAX_TIMESTAMP_MILLIS":               1,
-	"GLOBAL_WINDOW_MAX_TIMESTAMP_MILLIS": 2,
+func (x BeamConstants_Constants) Enum() *BeamConstants_Constants {
+	p := new(BeamConstants_Constants)
+	*p = x
+	return p
 }
 
 func (x BeamConstants_Constants) String() string {
-	return proto.EnumName(BeamConstants_Constants_name, int32(x))
+	return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x))
 }
 
+func (BeamConstants_Constants) Descriptor() protoreflect.EnumDescriptor {
+	return file_beam_runner_api_proto_enumTypes[0].Descriptor()
+}
+
+func (BeamConstants_Constants) Type() protoreflect.EnumType {
+	return &file_beam_runner_api_proto_enumTypes[0]
+}
+
+func (x BeamConstants_Constants) Number() protoreflect.EnumNumber {
+	return protoreflect.EnumNumber(x)
+}
+
+// Deprecated: Use BeamConstants_Constants.Descriptor instead.
 func (BeamConstants_Constants) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_cf57597c3a9659a9, []int{0, 0}
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{0, 0}
 }
 
 // Primitive transforms may not specify composite sub-transforms.
@@ -167,34 +206,55 @@
 	StandardPTransforms_MERGE_WINDOWS StandardPTransforms_Primitives = 7
 )
 
-var StandardPTransforms_Primitives_name = map[int32]string{
-	0: "PAR_DO",
-	1: "FLATTEN",
-	2: "GROUP_BY_KEY",
-	3: "IMPULSE",
-	4: "ASSIGN_WINDOWS",
-	5: "TEST_STREAM",
-	6: "MAP_WINDOWS",
-	7: "MERGE_WINDOWS",
-}
+// Enum value maps for StandardPTransforms_Primitives.
+var (
+	StandardPTransforms_Primitives_name = map[int32]string{
+		0: "PAR_DO",
+		1: "FLATTEN",
+		2: "GROUP_BY_KEY",
+		3: "IMPULSE",
+		4: "ASSIGN_WINDOWS",
+		5: "TEST_STREAM",
+		6: "MAP_WINDOWS",
+		7: "MERGE_WINDOWS",
+	}
+	StandardPTransforms_Primitives_value = map[string]int32{
+		"PAR_DO":         0,
+		"FLATTEN":        1,
+		"GROUP_BY_KEY":   2,
+		"IMPULSE":        3,
+		"ASSIGN_WINDOWS": 4,
+		"TEST_STREAM":    5,
+		"MAP_WINDOWS":    6,
+		"MERGE_WINDOWS":  7,
+	}
+)
 
-var StandardPTransforms_Primitives_value = map[string]int32{
-	"PAR_DO":         0,
-	"FLATTEN":        1,
-	"GROUP_BY_KEY":   2,
-	"IMPULSE":        3,
-	"ASSIGN_WINDOWS": 4,
-	"TEST_STREAM":    5,
-	"MAP_WINDOWS":    6,
-	"MERGE_WINDOWS":  7,
+func (x StandardPTransforms_Primitives) Enum() *StandardPTransforms_Primitives {
+	p := new(StandardPTransforms_Primitives)
+	*p = x
+	return p
 }
 
 func (x StandardPTransforms_Primitives) String() string {
-	return proto.EnumName(StandardPTransforms_Primitives_name, int32(x))
+	return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x))
 }
 
+func (StandardPTransforms_Primitives) Descriptor() protoreflect.EnumDescriptor {
+	return file_beam_runner_api_proto_enumTypes[1].Descriptor()
+}
+
+func (StandardPTransforms_Primitives) Type() protoreflect.EnumType {
+	return &file_beam_runner_api_proto_enumTypes[1]
+}
+
+func (x StandardPTransforms_Primitives) Number() protoreflect.EnumNumber {
+	return protoreflect.EnumNumber(x)
+}
+
+// Deprecated: Use StandardPTransforms_Primitives.Descriptor instead.
 func (StandardPTransforms_Primitives) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_cf57597c3a9659a9, []int{4, 0}
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{4, 0}
 }
 
 type StandardPTransforms_DeprecatedPrimitives int32
@@ -208,22 +268,43 @@
 	StandardPTransforms_CREATE_VIEW StandardPTransforms_DeprecatedPrimitives = 1
 )
 
-var StandardPTransforms_DeprecatedPrimitives_name = map[int32]string{
-	0: "READ",
-	1: "CREATE_VIEW",
-}
+// Enum value maps for StandardPTransforms_DeprecatedPrimitives.
+var (
+	StandardPTransforms_DeprecatedPrimitives_name = map[int32]string{
+		0: "READ",
+		1: "CREATE_VIEW",
+	}
+	StandardPTransforms_DeprecatedPrimitives_value = map[string]int32{
+		"READ":        0,
+		"CREATE_VIEW": 1,
+	}
+)
 
-var StandardPTransforms_DeprecatedPrimitives_value = map[string]int32{
-	"READ":        0,
-	"CREATE_VIEW": 1,
+func (x StandardPTransforms_DeprecatedPrimitives) Enum() *StandardPTransforms_DeprecatedPrimitives {
+	p := new(StandardPTransforms_DeprecatedPrimitives)
+	*p = x
+	return p
 }
 
 func (x StandardPTransforms_DeprecatedPrimitives) String() string {
-	return proto.EnumName(StandardPTransforms_DeprecatedPrimitives_name, int32(x))
+	return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x))
 }
 
+func (StandardPTransforms_DeprecatedPrimitives) Descriptor() protoreflect.EnumDescriptor {
+	return file_beam_runner_api_proto_enumTypes[2].Descriptor()
+}
+
+func (StandardPTransforms_DeprecatedPrimitives) Type() protoreflect.EnumType {
+	return &file_beam_runner_api_proto_enumTypes[2]
+}
+
+func (x StandardPTransforms_DeprecatedPrimitives) Number() protoreflect.EnumNumber {
+	return protoreflect.EnumNumber(x)
+}
+
+// Deprecated: Use StandardPTransforms_DeprecatedPrimitives.Descriptor instead.
 func (StandardPTransforms_DeprecatedPrimitives) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_cf57597c3a9659a9, []int{4, 1}
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{4, 1}
 }
 
 type StandardPTransforms_Composites int32
@@ -243,28 +324,57 @@
 	StandardPTransforms_RESHUFFLE StandardPTransforms_Composites = 2
 	// Less well-known. Payload: WriteFilesPayload.
 	StandardPTransforms_WRITE_FILES StandardPTransforms_Composites = 3
+	// Payload: PubSubReadPayload.
+	StandardPTransforms_PUBSUB_READ StandardPTransforms_Composites = 4
+	// Payload: PubSubWritePayload.
+	StandardPTransforms_PUBSUB_WRITE StandardPTransforms_Composites = 5
 )
 
-var StandardPTransforms_Composites_name = map[int32]string{
-	0: "COMBINE_PER_KEY",
-	1: "COMBINE_GLOBALLY",
-	2: "RESHUFFLE",
-	3: "WRITE_FILES",
-}
+// Enum value maps for StandardPTransforms_Composites.
+var (
+	StandardPTransforms_Composites_name = map[int32]string{
+		0: "COMBINE_PER_KEY",
+		1: "COMBINE_GLOBALLY",
+		2: "RESHUFFLE",
+		3: "WRITE_FILES",
+		4: "PUBSUB_READ",
+		5: "PUBSUB_WRITE",
+	}
+	StandardPTransforms_Composites_value = map[string]int32{
+		"COMBINE_PER_KEY":  0,
+		"COMBINE_GLOBALLY": 1,
+		"RESHUFFLE":        2,
+		"WRITE_FILES":      3,
+		"PUBSUB_READ":      4,
+		"PUBSUB_WRITE":     5,
+	}
+)
 
-var StandardPTransforms_Composites_value = map[string]int32{
-	"COMBINE_PER_KEY":  0,
-	"COMBINE_GLOBALLY": 1,
-	"RESHUFFLE":        2,
-	"WRITE_FILES":      3,
+func (x StandardPTransforms_Composites) Enum() *StandardPTransforms_Composites {
+	p := new(StandardPTransforms_Composites)
+	*p = x
+	return p
 }
 
 func (x StandardPTransforms_Composites) String() string {
-	return proto.EnumName(StandardPTransforms_Composites_name, int32(x))
+	return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x))
 }
 
+func (StandardPTransforms_Composites) Descriptor() protoreflect.EnumDescriptor {
+	return file_beam_runner_api_proto_enumTypes[3].Descriptor()
+}
+
+func (StandardPTransforms_Composites) Type() protoreflect.EnumType {
+	return &file_beam_runner_api_proto_enumTypes[3]
+}
+
+func (x StandardPTransforms_Composites) Number() protoreflect.EnumNumber {
+	return protoreflect.EnumNumber(x)
+}
+
+// Deprecated: Use StandardPTransforms_Composites.Descriptor instead.
 func (StandardPTransforms_Composites) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_cf57597c3a9659a9, []int{4, 2}
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{4, 2}
 }
 
 // Payload for all of these: CombinePayload
@@ -298,28 +408,49 @@
 	StandardPTransforms_COMBINE_PER_KEY_CONVERT_TO_ACCUMULATORS StandardPTransforms_CombineComponents = 4
 )
 
-var StandardPTransforms_CombineComponents_name = map[int32]string{
-	0: "COMBINE_PER_KEY_PRECOMBINE",
-	1: "COMBINE_PER_KEY_MERGE_ACCUMULATORS",
-	2: "COMBINE_PER_KEY_EXTRACT_OUTPUTS",
-	3: "COMBINE_GROUPED_VALUES",
-	4: "COMBINE_PER_KEY_CONVERT_TO_ACCUMULATORS",
-}
+// Enum value maps for StandardPTransforms_CombineComponents.
+var (
+	StandardPTransforms_CombineComponents_name = map[int32]string{
+		0: "COMBINE_PER_KEY_PRECOMBINE",
+		1: "COMBINE_PER_KEY_MERGE_ACCUMULATORS",
+		2: "COMBINE_PER_KEY_EXTRACT_OUTPUTS",
+		3: "COMBINE_GROUPED_VALUES",
+		4: "COMBINE_PER_KEY_CONVERT_TO_ACCUMULATORS",
+	}
+	StandardPTransforms_CombineComponents_value = map[string]int32{
+		"COMBINE_PER_KEY_PRECOMBINE":              0,
+		"COMBINE_PER_KEY_MERGE_ACCUMULATORS":      1,
+		"COMBINE_PER_KEY_EXTRACT_OUTPUTS":         2,
+		"COMBINE_GROUPED_VALUES":                  3,
+		"COMBINE_PER_KEY_CONVERT_TO_ACCUMULATORS": 4,
+	}
+)
 
-var StandardPTransforms_CombineComponents_value = map[string]int32{
-	"COMBINE_PER_KEY_PRECOMBINE":              0,
-	"COMBINE_PER_KEY_MERGE_ACCUMULATORS":      1,
-	"COMBINE_PER_KEY_EXTRACT_OUTPUTS":         2,
-	"COMBINE_GROUPED_VALUES":                  3,
-	"COMBINE_PER_KEY_CONVERT_TO_ACCUMULATORS": 4,
+func (x StandardPTransforms_CombineComponents) Enum() *StandardPTransforms_CombineComponents {
+	p := new(StandardPTransforms_CombineComponents)
+	*p = x
+	return p
 }
 
 func (x StandardPTransforms_CombineComponents) String() string {
-	return proto.EnumName(StandardPTransforms_CombineComponents_name, int32(x))
+	return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x))
 }
 
+func (StandardPTransforms_CombineComponents) Descriptor() protoreflect.EnumDescriptor {
+	return file_beam_runner_api_proto_enumTypes[4].Descriptor()
+}
+
+func (StandardPTransforms_CombineComponents) Type() protoreflect.EnumType {
+	return &file_beam_runner_api_proto_enumTypes[4]
+}
+
+func (x StandardPTransforms_CombineComponents) Number() protoreflect.EnumNumber {
+	return protoreflect.EnumNumber(x)
+}
+
+// Deprecated: Use StandardPTransforms_CombineComponents.Descriptor instead.
 func (StandardPTransforms_CombineComponents) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_cf57597c3a9659a9, []int{4, 3}
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{4, 3}
 }
 
 // Payload for all of these: ParDoPayload containing the user's SDF
@@ -329,59 +460,75 @@
 	// Pairs the input element with its initial restriction.
 	// Input: element; output: KV(element, restriction).
 	StandardPTransforms_PAIR_WITH_RESTRICTION StandardPTransforms_SplittableParDoComponents = 0
-	// Splits the restriction inside an element/restriction pair.
-	// Input: KV(element, restriction); output: KV(element, restriction).
-	StandardPTransforms_SPLIT_RESTRICTION StandardPTransforms_SplittableParDoComponents = 1
-	// Applies the DoFn to every element/restriction pair in a uniquely keyed
-	// collection, in a splittable fashion.
-	// Input: KV(bytes, KV(element, restriction)); output: DoFn's output.
-	// The first "bytes" is an opaque unique key using the standard bytes coder.
-	// Typically a runner would rewrite this into a runner-specific grouping
-	// operation supporting state and timers, followed by PROCESS_ELEMENTS,
-	// with some runner-specific glue code in between.
-	StandardPTransforms_PROCESS_KEYED_ELEMENTS StandardPTransforms_SplittableParDoComponents = 2
-	// Like PROCESS_KEYED_ELEMENTS, but without the unique key - just elements
-	// and restrictions.
-	// Input: KV(element, restriction); output: DoFn's output.
-	StandardPTransforms_PROCESS_ELEMENTS StandardPTransforms_SplittableParDoComponents = 3
 	// Splits the restriction of each element/restriction pair and returns the
-	// resulting splits, with a corresponding floating point size estimations
+	// resulting splits, with a corresponding floating point size estimation
 	// for each.
+	//
 	// A reasonable value for size is the number of bytes expected to be
 	// produced by this (element, restriction) pair.
+	//
 	// Input: KV(element, restriction)
 	// Output: KV(KV(element, restriction), size))
-	StandardPTransforms_SPLIT_AND_SIZE_RESTRICTIONS StandardPTransforms_SplittableParDoComponents = 4
-	// Like PROCESS_ELEMENTS, but accepts the sized output produced by
-	// SPLIT_RESTRICTION_WITH_SIZING.
+	StandardPTransforms_SPLIT_AND_SIZE_RESTRICTIONS StandardPTransforms_SplittableParDoComponents = 1
+	// Applies the DoFn to every element and restriction.
+	//
+	// All primary and residuals returned from checkpointing or splitting must
+	// have the same type as the input to this transform.
+	//
 	// Input: KV(KV(element, restriction), size); output: DoFn's output.
-	StandardPTransforms_PROCESS_SIZED_ELEMENTS_AND_RESTRICTIONS StandardPTransforms_SplittableParDoComponents = 5
+	StandardPTransforms_PROCESS_SIZED_ELEMENTS_AND_RESTRICTIONS StandardPTransforms_SplittableParDoComponents = 2
+	// Truncates the restriction of each element/restriction pair and returns
+	// the finite restriction which will be processed when a pipeline is
+	// drained. See
+	// https://docs.google.com/document/d/1NExwHlj-2q2WUGhSO4jTu8XGhDPmm3cllSN8IMmWci8/edit#.
+	// for additional details about drain.
+	//
+	// Input: KV(KV(element, restriction), size);
+	// Output: KV(KV(element, restriction), size).
+	StandardPTransforms_TRUNCATE_SIZED_RESTRICTION StandardPTransforms_SplittableParDoComponents = 3
 )
 
-var StandardPTransforms_SplittableParDoComponents_name = map[int32]string{
-	0: "PAIR_WITH_RESTRICTION",
-	1: "SPLIT_RESTRICTION",
-	2: "PROCESS_KEYED_ELEMENTS",
-	3: "PROCESS_ELEMENTS",
-	4: "SPLIT_AND_SIZE_RESTRICTIONS",
-	5: "PROCESS_SIZED_ELEMENTS_AND_RESTRICTIONS",
-}
+// Enum value maps for StandardPTransforms_SplittableParDoComponents.
+var (
+	StandardPTransforms_SplittableParDoComponents_name = map[int32]string{
+		0: "PAIR_WITH_RESTRICTION",
+		1: "SPLIT_AND_SIZE_RESTRICTIONS",
+		2: "PROCESS_SIZED_ELEMENTS_AND_RESTRICTIONS",
+		3: "TRUNCATE_SIZED_RESTRICTION",
+	}
+	StandardPTransforms_SplittableParDoComponents_value = map[string]int32{
+		"PAIR_WITH_RESTRICTION":                   0,
+		"SPLIT_AND_SIZE_RESTRICTIONS":             1,
+		"PROCESS_SIZED_ELEMENTS_AND_RESTRICTIONS": 2,
+		"TRUNCATE_SIZED_RESTRICTION":              3,
+	}
+)
 
-var StandardPTransforms_SplittableParDoComponents_value = map[string]int32{
-	"PAIR_WITH_RESTRICTION":                   0,
-	"SPLIT_RESTRICTION":                       1,
-	"PROCESS_KEYED_ELEMENTS":                  2,
-	"PROCESS_ELEMENTS":                        3,
-	"SPLIT_AND_SIZE_RESTRICTIONS":             4,
-	"PROCESS_SIZED_ELEMENTS_AND_RESTRICTIONS": 5,
+func (x StandardPTransforms_SplittableParDoComponents) Enum() *StandardPTransforms_SplittableParDoComponents {
+	p := new(StandardPTransforms_SplittableParDoComponents)
+	*p = x
+	return p
 }
 
 func (x StandardPTransforms_SplittableParDoComponents) String() string {
-	return proto.EnumName(StandardPTransforms_SplittableParDoComponents_name, int32(x))
+	return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x))
 }
 
+func (StandardPTransforms_SplittableParDoComponents) Descriptor() protoreflect.EnumDescriptor {
+	return file_beam_runner_api_proto_enumTypes[5].Descriptor()
+}
+
+func (StandardPTransforms_SplittableParDoComponents) Type() protoreflect.EnumType {
+	return &file_beam_runner_api_proto_enumTypes[5]
+}
+
+func (x StandardPTransforms_SplittableParDoComponents) Number() protoreflect.EnumNumber {
+	return protoreflect.EnumNumber(x)
+}
+
+// Deprecated: Use StandardPTransforms_SplittableParDoComponents.Descriptor instead.
 func (StandardPTransforms_SplittableParDoComponents) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_cf57597c3a9659a9, []int{4, 4}
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{4, 4}
 }
 
 type StandardSideInputTypes_Enum int32
@@ -399,22 +546,43 @@
 	StandardSideInputTypes_MULTIMAP StandardSideInputTypes_Enum = 1
 )
 
-var StandardSideInputTypes_Enum_name = map[int32]string{
-	0: "ITERABLE",
-	1: "MULTIMAP",
-}
+// Enum value maps for StandardSideInputTypes_Enum.
+var (
+	StandardSideInputTypes_Enum_name = map[int32]string{
+		0: "ITERABLE",
+		1: "MULTIMAP",
+	}
+	StandardSideInputTypes_Enum_value = map[string]int32{
+		"ITERABLE": 0,
+		"MULTIMAP": 1,
+	}
+)
 
-var StandardSideInputTypes_Enum_value = map[string]int32{
-	"ITERABLE": 0,
-	"MULTIMAP": 1,
+func (x StandardSideInputTypes_Enum) Enum() *StandardSideInputTypes_Enum {
+	p := new(StandardSideInputTypes_Enum)
+	*p = x
+	return p
 }
 
 func (x StandardSideInputTypes_Enum) String() string {
-	return proto.EnumName(StandardSideInputTypes_Enum_name, int32(x))
+	return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x))
 }
 
+func (StandardSideInputTypes_Enum) Descriptor() protoreflect.EnumDescriptor {
+	return file_beam_runner_api_proto_enumTypes[6].Descriptor()
+}
+
+func (StandardSideInputTypes_Enum) Type() protoreflect.EnumType {
+	return &file_beam_runner_api_proto_enumTypes[6]
+}
+
+func (x StandardSideInputTypes_Enum) Number() protoreflect.EnumNumber {
+	return protoreflect.EnumNumber(x)
+}
+
+// Deprecated: Use StandardSideInputTypes_Enum.Descriptor instead.
 func (StandardSideInputTypes_Enum) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_cf57597c3a9659a9, []int{5, 0}
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{5, 0}
 }
 
 type IsBounded_Enum int32
@@ -425,24 +593,45 @@
 	IsBounded_BOUNDED     IsBounded_Enum = 2
 )
 
-var IsBounded_Enum_name = map[int32]string{
-	0: "UNSPECIFIED",
-	1: "UNBOUNDED",
-	2: "BOUNDED",
-}
+// Enum value maps for IsBounded_Enum.
+var (
+	IsBounded_Enum_name = map[int32]string{
+		0: "UNSPECIFIED",
+		1: "UNBOUNDED",
+		2: "BOUNDED",
+	}
+	IsBounded_Enum_value = map[string]int32{
+		"UNSPECIFIED": 0,
+		"UNBOUNDED":   1,
+		"BOUNDED":     2,
+	}
+)
 
-var IsBounded_Enum_value = map[string]int32{
-	"UNSPECIFIED": 0,
-	"UNBOUNDED":   1,
-	"BOUNDED":     2,
+func (x IsBounded_Enum) Enum() *IsBounded_Enum {
+	p := new(IsBounded_Enum)
+	*p = x
+	return p
 }
 
 func (x IsBounded_Enum) String() string {
-	return proto.EnumName(IsBounded_Enum_name, int32(x))
+	return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x))
 }
 
+func (IsBounded_Enum) Descriptor() protoreflect.EnumDescriptor {
+	return file_beam_runner_api_proto_enumTypes[7].Descriptor()
+}
+
+func (IsBounded_Enum) Type() protoreflect.EnumType {
+	return &file_beam_runner_api_proto_enumTypes[7]
+}
+
+func (x IsBounded_Enum) Number() protoreflect.EnumNumber {
+	return protoreflect.EnumNumber(x)
+}
+
+// Deprecated: Use IsBounded_Enum.Descriptor instead.
 func (IsBounded_Enum) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_cf57597c3a9659a9, []int{15, 0}
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{16, 0}
 }
 
 type StandardCoders_Enum int32
@@ -615,58 +804,96 @@
 	//     BOOLEAN:   beam:coder:bool:v1
 	//     BYTES:     beam:coder:bytes:v1
 	//   ArrayType:   beam:coder:iterable:v1 (always has a known length)
-	//   MapType:     not yet a standard coder (BEAM-7996)
+	//   MapType:     not a standard coder, specification defined below.
 	//   RowType:     beam:coder:row:v1
 	//   LogicalType: Uses the coder for its representation.
 	//
+	// The MapType is encoded by:
+	//   - An INT32 representing the size of the map (N)
+	//   - Followed by N interleaved keys and values, encoded with their
+	//     corresponding coder.
+	//
+	// Nullable types in container types (ArrayType, MapType) are encoded by:
+	//   - A one byte null indicator, 0x00 for null values, or 0x01 for present
+	//     values.
+	//   - For present values the null indicator is followed by the value
+	//     encoded with it's corresponding coder.
+	//
+	// Well known logical types:
+	//   beam:logical_type:micros_instant:v1
+	//   - Representation type: ROW<seconds: INT64, micros: INT64>
+	//   - A timestamp without a timezone where seconds + micros represents the
+	//     amount of time since the epoch.
+	//
 	// The payload for RowCoder is an instance of Schema.
 	// Components: None
 	// Experimental.
 	StandardCoders_ROW StandardCoders_Enum = 13
 )
 
-var StandardCoders_Enum_name = map[int32]string{
-	0:  "BYTES",
-	10: "STRING_UTF8",
-	1:  "KV",
-	12: "BOOL",
-	2:  "VARINT",
-	11: "DOUBLE",
-	3:  "ITERABLE",
-	4:  "TIMER",
-	5:  "INTERVAL_WINDOW",
-	6:  "LENGTH_PREFIX",
-	7:  "GLOBAL_WINDOW",
-	8:  "WINDOWED_VALUE",
-	14: "PARAM_WINDOWED_VALUE",
-	9:  "STATE_BACKED_ITERABLE",
-	13: "ROW",
-}
+// Enum value maps for StandardCoders_Enum.
+var (
+	StandardCoders_Enum_name = map[int32]string{
+		0:  "BYTES",
+		10: "STRING_UTF8",
+		1:  "KV",
+		12: "BOOL",
+		2:  "VARINT",
+		11: "DOUBLE",
+		3:  "ITERABLE",
+		4:  "TIMER",
+		5:  "INTERVAL_WINDOW",
+		6:  "LENGTH_PREFIX",
+		7:  "GLOBAL_WINDOW",
+		8:  "WINDOWED_VALUE",
+		14: "PARAM_WINDOWED_VALUE",
+		9:  "STATE_BACKED_ITERABLE",
+		13: "ROW",
+	}
+	StandardCoders_Enum_value = map[string]int32{
+		"BYTES":                 0,
+		"STRING_UTF8":           10,
+		"KV":                    1,
+		"BOOL":                  12,
+		"VARINT":                2,
+		"DOUBLE":                11,
+		"ITERABLE":              3,
+		"TIMER":                 4,
+		"INTERVAL_WINDOW":       5,
+		"LENGTH_PREFIX":         6,
+		"GLOBAL_WINDOW":         7,
+		"WINDOWED_VALUE":        8,
+		"PARAM_WINDOWED_VALUE":  14,
+		"STATE_BACKED_ITERABLE": 9,
+		"ROW":                   13,
+	}
+)
 
-var StandardCoders_Enum_value = map[string]int32{
-	"BYTES":                 0,
-	"STRING_UTF8":           10,
-	"KV":                    1,
-	"BOOL":                  12,
-	"VARINT":                2,
-	"DOUBLE":                11,
-	"ITERABLE":              3,
-	"TIMER":                 4,
-	"INTERVAL_WINDOW":       5,
-	"LENGTH_PREFIX":         6,
-	"GLOBAL_WINDOW":         7,
-	"WINDOWED_VALUE":        8,
-	"PARAM_WINDOWED_VALUE":  14,
-	"STATE_BACKED_ITERABLE": 9,
-	"ROW":                   13,
+func (x StandardCoders_Enum) Enum() *StandardCoders_Enum {
+	p := new(StandardCoders_Enum)
+	*p = x
+	return p
 }
 
 func (x StandardCoders_Enum) String() string {
-	return proto.EnumName(StandardCoders_Enum_name, int32(x))
+	return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x))
 }
 
+func (StandardCoders_Enum) Descriptor() protoreflect.EnumDescriptor {
+	return file_beam_runner_api_proto_enumTypes[8].Descriptor()
+}
+
+func (StandardCoders_Enum) Type() protoreflect.EnumType {
+	return &file_beam_runner_api_proto_enumTypes[8]
+}
+
+func (x StandardCoders_Enum) Number() protoreflect.EnumNumber {
+	return protoreflect.EnumNumber(x)
+}
+
+// Deprecated: Use StandardCoders_Enum.Descriptor instead.
 func (StandardCoders_Enum) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_cf57597c3a9659a9, []int{23, 0}
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{26, 0}
 }
 
 type MergeStatus_Enum int32
@@ -686,26 +913,47 @@
 	MergeStatus_ALREADY_MERGED MergeStatus_Enum = 3
 )
 
-var MergeStatus_Enum_name = map[int32]string{
-	0: "UNSPECIFIED",
-	1: "NON_MERGING",
-	2: "NEEDS_MERGE",
-	3: "ALREADY_MERGED",
-}
+// Enum value maps for MergeStatus_Enum.
+var (
+	MergeStatus_Enum_name = map[int32]string{
+		0: "UNSPECIFIED",
+		1: "NON_MERGING",
+		2: "NEEDS_MERGE",
+		3: "ALREADY_MERGED",
+	}
+	MergeStatus_Enum_value = map[string]int32{
+		"UNSPECIFIED":    0,
+		"NON_MERGING":    1,
+		"NEEDS_MERGE":    2,
+		"ALREADY_MERGED": 3,
+	}
+)
 
-var MergeStatus_Enum_value = map[string]int32{
-	"UNSPECIFIED":    0,
-	"NON_MERGING":    1,
-	"NEEDS_MERGE":    2,
-	"ALREADY_MERGED": 3,
+func (x MergeStatus_Enum) Enum() *MergeStatus_Enum {
+	p := new(MergeStatus_Enum)
+	*p = x
+	return p
 }
 
 func (x MergeStatus_Enum) String() string {
-	return proto.EnumName(MergeStatus_Enum_name, int32(x))
+	return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x))
 }
 
+func (MergeStatus_Enum) Descriptor() protoreflect.EnumDescriptor {
+	return file_beam_runner_api_proto_enumTypes[9].Descriptor()
+}
+
+func (MergeStatus_Enum) Type() protoreflect.EnumType {
+	return &file_beam_runner_api_proto_enumTypes[9]
+}
+
+func (x MergeStatus_Enum) Number() protoreflect.EnumNumber {
+	return protoreflect.EnumNumber(x)
+}
+
+// Deprecated: Use MergeStatus_Enum.Descriptor instead.
 func (MergeStatus_Enum) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_cf57597c3a9659a9, []int{25, 0}
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{28, 0}
 }
 
 type AccumulationMode_Enum int32
@@ -720,26 +968,47 @@
 	AccumulationMode_RETRACTING AccumulationMode_Enum = 3
 )
 
-var AccumulationMode_Enum_name = map[int32]string{
-	0: "UNSPECIFIED",
-	1: "DISCARDING",
-	2: "ACCUMULATING",
-	3: "RETRACTING",
-}
+// Enum value maps for AccumulationMode_Enum.
+var (
+	AccumulationMode_Enum_name = map[int32]string{
+		0: "UNSPECIFIED",
+		1: "DISCARDING",
+		2: "ACCUMULATING",
+		3: "RETRACTING",
+	}
+	AccumulationMode_Enum_value = map[string]int32{
+		"UNSPECIFIED":  0,
+		"DISCARDING":   1,
+		"ACCUMULATING": 2,
+		"RETRACTING":   3,
+	}
+)
 
-var AccumulationMode_Enum_value = map[string]int32{
-	"UNSPECIFIED":  0,
-	"DISCARDING":   1,
-	"ACCUMULATING": 2,
-	"RETRACTING":   3,
+func (x AccumulationMode_Enum) Enum() *AccumulationMode_Enum {
+	p := new(AccumulationMode_Enum)
+	*p = x
+	return p
 }
 
 func (x AccumulationMode_Enum) String() string {
-	return proto.EnumName(AccumulationMode_Enum_name, int32(x))
+	return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x))
 }
 
+func (AccumulationMode_Enum) Descriptor() protoreflect.EnumDescriptor {
+	return file_beam_runner_api_proto_enumTypes[10].Descriptor()
+}
+
+func (AccumulationMode_Enum) Type() protoreflect.EnumType {
+	return &file_beam_runner_api_proto_enumTypes[10]
+}
+
+func (x AccumulationMode_Enum) Number() protoreflect.EnumNumber {
+	return protoreflect.EnumNumber(x)
+}
+
+// Deprecated: Use AccumulationMode_Enum.Descriptor instead.
 func (AccumulationMode_Enum) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_cf57597c3a9659a9, []int{26, 0}
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{29, 0}
 }
 
 type ClosingBehavior_Enum int32
@@ -753,24 +1022,45 @@
 	ClosingBehavior_EMIT_IF_NONEMPTY ClosingBehavior_Enum = 2
 )
 
-var ClosingBehavior_Enum_name = map[int32]string{
-	0: "UNSPECIFIED",
-	1: "EMIT_ALWAYS",
-	2: "EMIT_IF_NONEMPTY",
-}
+// Enum value maps for ClosingBehavior_Enum.
+var (
+	ClosingBehavior_Enum_name = map[int32]string{
+		0: "UNSPECIFIED",
+		1: "EMIT_ALWAYS",
+		2: "EMIT_IF_NONEMPTY",
+	}
+	ClosingBehavior_Enum_value = map[string]int32{
+		"UNSPECIFIED":      0,
+		"EMIT_ALWAYS":      1,
+		"EMIT_IF_NONEMPTY": 2,
+	}
+)
 
-var ClosingBehavior_Enum_value = map[string]int32{
-	"UNSPECIFIED":      0,
-	"EMIT_ALWAYS":      1,
-	"EMIT_IF_NONEMPTY": 2,
+func (x ClosingBehavior_Enum) Enum() *ClosingBehavior_Enum {
+	p := new(ClosingBehavior_Enum)
+	*p = x
+	return p
 }
 
 func (x ClosingBehavior_Enum) String() string {
-	return proto.EnumName(ClosingBehavior_Enum_name, int32(x))
+	return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x))
 }
 
+func (ClosingBehavior_Enum) Descriptor() protoreflect.EnumDescriptor {
+	return file_beam_runner_api_proto_enumTypes[11].Descriptor()
+}
+
+func (ClosingBehavior_Enum) Type() protoreflect.EnumType {
+	return &file_beam_runner_api_proto_enumTypes[11]
+}
+
+func (x ClosingBehavior_Enum) Number() protoreflect.EnumNumber {
+	return protoreflect.EnumNumber(x)
+}
+
+// Deprecated: Use ClosingBehavior_Enum.Descriptor instead.
 func (ClosingBehavior_Enum) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_cf57597c3a9659a9, []int{27, 0}
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{30, 0}
 }
 
 type OnTimeBehavior_Enum int32
@@ -784,24 +1074,45 @@
 	OnTimeBehavior_FIRE_IF_NONEMPTY OnTimeBehavior_Enum = 2
 )
 
-var OnTimeBehavior_Enum_name = map[int32]string{
-	0: "UNSPECIFIED",
-	1: "FIRE_ALWAYS",
-	2: "FIRE_IF_NONEMPTY",
-}
+// Enum value maps for OnTimeBehavior_Enum.
+var (
+	OnTimeBehavior_Enum_name = map[int32]string{
+		0: "UNSPECIFIED",
+		1: "FIRE_ALWAYS",
+		2: "FIRE_IF_NONEMPTY",
+	}
+	OnTimeBehavior_Enum_value = map[string]int32{
+		"UNSPECIFIED":      0,
+		"FIRE_ALWAYS":      1,
+		"FIRE_IF_NONEMPTY": 2,
+	}
+)
 
-var OnTimeBehavior_Enum_value = map[string]int32{
-	"UNSPECIFIED":      0,
-	"FIRE_ALWAYS":      1,
-	"FIRE_IF_NONEMPTY": 2,
+func (x OnTimeBehavior_Enum) Enum() *OnTimeBehavior_Enum {
+	p := new(OnTimeBehavior_Enum)
+	*p = x
+	return p
 }
 
 func (x OnTimeBehavior_Enum) String() string {
-	return proto.EnumName(OnTimeBehavior_Enum_name, int32(x))
+	return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x))
 }
 
+func (OnTimeBehavior_Enum) Descriptor() protoreflect.EnumDescriptor {
+	return file_beam_runner_api_proto_enumTypes[12].Descriptor()
+}
+
+func (OnTimeBehavior_Enum) Type() protoreflect.EnumType {
+	return &file_beam_runner_api_proto_enumTypes[12]
+}
+
+func (x OnTimeBehavior_Enum) Number() protoreflect.EnumNumber {
+	return protoreflect.EnumNumber(x)
+}
+
+// Deprecated: Use OnTimeBehavior_Enum.Descriptor instead.
 func (OnTimeBehavior_Enum) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_cf57597c3a9659a9, []int{28, 0}
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{31, 0}
 }
 
 type OutputTime_Enum int32
@@ -818,26 +1129,47 @@
 	OutputTime_EARLIEST_IN_PANE OutputTime_Enum = 3
 )
 
-var OutputTime_Enum_name = map[int32]string{
-	0: "UNSPECIFIED",
-	1: "END_OF_WINDOW",
-	2: "LATEST_IN_PANE",
-	3: "EARLIEST_IN_PANE",
-}
+// Enum value maps for OutputTime_Enum.
+var (
+	OutputTime_Enum_name = map[int32]string{
+		0: "UNSPECIFIED",
+		1: "END_OF_WINDOW",
+		2: "LATEST_IN_PANE",
+		3: "EARLIEST_IN_PANE",
+	}
+	OutputTime_Enum_value = map[string]int32{
+		"UNSPECIFIED":      0,
+		"END_OF_WINDOW":    1,
+		"LATEST_IN_PANE":   2,
+		"EARLIEST_IN_PANE": 3,
+	}
+)
 
-var OutputTime_Enum_value = map[string]int32{
-	"UNSPECIFIED":      0,
-	"END_OF_WINDOW":    1,
-	"LATEST_IN_PANE":   2,
-	"EARLIEST_IN_PANE": 3,
+func (x OutputTime_Enum) Enum() *OutputTime_Enum {
+	p := new(OutputTime_Enum)
+	*p = x
+	return p
 }
 
 func (x OutputTime_Enum) String() string {
-	return proto.EnumName(OutputTime_Enum_name, int32(x))
+	return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x))
 }
 
+func (OutputTime_Enum) Descriptor() protoreflect.EnumDescriptor {
+	return file_beam_runner_api_proto_enumTypes[13].Descriptor()
+}
+
+func (OutputTime_Enum) Type() protoreflect.EnumType {
+	return &file_beam_runner_api_proto_enumTypes[13]
+}
+
+func (x OutputTime_Enum) Number() protoreflect.EnumNumber {
+	return protoreflect.EnumNumber(x)
+}
+
+// Deprecated: Use OutputTime_Enum.Descriptor instead.
 func (OutputTime_Enum) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_cf57597c3a9659a9, []int{29, 0}
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{32, 0}
 }
 
 type TimeDomain_Enum int32
@@ -857,26 +1189,47 @@
 	TimeDomain_SYNCHRONIZED_PROCESSING_TIME TimeDomain_Enum = 3
 )
 
-var TimeDomain_Enum_name = map[int32]string{
-	0: "UNSPECIFIED",
-	1: "EVENT_TIME",
-	2: "PROCESSING_TIME",
-	3: "SYNCHRONIZED_PROCESSING_TIME",
-}
+// Enum value maps for TimeDomain_Enum.
+var (
+	TimeDomain_Enum_name = map[int32]string{
+		0: "UNSPECIFIED",
+		1: "EVENT_TIME",
+		2: "PROCESSING_TIME",
+		3: "SYNCHRONIZED_PROCESSING_TIME",
+	}
+	TimeDomain_Enum_value = map[string]int32{
+		"UNSPECIFIED":                  0,
+		"EVENT_TIME":                   1,
+		"PROCESSING_TIME":              2,
+		"SYNCHRONIZED_PROCESSING_TIME": 3,
+	}
+)
 
-var TimeDomain_Enum_value = map[string]int32{
-	"UNSPECIFIED":                  0,
-	"EVENT_TIME":                   1,
-	"PROCESSING_TIME":              2,
-	"SYNCHRONIZED_PROCESSING_TIME": 3,
+func (x TimeDomain_Enum) Enum() *TimeDomain_Enum {
+	p := new(TimeDomain_Enum)
+	*p = x
+	return p
 }
 
 func (x TimeDomain_Enum) String() string {
-	return proto.EnumName(TimeDomain_Enum_name, int32(x))
+	return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x))
 }
 
+func (TimeDomain_Enum) Descriptor() protoreflect.EnumDescriptor {
+	return file_beam_runner_api_proto_enumTypes[14].Descriptor()
+}
+
+func (TimeDomain_Enum) Type() protoreflect.EnumType {
+	return &file_beam_runner_api_proto_enumTypes[14]
+}
+
+func (x TimeDomain_Enum) Number() protoreflect.EnumNumber {
+	return protoreflect.EnumNumber(x)
+}
+
+// Deprecated: Use TimeDomain_Enum.Descriptor instead.
 func (TimeDomain_Enum) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_cf57597c3a9659a9, []int{30, 0}
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{33, 0}
 }
 
 type StandardArtifacts_Types int32
@@ -902,30 +1255,51 @@
 	StandardArtifacts_DEFERRED StandardArtifacts_Types = 5
 )
 
-var StandardArtifacts_Types_name = map[int32]string{
-	0: "FILE",
-	1: "URL",
-	2: "EMBEDDED",
-	3: "PYPI",
-	4: "MAVEN",
-	5: "DEFERRED",
-}
+// Enum value maps for StandardArtifacts_Types.
+var (
+	StandardArtifacts_Types_name = map[int32]string{
+		0: "FILE",
+		1: "URL",
+		2: "EMBEDDED",
+		3: "PYPI",
+		4: "MAVEN",
+		5: "DEFERRED",
+	}
+	StandardArtifacts_Types_value = map[string]int32{
+		"FILE":     0,
+		"URL":      1,
+		"EMBEDDED": 2,
+		"PYPI":     3,
+		"MAVEN":    4,
+		"DEFERRED": 5,
+	}
+)
 
-var StandardArtifacts_Types_value = map[string]int32{
-	"FILE":     0,
-	"URL":      1,
-	"EMBEDDED": 2,
-	"PYPI":     3,
-	"MAVEN":    4,
-	"DEFERRED": 5,
+func (x StandardArtifacts_Types) Enum() *StandardArtifacts_Types {
+	p := new(StandardArtifacts_Types)
+	*p = x
+	return p
 }
 
 func (x StandardArtifacts_Types) String() string {
-	return proto.EnumName(StandardArtifacts_Types_name, int32(x))
+	return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x))
 }
 
+func (StandardArtifacts_Types) Descriptor() protoreflect.EnumDescriptor {
+	return file_beam_runner_api_proto_enumTypes[15].Descriptor()
+}
+
+func (StandardArtifacts_Types) Type() protoreflect.EnumType {
+	return &file_beam_runner_api_proto_enumTypes[15]
+}
+
+func (x StandardArtifacts_Types) Number() protoreflect.EnumNumber {
+	return protoreflect.EnumNumber(x)
+}
+
+// Deprecated: Use StandardArtifacts_Types.Descriptor instead.
 func (StandardArtifacts_Types) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_cf57597c3a9659a9, []int{34, 0}
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{37, 0}
 }
 
 type StandardArtifacts_Roles int32
@@ -936,48 +1310,90 @@
 	StandardArtifacts_STAGING_TO StandardArtifacts_Roles = 0
 )
 
-var StandardArtifacts_Roles_name = map[int32]string{
-	0: "STAGING_TO",
-}
+// Enum value maps for StandardArtifacts_Roles.
+var (
+	StandardArtifacts_Roles_name = map[int32]string{
+		0: "STAGING_TO",
+	}
+	StandardArtifacts_Roles_value = map[string]int32{
+		"STAGING_TO": 0,
+	}
+)
 
-var StandardArtifacts_Roles_value = map[string]int32{
-	"STAGING_TO": 0,
+func (x StandardArtifacts_Roles) Enum() *StandardArtifacts_Roles {
+	p := new(StandardArtifacts_Roles)
+	*p = x
+	return p
 }
 
 func (x StandardArtifacts_Roles) String() string {
-	return proto.EnumName(StandardArtifacts_Roles_name, int32(x))
+	return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x))
 }
 
+func (StandardArtifacts_Roles) Descriptor() protoreflect.EnumDescriptor {
+	return file_beam_runner_api_proto_enumTypes[16].Descriptor()
+}
+
+func (StandardArtifacts_Roles) Type() protoreflect.EnumType {
+	return &file_beam_runner_api_proto_enumTypes[16]
+}
+
+func (x StandardArtifacts_Roles) Number() protoreflect.EnumNumber {
+	return protoreflect.EnumNumber(x)
+}
+
+// Deprecated: Use StandardArtifacts_Roles.Descriptor instead.
 func (StandardArtifacts_Roles) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_cf57597c3a9659a9, []int{34, 1}
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{37, 1}
 }
 
 type StandardEnvironments_Environments int32
 
 const (
-	StandardEnvironments_DOCKER   StandardEnvironments_Environments = 0
-	StandardEnvironments_PROCESS  StandardEnvironments_Environments = 1
-	StandardEnvironments_EXTERNAL StandardEnvironments_Environments = 2
+	StandardEnvironments_DOCKER   StandardEnvironments_Environments = 0 // A managed docker container to run user code.
+	StandardEnvironments_PROCESS  StandardEnvironments_Environments = 1 // A managed native process to run user code.
+	StandardEnvironments_EXTERNAL StandardEnvironments_Environments = 2 // An external non managed process to run user code.
 )
 
-var StandardEnvironments_Environments_name = map[int32]string{
-	0: "DOCKER",
-	1: "PROCESS",
-	2: "EXTERNAL",
-}
+// Enum value maps for StandardEnvironments_Environments.
+var (
+	StandardEnvironments_Environments_name = map[int32]string{
+		0: "DOCKER",
+		1: "PROCESS",
+		2: "EXTERNAL",
+	}
+	StandardEnvironments_Environments_value = map[string]int32{
+		"DOCKER":   0,
+		"PROCESS":  1,
+		"EXTERNAL": 2,
+	}
+)
 
-var StandardEnvironments_Environments_value = map[string]int32{
-	"DOCKER":   0,
-	"PROCESS":  1,
-	"EXTERNAL": 2,
+func (x StandardEnvironments_Environments) Enum() *StandardEnvironments_Environments {
+	p := new(StandardEnvironments_Environments)
+	*p = x
+	return p
 }
 
 func (x StandardEnvironments_Environments) String() string {
-	return proto.EnumName(StandardEnvironments_Environments_name, int32(x))
+	return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x))
 }
 
+func (StandardEnvironments_Environments) Descriptor() protoreflect.EnumDescriptor {
+	return file_beam_runner_api_proto_enumTypes[17].Descriptor()
+}
+
+func (StandardEnvironments_Environments) Type() protoreflect.EnumType {
+	return &file_beam_runner_api_proto_enumTypes[17]
+}
+
+func (x StandardEnvironments_Environments) Number() protoreflect.EnumNumber {
+	return protoreflect.EnumNumber(x)
+}
+
+// Deprecated: Use StandardEnvironments_Environments.Descriptor instead.
 func (StandardEnvironments_Environments) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_cf57597c3a9659a9, []int{44, 0}
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{47, 0}
 }
 
 type StandardProtocols_Enum int32
@@ -998,26 +1414,47 @@
 	StandardProtocols_MULTI_CORE_BUNDLE_PROCESSING StandardProtocols_Enum = 3
 )
 
-var StandardProtocols_Enum_name = map[int32]string{
-	0: "LEGACY_PROGRESS_REPORTING",
-	1: "PROGRESS_REPORTING",
-	2: "WORKER_STATUS",
-	3: "MULTI_CORE_BUNDLE_PROCESSING",
-}
+// Enum value maps for StandardProtocols_Enum.
+var (
+	StandardProtocols_Enum_name = map[int32]string{
+		0: "LEGACY_PROGRESS_REPORTING",
+		1: "PROGRESS_REPORTING",
+		2: "WORKER_STATUS",
+		3: "MULTI_CORE_BUNDLE_PROCESSING",
+	}
+	StandardProtocols_Enum_value = map[string]int32{
+		"LEGACY_PROGRESS_REPORTING":    0,
+		"PROGRESS_REPORTING":           1,
+		"WORKER_STATUS":                2,
+		"MULTI_CORE_BUNDLE_PROCESSING": 3,
+	}
+)
 
-var StandardProtocols_Enum_value = map[string]int32{
-	"LEGACY_PROGRESS_REPORTING":    0,
-	"PROGRESS_REPORTING":           1,
-	"WORKER_STATUS":                2,
-	"MULTI_CORE_BUNDLE_PROCESSING": 3,
+func (x StandardProtocols_Enum) Enum() *StandardProtocols_Enum {
+	p := new(StandardProtocols_Enum)
+	*p = x
+	return p
 }
 
 func (x StandardProtocols_Enum) String() string {
-	return proto.EnumName(StandardProtocols_Enum_name, int32(x))
+	return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x))
 }
 
+func (StandardProtocols_Enum) Descriptor() protoreflect.EnumDescriptor {
+	return file_beam_runner_api_proto_enumTypes[18].Descriptor()
+}
+
+func (StandardProtocols_Enum) Type() protoreflect.EnumType {
+	return &file_beam_runner_api_proto_enumTypes[18]
+}
+
+func (x StandardProtocols_Enum) Number() protoreflect.EnumNumber {
+	return protoreflect.EnumNumber(x)
+}
+
+// Deprecated: Use StandardProtocols_Enum.Descriptor instead.
 func (StandardProtocols_Enum) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_cf57597c3a9659a9, []int{48, 0}
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{51, 0}
 }
 
 type StandardRequirements_Enum int32
@@ -1040,28 +1477,49 @@
 	StandardRequirements_REQUIRES_SPLITTABLE_DOFN StandardRequirements_Enum = 4
 )
 
-var StandardRequirements_Enum_name = map[int32]string{
-	0: "REQUIRES_STATEFUL_PROCESSING",
-	1: "REQUIRES_BUNDLE_FINALIZATION",
-	2: "REQUIRES_STABLE_INPUT",
-	3: "REQUIRES_TIME_SORTED_INPUT",
-	4: "REQUIRES_SPLITTABLE_DOFN",
-}
+// Enum value maps for StandardRequirements_Enum.
+var (
+	StandardRequirements_Enum_name = map[int32]string{
+		0: "REQUIRES_STATEFUL_PROCESSING",
+		1: "REQUIRES_BUNDLE_FINALIZATION",
+		2: "REQUIRES_STABLE_INPUT",
+		3: "REQUIRES_TIME_SORTED_INPUT",
+		4: "REQUIRES_SPLITTABLE_DOFN",
+	}
+	StandardRequirements_Enum_value = map[string]int32{
+		"REQUIRES_STATEFUL_PROCESSING": 0,
+		"REQUIRES_BUNDLE_FINALIZATION": 1,
+		"REQUIRES_STABLE_INPUT":        2,
+		"REQUIRES_TIME_SORTED_INPUT":   3,
+		"REQUIRES_SPLITTABLE_DOFN":     4,
+	}
+)
 
-var StandardRequirements_Enum_value = map[string]int32{
-	"REQUIRES_STATEFUL_PROCESSING": 0,
-	"REQUIRES_BUNDLE_FINALIZATION": 1,
-	"REQUIRES_STABLE_INPUT":        2,
-	"REQUIRES_TIME_SORTED_INPUT":   3,
-	"REQUIRES_SPLITTABLE_DOFN":     4,
+func (x StandardRequirements_Enum) Enum() *StandardRequirements_Enum {
+	p := new(StandardRequirements_Enum)
+	*p = x
+	return p
 }
 
 func (x StandardRequirements_Enum) String() string {
-	return proto.EnumName(StandardRequirements_Enum_name, int32(x))
+	return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x))
 }
 
+func (StandardRequirements_Enum) Descriptor() protoreflect.EnumDescriptor {
+	return file_beam_runner_api_proto_enumTypes[19].Descriptor()
+}
+
+func (StandardRequirements_Enum) Type() protoreflect.EnumType {
+	return &file_beam_runner_api_proto_enumTypes[19]
+}
+
+func (x StandardRequirements_Enum) Number() protoreflect.EnumNumber {
+	return protoreflect.EnumNumber(x)
+}
+
+// Deprecated: Use StandardRequirements_Enum.Descriptor instead.
 func (StandardRequirements_Enum) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_cf57597c3a9659a9, []int{49, 0}
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{52, 0}
 }
 
 type StandardDisplayData_DisplayData int32
@@ -1072,56 +1530,88 @@
 	StandardDisplayData_LABELLED_STRING StandardDisplayData_DisplayData = 0
 )
 
-var StandardDisplayData_DisplayData_name = map[int32]string{
-	0: "LABELLED_STRING",
-}
+// Enum value maps for StandardDisplayData_DisplayData.
+var (
+	StandardDisplayData_DisplayData_name = map[int32]string{
+		0: "LABELLED_STRING",
+	}
+	StandardDisplayData_DisplayData_value = map[string]int32{
+		"LABELLED_STRING": 0,
+	}
+)
 
-var StandardDisplayData_DisplayData_value = map[string]int32{
-	"LABELLED_STRING": 0,
+func (x StandardDisplayData_DisplayData) Enum() *StandardDisplayData_DisplayData {
+	p := new(StandardDisplayData_DisplayData)
+	*p = x
+	return p
 }
 
 func (x StandardDisplayData_DisplayData) String() string {
-	return proto.EnumName(StandardDisplayData_DisplayData_name, int32(x))
+	return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x))
 }
 
+func (StandardDisplayData_DisplayData) Descriptor() protoreflect.EnumDescriptor {
+	return file_beam_runner_api_proto_enumTypes[20].Descriptor()
+}
+
+func (StandardDisplayData_DisplayData) Type() protoreflect.EnumType {
+	return &file_beam_runner_api_proto_enumTypes[20]
+}
+
+func (x StandardDisplayData_DisplayData) Number() protoreflect.EnumNumber {
+	return protoreflect.EnumNumber(x)
+}
+
+// Deprecated: Use StandardDisplayData_DisplayData.Descriptor instead.
 func (StandardDisplayData_DisplayData) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_cf57597c3a9659a9, []int{51, 0}
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{54, 0}
 }
 
 type BeamConstants struct {
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
 }
 
-func (m *BeamConstants) Reset()         { *m = BeamConstants{} }
-func (m *BeamConstants) String() string { return proto.CompactTextString(m) }
-func (*BeamConstants) ProtoMessage()    {}
+func (x *BeamConstants) Reset() {
+	*x = BeamConstants{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_runner_api_proto_msgTypes[0]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *BeamConstants) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*BeamConstants) ProtoMessage() {}
+
+func (x *BeamConstants) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_runner_api_proto_msgTypes[0]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use BeamConstants.ProtoReflect.Descriptor instead.
 func (*BeamConstants) Descriptor() ([]byte, []int) {
-	return fileDescriptor_cf57597c3a9659a9, []int{0}
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{0}
 }
 
-func (m *BeamConstants) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_BeamConstants.Unmarshal(m, b)
-}
-func (m *BeamConstants) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_BeamConstants.Marshal(b, m, deterministic)
-}
-func (m *BeamConstants) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_BeamConstants.Merge(m, src)
-}
-func (m *BeamConstants) XXX_Size() int {
-	return xxx_messageInfo_BeamConstants.Size(m)
-}
-func (m *BeamConstants) XXX_DiscardUnknown() {
-	xxx_messageInfo_BeamConstants.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_BeamConstants proto.InternalMessageInfo
-
 // A set of mappings from id to message. This is included as an optional field
 // on any proto message that may contain references needing resolution.
 type Components struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
 	// (Required) A map from pipeline-scoped id to PTransform.
 	Transforms map[string]*PTransform `protobuf:"bytes,1,rep,name=transforms,proto3" json:"transforms,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"`
 	// (Required) A map from pipeline-scoped id to PCollection.
@@ -1131,68 +1621,72 @@
 	// (Required) A map from pipeline-scoped id to Coder.
 	Coders map[string]*Coder `protobuf:"bytes,4,rep,name=coders,proto3" json:"coders,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"`
 	// (Required) A map from pipeline-scoped id to Environment.
-	Environments         map[string]*Environment `protobuf:"bytes,5,rep,name=environments,proto3" json:"environments,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"`
-	XXX_NoUnkeyedLiteral struct{}                `json:"-"`
-	XXX_unrecognized     []byte                  `json:"-"`
-	XXX_sizecache        int32                   `json:"-"`
+	Environments map[string]*Environment `protobuf:"bytes,5,rep,name=environments,proto3" json:"environments,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"`
 }
 
-func (m *Components) Reset()         { *m = Components{} }
-func (m *Components) String() string { return proto.CompactTextString(m) }
-func (*Components) ProtoMessage()    {}
+func (x *Components) Reset() {
+	*x = Components{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_runner_api_proto_msgTypes[1]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *Components) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*Components) ProtoMessage() {}
+
+func (x *Components) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_runner_api_proto_msgTypes[1]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use Components.ProtoReflect.Descriptor instead.
 func (*Components) Descriptor() ([]byte, []int) {
-	return fileDescriptor_cf57597c3a9659a9, []int{1}
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{1}
 }
 
-func (m *Components) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_Components.Unmarshal(m, b)
-}
-func (m *Components) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_Components.Marshal(b, m, deterministic)
-}
-func (m *Components) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Components.Merge(m, src)
-}
-func (m *Components) XXX_Size() int {
-	return xxx_messageInfo_Components.Size(m)
-}
-func (m *Components) XXX_DiscardUnknown() {
-	xxx_messageInfo_Components.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_Components proto.InternalMessageInfo
-
-func (m *Components) GetTransforms() map[string]*PTransform {
-	if m != nil {
-		return m.Transforms
+func (x *Components) GetTransforms() map[string]*PTransform {
+	if x != nil {
+		return x.Transforms
 	}
 	return nil
 }
 
-func (m *Components) GetPcollections() map[string]*PCollection {
-	if m != nil {
-		return m.Pcollections
+func (x *Components) GetPcollections() map[string]*PCollection {
+	if x != nil {
+		return x.Pcollections
 	}
 	return nil
 }
 
-func (m *Components) GetWindowingStrategies() map[string]*WindowingStrategy {
-	if m != nil {
-		return m.WindowingStrategies
+func (x *Components) GetWindowingStrategies() map[string]*WindowingStrategy {
+	if x != nil {
+		return x.WindowingStrategies
 	}
 	return nil
 }
 
-func (m *Components) GetCoders() map[string]*Coder {
-	if m != nil {
-		return m.Coders
+func (x *Components) GetCoders() map[string]*Coder {
+	if x != nil {
+		return x.Coders
 	}
 	return nil
 }
 
-func (m *Components) GetEnvironments() map[string]*Environment {
-	if m != nil {
-		return m.Environments
+func (x *Components) GetEnvironments() map[string]*Environment {
+	if x != nil {
+		return x.Environments
 	}
 	return nil
 }
@@ -1209,6 +1703,10 @@
 // PCollections, SDK environments, coders, etc., for
 // supporting compact reuse and arbitrary graph structure.
 type Pipeline struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
 	// (Required) The coders, UDFs, graph nodes, etc, that make up
 	// this pipeline.
 	Components *Components `protobuf:"bytes,1,opt,name=components,proto3" json:"components,omitempty"`
@@ -1224,61 +1722,65 @@
 	// may indicate that a runner must inspect new fields on a component or
 	// provide additional guarantees when processing specific transforms.
 	// A runner should reject any pipelines with unknown requirements.
-	Requirements         []string `protobuf:"bytes,4,rep,name=requirements,proto3" json:"requirements,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
+	Requirements []string `protobuf:"bytes,4,rep,name=requirements,proto3" json:"requirements,omitempty"`
 }
 
-func (m *Pipeline) Reset()         { *m = Pipeline{} }
-func (m *Pipeline) String() string { return proto.CompactTextString(m) }
-func (*Pipeline) ProtoMessage()    {}
+func (x *Pipeline) Reset() {
+	*x = Pipeline{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_runner_api_proto_msgTypes[2]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *Pipeline) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*Pipeline) ProtoMessage() {}
+
+func (x *Pipeline) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_runner_api_proto_msgTypes[2]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use Pipeline.ProtoReflect.Descriptor instead.
 func (*Pipeline) Descriptor() ([]byte, []int) {
-	return fileDescriptor_cf57597c3a9659a9, []int{2}
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{2}
 }
 
-func (m *Pipeline) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_Pipeline.Unmarshal(m, b)
-}
-func (m *Pipeline) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_Pipeline.Marshal(b, m, deterministic)
-}
-func (m *Pipeline) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Pipeline.Merge(m, src)
-}
-func (m *Pipeline) XXX_Size() int {
-	return xxx_messageInfo_Pipeline.Size(m)
-}
-func (m *Pipeline) XXX_DiscardUnknown() {
-	xxx_messageInfo_Pipeline.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_Pipeline proto.InternalMessageInfo
-
-func (m *Pipeline) GetComponents() *Components {
-	if m != nil {
-		return m.Components
+func (x *Pipeline) GetComponents() *Components {
+	if x != nil {
+		return x.Components
 	}
 	return nil
 }
 
-func (m *Pipeline) GetRootTransformIds() []string {
-	if m != nil {
-		return m.RootTransformIds
+func (x *Pipeline) GetRootTransformIds() []string {
+	if x != nil {
+		return x.RootTransformIds
 	}
 	return nil
 }
 
-func (m *Pipeline) GetDisplayData() []*DisplayData {
-	if m != nil {
-		return m.DisplayData
+func (x *Pipeline) GetDisplayData() []*DisplayData {
+	if x != nil {
+		return x.DisplayData
 	}
 	return nil
 }
 
-func (m *Pipeline) GetRequirements() []string {
-	if m != nil {
-		return m.Requirements
+func (x *Pipeline) GetRequirements() []string {
+	if x != nil {
+		return x.Requirements
 	}
 	return nil
 }
@@ -1301,6 +1803,10 @@
 // collection. You can also define your own more complex composite transforms to
 // fit your pipeline’s exact use case.
 type PTransform struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
 	// (Required) A unique name for the application node.
 	//
 	// Ideally, this should be stable over multiple evolutions of a pipeline
@@ -1356,150 +1862,172 @@
 	//
 	// Transforms that are required to be implemented by a runner must omit this.
 	// All other transforms are required to specify this.
-	EnvironmentId        string   `protobuf:"bytes,7,opt,name=environment_id,json=environmentId,proto3" json:"environment_id,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
+	EnvironmentId string `protobuf:"bytes,7,opt,name=environment_id,json=environmentId,proto3" json:"environment_id,omitempty"`
 }
 
-func (m *PTransform) Reset()         { *m = PTransform{} }
-func (m *PTransform) String() string { return proto.CompactTextString(m) }
-func (*PTransform) ProtoMessage()    {}
+func (x *PTransform) Reset() {
+	*x = PTransform{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_runner_api_proto_msgTypes[3]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *PTransform) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*PTransform) ProtoMessage() {}
+
+func (x *PTransform) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_runner_api_proto_msgTypes[3]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use PTransform.ProtoReflect.Descriptor instead.
 func (*PTransform) Descriptor() ([]byte, []int) {
-	return fileDescriptor_cf57597c3a9659a9, []int{3}
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{3}
 }
 
-func (m *PTransform) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_PTransform.Unmarshal(m, b)
-}
-func (m *PTransform) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_PTransform.Marshal(b, m, deterministic)
-}
-func (m *PTransform) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_PTransform.Merge(m, src)
-}
-func (m *PTransform) XXX_Size() int {
-	return xxx_messageInfo_PTransform.Size(m)
-}
-func (m *PTransform) XXX_DiscardUnknown() {
-	xxx_messageInfo_PTransform.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_PTransform proto.InternalMessageInfo
-
-func (m *PTransform) GetUniqueName() string {
-	if m != nil {
-		return m.UniqueName
+func (x *PTransform) GetUniqueName() string {
+	if x != nil {
+		return x.UniqueName
 	}
 	return ""
 }
 
-func (m *PTransform) GetSpec() *FunctionSpec {
-	if m != nil {
-		return m.Spec
+func (x *PTransform) GetSpec() *FunctionSpec {
+	if x != nil {
+		return x.Spec
 	}
 	return nil
 }
 
-func (m *PTransform) GetSubtransforms() []string {
-	if m != nil {
-		return m.Subtransforms
+func (x *PTransform) GetSubtransforms() []string {
+	if x != nil {
+		return x.Subtransforms
 	}
 	return nil
 }
 
-func (m *PTransform) GetInputs() map[string]string {
-	if m != nil {
-		return m.Inputs
+func (x *PTransform) GetInputs() map[string]string {
+	if x != nil {
+		return x.Inputs
 	}
 	return nil
 }
 
-func (m *PTransform) GetOutputs() map[string]string {
-	if m != nil {
-		return m.Outputs
+func (x *PTransform) GetOutputs() map[string]string {
+	if x != nil {
+		return x.Outputs
 	}
 	return nil
 }
 
-func (m *PTransform) GetDisplayData() []*DisplayData {
-	if m != nil {
-		return m.DisplayData
+func (x *PTransform) GetDisplayData() []*DisplayData {
+	if x != nil {
+		return x.DisplayData
 	}
 	return nil
 }
 
-func (m *PTransform) GetEnvironmentId() string {
-	if m != nil {
-		return m.EnvironmentId
+func (x *PTransform) GetEnvironmentId() string {
+	if x != nil {
+		return x.EnvironmentId
 	}
 	return ""
 }
 
 type StandardPTransforms struct {
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
 }
 
-func (m *StandardPTransforms) Reset()         { *m = StandardPTransforms{} }
-func (m *StandardPTransforms) String() string { return proto.CompactTextString(m) }
-func (*StandardPTransforms) ProtoMessage()    {}
+func (x *StandardPTransforms) Reset() {
+	*x = StandardPTransforms{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_runner_api_proto_msgTypes[4]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *StandardPTransforms) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*StandardPTransforms) ProtoMessage() {}
+
+func (x *StandardPTransforms) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_runner_api_proto_msgTypes[4]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use StandardPTransforms.ProtoReflect.Descriptor instead.
 func (*StandardPTransforms) Descriptor() ([]byte, []int) {
-	return fileDescriptor_cf57597c3a9659a9, []int{4}
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{4}
 }
 
-func (m *StandardPTransforms) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_StandardPTransforms.Unmarshal(m, b)
-}
-func (m *StandardPTransforms) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_StandardPTransforms.Marshal(b, m, deterministic)
-}
-func (m *StandardPTransforms) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_StandardPTransforms.Merge(m, src)
-}
-func (m *StandardPTransforms) XXX_Size() int {
-	return xxx_messageInfo_StandardPTransforms.Size(m)
-}
-func (m *StandardPTransforms) XXX_DiscardUnknown() {
-	xxx_messageInfo_StandardPTransforms.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_StandardPTransforms proto.InternalMessageInfo
-
 type StandardSideInputTypes struct {
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
 }
 
-func (m *StandardSideInputTypes) Reset()         { *m = StandardSideInputTypes{} }
-func (m *StandardSideInputTypes) String() string { return proto.CompactTextString(m) }
-func (*StandardSideInputTypes) ProtoMessage()    {}
+func (x *StandardSideInputTypes) Reset() {
+	*x = StandardSideInputTypes{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_runner_api_proto_msgTypes[5]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *StandardSideInputTypes) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*StandardSideInputTypes) ProtoMessage() {}
+
+func (x *StandardSideInputTypes) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_runner_api_proto_msgTypes[5]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use StandardSideInputTypes.ProtoReflect.Descriptor instead.
 func (*StandardSideInputTypes) Descriptor() ([]byte, []int) {
-	return fileDescriptor_cf57597c3a9659a9, []int{5}
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{5}
 }
 
-func (m *StandardSideInputTypes) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_StandardSideInputTypes.Unmarshal(m, b)
-}
-func (m *StandardSideInputTypes) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_StandardSideInputTypes.Marshal(b, m, deterministic)
-}
-func (m *StandardSideInputTypes) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_StandardSideInputTypes.Merge(m, src)
-}
-func (m *StandardSideInputTypes) XXX_Size() int {
-	return xxx_messageInfo_StandardSideInputTypes.Size(m)
-}
-func (m *StandardSideInputTypes) XXX_DiscardUnknown() {
-	xxx_messageInfo_StandardSideInputTypes.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_StandardSideInputTypes proto.InternalMessageInfo
-
 // A PCollection!
 type PCollection struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
 	// (Required) A unique name for the PCollection.
 	//
 	// Ideally, this should be stable over multiple evolutions of a pipeline
@@ -1517,74 +2045,82 @@
 	WindowingStrategyId string `protobuf:"bytes,4,opt,name=windowing_strategy_id,json=windowingStrategyId,proto3" json:"windowing_strategy_id,omitempty"`
 	// (Optional) Static display data for the PCollection. If there is none,
 	// it may be omitted.
-	DisplayData          []*DisplayData `protobuf:"bytes,5,rep,name=display_data,json=displayData,proto3" json:"display_data,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}       `json:"-"`
-	XXX_unrecognized     []byte         `json:"-"`
-	XXX_sizecache        int32          `json:"-"`
+	DisplayData []*DisplayData `protobuf:"bytes,5,rep,name=display_data,json=displayData,proto3" json:"display_data,omitempty"`
 }
 
-func (m *PCollection) Reset()         { *m = PCollection{} }
-func (m *PCollection) String() string { return proto.CompactTextString(m) }
-func (*PCollection) ProtoMessage()    {}
+func (x *PCollection) Reset() {
+	*x = PCollection{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_runner_api_proto_msgTypes[6]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *PCollection) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*PCollection) ProtoMessage() {}
+
+func (x *PCollection) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_runner_api_proto_msgTypes[6]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use PCollection.ProtoReflect.Descriptor instead.
 func (*PCollection) Descriptor() ([]byte, []int) {
-	return fileDescriptor_cf57597c3a9659a9, []int{6}
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{6}
 }
 
-func (m *PCollection) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_PCollection.Unmarshal(m, b)
-}
-func (m *PCollection) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_PCollection.Marshal(b, m, deterministic)
-}
-func (m *PCollection) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_PCollection.Merge(m, src)
-}
-func (m *PCollection) XXX_Size() int {
-	return xxx_messageInfo_PCollection.Size(m)
-}
-func (m *PCollection) XXX_DiscardUnknown() {
-	xxx_messageInfo_PCollection.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_PCollection proto.InternalMessageInfo
-
-func (m *PCollection) GetUniqueName() string {
-	if m != nil {
-		return m.UniqueName
+func (x *PCollection) GetUniqueName() string {
+	if x != nil {
+		return x.UniqueName
 	}
 	return ""
 }
 
-func (m *PCollection) GetCoderId() string {
-	if m != nil {
-		return m.CoderId
+func (x *PCollection) GetCoderId() string {
+	if x != nil {
+		return x.CoderId
 	}
 	return ""
 }
 
-func (m *PCollection) GetIsBounded() IsBounded_Enum {
-	if m != nil {
-		return m.IsBounded
+func (x *PCollection) GetIsBounded() IsBounded_Enum {
+	if x != nil {
+		return x.IsBounded
 	}
 	return IsBounded_UNSPECIFIED
 }
 
-func (m *PCollection) GetWindowingStrategyId() string {
-	if m != nil {
-		return m.WindowingStrategyId
+func (x *PCollection) GetWindowingStrategyId() string {
+	if x != nil {
+		return x.WindowingStrategyId
 	}
 	return ""
 }
 
-func (m *PCollection) GetDisplayData() []*DisplayData {
-	if m != nil {
-		return m.DisplayData
+func (x *PCollection) GetDisplayData() []*DisplayData {
+	if x != nil {
+		return x.DisplayData
 	}
 	return nil
 }
 
 // The payload for the primitive ParDo transform.
 type ParDoPayload struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
 	// (Required) The FunctionSpec of the DoFn.
 	DoFn *FunctionSpec `protobuf:"bytes,1,opt,name=do_fn,json=doFn,proto3" json:"do_fn,omitempty"`
 	// (Optional) A mapping of local input names to side inputs, describing
@@ -1613,130 +2149,192 @@
 	// Whether this stage requires stable input.
 	// If this is set, the corresponding standard requirement should also
 	// be placed in the pipeline requirements.
-	RequiresStableInput  bool     `protobuf:"varint,11,opt,name=requires_stable_input,json=requiresStableInput,proto3" json:"requires_stable_input,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
+	RequiresStableInput bool `protobuf:"varint,11,opt,name=requires_stable_input,json=requiresStableInput,proto3" json:"requires_stable_input,omitempty"`
 }
 
-func (m *ParDoPayload) Reset()         { *m = ParDoPayload{} }
-func (m *ParDoPayload) String() string { return proto.CompactTextString(m) }
-func (*ParDoPayload) ProtoMessage()    {}
+func (x *ParDoPayload) Reset() {
+	*x = ParDoPayload{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_runner_api_proto_msgTypes[7]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *ParDoPayload) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*ParDoPayload) ProtoMessage() {}
+
+func (x *ParDoPayload) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_runner_api_proto_msgTypes[7]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use ParDoPayload.ProtoReflect.Descriptor instead.
 func (*ParDoPayload) Descriptor() ([]byte, []int) {
-	return fileDescriptor_cf57597c3a9659a9, []int{7}
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{7}
 }
 
-func (m *ParDoPayload) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_ParDoPayload.Unmarshal(m, b)
-}
-func (m *ParDoPayload) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_ParDoPayload.Marshal(b, m, deterministic)
-}
-func (m *ParDoPayload) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ParDoPayload.Merge(m, src)
-}
-func (m *ParDoPayload) XXX_Size() int {
-	return xxx_messageInfo_ParDoPayload.Size(m)
-}
-func (m *ParDoPayload) XXX_DiscardUnknown() {
-	xxx_messageInfo_ParDoPayload.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_ParDoPayload proto.InternalMessageInfo
-
-func (m *ParDoPayload) GetDoFn() *FunctionSpec {
-	if m != nil {
-		return m.DoFn
+func (x *ParDoPayload) GetDoFn() *FunctionSpec {
+	if x != nil {
+		return x.DoFn
 	}
 	return nil
 }
 
-func (m *ParDoPayload) GetSideInputs() map[string]*SideInput {
-	if m != nil {
-		return m.SideInputs
+func (x *ParDoPayload) GetSideInputs() map[string]*SideInput {
+	if x != nil {
+		return x.SideInputs
 	}
 	return nil
 }
 
-func (m *ParDoPayload) GetStateSpecs() map[string]*StateSpec {
-	if m != nil {
-		return m.StateSpecs
+func (x *ParDoPayload) GetStateSpecs() map[string]*StateSpec {
+	if x != nil {
+		return x.StateSpecs
 	}
 	return nil
 }
 
-func (m *ParDoPayload) GetTimerFamilySpecs() map[string]*TimerFamilySpec {
-	if m != nil {
-		return m.TimerFamilySpecs
+func (x *ParDoPayload) GetTimerFamilySpecs() map[string]*TimerFamilySpec {
+	if x != nil {
+		return x.TimerFamilySpecs
 	}
 	return nil
 }
 
-func (m *ParDoPayload) GetRestrictionCoderId() string {
-	if m != nil {
-		return m.RestrictionCoderId
+func (x *ParDoPayload) GetRestrictionCoderId() string {
+	if x != nil {
+		return x.RestrictionCoderId
 	}
 	return ""
 }
 
-func (m *ParDoPayload) GetRequestsFinalization() bool {
-	if m != nil {
-		return m.RequestsFinalization
+func (x *ParDoPayload) GetRequestsFinalization() bool {
+	if x != nil {
+		return x.RequestsFinalization
 	}
 	return false
 }
 
-func (m *ParDoPayload) GetRequiresTimeSortedInput() bool {
-	if m != nil {
-		return m.RequiresTimeSortedInput
+func (x *ParDoPayload) GetRequiresTimeSortedInput() bool {
+	if x != nil {
+		return x.RequiresTimeSortedInput
 	}
 	return false
 }
 
-func (m *ParDoPayload) GetRequiresStableInput() bool {
-	if m != nil {
-		return m.RequiresStableInput
+func (x *ParDoPayload) GetRequiresStableInput() bool {
+	if x != nil {
+		return x.RequiresStableInput
 	}
 	return false
 }
 
 type StateSpec struct {
-	// Types that are valid to be assigned to Spec:
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	// Types that are assignable to Spec:
 	//	*StateSpec_ReadModifyWriteSpec
 	//	*StateSpec_BagSpec
 	//	*StateSpec_CombiningSpec
 	//	*StateSpec_MapSpec
 	//	*StateSpec_SetSpec
-	Spec                 isStateSpec_Spec `protobuf_oneof:"spec"`
-	XXX_NoUnkeyedLiteral struct{}         `json:"-"`
-	XXX_unrecognized     []byte           `json:"-"`
-	XXX_sizecache        int32            `json:"-"`
+	//	*StateSpec_OrderedListSpec
+	Spec isStateSpec_Spec `protobuf_oneof:"spec"`
 }
 
-func (m *StateSpec) Reset()         { *m = StateSpec{} }
-func (m *StateSpec) String() string { return proto.CompactTextString(m) }
-func (*StateSpec) ProtoMessage()    {}
+func (x *StateSpec) Reset() {
+	*x = StateSpec{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_runner_api_proto_msgTypes[8]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *StateSpec) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*StateSpec) ProtoMessage() {}
+
+func (x *StateSpec) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_runner_api_proto_msgTypes[8]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use StateSpec.ProtoReflect.Descriptor instead.
 func (*StateSpec) Descriptor() ([]byte, []int) {
-	return fileDescriptor_cf57597c3a9659a9, []int{8}
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{8}
 }
 
-func (m *StateSpec) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_StateSpec.Unmarshal(m, b)
-}
-func (m *StateSpec) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_StateSpec.Marshal(b, m, deterministic)
-}
-func (m *StateSpec) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_StateSpec.Merge(m, src)
-}
-func (m *StateSpec) XXX_Size() int {
-	return xxx_messageInfo_StateSpec.Size(m)
-}
-func (m *StateSpec) XXX_DiscardUnknown() {
-	xxx_messageInfo_StateSpec.DiscardUnknown(m)
+func (m *StateSpec) GetSpec() isStateSpec_Spec {
+	if m != nil {
+		return m.Spec
+	}
+	return nil
 }
 
-var xxx_messageInfo_StateSpec proto.InternalMessageInfo
+func (x *StateSpec) GetReadModifyWriteSpec() *ReadModifyWriteStateSpec {
+	if x, ok := x.GetSpec().(*StateSpec_ReadModifyWriteSpec); ok {
+		return x.ReadModifyWriteSpec
+	}
+	return nil
+}
+
+func (x *StateSpec) GetBagSpec() *BagStateSpec {
+	if x, ok := x.GetSpec().(*StateSpec_BagSpec); ok {
+		return x.BagSpec
+	}
+	return nil
+}
+
+func (x *StateSpec) GetCombiningSpec() *CombiningStateSpec {
+	if x, ok := x.GetSpec().(*StateSpec_CombiningSpec); ok {
+		return x.CombiningSpec
+	}
+	return nil
+}
+
+func (x *StateSpec) GetMapSpec() *MapStateSpec {
+	if x, ok := x.GetSpec().(*StateSpec_MapSpec); ok {
+		return x.MapSpec
+	}
+	return nil
+}
+
+func (x *StateSpec) GetSetSpec() *SetStateSpec {
+	if x, ok := x.GetSpec().(*StateSpec_SetSpec); ok {
+		return x.SetSpec
+	}
+	return nil
+}
+
+func (x *StateSpec) GetOrderedListSpec() *OrderedListStateSpec {
+	if x, ok := x.GetSpec().(*StateSpec_OrderedListSpec); ok {
+		return x.OrderedListSpec
+	}
+	return nil
+}
 
 type isStateSpec_Spec interface {
 	isStateSpec_Spec()
@@ -1762,6 +2360,10 @@
 	SetSpec *SetStateSpec `protobuf:"bytes,5,opt,name=set_spec,json=setSpec,proto3,oneof"`
 }
 
+type StateSpec_OrderedListSpec struct {
+	OrderedListSpec *OrderedListStateSpec `protobuf:"bytes,6,opt,name=ordered_list_spec,json=orderedListSpec,proto3,oneof"`
+}
+
 func (*StateSpec_ReadModifyWriteSpec) isStateSpec_Spec() {}
 
 func (*StateSpec_BagSpec) isStateSpec_Spec() {}
@@ -1772,891 +2374,689 @@
 
 func (*StateSpec_SetSpec) isStateSpec_Spec() {}
 
-func (m *StateSpec) GetSpec() isStateSpec_Spec {
-	if m != nil {
-		return m.Spec
-	}
-	return nil
-}
-
-func (m *StateSpec) GetReadModifyWriteSpec() *ReadModifyWriteStateSpec {
-	if x, ok := m.GetSpec().(*StateSpec_ReadModifyWriteSpec); ok {
-		return x.ReadModifyWriteSpec
-	}
-	return nil
-}
-
-func (m *StateSpec) GetBagSpec() *BagStateSpec {
-	if x, ok := m.GetSpec().(*StateSpec_BagSpec); ok {
-		return x.BagSpec
-	}
-	return nil
-}
-
-func (m *StateSpec) GetCombiningSpec() *CombiningStateSpec {
-	if x, ok := m.GetSpec().(*StateSpec_CombiningSpec); ok {
-		return x.CombiningSpec
-	}
-	return nil
-}
-
-func (m *StateSpec) GetMapSpec() *MapStateSpec {
-	if x, ok := m.GetSpec().(*StateSpec_MapSpec); ok {
-		return x.MapSpec
-	}
-	return nil
-}
-
-func (m *StateSpec) GetSetSpec() *SetStateSpec {
-	if x, ok := m.GetSpec().(*StateSpec_SetSpec); ok {
-		return x.SetSpec
-	}
-	return nil
-}
-
-// XXX_OneofWrappers is for the internal use of the proto package.
-func (*StateSpec) XXX_OneofWrappers() []interface{} {
-	return []interface{}{
-		(*StateSpec_ReadModifyWriteSpec)(nil),
-		(*StateSpec_BagSpec)(nil),
-		(*StateSpec_CombiningSpec)(nil),
-		(*StateSpec_MapSpec)(nil),
-		(*StateSpec_SetSpec)(nil),
-	}
-}
+func (*StateSpec_OrderedListSpec) isStateSpec_Spec() {}
 
 type ReadModifyWriteStateSpec struct {
-	CoderId              string   `protobuf:"bytes,1,opt,name=coder_id,json=coderId,proto3" json:"coder_id,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	CoderId string `protobuf:"bytes,1,opt,name=coder_id,json=coderId,proto3" json:"coder_id,omitempty"`
 }
 
-func (m *ReadModifyWriteStateSpec) Reset()         { *m = ReadModifyWriteStateSpec{} }
-func (m *ReadModifyWriteStateSpec) String() string { return proto.CompactTextString(m) }
-func (*ReadModifyWriteStateSpec) ProtoMessage()    {}
+func (x *ReadModifyWriteStateSpec) Reset() {
+	*x = ReadModifyWriteStateSpec{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_runner_api_proto_msgTypes[9]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *ReadModifyWriteStateSpec) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*ReadModifyWriteStateSpec) ProtoMessage() {}
+
+func (x *ReadModifyWriteStateSpec) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_runner_api_proto_msgTypes[9]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use ReadModifyWriteStateSpec.ProtoReflect.Descriptor instead.
 func (*ReadModifyWriteStateSpec) Descriptor() ([]byte, []int) {
-	return fileDescriptor_cf57597c3a9659a9, []int{9}
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{9}
 }
 
-func (m *ReadModifyWriteStateSpec) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_ReadModifyWriteStateSpec.Unmarshal(m, b)
-}
-func (m *ReadModifyWriteStateSpec) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_ReadModifyWriteStateSpec.Marshal(b, m, deterministic)
-}
-func (m *ReadModifyWriteStateSpec) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ReadModifyWriteStateSpec.Merge(m, src)
-}
-func (m *ReadModifyWriteStateSpec) XXX_Size() int {
-	return xxx_messageInfo_ReadModifyWriteStateSpec.Size(m)
-}
-func (m *ReadModifyWriteStateSpec) XXX_DiscardUnknown() {
-	xxx_messageInfo_ReadModifyWriteStateSpec.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_ReadModifyWriteStateSpec proto.InternalMessageInfo
-
-func (m *ReadModifyWriteStateSpec) GetCoderId() string {
-	if m != nil {
-		return m.CoderId
+func (x *ReadModifyWriteStateSpec) GetCoderId() string {
+	if x != nil {
+		return x.CoderId
 	}
 	return ""
 }
 
 type BagStateSpec struct {
-	ElementCoderId       string   `protobuf:"bytes,1,opt,name=element_coder_id,json=elementCoderId,proto3" json:"element_coder_id,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	ElementCoderId string `protobuf:"bytes,1,opt,name=element_coder_id,json=elementCoderId,proto3" json:"element_coder_id,omitempty"`
 }
 
-func (m *BagStateSpec) Reset()         { *m = BagStateSpec{} }
-func (m *BagStateSpec) String() string { return proto.CompactTextString(m) }
-func (*BagStateSpec) ProtoMessage()    {}
+func (x *BagStateSpec) Reset() {
+	*x = BagStateSpec{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_runner_api_proto_msgTypes[10]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *BagStateSpec) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*BagStateSpec) ProtoMessage() {}
+
+func (x *BagStateSpec) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_runner_api_proto_msgTypes[10]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use BagStateSpec.ProtoReflect.Descriptor instead.
 func (*BagStateSpec) Descriptor() ([]byte, []int) {
-	return fileDescriptor_cf57597c3a9659a9, []int{10}
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{10}
 }
 
-func (m *BagStateSpec) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_BagStateSpec.Unmarshal(m, b)
-}
-func (m *BagStateSpec) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_BagStateSpec.Marshal(b, m, deterministic)
-}
-func (m *BagStateSpec) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_BagStateSpec.Merge(m, src)
-}
-func (m *BagStateSpec) XXX_Size() int {
-	return xxx_messageInfo_BagStateSpec.Size(m)
-}
-func (m *BagStateSpec) XXX_DiscardUnknown() {
-	xxx_messageInfo_BagStateSpec.DiscardUnknown(m)
+func (x *BagStateSpec) GetElementCoderId() string {
+	if x != nil {
+		return x.ElementCoderId
+	}
+	return ""
 }
 
-var xxx_messageInfo_BagStateSpec proto.InternalMessageInfo
+type OrderedListStateSpec struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
 
-func (m *BagStateSpec) GetElementCoderId() string {
-	if m != nil {
-		return m.ElementCoderId
+	ElementCoderId string `protobuf:"bytes,1,opt,name=element_coder_id,json=elementCoderId,proto3" json:"element_coder_id,omitempty"`
+}
+
+func (x *OrderedListStateSpec) Reset() {
+	*x = OrderedListStateSpec{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_runner_api_proto_msgTypes[11]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *OrderedListStateSpec) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*OrderedListStateSpec) ProtoMessage() {}
+
+func (x *OrderedListStateSpec) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_runner_api_proto_msgTypes[11]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use OrderedListStateSpec.ProtoReflect.Descriptor instead.
+func (*OrderedListStateSpec) Descriptor() ([]byte, []int) {
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{11}
+}
+
+func (x *OrderedListStateSpec) GetElementCoderId() string {
+	if x != nil {
+		return x.ElementCoderId
 	}
 	return ""
 }
 
 type CombiningStateSpec struct {
-	AccumulatorCoderId   string        `protobuf:"bytes,1,opt,name=accumulator_coder_id,json=accumulatorCoderId,proto3" json:"accumulator_coder_id,omitempty"`
-	CombineFn            *FunctionSpec `protobuf:"bytes,2,opt,name=combine_fn,json=combineFn,proto3" json:"combine_fn,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}      `json:"-"`
-	XXX_unrecognized     []byte        `json:"-"`
-	XXX_sizecache        int32         `json:"-"`
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	AccumulatorCoderId string        `protobuf:"bytes,1,opt,name=accumulator_coder_id,json=accumulatorCoderId,proto3" json:"accumulator_coder_id,omitempty"`
+	CombineFn          *FunctionSpec `protobuf:"bytes,2,opt,name=combine_fn,json=combineFn,proto3" json:"combine_fn,omitempty"`
 }
 
-func (m *CombiningStateSpec) Reset()         { *m = CombiningStateSpec{} }
-func (m *CombiningStateSpec) String() string { return proto.CompactTextString(m) }
-func (*CombiningStateSpec) ProtoMessage()    {}
+func (x *CombiningStateSpec) Reset() {
+	*x = CombiningStateSpec{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_runner_api_proto_msgTypes[12]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *CombiningStateSpec) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*CombiningStateSpec) ProtoMessage() {}
+
+func (x *CombiningStateSpec) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_runner_api_proto_msgTypes[12]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use CombiningStateSpec.ProtoReflect.Descriptor instead.
 func (*CombiningStateSpec) Descriptor() ([]byte, []int) {
-	return fileDescriptor_cf57597c3a9659a9, []int{11}
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{12}
 }
 
-func (m *CombiningStateSpec) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_CombiningStateSpec.Unmarshal(m, b)
-}
-func (m *CombiningStateSpec) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_CombiningStateSpec.Marshal(b, m, deterministic)
-}
-func (m *CombiningStateSpec) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_CombiningStateSpec.Merge(m, src)
-}
-func (m *CombiningStateSpec) XXX_Size() int {
-	return xxx_messageInfo_CombiningStateSpec.Size(m)
-}
-func (m *CombiningStateSpec) XXX_DiscardUnknown() {
-	xxx_messageInfo_CombiningStateSpec.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_CombiningStateSpec proto.InternalMessageInfo
-
-func (m *CombiningStateSpec) GetAccumulatorCoderId() string {
-	if m != nil {
-		return m.AccumulatorCoderId
+func (x *CombiningStateSpec) GetAccumulatorCoderId() string {
+	if x != nil {
+		return x.AccumulatorCoderId
 	}
 	return ""
 }
 
-func (m *CombiningStateSpec) GetCombineFn() *FunctionSpec {
-	if m != nil {
-		return m.CombineFn
+func (x *CombiningStateSpec) GetCombineFn() *FunctionSpec {
+	if x != nil {
+		return x.CombineFn
 	}
 	return nil
 }
 
 type MapStateSpec struct {
-	KeyCoderId           string   `protobuf:"bytes,1,opt,name=key_coder_id,json=keyCoderId,proto3" json:"key_coder_id,omitempty"`
-	ValueCoderId         string   `protobuf:"bytes,2,opt,name=value_coder_id,json=valueCoderId,proto3" json:"value_coder_id,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	KeyCoderId   string `protobuf:"bytes,1,opt,name=key_coder_id,json=keyCoderId,proto3" json:"key_coder_id,omitempty"`
+	ValueCoderId string `protobuf:"bytes,2,opt,name=value_coder_id,json=valueCoderId,proto3" json:"value_coder_id,omitempty"`
 }
 
-func (m *MapStateSpec) Reset()         { *m = MapStateSpec{} }
-func (m *MapStateSpec) String() string { return proto.CompactTextString(m) }
-func (*MapStateSpec) ProtoMessage()    {}
+func (x *MapStateSpec) Reset() {
+	*x = MapStateSpec{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_runner_api_proto_msgTypes[13]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *MapStateSpec) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*MapStateSpec) ProtoMessage() {}
+
+func (x *MapStateSpec) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_runner_api_proto_msgTypes[13]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use MapStateSpec.ProtoReflect.Descriptor instead.
 func (*MapStateSpec) Descriptor() ([]byte, []int) {
-	return fileDescriptor_cf57597c3a9659a9, []int{12}
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{13}
 }
 
-func (m *MapStateSpec) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_MapStateSpec.Unmarshal(m, b)
-}
-func (m *MapStateSpec) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_MapStateSpec.Marshal(b, m, deterministic)
-}
-func (m *MapStateSpec) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_MapStateSpec.Merge(m, src)
-}
-func (m *MapStateSpec) XXX_Size() int {
-	return xxx_messageInfo_MapStateSpec.Size(m)
-}
-func (m *MapStateSpec) XXX_DiscardUnknown() {
-	xxx_messageInfo_MapStateSpec.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_MapStateSpec proto.InternalMessageInfo
-
-func (m *MapStateSpec) GetKeyCoderId() string {
-	if m != nil {
-		return m.KeyCoderId
+func (x *MapStateSpec) GetKeyCoderId() string {
+	if x != nil {
+		return x.KeyCoderId
 	}
 	return ""
 }
 
-func (m *MapStateSpec) GetValueCoderId() string {
-	if m != nil {
-		return m.ValueCoderId
+func (x *MapStateSpec) GetValueCoderId() string {
+	if x != nil {
+		return x.ValueCoderId
 	}
 	return ""
 }
 
 type SetStateSpec struct {
-	ElementCoderId       string   `protobuf:"bytes,1,opt,name=element_coder_id,json=elementCoderId,proto3" json:"element_coder_id,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	ElementCoderId string `protobuf:"bytes,1,opt,name=element_coder_id,json=elementCoderId,proto3" json:"element_coder_id,omitempty"`
 }
 
-func (m *SetStateSpec) Reset()         { *m = SetStateSpec{} }
-func (m *SetStateSpec) String() string { return proto.CompactTextString(m) }
-func (*SetStateSpec) ProtoMessage()    {}
+func (x *SetStateSpec) Reset() {
+	*x = SetStateSpec{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_runner_api_proto_msgTypes[14]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *SetStateSpec) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*SetStateSpec) ProtoMessage() {}
+
+func (x *SetStateSpec) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_runner_api_proto_msgTypes[14]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use SetStateSpec.ProtoReflect.Descriptor instead.
 func (*SetStateSpec) Descriptor() ([]byte, []int) {
-	return fileDescriptor_cf57597c3a9659a9, []int{13}
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{14}
 }
 
-func (m *SetStateSpec) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_SetStateSpec.Unmarshal(m, b)
-}
-func (m *SetStateSpec) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_SetStateSpec.Marshal(b, m, deterministic)
-}
-func (m *SetStateSpec) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_SetStateSpec.Merge(m, src)
-}
-func (m *SetStateSpec) XXX_Size() int {
-	return xxx_messageInfo_SetStateSpec.Size(m)
-}
-func (m *SetStateSpec) XXX_DiscardUnknown() {
-	xxx_messageInfo_SetStateSpec.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_SetStateSpec proto.InternalMessageInfo
-
-func (m *SetStateSpec) GetElementCoderId() string {
-	if m != nil {
-		return m.ElementCoderId
+func (x *SetStateSpec) GetElementCoderId() string {
+	if x != nil {
+		return x.ElementCoderId
 	}
 	return ""
 }
 
 type TimerFamilySpec struct {
-	TimeDomain           TimeDomain_Enum `protobuf:"varint,1,opt,name=time_domain,json=timeDomain,proto3,enum=org.apache.beam.model.pipeline.v1.TimeDomain_Enum" json:"time_domain,omitempty"`
-	TimerFamilyCoderId   string          `protobuf:"bytes,2,opt,name=timer_family_coder_id,json=timerFamilyCoderId,proto3" json:"timer_family_coder_id,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}        `json:"-"`
-	XXX_unrecognized     []byte          `json:"-"`
-	XXX_sizecache        int32           `json:"-"`
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	TimeDomain         TimeDomain_Enum `protobuf:"varint,1,opt,name=time_domain,json=timeDomain,proto3,enum=org.apache.beam.model.pipeline.v1.TimeDomain_Enum" json:"time_domain,omitempty"`
+	TimerFamilyCoderId string          `protobuf:"bytes,2,opt,name=timer_family_coder_id,json=timerFamilyCoderId,proto3" json:"timer_family_coder_id,omitempty"`
 }
 
-func (m *TimerFamilySpec) Reset()         { *m = TimerFamilySpec{} }
-func (m *TimerFamilySpec) String() string { return proto.CompactTextString(m) }
-func (*TimerFamilySpec) ProtoMessage()    {}
+func (x *TimerFamilySpec) Reset() {
+	*x = TimerFamilySpec{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_runner_api_proto_msgTypes[15]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *TimerFamilySpec) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*TimerFamilySpec) ProtoMessage() {}
+
+func (x *TimerFamilySpec) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_runner_api_proto_msgTypes[15]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use TimerFamilySpec.ProtoReflect.Descriptor instead.
 func (*TimerFamilySpec) Descriptor() ([]byte, []int) {
-	return fileDescriptor_cf57597c3a9659a9, []int{14}
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{15}
 }
 
-func (m *TimerFamilySpec) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_TimerFamilySpec.Unmarshal(m, b)
-}
-func (m *TimerFamilySpec) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_TimerFamilySpec.Marshal(b, m, deterministic)
-}
-func (m *TimerFamilySpec) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_TimerFamilySpec.Merge(m, src)
-}
-func (m *TimerFamilySpec) XXX_Size() int {
-	return xxx_messageInfo_TimerFamilySpec.Size(m)
-}
-func (m *TimerFamilySpec) XXX_DiscardUnknown() {
-	xxx_messageInfo_TimerFamilySpec.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_TimerFamilySpec proto.InternalMessageInfo
-
-func (m *TimerFamilySpec) GetTimeDomain() TimeDomain_Enum {
-	if m != nil {
-		return m.TimeDomain
+func (x *TimerFamilySpec) GetTimeDomain() TimeDomain_Enum {
+	if x != nil {
+		return x.TimeDomain
 	}
 	return TimeDomain_UNSPECIFIED
 }
 
-func (m *TimerFamilySpec) GetTimerFamilyCoderId() string {
-	if m != nil {
-		return m.TimerFamilyCoderId
+func (x *TimerFamilySpec) GetTimerFamilyCoderId() string {
+	if x != nil {
+		return x.TimerFamilyCoderId
 	}
 	return ""
 }
 
 type IsBounded struct {
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
 }
 
-func (m *IsBounded) Reset()         { *m = IsBounded{} }
-func (m *IsBounded) String() string { return proto.CompactTextString(m) }
-func (*IsBounded) ProtoMessage()    {}
+func (x *IsBounded) Reset() {
+	*x = IsBounded{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_runner_api_proto_msgTypes[16]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *IsBounded) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*IsBounded) ProtoMessage() {}
+
+func (x *IsBounded) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_runner_api_proto_msgTypes[16]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use IsBounded.ProtoReflect.Descriptor instead.
 func (*IsBounded) Descriptor() ([]byte, []int) {
-	return fileDescriptor_cf57597c3a9659a9, []int{15}
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{16}
 }
 
-func (m *IsBounded) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_IsBounded.Unmarshal(m, b)
-}
-func (m *IsBounded) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_IsBounded.Marshal(b, m, deterministic)
-}
-func (m *IsBounded) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_IsBounded.Merge(m, src)
-}
-func (m *IsBounded) XXX_Size() int {
-	return xxx_messageInfo_IsBounded.Size(m)
-}
-func (m *IsBounded) XXX_DiscardUnknown() {
-	xxx_messageInfo_IsBounded.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_IsBounded proto.InternalMessageInfo
-
 // The payload for the primitive Read transform.
 type ReadPayload struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
 	// (Required) The FunctionSpec of the source for this Read.
 	Source *FunctionSpec `protobuf:"bytes,1,opt,name=source,proto3" json:"source,omitempty"`
 	// (Required) Whether the source is bounded or unbounded
-	IsBounded            IsBounded_Enum `protobuf:"varint,2,opt,name=is_bounded,json=isBounded,proto3,enum=org.apache.beam.model.pipeline.v1.IsBounded_Enum" json:"is_bounded,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}       `json:"-"`
-	XXX_unrecognized     []byte         `json:"-"`
-	XXX_sizecache        int32          `json:"-"`
+	IsBounded IsBounded_Enum `protobuf:"varint,2,opt,name=is_bounded,json=isBounded,proto3,enum=org.apache.beam.model.pipeline.v1.IsBounded_Enum" json:"is_bounded,omitempty"`
 }
 
-func (m *ReadPayload) Reset()         { *m = ReadPayload{} }
-func (m *ReadPayload) String() string { return proto.CompactTextString(m) }
-func (*ReadPayload) ProtoMessage()    {}
+func (x *ReadPayload) Reset() {
+	*x = ReadPayload{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_runner_api_proto_msgTypes[17]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *ReadPayload) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*ReadPayload) ProtoMessage() {}
+
+func (x *ReadPayload) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_runner_api_proto_msgTypes[17]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use ReadPayload.ProtoReflect.Descriptor instead.
 func (*ReadPayload) Descriptor() ([]byte, []int) {
-	return fileDescriptor_cf57597c3a9659a9, []int{16}
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{17}
 }
 
-func (m *ReadPayload) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_ReadPayload.Unmarshal(m, b)
-}
-func (m *ReadPayload) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_ReadPayload.Marshal(b, m, deterministic)
-}
-func (m *ReadPayload) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ReadPayload.Merge(m, src)
-}
-func (m *ReadPayload) XXX_Size() int {
-	return xxx_messageInfo_ReadPayload.Size(m)
-}
-func (m *ReadPayload) XXX_DiscardUnknown() {
-	xxx_messageInfo_ReadPayload.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_ReadPayload proto.InternalMessageInfo
-
-func (m *ReadPayload) GetSource() *FunctionSpec {
-	if m != nil {
-		return m.Source
+func (x *ReadPayload) GetSource() *FunctionSpec {
+	if x != nil {
+		return x.Source
 	}
 	return nil
 }
 
-func (m *ReadPayload) GetIsBounded() IsBounded_Enum {
-	if m != nil {
-		return m.IsBounded
+func (x *ReadPayload) GetIsBounded() IsBounded_Enum {
+	if x != nil {
+		return x.IsBounded
 	}
 	return IsBounded_UNSPECIFIED
 }
 
 // The payload for the WindowInto transform.
 type WindowIntoPayload struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
 	// (Required) The FunctionSpec of the WindowFn.
-	WindowFn             *FunctionSpec `protobuf:"bytes,1,opt,name=window_fn,json=windowFn,proto3" json:"window_fn,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}      `json:"-"`
-	XXX_unrecognized     []byte        `json:"-"`
-	XXX_sizecache        int32         `json:"-"`
+	WindowFn *FunctionSpec `protobuf:"bytes,1,opt,name=window_fn,json=windowFn,proto3" json:"window_fn,omitempty"`
 }
 
-func (m *WindowIntoPayload) Reset()         { *m = WindowIntoPayload{} }
-func (m *WindowIntoPayload) String() string { return proto.CompactTextString(m) }
-func (*WindowIntoPayload) ProtoMessage()    {}
+func (x *WindowIntoPayload) Reset() {
+	*x = WindowIntoPayload{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_runner_api_proto_msgTypes[18]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *WindowIntoPayload) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*WindowIntoPayload) ProtoMessage() {}
+
+func (x *WindowIntoPayload) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_runner_api_proto_msgTypes[18]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use WindowIntoPayload.ProtoReflect.Descriptor instead.
 func (*WindowIntoPayload) Descriptor() ([]byte, []int) {
-	return fileDescriptor_cf57597c3a9659a9, []int{17}
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{18}
 }
 
-func (m *WindowIntoPayload) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_WindowIntoPayload.Unmarshal(m, b)
-}
-func (m *WindowIntoPayload) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_WindowIntoPayload.Marshal(b, m, deterministic)
-}
-func (m *WindowIntoPayload) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_WindowIntoPayload.Merge(m, src)
-}
-func (m *WindowIntoPayload) XXX_Size() int {
-	return xxx_messageInfo_WindowIntoPayload.Size(m)
-}
-func (m *WindowIntoPayload) XXX_DiscardUnknown() {
-	xxx_messageInfo_WindowIntoPayload.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_WindowIntoPayload proto.InternalMessageInfo
-
-func (m *WindowIntoPayload) GetWindowFn() *FunctionSpec {
-	if m != nil {
-		return m.WindowFn
+func (x *WindowIntoPayload) GetWindowFn() *FunctionSpec {
+	if x != nil {
+		return x.WindowFn
 	}
 	return nil
 }
 
 // The payload for the special-but-not-primitive Combine transform.
 type CombinePayload struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
 	// (Required) The FunctionSpec of the CombineFn.
 	CombineFn *FunctionSpec `protobuf:"bytes,1,opt,name=combine_fn,json=combineFn,proto3" json:"combine_fn,omitempty"`
 	// (Required) A reference to the Coder to use for accumulators of the CombineFn
-	AccumulatorCoderId   string   `protobuf:"bytes,2,opt,name=accumulator_coder_id,json=accumulatorCoderId,proto3" json:"accumulator_coder_id,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
+	AccumulatorCoderId string `protobuf:"bytes,2,opt,name=accumulator_coder_id,json=accumulatorCoderId,proto3" json:"accumulator_coder_id,omitempty"`
 }
 
-func (m *CombinePayload) Reset()         { *m = CombinePayload{} }
-func (m *CombinePayload) String() string { return proto.CompactTextString(m) }
-func (*CombinePayload) ProtoMessage()    {}
+func (x *CombinePayload) Reset() {
+	*x = CombinePayload{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_runner_api_proto_msgTypes[19]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *CombinePayload) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*CombinePayload) ProtoMessage() {}
+
+func (x *CombinePayload) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_runner_api_proto_msgTypes[19]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use CombinePayload.ProtoReflect.Descriptor instead.
 func (*CombinePayload) Descriptor() ([]byte, []int) {
-	return fileDescriptor_cf57597c3a9659a9, []int{18}
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{19}
 }
 
-func (m *CombinePayload) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_CombinePayload.Unmarshal(m, b)
-}
-func (m *CombinePayload) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_CombinePayload.Marshal(b, m, deterministic)
-}
-func (m *CombinePayload) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_CombinePayload.Merge(m, src)
-}
-func (m *CombinePayload) XXX_Size() int {
-	return xxx_messageInfo_CombinePayload.Size(m)
-}
-func (m *CombinePayload) XXX_DiscardUnknown() {
-	xxx_messageInfo_CombinePayload.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_CombinePayload proto.InternalMessageInfo
-
-func (m *CombinePayload) GetCombineFn() *FunctionSpec {
-	if m != nil {
-		return m.CombineFn
+func (x *CombinePayload) GetCombineFn() *FunctionSpec {
+	if x != nil {
+		return x.CombineFn
 	}
 	return nil
 }
 
-func (m *CombinePayload) GetAccumulatorCoderId() string {
-	if m != nil {
-		return m.AccumulatorCoderId
+func (x *CombinePayload) GetAccumulatorCoderId() string {
+	if x != nil {
+		return x.AccumulatorCoderId
 	}
 	return ""
 }
 
 // The payload for the test-only primitive TestStream
 type TestStreamPayload struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
 	// (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"`
 	// (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:"-"`
+	Endpoint *ApiServiceDescriptor `protobuf:"bytes,3,opt,name=endpoint,proto3" json:"endpoint,omitempty"`
 }
 
-func (m *TestStreamPayload) Reset()         { *m = TestStreamPayload{} }
-func (m *TestStreamPayload) String() string { return proto.CompactTextString(m) }
-func (*TestStreamPayload) ProtoMessage()    {}
+func (x *TestStreamPayload) Reset() {
+	*x = TestStreamPayload{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_runner_api_proto_msgTypes[20]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *TestStreamPayload) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*TestStreamPayload) ProtoMessage() {}
+
+func (x *TestStreamPayload) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_runner_api_proto_msgTypes[20]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use TestStreamPayload.ProtoReflect.Descriptor instead.
 func (*TestStreamPayload) Descriptor() ([]byte, []int) {
-	return fileDescriptor_cf57597c3a9659a9, []int{19}
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{20}
 }
 
-func (m *TestStreamPayload) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_TestStreamPayload.Unmarshal(m, b)
-}
-func (m *TestStreamPayload) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_TestStreamPayload.Marshal(b, m, deterministic)
-}
-func (m *TestStreamPayload) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_TestStreamPayload.Merge(m, src)
-}
-func (m *TestStreamPayload) XXX_Size() int {
-	return xxx_messageInfo_TestStreamPayload.Size(m)
-}
-func (m *TestStreamPayload) XXX_DiscardUnknown() {
-	xxx_messageInfo_TestStreamPayload.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_TestStreamPayload proto.InternalMessageInfo
-
-func (m *TestStreamPayload) GetCoderId() string {
-	if m != nil {
-		return m.CoderId
+func (x *TestStreamPayload) GetCoderId() string {
+	if x != nil {
+		return x.CoderId
 	}
 	return ""
 }
 
-func (m *TestStreamPayload) GetEvents() []*TestStreamPayload_Event {
-	if m != nil {
-		return m.Events
+func (x *TestStreamPayload) GetEvents() []*TestStreamPayload_Event {
+	if x != nil {
+		return x.Events
 	}
 	return nil
 }
 
-func (m *TestStreamPayload) GetEndpoint() *ApiServiceDescriptor {
-	if m != nil {
-		return m.Endpoint
+func (x *TestStreamPayload) GetEndpoint() *ApiServiceDescriptor {
+	if x != nil {
+		return x.Endpoint
 	}
 	return nil
 }
 
-type TestStreamPayload_Event struct {
-	// Types that are valid to be assigned to Event:
-	//	*TestStreamPayload_Event_WatermarkEvent
-	//	*TestStreamPayload_Event_ProcessingTimeEvent
-	//	*TestStreamPayload_Event_ElementEvent
-	Event                isTestStreamPayload_Event_Event `protobuf_oneof:"event"`
-	XXX_NoUnkeyedLiteral struct{}                        `json:"-"`
-	XXX_unrecognized     []byte                          `json:"-"`
-	XXX_sizecache        int32                           `json:"-"`
-}
-
-func (m *TestStreamPayload_Event) Reset()         { *m = TestStreamPayload_Event{} }
-func (m *TestStreamPayload_Event) String() string { return proto.CompactTextString(m) }
-func (*TestStreamPayload_Event) ProtoMessage()    {}
-func (*TestStreamPayload_Event) Descriptor() ([]byte, []int) {
-	return fileDescriptor_cf57597c3a9659a9, []int{19, 0}
-}
-
-func (m *TestStreamPayload_Event) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_TestStreamPayload_Event.Unmarshal(m, b)
-}
-func (m *TestStreamPayload_Event) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_TestStreamPayload_Event.Marshal(b, m, deterministic)
-}
-func (m *TestStreamPayload_Event) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_TestStreamPayload_Event.Merge(m, src)
-}
-func (m *TestStreamPayload_Event) XXX_Size() int {
-	return xxx_messageInfo_TestStreamPayload_Event.Size(m)
-}
-func (m *TestStreamPayload_Event) XXX_DiscardUnknown() {
-	xxx_messageInfo_TestStreamPayload_Event.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_TestStreamPayload_Event proto.InternalMessageInfo
-
-type isTestStreamPayload_Event_Event interface {
-	isTestStreamPayload_Event_Event()
-}
-
-type TestStreamPayload_Event_WatermarkEvent struct {
-	WatermarkEvent *TestStreamPayload_Event_AdvanceWatermark `protobuf:"bytes,1,opt,name=watermark_event,json=watermarkEvent,proto3,oneof"`
-}
-
-type TestStreamPayload_Event_ProcessingTimeEvent struct {
-	ProcessingTimeEvent *TestStreamPayload_Event_AdvanceProcessingTime `protobuf:"bytes,2,opt,name=processing_time_event,json=processingTimeEvent,proto3,oneof"`
-}
-
-type TestStreamPayload_Event_ElementEvent struct {
-	ElementEvent *TestStreamPayload_Event_AddElements `protobuf:"bytes,3,opt,name=element_event,json=elementEvent,proto3,oneof"`
-}
-
-func (*TestStreamPayload_Event_WatermarkEvent) isTestStreamPayload_Event_Event() {}
-
-func (*TestStreamPayload_Event_ProcessingTimeEvent) isTestStreamPayload_Event_Event() {}
-
-func (*TestStreamPayload_Event_ElementEvent) isTestStreamPayload_Event_Event() {}
-
-func (m *TestStreamPayload_Event) GetEvent() isTestStreamPayload_Event_Event {
-	if m != nil {
-		return m.Event
-	}
-	return nil
-}
-
-func (m *TestStreamPayload_Event) GetWatermarkEvent() *TestStreamPayload_Event_AdvanceWatermark {
-	if x, ok := m.GetEvent().(*TestStreamPayload_Event_WatermarkEvent); ok {
-		return x.WatermarkEvent
-	}
-	return nil
-}
-
-func (m *TestStreamPayload_Event) GetProcessingTimeEvent() *TestStreamPayload_Event_AdvanceProcessingTime {
-	if x, ok := m.GetEvent().(*TestStreamPayload_Event_ProcessingTimeEvent); ok {
-		return x.ProcessingTimeEvent
-	}
-	return nil
-}
-
-func (m *TestStreamPayload_Event) GetElementEvent() *TestStreamPayload_Event_AddElements {
-	if x, ok := m.GetEvent().(*TestStreamPayload_Event_ElementEvent); ok {
-		return x.ElementEvent
-	}
-	return nil
-}
-
-// XXX_OneofWrappers is for the internal use of the proto package.
-func (*TestStreamPayload_Event) XXX_OneofWrappers() []interface{} {
-	return []interface{}{
-		(*TestStreamPayload_Event_WatermarkEvent)(nil),
-		(*TestStreamPayload_Event_ProcessingTimeEvent)(nil),
-		(*TestStreamPayload_Event_ElementEvent)(nil),
-	}
-}
-
-// Advances the watermark to the specified timestamp.
-type TestStreamPayload_Event_AdvanceWatermark struct {
-	// (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:"-"`
-}
-
-func (m *TestStreamPayload_Event_AdvanceWatermark) Reset() {
-	*m = TestStreamPayload_Event_AdvanceWatermark{}
-}
-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_cf57597c3a9659a9, []int{19, 0, 0}
-}
-
-func (m *TestStreamPayload_Event_AdvanceWatermark) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_TestStreamPayload_Event_AdvanceWatermark.Unmarshal(m, b)
-}
-func (m *TestStreamPayload_Event_AdvanceWatermark) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_TestStreamPayload_Event_AdvanceWatermark.Marshal(b, m, deterministic)
-}
-func (m *TestStreamPayload_Event_AdvanceWatermark) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_TestStreamPayload_Event_AdvanceWatermark.Merge(m, src)
-}
-func (m *TestStreamPayload_Event_AdvanceWatermark) XXX_Size() int {
-	return xxx_messageInfo_TestStreamPayload_Event_AdvanceWatermark.Size(m)
-}
-func (m *TestStreamPayload_Event_AdvanceWatermark) XXX_DiscardUnknown() {
-	xxx_messageInfo_TestStreamPayload_Event_AdvanceWatermark.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_TestStreamPayload_Event_AdvanceWatermark proto.InternalMessageInfo
-
-func (m *TestStreamPayload_Event_AdvanceWatermark) GetNewWatermark() int64 {
-	if m != nil {
-		return m.NewWatermark
-	}
-	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:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *TestStreamPayload_Event_AdvanceProcessingTime) Reset() {
-	*m = TestStreamPayload_Event_AdvanceProcessingTime{}
-}
-func (m *TestStreamPayload_Event_AdvanceProcessingTime) String() string {
-	return proto.CompactTextString(m)
-}
-func (*TestStreamPayload_Event_AdvanceProcessingTime) ProtoMessage() {}
-func (*TestStreamPayload_Event_AdvanceProcessingTime) Descriptor() ([]byte, []int) {
-	return fileDescriptor_cf57597c3a9659a9, []int{19, 0, 1}
-}
-
-func (m *TestStreamPayload_Event_AdvanceProcessingTime) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_TestStreamPayload_Event_AdvanceProcessingTime.Unmarshal(m, b)
-}
-func (m *TestStreamPayload_Event_AdvanceProcessingTime) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_TestStreamPayload_Event_AdvanceProcessingTime.Marshal(b, m, deterministic)
-}
-func (m *TestStreamPayload_Event_AdvanceProcessingTime) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_TestStreamPayload_Event_AdvanceProcessingTime.Merge(m, src)
-}
-func (m *TestStreamPayload_Event_AdvanceProcessingTime) XXX_Size() int {
-	return xxx_messageInfo_TestStreamPayload_Event_AdvanceProcessingTime.Size(m)
-}
-func (m *TestStreamPayload_Event_AdvanceProcessingTime) XXX_DiscardUnknown() {
-	xxx_messageInfo_TestStreamPayload_Event_AdvanceProcessingTime.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_TestStreamPayload_Event_AdvanceProcessingTime proto.InternalMessageInfo
-
-func (m *TestStreamPayload_Event_AdvanceProcessingTime) GetAdvanceDuration() int64 {
-	if m != nil {
-		return m.AdvanceDuration
-	}
-	return 0
-}
-
-// Adds elements to the stream to be emitted.
-type TestStreamPayload_Event_AddElements struct {
-	// (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_cf57597c3a9659a9, []int{19, 0, 2}
-}
-
-func (m *TestStreamPayload_Event_AddElements) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_TestStreamPayload_Event_AddElements.Unmarshal(m, b)
-}
-func (m *TestStreamPayload_Event_AddElements) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_TestStreamPayload_Event_AddElements.Marshal(b, m, deterministic)
-}
-func (m *TestStreamPayload_Event_AddElements) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_TestStreamPayload_Event_AddElements.Merge(m, src)
-}
-func (m *TestStreamPayload_Event_AddElements) XXX_Size() int {
-	return xxx_messageInfo_TestStreamPayload_Event_AddElements.Size(m)
-}
-func (m *TestStreamPayload_Event_AddElements) XXX_DiscardUnknown() {
-	xxx_messageInfo_TestStreamPayload_Event_AddElements.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_TestStreamPayload_Event_AddElements proto.InternalMessageInfo
-
-func (m *TestStreamPayload_Event_AddElements) GetElements() []*TestStreamPayload_TimestampedElement {
-	if m != nil {
-		return m.Elements
-	}
-	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 {
-	// (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:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *TestStreamPayload_TimestampedElement) Reset()         { *m = TestStreamPayload_TimestampedElement{} }
-func (m *TestStreamPayload_TimestampedElement) String() string { return proto.CompactTextString(m) }
-func (*TestStreamPayload_TimestampedElement) ProtoMessage()    {}
-func (*TestStreamPayload_TimestampedElement) Descriptor() ([]byte, []int) {
-	return fileDescriptor_cf57597c3a9659a9, []int{19, 1}
-}
-
-func (m *TestStreamPayload_TimestampedElement) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_TestStreamPayload_TimestampedElement.Unmarshal(m, b)
-}
-func (m *TestStreamPayload_TimestampedElement) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_TestStreamPayload_TimestampedElement.Marshal(b, m, deterministic)
-}
-func (m *TestStreamPayload_TimestampedElement) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_TestStreamPayload_TimestampedElement.Merge(m, src)
-}
-func (m *TestStreamPayload_TimestampedElement) XXX_Size() int {
-	return xxx_messageInfo_TestStreamPayload_TimestampedElement.Size(m)
-}
-func (m *TestStreamPayload_TimestampedElement) XXX_DiscardUnknown() {
-	xxx_messageInfo_TestStreamPayload_TimestampedElement.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_TestStreamPayload_TimestampedElement proto.InternalMessageInfo
-
-func (m *TestStreamPayload_TimestampedElement) GetEncodedElement() []byte {
-	if m != nil {
-		return m.EncodedElement
-	}
-	return nil
-}
-
-func (m *TestStreamPayload_TimestampedElement) GetTimestamp() int64 {
-	if m != nil {
-		return m.Timestamp
-	}
-	return 0
-}
-
 type EventsRequest struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
 	// The set of PCollections to read from. These are the PTransform outputs
 	// local names. These are a subset of the TestStream's outputs. This allows
 	// Interactive Beam to cache many PCollections from a pipeline then replay a
 	// subset of them.
-	OutputIds            []string `protobuf:"bytes,1,rep,name=output_ids,json=outputIds,proto3" json:"output_ids,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
+	OutputIds []string `protobuf:"bytes,1,rep,name=output_ids,json=outputIds,proto3" json:"output_ids,omitempty"`
 }
 
-func (m *EventsRequest) Reset()         { *m = EventsRequest{} }
-func (m *EventsRequest) String() string { return proto.CompactTextString(m) }
-func (*EventsRequest) ProtoMessage()    {}
+func (x *EventsRequest) Reset() {
+	*x = EventsRequest{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_runner_api_proto_msgTypes[21]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *EventsRequest) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*EventsRequest) ProtoMessage() {}
+
+func (x *EventsRequest) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_runner_api_proto_msgTypes[21]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use EventsRequest.ProtoReflect.Descriptor instead.
 func (*EventsRequest) Descriptor() ([]byte, []int) {
-	return fileDescriptor_cf57597c3a9659a9, []int{20}
+	return file_beam_runner_api_proto_rawDescGZIP(), []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 (m *EventsRequest) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_EventsRequest.Merge(m, 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
-
-func (m *EventsRequest) GetOutputIds() []string {
-	if m != nil {
-		return m.OutputIds
+func (x *EventsRequest) GetOutputIds() []string {
+	if x != nil {
+		return x.OutputIds
 	}
 	return nil
 }
 
 // The payload for the special-but-not-primitive WriteFiles transform.
 type WriteFilesPayload struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
 	// (Required) The FunctionSpec of the FileBasedSink.
 	Sink *FunctionSpec `protobuf:"bytes,1,opt,name=sink,proto3" json:"sink,omitempty"`
 	// (Required) The format function.
@@ -2664,74 +3064,248 @@
 	WindowedWrites           bool                  `protobuf:"varint,3,opt,name=windowed_writes,json=windowedWrites,proto3" json:"windowed_writes,omitempty"`
 	RunnerDeterminedSharding bool                  `protobuf:"varint,4,opt,name=runner_determined_sharding,json=runnerDeterminedSharding,proto3" json:"runner_determined_sharding,omitempty"`
 	SideInputs               map[string]*SideInput `protobuf:"bytes,5,rep,name=side_inputs,json=sideInputs,proto3" json:"side_inputs,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"`
-	XXX_NoUnkeyedLiteral     struct{}              `json:"-"`
-	XXX_unrecognized         []byte                `json:"-"`
-	XXX_sizecache            int32                 `json:"-"`
 }
 
-func (m *WriteFilesPayload) Reset()         { *m = WriteFilesPayload{} }
-func (m *WriteFilesPayload) String() string { return proto.CompactTextString(m) }
-func (*WriteFilesPayload) ProtoMessage()    {}
+func (x *WriteFilesPayload) Reset() {
+	*x = WriteFilesPayload{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_runner_api_proto_msgTypes[22]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *WriteFilesPayload) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*WriteFilesPayload) ProtoMessage() {}
+
+func (x *WriteFilesPayload) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_runner_api_proto_msgTypes[22]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use WriteFilesPayload.ProtoReflect.Descriptor instead.
 func (*WriteFilesPayload) Descriptor() ([]byte, []int) {
-	return fileDescriptor_cf57597c3a9659a9, []int{21}
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{22}
 }
 
-func (m *WriteFilesPayload) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_WriteFilesPayload.Unmarshal(m, b)
-}
-func (m *WriteFilesPayload) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_WriteFilesPayload.Marshal(b, m, deterministic)
-}
-func (m *WriteFilesPayload) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_WriteFilesPayload.Merge(m, src)
-}
-func (m *WriteFilesPayload) XXX_Size() int {
-	return xxx_messageInfo_WriteFilesPayload.Size(m)
-}
-func (m *WriteFilesPayload) XXX_DiscardUnknown() {
-	xxx_messageInfo_WriteFilesPayload.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_WriteFilesPayload proto.InternalMessageInfo
-
-func (m *WriteFilesPayload) GetSink() *FunctionSpec {
-	if m != nil {
-		return m.Sink
+func (x *WriteFilesPayload) GetSink() *FunctionSpec {
+	if x != nil {
+		return x.Sink
 	}
 	return nil
 }
 
-func (m *WriteFilesPayload) GetFormatFunction() *FunctionSpec {
-	if m != nil {
-		return m.FormatFunction
+func (x *WriteFilesPayload) GetFormatFunction() *FunctionSpec {
+	if x != nil {
+		return x.FormatFunction
 	}
 	return nil
 }
 
-func (m *WriteFilesPayload) GetWindowedWrites() bool {
-	if m != nil {
-		return m.WindowedWrites
+func (x *WriteFilesPayload) GetWindowedWrites() bool {
+	if x != nil {
+		return x.WindowedWrites
 	}
 	return false
 }
 
-func (m *WriteFilesPayload) GetRunnerDeterminedSharding() bool {
-	if m != nil {
-		return m.RunnerDeterminedSharding
+func (x *WriteFilesPayload) GetRunnerDeterminedSharding() bool {
+	if x != nil {
+		return x.RunnerDeterminedSharding
 	}
 	return false
 }
 
-func (m *WriteFilesPayload) GetSideInputs() map[string]*SideInput {
-	if m != nil {
-		return m.SideInputs
+func (x *WriteFilesPayload) GetSideInputs() map[string]*SideInput {
+	if x != nil {
+		return x.SideInputs
 	}
 	return nil
 }
 
+// Payload used by Google Cloud Pub/Sub read transform.
+// This can be used by runners that wish to override Beam Pub/Sub read transform
+// with a native implementation.
+type PubSubReadPayload struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	// Topic to read from. Exactly one of topic or subscription should be set.
+	Topic string `protobuf:"bytes,1,opt,name=topic,proto3" json:"topic,omitempty"`
+	// Subscription to read from. Exactly one of topic or subscription should be set.
+	Subscription string `protobuf:"bytes,2,opt,name=subscription,proto3" json:"subscription,omitempty"`
+	// Attribute that provides element timestamps.
+	TimestampAttribute string `protobuf:"bytes,3,opt,name=timestamp_attribute,json=timestampAttribute,proto3" json:"timestamp_attribute,omitempty"`
+	// Attribute to be used for uniquely identifying messages.
+	IdAttribute string `protobuf:"bytes,4,opt,name=id_attribute,json=idAttribute,proto3" json:"id_attribute,omitempty"`
+	// If true, reads Pub/Sub payload as well as attributes. If false, reads only the payload.
+	WithAttributes bool `protobuf:"varint,5,opt,name=with_attributes,json=withAttributes,proto3" json:"with_attributes,omitempty"`
+}
+
+func (x *PubSubReadPayload) Reset() {
+	*x = PubSubReadPayload{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_runner_api_proto_msgTypes[23]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *PubSubReadPayload) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*PubSubReadPayload) ProtoMessage() {}
+
+func (x *PubSubReadPayload) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_runner_api_proto_msgTypes[23]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use PubSubReadPayload.ProtoReflect.Descriptor instead.
+func (*PubSubReadPayload) Descriptor() ([]byte, []int) {
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{23}
+}
+
+func (x *PubSubReadPayload) GetTopic() string {
+	if x != nil {
+		return x.Topic
+	}
+	return ""
+}
+
+func (x *PubSubReadPayload) GetSubscription() string {
+	if x != nil {
+		return x.Subscription
+	}
+	return ""
+}
+
+func (x *PubSubReadPayload) GetTimestampAttribute() string {
+	if x != nil {
+		return x.TimestampAttribute
+	}
+	return ""
+}
+
+func (x *PubSubReadPayload) GetIdAttribute() string {
+	if x != nil {
+		return x.IdAttribute
+	}
+	return ""
+}
+
+func (x *PubSubReadPayload) GetWithAttributes() bool {
+	if x != nil {
+		return x.WithAttributes
+	}
+	return false
+}
+
+// Payload used by Google Cloud Pub/Sub write transform.
+// This can be used by runners that wish to override Beam Pub/Sub write transform
+// with a native implementation.
+type PubSubWritePayload struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	// Topic to write to.
+	Topic string `protobuf:"bytes,1,opt,name=topic,proto3" json:"topic,omitempty"`
+	// Attribute that provides element timestamps.
+	TimestampAttribute string `protobuf:"bytes,2,opt,name=timestamp_attribute,json=timestampAttribute,proto3" json:"timestamp_attribute,omitempty"`
+	// Attribute that uniquely identify messages.
+	IdAttribute string `protobuf:"bytes,3,opt,name=id_attribute,json=idAttribute,proto3" json:"id_attribute,omitempty"`
+	// If true, writes Pub/Sub payload as well as attributes. If false, reads only the payload.
+	// TODO(BEAM-10869): consider removing/deprecating this field when fixed.
+	WithAttributes bool `protobuf:"varint,4,opt,name=with_attributes,json=withAttributes,proto3" json:"with_attributes,omitempty"`
+}
+
+func (x *PubSubWritePayload) Reset() {
+	*x = PubSubWritePayload{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_runner_api_proto_msgTypes[24]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *PubSubWritePayload) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*PubSubWritePayload) ProtoMessage() {}
+
+func (x *PubSubWritePayload) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_runner_api_proto_msgTypes[24]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use PubSubWritePayload.ProtoReflect.Descriptor instead.
+func (*PubSubWritePayload) Descriptor() ([]byte, []int) {
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{24}
+}
+
+func (x *PubSubWritePayload) GetTopic() string {
+	if x != nil {
+		return x.Topic
+	}
+	return ""
+}
+
+func (x *PubSubWritePayload) GetTimestampAttribute() string {
+	if x != nil {
+		return x.TimestampAttribute
+	}
+	return ""
+}
+
+func (x *PubSubWritePayload) GetIdAttribute() string {
+	if x != nil {
+		return x.IdAttribute
+	}
+	return ""
+}
+
+func (x *PubSubWritePayload) GetWithAttributes() bool {
+	if x != nil {
+		return x.WithAttributes
+	}
+	return false
+}
+
 // A coder, the binary format for serialization and deserialization of data in
 // a pipeline.
 type Coder struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
 	// (Required) A specification for the coder, as a URN plus parameters. This
 	// may be a cross-language agreed-upon format, or it may be a "custom coder"
 	// that can only be used by a particular SDK. It does not include component
@@ -2741,87 +3315,102 @@
 	// (Optional) If this coder is parametric, such as ListCoder(VarIntCoder),
 	// this is a list of the components. In order for encodings to be identical,
 	// the FunctionSpec and all components must be identical, recursively.
-	ComponentCoderIds    []string `protobuf:"bytes,2,rep,name=component_coder_ids,json=componentCoderIds,proto3" json:"component_coder_ids,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
+	ComponentCoderIds []string `protobuf:"bytes,2,rep,name=component_coder_ids,json=componentCoderIds,proto3" json:"component_coder_ids,omitempty"`
 }
 
-func (m *Coder) Reset()         { *m = Coder{} }
-func (m *Coder) String() string { return proto.CompactTextString(m) }
-func (*Coder) ProtoMessage()    {}
+func (x *Coder) Reset() {
+	*x = Coder{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_runner_api_proto_msgTypes[25]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *Coder) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*Coder) ProtoMessage() {}
+
+func (x *Coder) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_runner_api_proto_msgTypes[25]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use Coder.ProtoReflect.Descriptor instead.
 func (*Coder) Descriptor() ([]byte, []int) {
-	return fileDescriptor_cf57597c3a9659a9, []int{22}
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{25}
 }
 
-func (m *Coder) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_Coder.Unmarshal(m, b)
-}
-func (m *Coder) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_Coder.Marshal(b, m, deterministic)
-}
-func (m *Coder) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Coder.Merge(m, src)
-}
-func (m *Coder) XXX_Size() int {
-	return xxx_messageInfo_Coder.Size(m)
-}
-func (m *Coder) XXX_DiscardUnknown() {
-	xxx_messageInfo_Coder.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_Coder proto.InternalMessageInfo
-
-func (m *Coder) GetSpec() *FunctionSpec {
-	if m != nil {
-		return m.Spec
+func (x *Coder) GetSpec() *FunctionSpec {
+	if x != nil {
+		return x.Spec
 	}
 	return nil
 }
 
-func (m *Coder) GetComponentCoderIds() []string {
-	if m != nil {
-		return m.ComponentCoderIds
+func (x *Coder) GetComponentCoderIds() []string {
+	if x != nil {
+		return x.ComponentCoderIds
 	}
 	return nil
 }
 
 type StandardCoders struct {
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
 }
 
-func (m *StandardCoders) Reset()         { *m = StandardCoders{} }
-func (m *StandardCoders) String() string { return proto.CompactTextString(m) }
-func (*StandardCoders) ProtoMessage()    {}
+func (x *StandardCoders) Reset() {
+	*x = StandardCoders{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_runner_api_proto_msgTypes[26]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *StandardCoders) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*StandardCoders) ProtoMessage() {}
+
+func (x *StandardCoders) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_runner_api_proto_msgTypes[26]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use StandardCoders.ProtoReflect.Descriptor instead.
 func (*StandardCoders) Descriptor() ([]byte, []int) {
-	return fileDescriptor_cf57597c3a9659a9, []int{23}
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{26}
 }
 
-func (m *StandardCoders) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_StandardCoders.Unmarshal(m, b)
-}
-func (m *StandardCoders) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_StandardCoders.Marshal(b, m, deterministic)
-}
-func (m *StandardCoders) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_StandardCoders.Merge(m, src)
-}
-func (m *StandardCoders) XXX_Size() int {
-	return xxx_messageInfo_StandardCoders.Size(m)
-}
-func (m *StandardCoders) XXX_DiscardUnknown() {
-	xxx_messageInfo_StandardCoders.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_StandardCoders proto.InternalMessageInfo
-
 // A windowing strategy describes the window function, triggering, allowed
 // lateness, and accumulation mode for a PCollection.
 //
 // TODO: consider inlining field on PCollection
 type WindowingStrategy struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
 	// (Required) The FunctionSpec of the UDF that assigns windows,
 	// merges windows, and shifts timestamps before they are
 	// combined according to the OutputTime.
@@ -2862,110 +3451,114 @@
 	// (Optional) Environment where the current window_fn should be applied in.
 	// Runner that executes the pipeline may choose to override this if needed.
 	// If not specified, environment will be decided by the runner.
-	EnvironmentId        string   `protobuf:"bytes,11,opt,name=environment_id,json=environmentId,proto3" json:"environment_id,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
+	EnvironmentId string `protobuf:"bytes,11,opt,name=environment_id,json=environmentId,proto3" json:"environment_id,omitempty"`
 }
 
-func (m *WindowingStrategy) Reset()         { *m = WindowingStrategy{} }
-func (m *WindowingStrategy) String() string { return proto.CompactTextString(m) }
-func (*WindowingStrategy) ProtoMessage()    {}
+func (x *WindowingStrategy) Reset() {
+	*x = WindowingStrategy{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_runner_api_proto_msgTypes[27]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *WindowingStrategy) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*WindowingStrategy) ProtoMessage() {}
+
+func (x *WindowingStrategy) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_runner_api_proto_msgTypes[27]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use WindowingStrategy.ProtoReflect.Descriptor instead.
 func (*WindowingStrategy) Descriptor() ([]byte, []int) {
-	return fileDescriptor_cf57597c3a9659a9, []int{24}
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{27}
 }
 
-func (m *WindowingStrategy) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_WindowingStrategy.Unmarshal(m, b)
-}
-func (m *WindowingStrategy) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_WindowingStrategy.Marshal(b, m, deterministic)
-}
-func (m *WindowingStrategy) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_WindowingStrategy.Merge(m, src)
-}
-func (m *WindowingStrategy) XXX_Size() int {
-	return xxx_messageInfo_WindowingStrategy.Size(m)
-}
-func (m *WindowingStrategy) XXX_DiscardUnknown() {
-	xxx_messageInfo_WindowingStrategy.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_WindowingStrategy proto.InternalMessageInfo
-
-func (m *WindowingStrategy) GetWindowFn() *FunctionSpec {
-	if m != nil {
-		return m.WindowFn
+func (x *WindowingStrategy) GetWindowFn() *FunctionSpec {
+	if x != nil {
+		return x.WindowFn
 	}
 	return nil
 }
 
-func (m *WindowingStrategy) GetMergeStatus() MergeStatus_Enum {
-	if m != nil {
-		return m.MergeStatus
+func (x *WindowingStrategy) GetMergeStatus() MergeStatus_Enum {
+	if x != nil {
+		return x.MergeStatus
 	}
 	return MergeStatus_UNSPECIFIED
 }
 
-func (m *WindowingStrategy) GetWindowCoderId() string {
-	if m != nil {
-		return m.WindowCoderId
+func (x *WindowingStrategy) GetWindowCoderId() string {
+	if x != nil {
+		return x.WindowCoderId
 	}
 	return ""
 }
 
-func (m *WindowingStrategy) GetTrigger() *Trigger {
-	if m != nil {
-		return m.Trigger
+func (x *WindowingStrategy) GetTrigger() *Trigger {
+	if x != nil {
+		return x.Trigger
 	}
 	return nil
 }
 
-func (m *WindowingStrategy) GetAccumulationMode() AccumulationMode_Enum {
-	if m != nil {
-		return m.AccumulationMode
+func (x *WindowingStrategy) GetAccumulationMode() AccumulationMode_Enum {
+	if x != nil {
+		return x.AccumulationMode
 	}
 	return AccumulationMode_UNSPECIFIED
 }
 
-func (m *WindowingStrategy) GetOutputTime() OutputTime_Enum {
-	if m != nil {
-		return m.OutputTime
+func (x *WindowingStrategy) GetOutputTime() OutputTime_Enum {
+	if x != nil {
+		return x.OutputTime
 	}
 	return OutputTime_UNSPECIFIED
 }
 
-func (m *WindowingStrategy) GetClosingBehavior() ClosingBehavior_Enum {
-	if m != nil {
-		return m.ClosingBehavior
+func (x *WindowingStrategy) GetClosingBehavior() ClosingBehavior_Enum {
+	if x != nil {
+		return x.ClosingBehavior
 	}
 	return ClosingBehavior_UNSPECIFIED
 }
 
-func (m *WindowingStrategy) GetAllowedLateness() int64 {
-	if m != nil {
-		return m.AllowedLateness
+func (x *WindowingStrategy) GetAllowedLateness() int64 {
+	if x != nil {
+		return x.AllowedLateness
 	}
 	return 0
 }
 
-func (m *WindowingStrategy) GetOnTimeBehavior() OnTimeBehavior_Enum {
-	if m != nil {
-		return m.OnTimeBehavior
+func (x *WindowingStrategy) GetOnTimeBehavior() OnTimeBehavior_Enum {
+	if x != nil {
+		return x.OnTimeBehavior
 	}
 	return OnTimeBehavior_UNSPECIFIED
 }
 
-func (m *WindowingStrategy) GetAssignsToOneWindow() bool {
-	if m != nil {
-		return m.AssignsToOneWindow
+func (x *WindowingStrategy) GetAssignsToOneWindow() bool {
+	if x != nil {
+		return x.AssignsToOneWindow
 	}
 	return false
 }
 
-func (m *WindowingStrategy) GetEnvironmentId() string {
-	if m != nil {
-		return m.EnvironmentId
+func (x *WindowingStrategy) GetEnvironmentId() string {
+	if x != nil {
+		return x.EnvironmentId
 	}
 	return ""
 }
@@ -2974,209 +3567,255 @@
 // merging-but-already-merged, in which case a subsequent GroupByKey is almost
 // always going to do something the user does not want
 type MergeStatus struct {
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
 }
 
-func (m *MergeStatus) Reset()         { *m = MergeStatus{} }
-func (m *MergeStatus) String() string { return proto.CompactTextString(m) }
-func (*MergeStatus) ProtoMessage()    {}
+func (x *MergeStatus) Reset() {
+	*x = MergeStatus{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_runner_api_proto_msgTypes[28]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *MergeStatus) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*MergeStatus) ProtoMessage() {}
+
+func (x *MergeStatus) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_runner_api_proto_msgTypes[28]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use MergeStatus.ProtoReflect.Descriptor instead.
 func (*MergeStatus) Descriptor() ([]byte, []int) {
-	return fileDescriptor_cf57597c3a9659a9, []int{25}
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{28}
 }
 
-func (m *MergeStatus) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_MergeStatus.Unmarshal(m, b)
-}
-func (m *MergeStatus) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_MergeStatus.Marshal(b, m, deterministic)
-}
-func (m *MergeStatus) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_MergeStatus.Merge(m, src)
-}
-func (m *MergeStatus) XXX_Size() int {
-	return xxx_messageInfo_MergeStatus.Size(m)
-}
-func (m *MergeStatus) XXX_DiscardUnknown() {
-	xxx_messageInfo_MergeStatus.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_MergeStatus proto.InternalMessageInfo
-
 // Whether or not subsequent outputs of aggregations should be entire
 // replacement values or just the aggregation of inputs received since
 // the prior output.
 type AccumulationMode struct {
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
 }
 
-func (m *AccumulationMode) Reset()         { *m = AccumulationMode{} }
-func (m *AccumulationMode) String() string { return proto.CompactTextString(m) }
-func (*AccumulationMode) ProtoMessage()    {}
+func (x *AccumulationMode) Reset() {
+	*x = AccumulationMode{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_runner_api_proto_msgTypes[29]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *AccumulationMode) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*AccumulationMode) ProtoMessage() {}
+
+func (x *AccumulationMode) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_runner_api_proto_msgTypes[29]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use AccumulationMode.ProtoReflect.Descriptor instead.
 func (*AccumulationMode) Descriptor() ([]byte, []int) {
-	return fileDescriptor_cf57597c3a9659a9, []int{26}
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{29}
 }
 
-func (m *AccumulationMode) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_AccumulationMode.Unmarshal(m, b)
-}
-func (m *AccumulationMode) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_AccumulationMode.Marshal(b, m, deterministic)
-}
-func (m *AccumulationMode) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_AccumulationMode.Merge(m, src)
-}
-func (m *AccumulationMode) XXX_Size() int {
-	return xxx_messageInfo_AccumulationMode.Size(m)
-}
-func (m *AccumulationMode) XXX_DiscardUnknown() {
-	xxx_messageInfo_AccumulationMode.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_AccumulationMode proto.InternalMessageInfo
-
 // Controls whether or not an aggregating transform should output data
 // when a window expires.
 type ClosingBehavior struct {
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
 }
 
-func (m *ClosingBehavior) Reset()         { *m = ClosingBehavior{} }
-func (m *ClosingBehavior) String() string { return proto.CompactTextString(m) }
-func (*ClosingBehavior) ProtoMessage()    {}
+func (x *ClosingBehavior) Reset() {
+	*x = ClosingBehavior{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_runner_api_proto_msgTypes[30]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *ClosingBehavior) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*ClosingBehavior) ProtoMessage() {}
+
+func (x *ClosingBehavior) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_runner_api_proto_msgTypes[30]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use ClosingBehavior.ProtoReflect.Descriptor instead.
 func (*ClosingBehavior) Descriptor() ([]byte, []int) {
-	return fileDescriptor_cf57597c3a9659a9, []int{27}
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{30}
 }
 
-func (m *ClosingBehavior) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_ClosingBehavior.Unmarshal(m, b)
-}
-func (m *ClosingBehavior) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_ClosingBehavior.Marshal(b, m, deterministic)
-}
-func (m *ClosingBehavior) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ClosingBehavior.Merge(m, src)
-}
-func (m *ClosingBehavior) XXX_Size() int {
-	return xxx_messageInfo_ClosingBehavior.Size(m)
-}
-func (m *ClosingBehavior) XXX_DiscardUnknown() {
-	xxx_messageInfo_ClosingBehavior.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_ClosingBehavior proto.InternalMessageInfo
-
 // Controls whether or not an aggregating transform should output data
 // when an on-time pane is empty.
 type OnTimeBehavior struct {
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
 }
 
-func (m *OnTimeBehavior) Reset()         { *m = OnTimeBehavior{} }
-func (m *OnTimeBehavior) String() string { return proto.CompactTextString(m) }
-func (*OnTimeBehavior) ProtoMessage()    {}
+func (x *OnTimeBehavior) Reset() {
+	*x = OnTimeBehavior{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_runner_api_proto_msgTypes[31]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *OnTimeBehavior) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*OnTimeBehavior) ProtoMessage() {}
+
+func (x *OnTimeBehavior) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_runner_api_proto_msgTypes[31]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use OnTimeBehavior.ProtoReflect.Descriptor instead.
 func (*OnTimeBehavior) Descriptor() ([]byte, []int) {
-	return fileDescriptor_cf57597c3a9659a9, []int{28}
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{31}
 }
 
-func (m *OnTimeBehavior) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OnTimeBehavior.Unmarshal(m, b)
-}
-func (m *OnTimeBehavior) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OnTimeBehavior.Marshal(b, m, deterministic)
-}
-func (m *OnTimeBehavior) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OnTimeBehavior.Merge(m, src)
-}
-func (m *OnTimeBehavior) XXX_Size() int {
-	return xxx_messageInfo_OnTimeBehavior.Size(m)
-}
-func (m *OnTimeBehavior) XXX_DiscardUnknown() {
-	xxx_messageInfo_OnTimeBehavior.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OnTimeBehavior proto.InternalMessageInfo
-
 // When a number of windowed, timestamped inputs are aggregated, the timestamp
 // for the resulting output.
 type OutputTime struct {
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
 }
 
-func (m *OutputTime) Reset()         { *m = OutputTime{} }
-func (m *OutputTime) String() string { return proto.CompactTextString(m) }
-func (*OutputTime) ProtoMessage()    {}
+func (x *OutputTime) Reset() {
+	*x = OutputTime{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_runner_api_proto_msgTypes[32]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *OutputTime) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*OutputTime) ProtoMessage() {}
+
+func (x *OutputTime) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_runner_api_proto_msgTypes[32]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use OutputTime.ProtoReflect.Descriptor instead.
 func (*OutputTime) Descriptor() ([]byte, []int) {
-	return fileDescriptor_cf57597c3a9659a9, []int{29}
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{32}
 }
 
-func (m *OutputTime) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_OutputTime.Unmarshal(m, b)
-}
-func (m *OutputTime) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_OutputTime.Marshal(b, m, deterministic)
-}
-func (m *OutputTime) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_OutputTime.Merge(m, src)
-}
-func (m *OutputTime) XXX_Size() int {
-	return xxx_messageInfo_OutputTime.Size(m)
-}
-func (m *OutputTime) XXX_DiscardUnknown() {
-	xxx_messageInfo_OutputTime.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_OutputTime proto.InternalMessageInfo
-
 // The different time domains in the Beam model.
 type TimeDomain struct {
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
 }
 
-func (m *TimeDomain) Reset()         { *m = TimeDomain{} }
-func (m *TimeDomain) String() string { return proto.CompactTextString(m) }
-func (*TimeDomain) ProtoMessage()    {}
+func (x *TimeDomain) Reset() {
+	*x = TimeDomain{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_runner_api_proto_msgTypes[33]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *TimeDomain) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*TimeDomain) ProtoMessage() {}
+
+func (x *TimeDomain) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_runner_api_proto_msgTypes[33]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use TimeDomain.ProtoReflect.Descriptor instead.
 func (*TimeDomain) Descriptor() ([]byte, []int) {
-	return fileDescriptor_cf57597c3a9659a9, []int{30}
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{33}
 }
 
-func (m *TimeDomain) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_TimeDomain.Unmarshal(m, b)
-}
-func (m *TimeDomain) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_TimeDomain.Marshal(b, m, deterministic)
-}
-func (m *TimeDomain) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_TimeDomain.Merge(m, src)
-}
-func (m *TimeDomain) XXX_Size() int {
-	return xxx_messageInfo_TimeDomain.Size(m)
-}
-func (m *TimeDomain) XXX_DiscardUnknown() {
-	xxx_messageInfo_TimeDomain.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_TimeDomain proto.InternalMessageInfo
-
 // A small DSL for expressing when to emit new aggregations
 // from a GroupByKey or CombinePerKey
 //
 // A trigger is described in terms of when it is _ready_ to permit output.
 type Trigger struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
 	// The full disjoint union of possible triggers.
 	//
-	// Types that are valid to be assigned to Trigger:
+	// Types that are assignable to Trigger:
 	//	*Trigger_AfterAll_
 	//	*Trigger_AfterAny_
 	//	*Trigger_AfterEach_
@@ -3189,36 +3828,131 @@
 	//	*Trigger_Never_
 	//	*Trigger_OrFinally_
 	//	*Trigger_Repeat_
-	Trigger              isTrigger_Trigger `protobuf_oneof:"trigger"`
-	XXX_NoUnkeyedLiteral struct{}          `json:"-"`
-	XXX_unrecognized     []byte            `json:"-"`
-	XXX_sizecache        int32             `json:"-"`
+	Trigger isTrigger_Trigger `protobuf_oneof:"trigger"`
 }
 
-func (m *Trigger) Reset()         { *m = Trigger{} }
-func (m *Trigger) String() string { return proto.CompactTextString(m) }
-func (*Trigger) ProtoMessage()    {}
+func (x *Trigger) Reset() {
+	*x = Trigger{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_runner_api_proto_msgTypes[34]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *Trigger) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*Trigger) ProtoMessage() {}
+
+func (x *Trigger) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_runner_api_proto_msgTypes[34]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use Trigger.ProtoReflect.Descriptor instead.
 func (*Trigger) Descriptor() ([]byte, []int) {
-	return fileDescriptor_cf57597c3a9659a9, []int{31}
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{34}
 }
 
-func (m *Trigger) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_Trigger.Unmarshal(m, b)
-}
-func (m *Trigger) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_Trigger.Marshal(b, m, deterministic)
-}
-func (m *Trigger) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Trigger.Merge(m, src)
-}
-func (m *Trigger) XXX_Size() int {
-	return xxx_messageInfo_Trigger.Size(m)
-}
-func (m *Trigger) XXX_DiscardUnknown() {
-	xxx_messageInfo_Trigger.DiscardUnknown(m)
+func (m *Trigger) GetTrigger() isTrigger_Trigger {
+	if m != nil {
+		return m.Trigger
+	}
+	return nil
 }
 
-var xxx_messageInfo_Trigger proto.InternalMessageInfo
+func (x *Trigger) GetAfterAll() *Trigger_AfterAll {
+	if x, ok := x.GetTrigger().(*Trigger_AfterAll_); ok {
+		return x.AfterAll
+	}
+	return nil
+}
+
+func (x *Trigger) GetAfterAny() *Trigger_AfterAny {
+	if x, ok := x.GetTrigger().(*Trigger_AfterAny_); ok {
+		return x.AfterAny
+	}
+	return nil
+}
+
+func (x *Trigger) GetAfterEach() *Trigger_AfterEach {
+	if x, ok := x.GetTrigger().(*Trigger_AfterEach_); ok {
+		return x.AfterEach
+	}
+	return nil
+}
+
+func (x *Trigger) GetAfterEndOfWindow() *Trigger_AfterEndOfWindow {
+	if x, ok := x.GetTrigger().(*Trigger_AfterEndOfWindow_); ok {
+		return x.AfterEndOfWindow
+	}
+	return nil
+}
+
+func (x *Trigger) GetAfterProcessingTime() *Trigger_AfterProcessingTime {
+	if x, ok := x.GetTrigger().(*Trigger_AfterProcessingTime_); ok {
+		return x.AfterProcessingTime
+	}
+	return nil
+}
+
+func (x *Trigger) GetAfterSynchronizedProcessingTime() *Trigger_AfterSynchronizedProcessingTime {
+	if x, ok := x.GetTrigger().(*Trigger_AfterSynchronizedProcessingTime_); ok {
+		return x.AfterSynchronizedProcessingTime
+	}
+	return nil
+}
+
+func (x *Trigger) GetAlways() *Trigger_Always {
+	if x, ok := x.GetTrigger().(*Trigger_Always_); ok {
+		return x.Always
+	}
+	return nil
+}
+
+func (x *Trigger) GetDefault() *Trigger_Default {
+	if x, ok := x.GetTrigger().(*Trigger_Default_); ok {
+		return x.Default
+	}
+	return nil
+}
+
+func (x *Trigger) GetElementCount() *Trigger_ElementCount {
+	if x, ok := x.GetTrigger().(*Trigger_ElementCount_); ok {
+		return x.ElementCount
+	}
+	return nil
+}
+
+func (x *Trigger) GetNever() *Trigger_Never {
+	if x, ok := x.GetTrigger().(*Trigger_Never_); ok {
+		return x.Never
+	}
+	return nil
+}
+
+func (x *Trigger) GetOrFinally() *Trigger_OrFinally {
+	if x, ok := x.GetTrigger().(*Trigger_OrFinally_); ok {
+		return x.OrFinally
+	}
+	return nil
+}
+
+func (x *Trigger) GetRepeat() *Trigger_Repeat {
+	if x, ok := x.GetTrigger().(*Trigger_Repeat_); ok {
+		return x.Repeat
+	}
+	return nil
+}
 
 type isTrigger_Trigger interface {
 	isTrigger_Trigger()
@@ -3296,637 +4030,73 @@
 
 func (*Trigger_Repeat_) isTrigger_Trigger() {}
 
-func (m *Trigger) GetTrigger() isTrigger_Trigger {
-	if m != nil {
-		return m.Trigger
-	}
-	return nil
-}
-
-func (m *Trigger) GetAfterAll() *Trigger_AfterAll {
-	if x, ok := m.GetTrigger().(*Trigger_AfterAll_); ok {
-		return x.AfterAll
-	}
-	return nil
-}
-
-func (m *Trigger) GetAfterAny() *Trigger_AfterAny {
-	if x, ok := m.GetTrigger().(*Trigger_AfterAny_); ok {
-		return x.AfterAny
-	}
-	return nil
-}
-
-func (m *Trigger) GetAfterEach() *Trigger_AfterEach {
-	if x, ok := m.GetTrigger().(*Trigger_AfterEach_); ok {
-		return x.AfterEach
-	}
-	return nil
-}
-
-func (m *Trigger) GetAfterEndOfWindow() *Trigger_AfterEndOfWindow {
-	if x, ok := m.GetTrigger().(*Trigger_AfterEndOfWindow_); ok {
-		return x.AfterEndOfWindow
-	}
-	return nil
-}
-
-func (m *Trigger) GetAfterProcessingTime() *Trigger_AfterProcessingTime {
-	if x, ok := m.GetTrigger().(*Trigger_AfterProcessingTime_); ok {
-		return x.AfterProcessingTime
-	}
-	return nil
-}
-
-func (m *Trigger) GetAfterSynchronizedProcessingTime() *Trigger_AfterSynchronizedProcessingTime {
-	if x, ok := m.GetTrigger().(*Trigger_AfterSynchronizedProcessingTime_); ok {
-		return x.AfterSynchronizedProcessingTime
-	}
-	return nil
-}
-
-func (m *Trigger) GetAlways() *Trigger_Always {
-	if x, ok := m.GetTrigger().(*Trigger_Always_); ok {
-		return x.Always
-	}
-	return nil
-}
-
-func (m *Trigger) GetDefault() *Trigger_Default {
-	if x, ok := m.GetTrigger().(*Trigger_Default_); ok {
-		return x.Default
-	}
-	return nil
-}
-
-func (m *Trigger) GetElementCount() *Trigger_ElementCount {
-	if x, ok := m.GetTrigger().(*Trigger_ElementCount_); ok {
-		return x.ElementCount
-	}
-	return nil
-}
-
-func (m *Trigger) GetNever() *Trigger_Never {
-	if x, ok := m.GetTrigger().(*Trigger_Never_); ok {
-		return x.Never
-	}
-	return nil
-}
-
-func (m *Trigger) GetOrFinally() *Trigger_OrFinally {
-	if x, ok := m.GetTrigger().(*Trigger_OrFinally_); ok {
-		return x.OrFinally
-	}
-	return nil
-}
-
-func (m *Trigger) GetRepeat() *Trigger_Repeat {
-	if x, ok := m.GetTrigger().(*Trigger_Repeat_); ok {
-		return x.Repeat
-	}
-	return nil
-}
-
-// XXX_OneofWrappers is for the internal use of the proto package.
-func (*Trigger) XXX_OneofWrappers() []interface{} {
-	return []interface{}{
-		(*Trigger_AfterAll_)(nil),
-		(*Trigger_AfterAny_)(nil),
-		(*Trigger_AfterEach_)(nil),
-		(*Trigger_AfterEndOfWindow_)(nil),
-		(*Trigger_AfterProcessingTime_)(nil),
-		(*Trigger_AfterSynchronizedProcessingTime_)(nil),
-		(*Trigger_Always_)(nil),
-		(*Trigger_Default_)(nil),
-		(*Trigger_ElementCount_)(nil),
-		(*Trigger_Never_)(nil),
-		(*Trigger_OrFinally_)(nil),
-		(*Trigger_Repeat_)(nil),
-	}
-}
-
-// Ready when all subtriggers are ready.
-type Trigger_AfterAll struct {
-	Subtriggers          []*Trigger `protobuf:"bytes,1,rep,name=subtriggers,proto3" json:"subtriggers,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}   `json:"-"`
-	XXX_unrecognized     []byte     `json:"-"`
-	XXX_sizecache        int32      `json:"-"`
-}
-
-func (m *Trigger_AfterAll) Reset()         { *m = Trigger_AfterAll{} }
-func (m *Trigger_AfterAll) String() string { return proto.CompactTextString(m) }
-func (*Trigger_AfterAll) ProtoMessage()    {}
-func (*Trigger_AfterAll) Descriptor() ([]byte, []int) {
-	return fileDescriptor_cf57597c3a9659a9, []int{31, 0}
-}
-
-func (m *Trigger_AfterAll) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_Trigger_AfterAll.Unmarshal(m, b)
-}
-func (m *Trigger_AfterAll) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_Trigger_AfterAll.Marshal(b, m, deterministic)
-}
-func (m *Trigger_AfterAll) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Trigger_AfterAll.Merge(m, src)
-}
-func (m *Trigger_AfterAll) XXX_Size() int {
-	return xxx_messageInfo_Trigger_AfterAll.Size(m)
-}
-func (m *Trigger_AfterAll) XXX_DiscardUnknown() {
-	xxx_messageInfo_Trigger_AfterAll.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_Trigger_AfterAll proto.InternalMessageInfo
-
-func (m *Trigger_AfterAll) GetSubtriggers() []*Trigger {
-	if m != nil {
-		return m.Subtriggers
-	}
-	return nil
-}
-
-// Ready when any subtrigger is ready.
-type Trigger_AfterAny struct {
-	Subtriggers          []*Trigger `protobuf:"bytes,1,rep,name=subtriggers,proto3" json:"subtriggers,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}   `json:"-"`
-	XXX_unrecognized     []byte     `json:"-"`
-	XXX_sizecache        int32      `json:"-"`
-}
-
-func (m *Trigger_AfterAny) Reset()         { *m = Trigger_AfterAny{} }
-func (m *Trigger_AfterAny) String() string { return proto.CompactTextString(m) }
-func (*Trigger_AfterAny) ProtoMessage()    {}
-func (*Trigger_AfterAny) Descriptor() ([]byte, []int) {
-	return fileDescriptor_cf57597c3a9659a9, []int{31, 1}
-}
-
-func (m *Trigger_AfterAny) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_Trigger_AfterAny.Unmarshal(m, b)
-}
-func (m *Trigger_AfterAny) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_Trigger_AfterAny.Marshal(b, m, deterministic)
-}
-func (m *Trigger_AfterAny) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Trigger_AfterAny.Merge(m, src)
-}
-func (m *Trigger_AfterAny) XXX_Size() int {
-	return xxx_messageInfo_Trigger_AfterAny.Size(m)
-}
-func (m *Trigger_AfterAny) XXX_DiscardUnknown() {
-	xxx_messageInfo_Trigger_AfterAny.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_Trigger_AfterAny proto.InternalMessageInfo
-
-func (m *Trigger_AfterAny) GetSubtriggers() []*Trigger {
-	if m != nil {
-		return m.Subtriggers
-	}
-	return nil
-}
-
-// Starting with the first subtrigger, ready when the _current_ subtrigger
-// is ready. After output, advances the current trigger by one.
-type Trigger_AfterEach struct {
-	Subtriggers          []*Trigger `protobuf:"bytes,1,rep,name=subtriggers,proto3" json:"subtriggers,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}   `json:"-"`
-	XXX_unrecognized     []byte     `json:"-"`
-	XXX_sizecache        int32      `json:"-"`
-}
-
-func (m *Trigger_AfterEach) Reset()         { *m = Trigger_AfterEach{} }
-func (m *Trigger_AfterEach) String() string { return proto.CompactTextString(m) }
-func (*Trigger_AfterEach) ProtoMessage()    {}
-func (*Trigger_AfterEach) Descriptor() ([]byte, []int) {
-	return fileDescriptor_cf57597c3a9659a9, []int{31, 2}
-}
-
-func (m *Trigger_AfterEach) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_Trigger_AfterEach.Unmarshal(m, b)
-}
-func (m *Trigger_AfterEach) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_Trigger_AfterEach.Marshal(b, m, deterministic)
-}
-func (m *Trigger_AfterEach) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Trigger_AfterEach.Merge(m, src)
-}
-func (m *Trigger_AfterEach) XXX_Size() int {
-	return xxx_messageInfo_Trigger_AfterEach.Size(m)
-}
-func (m *Trigger_AfterEach) XXX_DiscardUnknown() {
-	xxx_messageInfo_Trigger_AfterEach.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_Trigger_AfterEach proto.InternalMessageInfo
-
-func (m *Trigger_AfterEach) GetSubtriggers() []*Trigger {
-	if m != nil {
-		return m.Subtriggers
-	}
-	return nil
-}
-
-// Ready after the input watermark is past the end of the window.
-//
-// May have implicitly-repeated subtriggers for early and late firings.
-// When the end of the window is reached, the trigger transitions between
-// the subtriggers.
-type Trigger_AfterEndOfWindow struct {
-	// (Optional) A trigger governing output prior to the end of the window.
-	EarlyFirings *Trigger `protobuf:"bytes,1,opt,name=early_firings,json=earlyFirings,proto3" json:"early_firings,omitempty"`
-	// (Optional) A trigger governing output after the end of the window.
-	LateFirings          *Trigger `protobuf:"bytes,2,opt,name=late_firings,json=lateFirings,proto3" json:"late_firings,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *Trigger_AfterEndOfWindow) Reset()         { *m = Trigger_AfterEndOfWindow{} }
-func (m *Trigger_AfterEndOfWindow) String() string { return proto.CompactTextString(m) }
-func (*Trigger_AfterEndOfWindow) ProtoMessage()    {}
-func (*Trigger_AfterEndOfWindow) Descriptor() ([]byte, []int) {
-	return fileDescriptor_cf57597c3a9659a9, []int{31, 3}
-}
-
-func (m *Trigger_AfterEndOfWindow) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_Trigger_AfterEndOfWindow.Unmarshal(m, b)
-}
-func (m *Trigger_AfterEndOfWindow) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_Trigger_AfterEndOfWindow.Marshal(b, m, deterministic)
-}
-func (m *Trigger_AfterEndOfWindow) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Trigger_AfterEndOfWindow.Merge(m, src)
-}
-func (m *Trigger_AfterEndOfWindow) XXX_Size() int {
-	return xxx_messageInfo_Trigger_AfterEndOfWindow.Size(m)
-}
-func (m *Trigger_AfterEndOfWindow) XXX_DiscardUnknown() {
-	xxx_messageInfo_Trigger_AfterEndOfWindow.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_Trigger_AfterEndOfWindow proto.InternalMessageInfo
-
-func (m *Trigger_AfterEndOfWindow) GetEarlyFirings() *Trigger {
-	if m != nil {
-		return m.EarlyFirings
-	}
-	return nil
-}
-
-func (m *Trigger_AfterEndOfWindow) GetLateFirings() *Trigger {
-	if m != nil {
-		return m.LateFirings
-	}
-	return nil
-}
-
-// After input arrives, ready when the specified delay has passed.
-type Trigger_AfterProcessingTime struct {
-	// (Required) The transforms to apply to an arriving element's timestamp,
-	// in order
-	TimestampTransforms  []*TimestampTransform `protobuf:"bytes,1,rep,name=timestamp_transforms,json=timestampTransforms,proto3" json:"timestamp_transforms,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}              `json:"-"`
-	XXX_unrecognized     []byte                `json:"-"`
-	XXX_sizecache        int32                 `json:"-"`
-}
-
-func (m *Trigger_AfterProcessingTime) Reset()         { *m = Trigger_AfterProcessingTime{} }
-func (m *Trigger_AfterProcessingTime) String() string { return proto.CompactTextString(m) }
-func (*Trigger_AfterProcessingTime) ProtoMessage()    {}
-func (*Trigger_AfterProcessingTime) Descriptor() ([]byte, []int) {
-	return fileDescriptor_cf57597c3a9659a9, []int{31, 4}
-}
-
-func (m *Trigger_AfterProcessingTime) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_Trigger_AfterProcessingTime.Unmarshal(m, b)
-}
-func (m *Trigger_AfterProcessingTime) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_Trigger_AfterProcessingTime.Marshal(b, m, deterministic)
-}
-func (m *Trigger_AfterProcessingTime) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Trigger_AfterProcessingTime.Merge(m, src)
-}
-func (m *Trigger_AfterProcessingTime) XXX_Size() int {
-	return xxx_messageInfo_Trigger_AfterProcessingTime.Size(m)
-}
-func (m *Trigger_AfterProcessingTime) XXX_DiscardUnknown() {
-	xxx_messageInfo_Trigger_AfterProcessingTime.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_Trigger_AfterProcessingTime proto.InternalMessageInfo
-
-func (m *Trigger_AfterProcessingTime) GetTimestampTransforms() []*TimestampTransform {
-	if m != nil {
-		return m.TimestampTransforms
-	}
-	return nil
-}
-
-// Ready whenever upstream processing time has all caught up with
-// the arrival time of an input element
-type Trigger_AfterSynchronizedProcessingTime struct {
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *Trigger_AfterSynchronizedProcessingTime) Reset() {
-	*m = Trigger_AfterSynchronizedProcessingTime{}
-}
-func (m *Trigger_AfterSynchronizedProcessingTime) String() string { return proto.CompactTextString(m) }
-func (*Trigger_AfterSynchronizedProcessingTime) ProtoMessage()    {}
-func (*Trigger_AfterSynchronizedProcessingTime) Descriptor() ([]byte, []int) {
-	return fileDescriptor_cf57597c3a9659a9, []int{31, 5}
-}
-
-func (m *Trigger_AfterSynchronizedProcessingTime) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_Trigger_AfterSynchronizedProcessingTime.Unmarshal(m, b)
-}
-func (m *Trigger_AfterSynchronizedProcessingTime) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_Trigger_AfterSynchronizedProcessingTime.Marshal(b, m, deterministic)
-}
-func (m *Trigger_AfterSynchronizedProcessingTime) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Trigger_AfterSynchronizedProcessingTime.Merge(m, src)
-}
-func (m *Trigger_AfterSynchronizedProcessingTime) XXX_Size() int {
-	return xxx_messageInfo_Trigger_AfterSynchronizedProcessingTime.Size(m)
-}
-func (m *Trigger_AfterSynchronizedProcessingTime) XXX_DiscardUnknown() {
-	xxx_messageInfo_Trigger_AfterSynchronizedProcessingTime.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_Trigger_AfterSynchronizedProcessingTime proto.InternalMessageInfo
-
-// The default trigger. Equivalent to Repeat { AfterEndOfWindow } but
-// specially denoted to indicate the user did not alter the triggering.
-type Trigger_Default struct {
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *Trigger_Default) Reset()         { *m = Trigger_Default{} }
-func (m *Trigger_Default) String() string { return proto.CompactTextString(m) }
-func (*Trigger_Default) ProtoMessage()    {}
-func (*Trigger_Default) Descriptor() ([]byte, []int) {
-	return fileDescriptor_cf57597c3a9659a9, []int{31, 6}
-}
-
-func (m *Trigger_Default) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_Trigger_Default.Unmarshal(m, b)
-}
-func (m *Trigger_Default) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_Trigger_Default.Marshal(b, m, deterministic)
-}
-func (m *Trigger_Default) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Trigger_Default.Merge(m, src)
-}
-func (m *Trigger_Default) XXX_Size() int {
-	return xxx_messageInfo_Trigger_Default.Size(m)
-}
-func (m *Trigger_Default) XXX_DiscardUnknown() {
-	xxx_messageInfo_Trigger_Default.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_Trigger_Default proto.InternalMessageInfo
-
-// Ready whenever the requisite number of input elements have arrived
-type Trigger_ElementCount struct {
-	ElementCount         int32    `protobuf:"varint,1,opt,name=element_count,json=elementCount,proto3" json:"element_count,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *Trigger_ElementCount) Reset()         { *m = Trigger_ElementCount{} }
-func (m *Trigger_ElementCount) String() string { return proto.CompactTextString(m) }
-func (*Trigger_ElementCount) ProtoMessage()    {}
-func (*Trigger_ElementCount) Descriptor() ([]byte, []int) {
-	return fileDescriptor_cf57597c3a9659a9, []int{31, 7}
-}
-
-func (m *Trigger_ElementCount) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_Trigger_ElementCount.Unmarshal(m, b)
-}
-func (m *Trigger_ElementCount) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_Trigger_ElementCount.Marshal(b, m, deterministic)
-}
-func (m *Trigger_ElementCount) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Trigger_ElementCount.Merge(m, src)
-}
-func (m *Trigger_ElementCount) XXX_Size() int {
-	return xxx_messageInfo_Trigger_ElementCount.Size(m)
-}
-func (m *Trigger_ElementCount) XXX_DiscardUnknown() {
-	xxx_messageInfo_Trigger_ElementCount.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_Trigger_ElementCount proto.InternalMessageInfo
-
-func (m *Trigger_ElementCount) GetElementCount() int32 {
-	if m != nil {
-		return m.ElementCount
-	}
-	return 0
-}
-
-// Never ready. There will only be an ON_TIME output and a final
-// output at window expiration.
-type Trigger_Never struct {
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *Trigger_Never) Reset()         { *m = Trigger_Never{} }
-func (m *Trigger_Never) String() string { return proto.CompactTextString(m) }
-func (*Trigger_Never) ProtoMessage()    {}
-func (*Trigger_Never) Descriptor() ([]byte, []int) {
-	return fileDescriptor_cf57597c3a9659a9, []int{31, 8}
-}
-
-func (m *Trigger_Never) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_Trigger_Never.Unmarshal(m, b)
-}
-func (m *Trigger_Never) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_Trigger_Never.Marshal(b, m, deterministic)
-}
-func (m *Trigger_Never) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Trigger_Never.Merge(m, src)
-}
-func (m *Trigger_Never) XXX_Size() int {
-	return xxx_messageInfo_Trigger_Never.Size(m)
-}
-func (m *Trigger_Never) XXX_DiscardUnknown() {
-	xxx_messageInfo_Trigger_Never.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_Trigger_Never proto.InternalMessageInfo
-
-// Always ready. This can also be expressed as ElementCount(1) but
-// is more explicit.
-type Trigger_Always struct {
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *Trigger_Always) Reset()         { *m = Trigger_Always{} }
-func (m *Trigger_Always) String() string { return proto.CompactTextString(m) }
-func (*Trigger_Always) ProtoMessage()    {}
-func (*Trigger_Always) Descriptor() ([]byte, []int) {
-	return fileDescriptor_cf57597c3a9659a9, []int{31, 9}
-}
-
-func (m *Trigger_Always) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_Trigger_Always.Unmarshal(m, b)
-}
-func (m *Trigger_Always) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_Trigger_Always.Marshal(b, m, deterministic)
-}
-func (m *Trigger_Always) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Trigger_Always.Merge(m, src)
-}
-func (m *Trigger_Always) XXX_Size() int {
-	return xxx_messageInfo_Trigger_Always.Size(m)
-}
-func (m *Trigger_Always) XXX_DiscardUnknown() {
-	xxx_messageInfo_Trigger_Always.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_Trigger_Always proto.InternalMessageInfo
-
-// Ready whenever either of its subtriggers are ready, but finishes output
-// when the finally subtrigger fires.
-type Trigger_OrFinally struct {
-	// (Required) Trigger governing main output; may fire repeatedly.
-	Main *Trigger `protobuf:"bytes,1,opt,name=main,proto3" json:"main,omitempty"`
-	// (Required) Trigger governing termination of output.
-	Finally              *Trigger `protobuf:"bytes,2,opt,name=finally,proto3" json:"finally,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *Trigger_OrFinally) Reset()         { *m = Trigger_OrFinally{} }
-func (m *Trigger_OrFinally) String() string { return proto.CompactTextString(m) }
-func (*Trigger_OrFinally) ProtoMessage()    {}
-func (*Trigger_OrFinally) Descriptor() ([]byte, []int) {
-	return fileDescriptor_cf57597c3a9659a9, []int{31, 10}
-}
-
-func (m *Trigger_OrFinally) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_Trigger_OrFinally.Unmarshal(m, b)
-}
-func (m *Trigger_OrFinally) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_Trigger_OrFinally.Marshal(b, m, deterministic)
-}
-func (m *Trigger_OrFinally) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Trigger_OrFinally.Merge(m, src)
-}
-func (m *Trigger_OrFinally) XXX_Size() int {
-	return xxx_messageInfo_Trigger_OrFinally.Size(m)
-}
-func (m *Trigger_OrFinally) XXX_DiscardUnknown() {
-	xxx_messageInfo_Trigger_OrFinally.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_Trigger_OrFinally proto.InternalMessageInfo
-
-func (m *Trigger_OrFinally) GetMain() *Trigger {
-	if m != nil {
-		return m.Main
-	}
-	return nil
-}
-
-func (m *Trigger_OrFinally) GetFinally() *Trigger {
-	if m != nil {
-		return m.Finally
-	}
-	return nil
-}
-
-// Ready whenever the subtrigger is ready; resets state when the subtrigger
-// completes.
-type Trigger_Repeat struct {
-	// (Require) Trigger that is run repeatedly.
-	Subtrigger           *Trigger `protobuf:"bytes,1,opt,name=subtrigger,proto3" json:"subtrigger,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *Trigger_Repeat) Reset()         { *m = Trigger_Repeat{} }
-func (m *Trigger_Repeat) String() string { return proto.CompactTextString(m) }
-func (*Trigger_Repeat) ProtoMessage()    {}
-func (*Trigger_Repeat) Descriptor() ([]byte, []int) {
-	return fileDescriptor_cf57597c3a9659a9, []int{31, 11}
-}
-
-func (m *Trigger_Repeat) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_Trigger_Repeat.Unmarshal(m, b)
-}
-func (m *Trigger_Repeat) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_Trigger_Repeat.Marshal(b, m, deterministic)
-}
-func (m *Trigger_Repeat) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Trigger_Repeat.Merge(m, src)
-}
-func (m *Trigger_Repeat) XXX_Size() int {
-	return xxx_messageInfo_Trigger_Repeat.Size(m)
-}
-func (m *Trigger_Repeat) XXX_DiscardUnknown() {
-	xxx_messageInfo_Trigger_Repeat.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_Trigger_Repeat proto.InternalMessageInfo
-
-func (m *Trigger_Repeat) GetSubtrigger() *Trigger {
-	if m != nil {
-		return m.Subtrigger
-	}
-	return nil
-}
-
 // A specification for a transformation on a timestamp.
 //
 // Primarily used by AfterProcessingTime triggers to transform
 // the arrival time of input to a target time for firing.
 type TimestampTransform struct {
-	// Types that are valid to be assigned to TimestampTransform:
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	// Types that are assignable to TimestampTransform:
 	//	*TimestampTransform_Delay_
 	//	*TimestampTransform_AlignTo_
-	TimestampTransform   isTimestampTransform_TimestampTransform `protobuf_oneof:"timestamp_transform"`
-	XXX_NoUnkeyedLiteral struct{}                                `json:"-"`
-	XXX_unrecognized     []byte                                  `json:"-"`
-	XXX_sizecache        int32                                   `json:"-"`
+	TimestampTransform isTimestampTransform_TimestampTransform `protobuf_oneof:"timestamp_transform"`
 }
 
-func (m *TimestampTransform) Reset()         { *m = TimestampTransform{} }
-func (m *TimestampTransform) String() string { return proto.CompactTextString(m) }
-func (*TimestampTransform) ProtoMessage()    {}
+func (x *TimestampTransform) Reset() {
+	*x = TimestampTransform{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_runner_api_proto_msgTypes[35]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *TimestampTransform) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*TimestampTransform) ProtoMessage() {}
+
+func (x *TimestampTransform) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_runner_api_proto_msgTypes[35]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use TimestampTransform.ProtoReflect.Descriptor instead.
 func (*TimestampTransform) Descriptor() ([]byte, []int) {
-	return fileDescriptor_cf57597c3a9659a9, []int{32}
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{35}
 }
 
-func (m *TimestampTransform) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_TimestampTransform.Unmarshal(m, b)
-}
-func (m *TimestampTransform) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_TimestampTransform.Marshal(b, m, deterministic)
-}
-func (m *TimestampTransform) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_TimestampTransform.Merge(m, src)
-}
-func (m *TimestampTransform) XXX_Size() int {
-	return xxx_messageInfo_TimestampTransform.Size(m)
-}
-func (m *TimestampTransform) XXX_DiscardUnknown() {
-	xxx_messageInfo_TimestampTransform.DiscardUnknown(m)
+func (m *TimestampTransform) GetTimestampTransform() isTimestampTransform_TimestampTransform {
+	if m != nil {
+		return m.TimestampTransform
+	}
+	return nil
 }
 
-var xxx_messageInfo_TimestampTransform proto.InternalMessageInfo
+func (x *TimestampTransform) GetDelay() *TimestampTransform_Delay {
+	if x, ok := x.GetTimestampTransform().(*TimestampTransform_Delay_); ok {
+		return x.Delay
+	}
+	return nil
+}
+
+func (x *TimestampTransform) GetAlignTo() *TimestampTransform_AlignTo {
+	if x, ok := x.GetTimestampTransform().(*TimestampTransform_AlignTo_); ok {
+		return x.AlignTo
+	}
+	return nil
+}
 
 type isTimestampTransform_TimestampTransform interface {
 	isTimestampTransform_TimestampTransform()
@@ -3944,128 +4114,12 @@
 
 func (*TimestampTransform_AlignTo_) isTimestampTransform_TimestampTransform() {}
 
-func (m *TimestampTransform) GetTimestampTransform() isTimestampTransform_TimestampTransform {
-	if m != nil {
-		return m.TimestampTransform
-	}
-	return nil
-}
-
-func (m *TimestampTransform) GetDelay() *TimestampTransform_Delay {
-	if x, ok := m.GetTimestampTransform().(*TimestampTransform_Delay_); ok {
-		return x.Delay
-	}
-	return nil
-}
-
-func (m *TimestampTransform) GetAlignTo() *TimestampTransform_AlignTo {
-	if x, ok := m.GetTimestampTransform().(*TimestampTransform_AlignTo_); ok {
-		return x.AlignTo
-	}
-	return nil
-}
-
-// XXX_OneofWrappers is for the internal use of the proto package.
-func (*TimestampTransform) XXX_OneofWrappers() []interface{} {
-	return []interface{}{
-		(*TimestampTransform_Delay_)(nil),
-		(*TimestampTransform_AlignTo_)(nil),
-	}
-}
-
-type TimestampTransform_Delay struct {
-	// (Required) The delay, in milliseconds.
-	DelayMillis          int64    `protobuf:"varint,1,opt,name=delay_millis,json=delayMillis,proto3" json:"delay_millis,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *TimestampTransform_Delay) Reset()         { *m = TimestampTransform_Delay{} }
-func (m *TimestampTransform_Delay) String() string { return proto.CompactTextString(m) }
-func (*TimestampTransform_Delay) ProtoMessage()    {}
-func (*TimestampTransform_Delay) Descriptor() ([]byte, []int) {
-	return fileDescriptor_cf57597c3a9659a9, []int{32, 0}
-}
-
-func (m *TimestampTransform_Delay) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_TimestampTransform_Delay.Unmarshal(m, b)
-}
-func (m *TimestampTransform_Delay) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_TimestampTransform_Delay.Marshal(b, m, deterministic)
-}
-func (m *TimestampTransform_Delay) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_TimestampTransform_Delay.Merge(m, src)
-}
-func (m *TimestampTransform_Delay) XXX_Size() int {
-	return xxx_messageInfo_TimestampTransform_Delay.Size(m)
-}
-func (m *TimestampTransform_Delay) XXX_DiscardUnknown() {
-	xxx_messageInfo_TimestampTransform_Delay.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_TimestampTransform_Delay proto.InternalMessageInfo
-
-func (m *TimestampTransform_Delay) GetDelayMillis() int64 {
-	if m != nil {
-		return m.DelayMillis
-	}
-	return 0
-}
-
-type TimestampTransform_AlignTo struct {
-	// (Required) A duration to which delays should be quantized
-	// in milliseconds.
-	Period int64 `protobuf:"varint,3,opt,name=period,proto3" json:"period,omitempty"`
-	// (Required) An offset from 0 for the quantization specified by
-	// alignment_size, in milliseconds
-	Offset               int64    `protobuf:"varint,4,opt,name=offset,proto3" json:"offset,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *TimestampTransform_AlignTo) Reset()         { *m = TimestampTransform_AlignTo{} }
-func (m *TimestampTransform_AlignTo) String() string { return proto.CompactTextString(m) }
-func (*TimestampTransform_AlignTo) ProtoMessage()    {}
-func (*TimestampTransform_AlignTo) Descriptor() ([]byte, []int) {
-	return fileDescriptor_cf57597c3a9659a9, []int{32, 1}
-}
-
-func (m *TimestampTransform_AlignTo) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_TimestampTransform_AlignTo.Unmarshal(m, b)
-}
-func (m *TimestampTransform_AlignTo) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_TimestampTransform_AlignTo.Marshal(b, m, deterministic)
-}
-func (m *TimestampTransform_AlignTo) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_TimestampTransform_AlignTo.Merge(m, src)
-}
-func (m *TimestampTransform_AlignTo) XXX_Size() int {
-	return xxx_messageInfo_TimestampTransform_AlignTo.Size(m)
-}
-func (m *TimestampTransform_AlignTo) XXX_DiscardUnknown() {
-	xxx_messageInfo_TimestampTransform_AlignTo.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_TimestampTransform_AlignTo proto.InternalMessageInfo
-
-func (m *TimestampTransform_AlignTo) GetPeriod() int64 {
-	if m != nil {
-		return m.Period
-	}
-	return 0
-}
-
-func (m *TimestampTransform_AlignTo) GetOffset() int64 {
-	if m != nil {
-		return m.Offset
-	}
-	return 0
-}
-
 // A specification for how to "side input" a PCollection.
 type SideInput struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
 	// (Required) URN of the access pattern required by the `view_fn` to present
 	// the desired SDK-specific interface to a UDF.
 	//
@@ -4088,468 +4142,543 @@
 	// For example, when the main input is in fixed windows of one hour, this
 	// can specify that the side input should be accessed according to the day
 	// in which that hour falls.
-	WindowMappingFn      *FunctionSpec `protobuf:"bytes,3,opt,name=window_mapping_fn,json=windowMappingFn,proto3" json:"window_mapping_fn,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}      `json:"-"`
-	XXX_unrecognized     []byte        `json:"-"`
-	XXX_sizecache        int32         `json:"-"`
+	WindowMappingFn *FunctionSpec `protobuf:"bytes,3,opt,name=window_mapping_fn,json=windowMappingFn,proto3" json:"window_mapping_fn,omitempty"`
 }
 
-func (m *SideInput) Reset()         { *m = SideInput{} }
-func (m *SideInput) String() string { return proto.CompactTextString(m) }
-func (*SideInput) ProtoMessage()    {}
+func (x *SideInput) Reset() {
+	*x = SideInput{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_runner_api_proto_msgTypes[36]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *SideInput) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*SideInput) ProtoMessage() {}
+
+func (x *SideInput) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_runner_api_proto_msgTypes[36]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use SideInput.ProtoReflect.Descriptor instead.
 func (*SideInput) Descriptor() ([]byte, []int) {
-	return fileDescriptor_cf57597c3a9659a9, []int{33}
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{36}
 }
 
-func (m *SideInput) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_SideInput.Unmarshal(m, b)
-}
-func (m *SideInput) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_SideInput.Marshal(b, m, deterministic)
-}
-func (m *SideInput) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_SideInput.Merge(m, src)
-}
-func (m *SideInput) XXX_Size() int {
-	return xxx_messageInfo_SideInput.Size(m)
-}
-func (m *SideInput) XXX_DiscardUnknown() {
-	xxx_messageInfo_SideInput.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_SideInput proto.InternalMessageInfo
-
-func (m *SideInput) GetAccessPattern() *FunctionSpec {
-	if m != nil {
-		return m.AccessPattern
+func (x *SideInput) GetAccessPattern() *FunctionSpec {
+	if x != nil {
+		return x.AccessPattern
 	}
 	return nil
 }
 
-func (m *SideInput) GetViewFn() *FunctionSpec {
-	if m != nil {
-		return m.ViewFn
+func (x *SideInput) GetViewFn() *FunctionSpec {
+	if x != nil {
+		return x.ViewFn
 	}
 	return nil
 }
 
-func (m *SideInput) GetWindowMappingFn() *FunctionSpec {
-	if m != nil {
-		return m.WindowMappingFn
+func (x *SideInput) GetWindowMappingFn() *FunctionSpec {
+	if x != nil {
+		return x.WindowMappingFn
 	}
 	return nil
 }
 
 type StandardArtifacts struct {
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
 }
 
-func (m *StandardArtifacts) Reset()         { *m = StandardArtifacts{} }
-func (m *StandardArtifacts) String() string { return proto.CompactTextString(m) }
-func (*StandardArtifacts) ProtoMessage()    {}
+func (x *StandardArtifacts) Reset() {
+	*x = StandardArtifacts{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_runner_api_proto_msgTypes[37]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *StandardArtifacts) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*StandardArtifacts) ProtoMessage() {}
+
+func (x *StandardArtifacts) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_runner_api_proto_msgTypes[37]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use StandardArtifacts.ProtoReflect.Descriptor instead.
 func (*StandardArtifacts) Descriptor() ([]byte, []int) {
-	return fileDescriptor_cf57597c3a9659a9, []int{34}
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{37}
 }
 
-func (m *StandardArtifacts) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_StandardArtifacts.Unmarshal(m, b)
-}
-func (m *StandardArtifacts) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_StandardArtifacts.Marshal(b, m, deterministic)
-}
-func (m *StandardArtifacts) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_StandardArtifacts.Merge(m, src)
-}
-func (m *StandardArtifacts) XXX_Size() int {
-	return xxx_messageInfo_StandardArtifacts.Size(m)
-}
-func (m *StandardArtifacts) XXX_DiscardUnknown() {
-	xxx_messageInfo_StandardArtifacts.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_StandardArtifacts proto.InternalMessageInfo
-
 type ArtifactFilePayload struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
 	// a string for an artifact file path e.g. "/tmp/foo.jar"
 	Path string `protobuf:"bytes,1,opt,name=path,proto3" json:"path,omitempty"`
 	// The hex-encoded sha256 checksum of the artifact.
-	Sha256               string   `protobuf:"bytes,2,opt,name=sha256,proto3" json:"sha256,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
+	Sha256 string `protobuf:"bytes,2,opt,name=sha256,proto3" json:"sha256,omitempty"`
 }
 
-func (m *ArtifactFilePayload) Reset()         { *m = ArtifactFilePayload{} }
-func (m *ArtifactFilePayload) String() string { return proto.CompactTextString(m) }
-func (*ArtifactFilePayload) ProtoMessage()    {}
+func (x *ArtifactFilePayload) Reset() {
+	*x = ArtifactFilePayload{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_runner_api_proto_msgTypes[38]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *ArtifactFilePayload) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*ArtifactFilePayload) ProtoMessage() {}
+
+func (x *ArtifactFilePayload) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_runner_api_proto_msgTypes[38]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use ArtifactFilePayload.ProtoReflect.Descriptor instead.
 func (*ArtifactFilePayload) Descriptor() ([]byte, []int) {
-	return fileDescriptor_cf57597c3a9659a9, []int{35}
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{38}
 }
 
-func (m *ArtifactFilePayload) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_ArtifactFilePayload.Unmarshal(m, b)
-}
-func (m *ArtifactFilePayload) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_ArtifactFilePayload.Marshal(b, m, deterministic)
-}
-func (m *ArtifactFilePayload) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ArtifactFilePayload.Merge(m, src)
-}
-func (m *ArtifactFilePayload) XXX_Size() int {
-	return xxx_messageInfo_ArtifactFilePayload.Size(m)
-}
-func (m *ArtifactFilePayload) XXX_DiscardUnknown() {
-	xxx_messageInfo_ArtifactFilePayload.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_ArtifactFilePayload proto.InternalMessageInfo
-
-func (m *ArtifactFilePayload) GetPath() string {
-	if m != nil {
-		return m.Path
+func (x *ArtifactFilePayload) GetPath() string {
+	if x != nil {
+		return x.Path
 	}
 	return ""
 }
 
-func (m *ArtifactFilePayload) GetSha256() string {
-	if m != nil {
-		return m.Sha256
+func (x *ArtifactFilePayload) GetSha256() string {
+	if x != nil {
+		return x.Sha256
 	}
 	return ""
 }
 
 type ArtifactUrlPayload struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
 	// a string for an artifact URL e.g. "https://.../foo.jar" or "gs://tmp/foo.jar"
-	Url                  string   `protobuf:"bytes,1,opt,name=url,proto3" json:"url,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
+	Url string `protobuf:"bytes,1,opt,name=url,proto3" json:"url,omitempty"`
 }
 
-func (m *ArtifactUrlPayload) Reset()         { *m = ArtifactUrlPayload{} }
-func (m *ArtifactUrlPayload) String() string { return proto.CompactTextString(m) }
-func (*ArtifactUrlPayload) ProtoMessage()    {}
+func (x *ArtifactUrlPayload) Reset() {
+	*x = ArtifactUrlPayload{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_runner_api_proto_msgTypes[39]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *ArtifactUrlPayload) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*ArtifactUrlPayload) ProtoMessage() {}
+
+func (x *ArtifactUrlPayload) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_runner_api_proto_msgTypes[39]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use ArtifactUrlPayload.ProtoReflect.Descriptor instead.
 func (*ArtifactUrlPayload) Descriptor() ([]byte, []int) {
-	return fileDescriptor_cf57597c3a9659a9, []int{36}
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{39}
 }
 
-func (m *ArtifactUrlPayload) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_ArtifactUrlPayload.Unmarshal(m, b)
-}
-func (m *ArtifactUrlPayload) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_ArtifactUrlPayload.Marshal(b, m, deterministic)
-}
-func (m *ArtifactUrlPayload) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ArtifactUrlPayload.Merge(m, src)
-}
-func (m *ArtifactUrlPayload) XXX_Size() int {
-	return xxx_messageInfo_ArtifactUrlPayload.Size(m)
-}
-func (m *ArtifactUrlPayload) XXX_DiscardUnknown() {
-	xxx_messageInfo_ArtifactUrlPayload.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_ArtifactUrlPayload proto.InternalMessageInfo
-
-func (m *ArtifactUrlPayload) GetUrl() string {
-	if m != nil {
-		return m.Url
+func (x *ArtifactUrlPayload) GetUrl() string {
+	if x != nil {
+		return x.Url
 	}
 	return ""
 }
 
 type EmbeddedFilePayload struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
 	// raw data bytes for an embedded artifact
-	Data                 []byte   `protobuf:"bytes,1,opt,name=data,proto3" json:"data,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
+	Data []byte `protobuf:"bytes,1,opt,name=data,proto3" json:"data,omitempty"`
 }
 
-func (m *EmbeddedFilePayload) Reset()         { *m = EmbeddedFilePayload{} }
-func (m *EmbeddedFilePayload) String() string { return proto.CompactTextString(m) }
-func (*EmbeddedFilePayload) ProtoMessage()    {}
+func (x *EmbeddedFilePayload) Reset() {
+	*x = EmbeddedFilePayload{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_runner_api_proto_msgTypes[40]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *EmbeddedFilePayload) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*EmbeddedFilePayload) ProtoMessage() {}
+
+func (x *EmbeddedFilePayload) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_runner_api_proto_msgTypes[40]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use EmbeddedFilePayload.ProtoReflect.Descriptor instead.
 func (*EmbeddedFilePayload) Descriptor() ([]byte, []int) {
-	return fileDescriptor_cf57597c3a9659a9, []int{37}
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{40}
 }
 
-func (m *EmbeddedFilePayload) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_EmbeddedFilePayload.Unmarshal(m, b)
-}
-func (m *EmbeddedFilePayload) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_EmbeddedFilePayload.Marshal(b, m, deterministic)
-}
-func (m *EmbeddedFilePayload) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_EmbeddedFilePayload.Merge(m, src)
-}
-func (m *EmbeddedFilePayload) XXX_Size() int {
-	return xxx_messageInfo_EmbeddedFilePayload.Size(m)
-}
-func (m *EmbeddedFilePayload) XXX_DiscardUnknown() {
-	xxx_messageInfo_EmbeddedFilePayload.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_EmbeddedFilePayload proto.InternalMessageInfo
-
-func (m *EmbeddedFilePayload) GetData() []byte {
-	if m != nil {
-		return m.Data
+func (x *EmbeddedFilePayload) GetData() []byte {
+	if x != nil {
+		return x.Data
 	}
 	return nil
 }
 
 type PyPIPayload struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
 	// Pypi compatible artifact id e.g. "apache-beam"
 	ArtifactId string `protobuf:"bytes,1,opt,name=artifact_id,json=artifactId,proto3" json:"artifact_id,omitempty"`
 	// Pypi compatible version string.
-	Version              string   `protobuf:"bytes,2,opt,name=version,proto3" json:"version,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
+	Version string `protobuf:"bytes,2,opt,name=version,proto3" json:"version,omitempty"`
 }
 
-func (m *PyPIPayload) Reset()         { *m = PyPIPayload{} }
-func (m *PyPIPayload) String() string { return proto.CompactTextString(m) }
-func (*PyPIPayload) ProtoMessage()    {}
+func (x *PyPIPayload) Reset() {
+	*x = PyPIPayload{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_runner_api_proto_msgTypes[41]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *PyPIPayload) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*PyPIPayload) ProtoMessage() {}
+
+func (x *PyPIPayload) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_runner_api_proto_msgTypes[41]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use PyPIPayload.ProtoReflect.Descriptor instead.
 func (*PyPIPayload) Descriptor() ([]byte, []int) {
-	return fileDescriptor_cf57597c3a9659a9, []int{38}
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{41}
 }
 
-func (m *PyPIPayload) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_PyPIPayload.Unmarshal(m, b)
-}
-func (m *PyPIPayload) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_PyPIPayload.Marshal(b, m, deterministic)
-}
-func (m *PyPIPayload) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_PyPIPayload.Merge(m, src)
-}
-func (m *PyPIPayload) XXX_Size() int {
-	return xxx_messageInfo_PyPIPayload.Size(m)
-}
-func (m *PyPIPayload) XXX_DiscardUnknown() {
-	xxx_messageInfo_PyPIPayload.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_PyPIPayload proto.InternalMessageInfo
-
-func (m *PyPIPayload) GetArtifactId() string {
-	if m != nil {
-		return m.ArtifactId
+func (x *PyPIPayload) GetArtifactId() string {
+	if x != nil {
+		return x.ArtifactId
 	}
 	return ""
 }
 
-func (m *PyPIPayload) GetVersion() string {
-	if m != nil {
-		return m.Version
+func (x *PyPIPayload) GetVersion() string {
+	if x != nil {
+		return x.Version
 	}
 	return ""
 }
 
 type MavenPayload struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
 	// A string specifying Maven artifact.
 	// The standard format is "groupId:artifactId:version[:packaging[:classifier]]"
 	Artifact string `protobuf:"bytes,1,opt,name=artifact,proto3" json:"artifact,omitempty"`
 	// (Optional) Repository URL. If not specified, Maven central is used by default.
-	RepositoryUrl        string   `protobuf:"bytes,2,opt,name=repository_url,json=repositoryUrl,proto3" json:"repository_url,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
+	RepositoryUrl string `protobuf:"bytes,2,opt,name=repository_url,json=repositoryUrl,proto3" json:"repository_url,omitempty"`
 }
 
-func (m *MavenPayload) Reset()         { *m = MavenPayload{} }
-func (m *MavenPayload) String() string { return proto.CompactTextString(m) }
-func (*MavenPayload) ProtoMessage()    {}
+func (x *MavenPayload) Reset() {
+	*x = MavenPayload{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_runner_api_proto_msgTypes[42]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *MavenPayload) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*MavenPayload) ProtoMessage() {}
+
+func (x *MavenPayload) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_runner_api_proto_msgTypes[42]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use MavenPayload.ProtoReflect.Descriptor instead.
 func (*MavenPayload) Descriptor() ([]byte, []int) {
-	return fileDescriptor_cf57597c3a9659a9, []int{39}
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{42}
 }
 
-func (m *MavenPayload) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_MavenPayload.Unmarshal(m, b)
-}
-func (m *MavenPayload) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_MavenPayload.Marshal(b, m, deterministic)
-}
-func (m *MavenPayload) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_MavenPayload.Merge(m, src)
-}
-func (m *MavenPayload) XXX_Size() int {
-	return xxx_messageInfo_MavenPayload.Size(m)
-}
-func (m *MavenPayload) XXX_DiscardUnknown() {
-	xxx_messageInfo_MavenPayload.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_MavenPayload proto.InternalMessageInfo
-
-func (m *MavenPayload) GetArtifact() string {
-	if m != nil {
-		return m.Artifact
+func (x *MavenPayload) GetArtifact() string {
+	if x != nil {
+		return x.Artifact
 	}
 	return ""
 }
 
-func (m *MavenPayload) GetRepositoryUrl() string {
-	if m != nil {
-		return m.RepositoryUrl
+func (x *MavenPayload) GetRepositoryUrl() string {
+	if x != nil {
+		return x.RepositoryUrl
 	}
 	return ""
 }
 
 type DeferredArtifactPayload struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
 	// A unique string identifier assigned by the creator of this payload. The creator may use this key to confirm
 	// whether they can parse the data.
 	Key string `protobuf:"bytes,1,opt,name=key,proto3" json:"key,omitempty"`
 	// Data for deferred artifacts. Interpretation of bytes is delegated to the creator of this payload.
-	Data                 []byte   `protobuf:"bytes,2,opt,name=data,proto3" json:"data,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
+	Data []byte `protobuf:"bytes,2,opt,name=data,proto3" json:"data,omitempty"`
 }
 
-func (m *DeferredArtifactPayload) Reset()         { *m = DeferredArtifactPayload{} }
-func (m *DeferredArtifactPayload) String() string { return proto.CompactTextString(m) }
-func (*DeferredArtifactPayload) ProtoMessage()    {}
+func (x *DeferredArtifactPayload) Reset() {
+	*x = DeferredArtifactPayload{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_runner_api_proto_msgTypes[43]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *DeferredArtifactPayload) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*DeferredArtifactPayload) ProtoMessage() {}
+
+func (x *DeferredArtifactPayload) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_runner_api_proto_msgTypes[43]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use DeferredArtifactPayload.ProtoReflect.Descriptor instead.
 func (*DeferredArtifactPayload) Descriptor() ([]byte, []int) {
-	return fileDescriptor_cf57597c3a9659a9, []int{40}
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{43}
 }
 
-func (m *DeferredArtifactPayload) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_DeferredArtifactPayload.Unmarshal(m, b)
-}
-func (m *DeferredArtifactPayload) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_DeferredArtifactPayload.Marshal(b, m, deterministic)
-}
-func (m *DeferredArtifactPayload) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_DeferredArtifactPayload.Merge(m, src)
-}
-func (m *DeferredArtifactPayload) XXX_Size() int {
-	return xxx_messageInfo_DeferredArtifactPayload.Size(m)
-}
-func (m *DeferredArtifactPayload) XXX_DiscardUnknown() {
-	xxx_messageInfo_DeferredArtifactPayload.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_DeferredArtifactPayload proto.InternalMessageInfo
-
-func (m *DeferredArtifactPayload) GetKey() string {
-	if m != nil {
-		return m.Key
+func (x *DeferredArtifactPayload) GetKey() string {
+	if x != nil {
+		return x.Key
 	}
 	return ""
 }
 
-func (m *DeferredArtifactPayload) GetData() []byte {
-	if m != nil {
-		return m.Data
+func (x *DeferredArtifactPayload) GetData() []byte {
+	if x != nil {
+		return x.Data
 	}
 	return nil
 }
 
 type ArtifactStagingToRolePayload struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
 	// A generated staged name (relative path under staging directory).
-	StagedName           string   `protobuf:"bytes,1,opt,name=staged_name,json=stagedName,proto3" json:"staged_name,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
+	StagedName string `protobuf:"bytes,1,opt,name=staged_name,json=stagedName,proto3" json:"staged_name,omitempty"`
 }
 
-func (m *ArtifactStagingToRolePayload) Reset()         { *m = ArtifactStagingToRolePayload{} }
-func (m *ArtifactStagingToRolePayload) String() string { return proto.CompactTextString(m) }
-func (*ArtifactStagingToRolePayload) ProtoMessage()    {}
+func (x *ArtifactStagingToRolePayload) Reset() {
+	*x = ArtifactStagingToRolePayload{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_runner_api_proto_msgTypes[44]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *ArtifactStagingToRolePayload) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*ArtifactStagingToRolePayload) ProtoMessage() {}
+
+func (x *ArtifactStagingToRolePayload) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_runner_api_proto_msgTypes[44]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use ArtifactStagingToRolePayload.ProtoReflect.Descriptor instead.
 func (*ArtifactStagingToRolePayload) Descriptor() ([]byte, []int) {
-	return fileDescriptor_cf57597c3a9659a9, []int{41}
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{44}
 }
 
-func (m *ArtifactStagingToRolePayload) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_ArtifactStagingToRolePayload.Unmarshal(m, b)
-}
-func (m *ArtifactStagingToRolePayload) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_ArtifactStagingToRolePayload.Marshal(b, m, deterministic)
-}
-func (m *ArtifactStagingToRolePayload) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ArtifactStagingToRolePayload.Merge(m, src)
-}
-func (m *ArtifactStagingToRolePayload) XXX_Size() int {
-	return xxx_messageInfo_ArtifactStagingToRolePayload.Size(m)
-}
-func (m *ArtifactStagingToRolePayload) XXX_DiscardUnknown() {
-	xxx_messageInfo_ArtifactStagingToRolePayload.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_ArtifactStagingToRolePayload proto.InternalMessageInfo
-
-func (m *ArtifactStagingToRolePayload) GetStagedName() string {
-	if m != nil {
-		return m.StagedName
+func (x *ArtifactStagingToRolePayload) GetStagedName() string {
+	if x != nil {
+		return x.StagedName
 	}
 	return ""
 }
 
 type ArtifactInformation struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
 	// A URN that describes the type of artifact
 	TypeUrn     string `protobuf:"bytes,1,opt,name=type_urn,json=typeUrn,proto3" json:"type_urn,omitempty"`
 	TypePayload []byte `protobuf:"bytes,2,opt,name=type_payload,json=typePayload,proto3" json:"type_payload,omitempty"`
 	// A URN that describes the role of artifact
-	RoleUrn              string   `protobuf:"bytes,3,opt,name=role_urn,json=roleUrn,proto3" json:"role_urn,omitempty"`
-	RolePayload          []byte   `protobuf:"bytes,4,opt,name=role_payload,json=rolePayload,proto3" json:"role_payload,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
+	RoleUrn     string `protobuf:"bytes,3,opt,name=role_urn,json=roleUrn,proto3" json:"role_urn,omitempty"`
+	RolePayload []byte `protobuf:"bytes,4,opt,name=role_payload,json=rolePayload,proto3" json:"role_payload,omitempty"`
 }
 
-func (m *ArtifactInformation) Reset()         { *m = ArtifactInformation{} }
-func (m *ArtifactInformation) String() string { return proto.CompactTextString(m) }
-func (*ArtifactInformation) ProtoMessage()    {}
+func (x *ArtifactInformation) Reset() {
+	*x = ArtifactInformation{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_runner_api_proto_msgTypes[45]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *ArtifactInformation) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*ArtifactInformation) ProtoMessage() {}
+
+func (x *ArtifactInformation) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_runner_api_proto_msgTypes[45]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use ArtifactInformation.ProtoReflect.Descriptor instead.
 func (*ArtifactInformation) Descriptor() ([]byte, []int) {
-	return fileDescriptor_cf57597c3a9659a9, []int{42}
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{45}
 }
 
-func (m *ArtifactInformation) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_ArtifactInformation.Unmarshal(m, b)
-}
-func (m *ArtifactInformation) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_ArtifactInformation.Marshal(b, m, deterministic)
-}
-func (m *ArtifactInformation) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ArtifactInformation.Merge(m, src)
-}
-func (m *ArtifactInformation) XXX_Size() int {
-	return xxx_messageInfo_ArtifactInformation.Size(m)
-}
-func (m *ArtifactInformation) XXX_DiscardUnknown() {
-	xxx_messageInfo_ArtifactInformation.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_ArtifactInformation proto.InternalMessageInfo
-
-func (m *ArtifactInformation) GetTypeUrn() string {
-	if m != nil {
-		return m.TypeUrn
+func (x *ArtifactInformation) GetTypeUrn() string {
+	if x != nil {
+		return x.TypeUrn
 	}
 	return ""
 }
 
-func (m *ArtifactInformation) GetTypePayload() []byte {
-	if m != nil {
-		return m.TypePayload
+func (x *ArtifactInformation) GetTypePayload() []byte {
+	if x != nil {
+		return x.TypePayload
 	}
 	return nil
 }
 
-func (m *ArtifactInformation) GetRoleUrn() string {
-	if m != nil {
-		return m.RoleUrn
+func (x *ArtifactInformation) GetRoleUrn() string {
+	if x != nil {
+		return x.RoleUrn
 	}
 	return ""
 }
 
-func (m *ArtifactInformation) GetRolePayload() []byte {
-	if m != nil {
-		return m.RolePayload
+func (x *ArtifactInformation) GetRolePayload() []byte {
+	if x != nil {
+		return x.RolePayload
 	}
 	return nil
 }
@@ -4557,6 +4686,10 @@
 // An environment for executing UDFs. By default, an SDK container URL, but
 // can also be a process forked by a command, or an externally managed process.
 type Environment struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
 	// (Required) The URN of the payload
 	Urn string `protobuf:"bytes,2,opt,name=urn,proto3" json:"urn,omitempty"`
 	// (Optional) The data specifying any parameters to the URN. If
@@ -4572,249 +4705,284 @@
 	// support of optional protocols not tied to a concrete component.
 	Capabilities []string `protobuf:"bytes,5,rep,name=capabilities,proto3" json:"capabilities,omitempty"`
 	// (Optional) artifact dependency information used for executing UDFs in this environment.
-	Dependencies         []*ArtifactInformation `protobuf:"bytes,6,rep,name=dependencies,proto3" json:"dependencies,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}               `json:"-"`
-	XXX_unrecognized     []byte                 `json:"-"`
-	XXX_sizecache        int32                  `json:"-"`
+	Dependencies []*ArtifactInformation `protobuf:"bytes,6,rep,name=dependencies,proto3" json:"dependencies,omitempty"`
 }
 
-func (m *Environment) Reset()         { *m = Environment{} }
-func (m *Environment) String() string { return proto.CompactTextString(m) }
-func (*Environment) ProtoMessage()    {}
+func (x *Environment) Reset() {
+	*x = Environment{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_runner_api_proto_msgTypes[46]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *Environment) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*Environment) ProtoMessage() {}
+
+func (x *Environment) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_runner_api_proto_msgTypes[46]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use Environment.ProtoReflect.Descriptor instead.
 func (*Environment) Descriptor() ([]byte, []int) {
-	return fileDescriptor_cf57597c3a9659a9, []int{43}
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{46}
 }
 
-func (m *Environment) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_Environment.Unmarshal(m, b)
-}
-func (m *Environment) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_Environment.Marshal(b, m, deterministic)
-}
-func (m *Environment) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Environment.Merge(m, src)
-}
-func (m *Environment) XXX_Size() int {
-	return xxx_messageInfo_Environment.Size(m)
-}
-func (m *Environment) XXX_DiscardUnknown() {
-	xxx_messageInfo_Environment.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_Environment proto.InternalMessageInfo
-
-func (m *Environment) GetUrn() string {
-	if m != nil {
-		return m.Urn
+func (x *Environment) GetUrn() string {
+	if x != nil {
+		return x.Urn
 	}
 	return ""
 }
 
-func (m *Environment) GetPayload() []byte {
-	if m != nil {
-		return m.Payload
+func (x *Environment) GetPayload() []byte {
+	if x != nil {
+		return x.Payload
 	}
 	return nil
 }
 
-func (m *Environment) GetDisplayData() []*DisplayData {
-	if m != nil {
-		return m.DisplayData
+func (x *Environment) GetDisplayData() []*DisplayData {
+	if x != nil {
+		return x.DisplayData
 	}
 	return nil
 }
 
-func (m *Environment) GetCapabilities() []string {
-	if m != nil {
-		return m.Capabilities
+func (x *Environment) GetCapabilities() []string {
+	if x != nil {
+		return x.Capabilities
 	}
 	return nil
 }
 
-func (m *Environment) GetDependencies() []*ArtifactInformation {
-	if m != nil {
-		return m.Dependencies
+func (x *Environment) GetDependencies() []*ArtifactInformation {
+	if x != nil {
+		return x.Dependencies
 	}
 	return nil
 }
 
 type StandardEnvironments struct {
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
 }
 
-func (m *StandardEnvironments) Reset()         { *m = StandardEnvironments{} }
-func (m *StandardEnvironments) String() string { return proto.CompactTextString(m) }
-func (*StandardEnvironments) ProtoMessage()    {}
+func (x *StandardEnvironments) Reset() {
+	*x = StandardEnvironments{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_runner_api_proto_msgTypes[47]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *StandardEnvironments) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*StandardEnvironments) ProtoMessage() {}
+
+func (x *StandardEnvironments) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_runner_api_proto_msgTypes[47]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use StandardEnvironments.ProtoReflect.Descriptor instead.
 func (*StandardEnvironments) Descriptor() ([]byte, []int) {
-	return fileDescriptor_cf57597c3a9659a9, []int{44}
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{47}
 }
 
-func (m *StandardEnvironments) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_StandardEnvironments.Unmarshal(m, b)
-}
-func (m *StandardEnvironments) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_StandardEnvironments.Marshal(b, m, deterministic)
-}
-func (m *StandardEnvironments) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_StandardEnvironments.Merge(m, src)
-}
-func (m *StandardEnvironments) XXX_Size() int {
-	return xxx_messageInfo_StandardEnvironments.Size(m)
-}
-func (m *StandardEnvironments) XXX_DiscardUnknown() {
-	xxx_messageInfo_StandardEnvironments.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_StandardEnvironments proto.InternalMessageInfo
-
 // The payload of a Docker image
 type DockerPayload struct {
-	ContainerImage       string   `protobuf:"bytes,1,opt,name=container_image,json=containerImage,proto3" json:"container_image,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	ContainerImage string `protobuf:"bytes,1,opt,name=container_image,json=containerImage,proto3" json:"container_image,omitempty"` // implicitly linux_amd64.
 }
 
-func (m *DockerPayload) Reset()         { *m = DockerPayload{} }
-func (m *DockerPayload) String() string { return proto.CompactTextString(m) }
-func (*DockerPayload) ProtoMessage()    {}
+func (x *DockerPayload) Reset() {
+	*x = DockerPayload{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_runner_api_proto_msgTypes[48]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *DockerPayload) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*DockerPayload) ProtoMessage() {}
+
+func (x *DockerPayload) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_runner_api_proto_msgTypes[48]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use DockerPayload.ProtoReflect.Descriptor instead.
 func (*DockerPayload) Descriptor() ([]byte, []int) {
-	return fileDescriptor_cf57597c3a9659a9, []int{45}
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{48}
 }
 
-func (m *DockerPayload) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_DockerPayload.Unmarshal(m, b)
-}
-func (m *DockerPayload) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_DockerPayload.Marshal(b, m, deterministic)
-}
-func (m *DockerPayload) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_DockerPayload.Merge(m, src)
-}
-func (m *DockerPayload) XXX_Size() int {
-	return xxx_messageInfo_DockerPayload.Size(m)
-}
-func (m *DockerPayload) XXX_DiscardUnknown() {
-	xxx_messageInfo_DockerPayload.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_DockerPayload proto.InternalMessageInfo
-
-func (m *DockerPayload) GetContainerImage() string {
-	if m != nil {
-		return m.ContainerImage
+func (x *DockerPayload) GetContainerImage() string {
+	if x != nil {
+		return x.ContainerImage
 	}
 	return ""
 }
 
 type ProcessPayload struct {
-	Os                   string            `protobuf:"bytes,1,opt,name=os,proto3" json:"os,omitempty"`
-	Arch                 string            `protobuf:"bytes,2,opt,name=arch,proto3" json:"arch,omitempty"`
-	Command              string            `protobuf:"bytes,3,opt,name=command,proto3" json:"command,omitempty"`
-	Env                  map[string]string `protobuf:"bytes,4,rep,name=env,proto3" json:"env,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"`
-	XXX_NoUnkeyedLiteral struct{}          `json:"-"`
-	XXX_unrecognized     []byte            `json:"-"`
-	XXX_sizecache        int32             `json:"-"`
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	Os      string            `protobuf:"bytes,1,opt,name=os,proto3" json:"os,omitempty"`                                                                                           // "linux", "darwin", ..
+	Arch    string            `protobuf:"bytes,2,opt,name=arch,proto3" json:"arch,omitempty"`                                                                                       // "amd64", ..
+	Command string            `protobuf:"bytes,3,opt,name=command,proto3" json:"command,omitempty"`                                                                                 // process to execute
+	Env     map[string]string `protobuf:"bytes,4,rep,name=env,proto3" json:"env,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` // Environment variables
 }
 
-func (m *ProcessPayload) Reset()         { *m = ProcessPayload{} }
-func (m *ProcessPayload) String() string { return proto.CompactTextString(m) }
-func (*ProcessPayload) ProtoMessage()    {}
+func (x *ProcessPayload) Reset() {
+	*x = ProcessPayload{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_runner_api_proto_msgTypes[49]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *ProcessPayload) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*ProcessPayload) ProtoMessage() {}
+
+func (x *ProcessPayload) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_runner_api_proto_msgTypes[49]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use ProcessPayload.ProtoReflect.Descriptor instead.
 func (*ProcessPayload) Descriptor() ([]byte, []int) {
-	return fileDescriptor_cf57597c3a9659a9, []int{46}
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{49}
 }
 
-func (m *ProcessPayload) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_ProcessPayload.Unmarshal(m, b)
-}
-func (m *ProcessPayload) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_ProcessPayload.Marshal(b, m, deterministic)
-}
-func (m *ProcessPayload) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ProcessPayload.Merge(m, src)
-}
-func (m *ProcessPayload) XXX_Size() int {
-	return xxx_messageInfo_ProcessPayload.Size(m)
-}
-func (m *ProcessPayload) XXX_DiscardUnknown() {
-	xxx_messageInfo_ProcessPayload.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_ProcessPayload proto.InternalMessageInfo
-
-func (m *ProcessPayload) GetOs() string {
-	if m != nil {
-		return m.Os
+func (x *ProcessPayload) GetOs() string {
+	if x != nil {
+		return x.Os
 	}
 	return ""
 }
 
-func (m *ProcessPayload) GetArch() string {
-	if m != nil {
-		return m.Arch
+func (x *ProcessPayload) GetArch() string {
+	if x != nil {
+		return x.Arch
 	}
 	return ""
 }
 
-func (m *ProcessPayload) GetCommand() string {
-	if m != nil {
-		return m.Command
+func (x *ProcessPayload) GetCommand() string {
+	if x != nil {
+		return x.Command
 	}
 	return ""
 }
 
-func (m *ProcessPayload) GetEnv() map[string]string {
-	if m != nil {
-		return m.Env
+func (x *ProcessPayload) GetEnv() map[string]string {
+	if x != nil {
+		return x.Env
 	}
 	return nil
 }
 
 type ExternalPayload struct {
-	Endpoint             *ApiServiceDescriptor `protobuf:"bytes,1,opt,name=endpoint,proto3" json:"endpoint,omitempty"`
-	Params               map[string]string     `protobuf:"bytes,2,rep,name=params,proto3" json:"params,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"`
-	XXX_NoUnkeyedLiteral struct{}              `json:"-"`
-	XXX_unrecognized     []byte                `json:"-"`
-	XXX_sizecache        int32                 `json:"-"`
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	Endpoint *ApiServiceDescriptor `protobuf:"bytes,1,opt,name=endpoint,proto3" json:"endpoint,omitempty"`
+	Params   map[string]string     `protobuf:"bytes,2,rep,name=params,proto3" json:"params,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"` // Arbitrary extra parameters to pass
 }
 
-func (m *ExternalPayload) Reset()         { *m = ExternalPayload{} }
-func (m *ExternalPayload) String() string { return proto.CompactTextString(m) }
-func (*ExternalPayload) ProtoMessage()    {}
+func (x *ExternalPayload) Reset() {
+	*x = ExternalPayload{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_runner_api_proto_msgTypes[50]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *ExternalPayload) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*ExternalPayload) ProtoMessage() {}
+
+func (x *ExternalPayload) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_runner_api_proto_msgTypes[50]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use ExternalPayload.ProtoReflect.Descriptor instead.
 func (*ExternalPayload) Descriptor() ([]byte, []int) {
-	return fileDescriptor_cf57597c3a9659a9, []int{47}
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{50}
 }
 
-func (m *ExternalPayload) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_ExternalPayload.Unmarshal(m, b)
-}
-func (m *ExternalPayload) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_ExternalPayload.Marshal(b, m, deterministic)
-}
-func (m *ExternalPayload) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ExternalPayload.Merge(m, src)
-}
-func (m *ExternalPayload) XXX_Size() int {
-	return xxx_messageInfo_ExternalPayload.Size(m)
-}
-func (m *ExternalPayload) XXX_DiscardUnknown() {
-	xxx_messageInfo_ExternalPayload.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_ExternalPayload proto.InternalMessageInfo
-
-func (m *ExternalPayload) GetEndpoint() *ApiServiceDescriptor {
-	if m != nil {
-		return m.Endpoint
+func (x *ExternalPayload) GetEndpoint() *ApiServiceDescriptor {
+	if x != nil {
+		return x.Endpoint
 	}
 	return nil
 }
 
-func (m *ExternalPayload) GetParams() map[string]string {
-	if m != nil {
-		return m.Params
+func (x *ExternalPayload) GetParams() map[string]string {
+	if x != nil {
+		return x.Params
 	}
 	return nil
 }
@@ -4823,72 +4991,86 @@
 // simply be expressed as a component (such as a Coder or PTransform) that this
 // environment understands.
 type StandardProtocols struct {
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
 }
 
-func (m *StandardProtocols) Reset()         { *m = StandardProtocols{} }
-func (m *StandardProtocols) String() string { return proto.CompactTextString(m) }
-func (*StandardProtocols) ProtoMessage()    {}
+func (x *StandardProtocols) Reset() {
+	*x = StandardProtocols{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_runner_api_proto_msgTypes[51]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *StandardProtocols) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*StandardProtocols) ProtoMessage() {}
+
+func (x *StandardProtocols) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_runner_api_proto_msgTypes[51]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use StandardProtocols.ProtoReflect.Descriptor instead.
 func (*StandardProtocols) Descriptor() ([]byte, []int) {
-	return fileDescriptor_cf57597c3a9659a9, []int{48}
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{51}
 }
 
-func (m *StandardProtocols) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_StandardProtocols.Unmarshal(m, b)
-}
-func (m *StandardProtocols) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_StandardProtocols.Marshal(b, m, deterministic)
-}
-func (m *StandardProtocols) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_StandardProtocols.Merge(m, src)
-}
-func (m *StandardProtocols) XXX_Size() int {
-	return xxx_messageInfo_StandardProtocols.Size(m)
-}
-func (m *StandardProtocols) XXX_DiscardUnknown() {
-	xxx_messageInfo_StandardProtocols.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_StandardProtocols proto.InternalMessageInfo
-
 // These URNs are used to indicate requirements of a pipeline that cannot
 // simply be expressed as a component (such as a Coder or PTransform) that the
 // runner must understand. In many cases, this indicates a particular field
 // of a transform must be inspected and respected (which allows new fields
 // to be added in a forwards-compatible way).
 type StandardRequirements struct {
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
 }
 
-func (m *StandardRequirements) Reset()         { *m = StandardRequirements{} }
-func (m *StandardRequirements) String() string { return proto.CompactTextString(m) }
-func (*StandardRequirements) ProtoMessage()    {}
+func (x *StandardRequirements) Reset() {
+	*x = StandardRequirements{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_runner_api_proto_msgTypes[52]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *StandardRequirements) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*StandardRequirements) ProtoMessage() {}
+
+func (x *StandardRequirements) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_runner_api_proto_msgTypes[52]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use StandardRequirements.ProtoReflect.Descriptor instead.
 func (*StandardRequirements) Descriptor() ([]byte, []int) {
-	return fileDescriptor_cf57597c3a9659a9, []int{49}
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{52}
 }
 
-func (m *StandardRequirements) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_StandardRequirements.Unmarshal(m, b)
-}
-func (m *StandardRequirements) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_StandardRequirements.Marshal(b, m, deterministic)
-}
-func (m *StandardRequirements) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_StandardRequirements.Merge(m, src)
-}
-func (m *StandardRequirements) XXX_Size() int {
-	return xxx_messageInfo_StandardRequirements.Size(m)
-}
-func (m *StandardRequirements) XXX_DiscardUnknown() {
-	xxx_messageInfo_StandardRequirements.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_StandardRequirements proto.InternalMessageInfo
-
 // A URN along with a parameter object whose schema is determined by the
 // URN.
 //
@@ -4917,6 +5099,10 @@
 //    then be an opaque message such as bytes in a
 //    language-specific serialization format.
 type FunctionSpec struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
 	// (Required) A URN that describes the accompanying payload.
 	// For any URN that is not recognized (by whomever is inspecting
 	// it) the parameter payload should be treated as opaque and
@@ -4924,47 +5110,51 @@
 	Urn string `protobuf:"bytes,1,opt,name=urn,proto3" json:"urn,omitempty"`
 	// (Optional) The data specifying any parameters to the URN. If
 	// the URN does not require any arguments, this may be omitted.
-	Payload              []byte   `protobuf:"bytes,3,opt,name=payload,proto3" json:"payload,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
+	Payload []byte `protobuf:"bytes,3,opt,name=payload,proto3" json:"payload,omitempty"`
 }
 
-func (m *FunctionSpec) Reset()         { *m = FunctionSpec{} }
-func (m *FunctionSpec) String() string { return proto.CompactTextString(m) }
-func (*FunctionSpec) ProtoMessage()    {}
+func (x *FunctionSpec) Reset() {
+	*x = FunctionSpec{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_runner_api_proto_msgTypes[53]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *FunctionSpec) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*FunctionSpec) ProtoMessage() {}
+
+func (x *FunctionSpec) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_runner_api_proto_msgTypes[53]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use FunctionSpec.ProtoReflect.Descriptor instead.
 func (*FunctionSpec) Descriptor() ([]byte, []int) {
-	return fileDescriptor_cf57597c3a9659a9, []int{50}
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{53}
 }
 
-func (m *FunctionSpec) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_FunctionSpec.Unmarshal(m, b)
-}
-func (m *FunctionSpec) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_FunctionSpec.Marshal(b, m, deterministic)
-}
-func (m *FunctionSpec) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_FunctionSpec.Merge(m, src)
-}
-func (m *FunctionSpec) XXX_Size() int {
-	return xxx_messageInfo_FunctionSpec.Size(m)
-}
-func (m *FunctionSpec) XXX_DiscardUnknown() {
-	xxx_messageInfo_FunctionSpec.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_FunctionSpec proto.InternalMessageInfo
-
-func (m *FunctionSpec) GetUrn() string {
-	if m != nil {
-		return m.Urn
+func (x *FunctionSpec) GetUrn() string {
+	if x != nil {
+		return x.Urn
 	}
 	return ""
 }
 
-func (m *FunctionSpec) GetPayload() []byte {
-	if m != nil {
-		return m.Payload
+func (x *FunctionSpec) GetPayload() []byte {
+	if x != nil {
+		return x.Payload
 	}
 	return nil
 }
@@ -4976,82 +5166,97 @@
 // information about the pipeline in addition to an encoding while others only
 // specify the encoding itself.
 type StandardDisplayData struct {
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
 }
 
-func (m *StandardDisplayData) Reset()         { *m = StandardDisplayData{} }
-func (m *StandardDisplayData) String() string { return proto.CompactTextString(m) }
-func (*StandardDisplayData) ProtoMessage()    {}
+func (x *StandardDisplayData) Reset() {
+	*x = StandardDisplayData{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_runner_api_proto_msgTypes[54]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *StandardDisplayData) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*StandardDisplayData) ProtoMessage() {}
+
+func (x *StandardDisplayData) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_runner_api_proto_msgTypes[54]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use StandardDisplayData.ProtoReflect.Descriptor instead.
 func (*StandardDisplayData) Descriptor() ([]byte, []int) {
-	return fileDescriptor_cf57597c3a9659a9, []int{51}
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{54}
 }
 
-func (m *StandardDisplayData) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_StandardDisplayData.Unmarshal(m, b)
-}
-func (m *StandardDisplayData) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_StandardDisplayData.Marshal(b, m, deterministic)
-}
-func (m *StandardDisplayData) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_StandardDisplayData.Merge(m, src)
-}
-func (m *StandardDisplayData) XXX_Size() int {
-	return xxx_messageInfo_StandardDisplayData.Size(m)
-}
-func (m *StandardDisplayData) XXX_DiscardUnknown() {
-	xxx_messageInfo_StandardDisplayData.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_StandardDisplayData proto.InternalMessageInfo
-
 type LabelledStringPayload struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
 	// (Required) A human readable label for the value.
 	Label string `protobuf:"bytes,1,opt,name=label,proto3" json:"label,omitempty"`
 	// (Required) A value which will be displayed to the user. The urn describes
 	// how the value can be interpreted and/or categorized.
-	Value                string   `protobuf:"bytes,2,opt,name=value,proto3" json:"value,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
+	Value string `protobuf:"bytes,2,opt,name=value,proto3" json:"value,omitempty"`
 }
 
-func (m *LabelledStringPayload) Reset()         { *m = LabelledStringPayload{} }
-func (m *LabelledStringPayload) String() string { return proto.CompactTextString(m) }
-func (*LabelledStringPayload) ProtoMessage()    {}
+func (x *LabelledStringPayload) Reset() {
+	*x = LabelledStringPayload{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_runner_api_proto_msgTypes[55]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *LabelledStringPayload) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*LabelledStringPayload) ProtoMessage() {}
+
+func (x *LabelledStringPayload) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_runner_api_proto_msgTypes[55]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use LabelledStringPayload.ProtoReflect.Descriptor instead.
 func (*LabelledStringPayload) Descriptor() ([]byte, []int) {
-	return fileDescriptor_cf57597c3a9659a9, []int{52}
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{55}
 }
 
-func (m *LabelledStringPayload) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_LabelledStringPayload.Unmarshal(m, b)
-}
-func (m *LabelledStringPayload) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_LabelledStringPayload.Marshal(b, m, deterministic)
-}
-func (m *LabelledStringPayload) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_LabelledStringPayload.Merge(m, src)
-}
-func (m *LabelledStringPayload) XXX_Size() int {
-	return xxx_messageInfo_LabelledStringPayload.Size(m)
-}
-func (m *LabelledStringPayload) XXX_DiscardUnknown() {
-	xxx_messageInfo_LabelledStringPayload.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_LabelledStringPayload proto.InternalMessageInfo
-
-func (m *LabelledStringPayload) GetLabel() string {
-	if m != nil {
-		return m.Label
+func (x *LabelledStringPayload) GetLabel() string {
+	if x != nil {
+		return x.Label
 	}
 	return ""
 }
 
-func (m *LabelledStringPayload) GetValue() string {
-	if m != nil {
-		return m.Value
+func (x *LabelledStringPayload) GetValue() string {
+	if x != nil {
+		return x.Value
 	}
 	return ""
 }
@@ -5060,53 +5265,61 @@
 // useful for pipeline runners IOs and diagnostic dashboards to display details
 // about annotated components.
 type DisplayData struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
 	// A key used to describe the type of display data. See StandardDisplayData
 	// for the set of well known urns describing how the payload is meant to be
 	// interpreted.
 	Urn string `protobuf:"bytes,1,opt,name=urn,proto3" json:"urn,omitempty"`
 	// (Optional) The data specifying any parameters to the URN. If
 	// the URN does not require any arguments, this may be omitted.
-	Payload              []byte   `protobuf:"bytes,2,opt,name=payload,proto3" json:"payload,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
+	Payload []byte `protobuf:"bytes,2,opt,name=payload,proto3" json:"payload,omitempty"`
 }
 
-func (m *DisplayData) Reset()         { *m = DisplayData{} }
-func (m *DisplayData) String() string { return proto.CompactTextString(m) }
-func (*DisplayData) ProtoMessage()    {}
+func (x *DisplayData) Reset() {
+	*x = DisplayData{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_runner_api_proto_msgTypes[56]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *DisplayData) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*DisplayData) ProtoMessage() {}
+
+func (x *DisplayData) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_runner_api_proto_msgTypes[56]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use DisplayData.ProtoReflect.Descriptor instead.
 func (*DisplayData) Descriptor() ([]byte, []int) {
-	return fileDescriptor_cf57597c3a9659a9, []int{53}
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{56}
 }
 
-func (m *DisplayData) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_DisplayData.Unmarshal(m, b)
-}
-func (m *DisplayData) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_DisplayData.Marshal(b, m, deterministic)
-}
-func (m *DisplayData) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_DisplayData.Merge(m, src)
-}
-func (m *DisplayData) XXX_Size() int {
-	return xxx_messageInfo_DisplayData.Size(m)
-}
-func (m *DisplayData) XXX_DiscardUnknown() {
-	xxx_messageInfo_DisplayData.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_DisplayData proto.InternalMessageInfo
-
-func (m *DisplayData) GetUrn() string {
-	if m != nil {
-		return m.Urn
+func (x *DisplayData) GetUrn() string {
+	if x != nil {
+		return x.Urn
 	}
 	return ""
 }
 
-func (m *DisplayData) GetPayload() []byte {
-	if m != nil {
-		return m.Payload
+func (x *DisplayData) GetPayload() []byte {
+	if x != nil {
+		return x.Payload
 	}
 	return nil
 }
@@ -5114,6 +5327,10 @@
 // A disjoint union of all the things that may contain references
 // that require Components to resolve.
 type MessageWithComponents struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
 	// (Optional) The by-reference components of the root message,
 	// enabling a standalone message.
 	//
@@ -5123,7 +5340,7 @@
 	// (Required) The root message that may contain pointers
 	// that should be resolved by looking inside components.
 	//
-	// Types that are valid to be assigned to Root:
+	// Types that are assignable to Root:
 	//	*MessageWithComponents_Coder
 	//	*MessageWithComponents_CombinePayload
 	//	*MessageWithComponents_FunctionSpec
@@ -5134,40 +5351,121 @@
 	//	*MessageWithComponents_SideInput
 	//	*MessageWithComponents_WindowIntoPayload
 	//	*MessageWithComponents_WindowingStrategy
-	Root                 isMessageWithComponents_Root `protobuf_oneof:"root"`
-	XXX_NoUnkeyedLiteral struct{}                     `json:"-"`
-	XXX_unrecognized     []byte                       `json:"-"`
-	XXX_sizecache        int32                        `json:"-"`
+	Root isMessageWithComponents_Root `protobuf_oneof:"root"`
 }
 
-func (m *MessageWithComponents) Reset()         { *m = MessageWithComponents{} }
-func (m *MessageWithComponents) String() string { return proto.CompactTextString(m) }
-func (*MessageWithComponents) ProtoMessage()    {}
+func (x *MessageWithComponents) Reset() {
+	*x = MessageWithComponents{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_runner_api_proto_msgTypes[57]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *MessageWithComponents) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*MessageWithComponents) ProtoMessage() {}
+
+func (x *MessageWithComponents) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_runner_api_proto_msgTypes[57]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use MessageWithComponents.ProtoReflect.Descriptor instead.
 func (*MessageWithComponents) Descriptor() ([]byte, []int) {
-	return fileDescriptor_cf57597c3a9659a9, []int{54}
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{57}
 }
 
-func (m *MessageWithComponents) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_MessageWithComponents.Unmarshal(m, b)
-}
-func (m *MessageWithComponents) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_MessageWithComponents.Marshal(b, m, deterministic)
-}
-func (m *MessageWithComponents) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_MessageWithComponents.Merge(m, src)
-}
-func (m *MessageWithComponents) XXX_Size() int {
-	return xxx_messageInfo_MessageWithComponents.Size(m)
-}
-func (m *MessageWithComponents) XXX_DiscardUnknown() {
-	xxx_messageInfo_MessageWithComponents.DiscardUnknown(m)
+func (x *MessageWithComponents) GetComponents() *Components {
+	if x != nil {
+		return x.Components
+	}
+	return nil
 }
 
-var xxx_messageInfo_MessageWithComponents proto.InternalMessageInfo
-
-func (m *MessageWithComponents) GetComponents() *Components {
+func (m *MessageWithComponents) GetRoot() isMessageWithComponents_Root {
 	if m != nil {
-		return m.Components
+		return m.Root
+	}
+	return nil
+}
+
+func (x *MessageWithComponents) GetCoder() *Coder {
+	if x, ok := x.GetRoot().(*MessageWithComponents_Coder); ok {
+		return x.Coder
+	}
+	return nil
+}
+
+func (x *MessageWithComponents) GetCombinePayload() *CombinePayload {
+	if x, ok := x.GetRoot().(*MessageWithComponents_CombinePayload); ok {
+		return x.CombinePayload
+	}
+	return nil
+}
+
+func (x *MessageWithComponents) GetFunctionSpec() *FunctionSpec {
+	if x, ok := x.GetRoot().(*MessageWithComponents_FunctionSpec); ok {
+		return x.FunctionSpec
+	}
+	return nil
+}
+
+func (x *MessageWithComponents) GetParDoPayload() *ParDoPayload {
+	if x, ok := x.GetRoot().(*MessageWithComponents_ParDoPayload); ok {
+		return x.ParDoPayload
+	}
+	return nil
+}
+
+func (x *MessageWithComponents) GetPtransform() *PTransform {
+	if x, ok := x.GetRoot().(*MessageWithComponents_Ptransform); ok {
+		return x.Ptransform
+	}
+	return nil
+}
+
+func (x *MessageWithComponents) GetPcollection() *PCollection {
+	if x, ok := x.GetRoot().(*MessageWithComponents_Pcollection); ok {
+		return x.Pcollection
+	}
+	return nil
+}
+
+func (x *MessageWithComponents) GetReadPayload() *ReadPayload {
+	if x, ok := x.GetRoot().(*MessageWithComponents_ReadPayload); ok {
+		return x.ReadPayload
+	}
+	return nil
+}
+
+func (x *MessageWithComponents) GetSideInput() *SideInput {
+	if x, ok := x.GetRoot().(*MessageWithComponents_SideInput); ok {
+		return x.SideInput
+	}
+	return nil
+}
+
+func (x *MessageWithComponents) GetWindowIntoPayload() *WindowIntoPayload {
+	if x, ok := x.GetRoot().(*MessageWithComponents_WindowIntoPayload); ok {
+		return x.WindowIntoPayload
+	}
+	return nil
+}
+
+func (x *MessageWithComponents) GetWindowingStrategy() *WindowingStrategy {
+	if x, ok := x.GetRoot().(*MessageWithComponents_WindowingStrategy); ok {
+		return x.WindowingStrategy
 	}
 	return nil
 }
@@ -5236,102 +5534,13 @@
 
 func (*MessageWithComponents_WindowingStrategy) isMessageWithComponents_Root() {}
 
-func (m *MessageWithComponents) GetRoot() isMessageWithComponents_Root {
-	if m != nil {
-		return m.Root
-	}
-	return nil
-}
-
-func (m *MessageWithComponents) GetCoder() *Coder {
-	if x, ok := m.GetRoot().(*MessageWithComponents_Coder); ok {
-		return x.Coder
-	}
-	return nil
-}
-
-func (m *MessageWithComponents) GetCombinePayload() *CombinePayload {
-	if x, ok := m.GetRoot().(*MessageWithComponents_CombinePayload); ok {
-		return x.CombinePayload
-	}
-	return nil
-}
-
-func (m *MessageWithComponents) GetFunctionSpec() *FunctionSpec {
-	if x, ok := m.GetRoot().(*MessageWithComponents_FunctionSpec); ok {
-		return x.FunctionSpec
-	}
-	return nil
-}
-
-func (m *MessageWithComponents) GetParDoPayload() *ParDoPayload {
-	if x, ok := m.GetRoot().(*MessageWithComponents_ParDoPayload); ok {
-		return x.ParDoPayload
-	}
-	return nil
-}
-
-func (m *MessageWithComponents) GetPtransform() *PTransform {
-	if x, ok := m.GetRoot().(*MessageWithComponents_Ptransform); ok {
-		return x.Ptransform
-	}
-	return nil
-}
-
-func (m *MessageWithComponents) GetPcollection() *PCollection {
-	if x, ok := m.GetRoot().(*MessageWithComponents_Pcollection); ok {
-		return x.Pcollection
-	}
-	return nil
-}
-
-func (m *MessageWithComponents) GetReadPayload() *ReadPayload {
-	if x, ok := m.GetRoot().(*MessageWithComponents_ReadPayload); ok {
-		return x.ReadPayload
-	}
-	return nil
-}
-
-func (m *MessageWithComponents) GetSideInput() *SideInput {
-	if x, ok := m.GetRoot().(*MessageWithComponents_SideInput); ok {
-		return x.SideInput
-	}
-	return nil
-}
-
-func (m *MessageWithComponents) GetWindowIntoPayload() *WindowIntoPayload {
-	if x, ok := m.GetRoot().(*MessageWithComponents_WindowIntoPayload); ok {
-		return x.WindowIntoPayload
-	}
-	return nil
-}
-
-func (m *MessageWithComponents) GetWindowingStrategy() *WindowingStrategy {
-	if x, ok := m.GetRoot().(*MessageWithComponents_WindowingStrategy); ok {
-		return x.WindowingStrategy
-	}
-	return nil
-}
-
-// XXX_OneofWrappers is for the internal use of the proto package.
-func (*MessageWithComponents) XXX_OneofWrappers() []interface{} {
-	return []interface{}{
-		(*MessageWithComponents_Coder)(nil),
-		(*MessageWithComponents_CombinePayload)(nil),
-		(*MessageWithComponents_FunctionSpec)(nil),
-		(*MessageWithComponents_ParDoPayload)(nil),
-		(*MessageWithComponents_Ptransform)(nil),
-		(*MessageWithComponents_Pcollection)(nil),
-		(*MessageWithComponents_ReadPayload)(nil),
-		(*MessageWithComponents_SideInput)(nil),
-		(*MessageWithComponents_WindowIntoPayload)(nil),
-		(*MessageWithComponents_WindowingStrategy)(nil),
-	}
-}
-
 // The payload for an executable stage. This will eventually be passed to an SDK in the form of a
 // ProcessBundleDescriptor.
 type ExecutableStagePayload struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
 	// (Required) Environment in which this stage executes.
 	//
 	// We use an environment rather than environment id
@@ -5362,154 +5571,1169 @@
 	Timers []*ExecutableStagePayload_TimerId `protobuf:"bytes,8,rep,name=timers,proto3" json:"timers,omitempty"`
 	// The timerfamilies required for this executable stage. Each timer familyof each PTransform within
 	// this ExecutableStagePayload must be represented within this field.
-	TimerFamilies        []*ExecutableStagePayload_TimerFamilyId `protobuf:"bytes,10,rep,name=timerFamilies,proto3" json:"timerFamilies,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}                                `json:"-"`
-	XXX_unrecognized     []byte                                  `json:"-"`
-	XXX_sizecache        int32                                   `json:"-"`
+	TimerFamilies []*ExecutableStagePayload_TimerFamilyId `protobuf:"bytes,10,rep,name=timerFamilies,proto3" json:"timerFamilies,omitempty"`
 }
 
-func (m *ExecutableStagePayload) Reset()         { *m = ExecutableStagePayload{} }
-func (m *ExecutableStagePayload) String() string { return proto.CompactTextString(m) }
-func (*ExecutableStagePayload) ProtoMessage()    {}
+func (x *ExecutableStagePayload) Reset() {
+	*x = ExecutableStagePayload{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_runner_api_proto_msgTypes[58]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *ExecutableStagePayload) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*ExecutableStagePayload) ProtoMessage() {}
+
+func (x *ExecutableStagePayload) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_runner_api_proto_msgTypes[58]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use ExecutableStagePayload.ProtoReflect.Descriptor instead.
 func (*ExecutableStagePayload) Descriptor() ([]byte, []int) {
-	return fileDescriptor_cf57597c3a9659a9, []int{55}
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{58}
 }
 
-func (m *ExecutableStagePayload) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_ExecutableStagePayload.Unmarshal(m, b)
-}
-func (m *ExecutableStagePayload) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_ExecutableStagePayload.Marshal(b, m, deterministic)
-}
-func (m *ExecutableStagePayload) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ExecutableStagePayload.Merge(m, src)
-}
-func (m *ExecutableStagePayload) XXX_Size() int {
-	return xxx_messageInfo_ExecutableStagePayload.Size(m)
-}
-func (m *ExecutableStagePayload) XXX_DiscardUnknown() {
-	xxx_messageInfo_ExecutableStagePayload.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_ExecutableStagePayload proto.InternalMessageInfo
-
-func (m *ExecutableStagePayload) GetEnvironment() *Environment {
-	if m != nil {
-		return m.Environment
+func (x *ExecutableStagePayload) GetEnvironment() *Environment {
+	if x != nil {
+		return x.Environment
 	}
 	return nil
 }
 
-func (m *ExecutableStagePayload) GetWireCoderSettings() []*ExecutableStagePayload_WireCoderSetting {
-	if m != nil {
-		return m.WireCoderSettings
+func (x *ExecutableStagePayload) GetWireCoderSettings() []*ExecutableStagePayload_WireCoderSetting {
+	if x != nil {
+		return x.WireCoderSettings
 	}
 	return nil
 }
 
-func (m *ExecutableStagePayload) GetInput() string {
-	if m != nil {
-		return m.Input
+func (x *ExecutableStagePayload) GetInput() string {
+	if x != nil {
+		return x.Input
 	}
 	return ""
 }
 
-func (m *ExecutableStagePayload) GetSideInputs() []*ExecutableStagePayload_SideInputId {
-	if m != nil {
-		return m.SideInputs
+func (x *ExecutableStagePayload) GetSideInputs() []*ExecutableStagePayload_SideInputId {
+	if x != nil {
+		return x.SideInputs
 	}
 	return nil
 }
 
-func (m *ExecutableStagePayload) GetTransforms() []string {
-	if m != nil {
-		return m.Transforms
+func (x *ExecutableStagePayload) GetTransforms() []string {
+	if x != nil {
+		return x.Transforms
 	}
 	return nil
 }
 
-func (m *ExecutableStagePayload) GetOutputs() []string {
-	if m != nil {
-		return m.Outputs
+func (x *ExecutableStagePayload) GetOutputs() []string {
+	if x != nil {
+		return x.Outputs
 	}
 	return nil
 }
 
-func (m *ExecutableStagePayload) GetComponents() *Components {
-	if m != nil {
-		return m.Components
+func (x *ExecutableStagePayload) GetComponents() *Components {
+	if x != nil {
+		return x.Components
 	}
 	return nil
 }
 
-func (m *ExecutableStagePayload) GetUserStates() []*ExecutableStagePayload_UserStateId {
-	if m != nil {
-		return m.UserStates
+func (x *ExecutableStagePayload) GetUserStates() []*ExecutableStagePayload_UserStateId {
+	if x != nil {
+		return x.UserStates
 	}
 	return nil
 }
 
-func (m *ExecutableStagePayload) GetTimers() []*ExecutableStagePayload_TimerId {
-	if m != nil {
-		return m.Timers
+func (x *ExecutableStagePayload) GetTimers() []*ExecutableStagePayload_TimerId {
+	if x != nil {
+		return x.Timers
 	}
 	return nil
 }
 
-func (m *ExecutableStagePayload) GetTimerFamilies() []*ExecutableStagePayload_TimerFamilyId {
-	if m != nil {
-		return m.TimerFamilies
+func (x *ExecutableStagePayload) GetTimerFamilies() []*ExecutableStagePayload_TimerFamilyId {
+	if x != nil {
+		return x.TimerFamilies
 	}
 	return nil
 }
 
+type TestStreamPayload_Event struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	// Types that are assignable to Event:
+	//	*TestStreamPayload_Event_WatermarkEvent
+	//	*TestStreamPayload_Event_ProcessingTimeEvent
+	//	*TestStreamPayload_Event_ElementEvent
+	Event isTestStreamPayload_Event_Event `protobuf_oneof:"event"`
+}
+
+func (x *TestStreamPayload_Event) Reset() {
+	*x = TestStreamPayload_Event{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_runner_api_proto_msgTypes[69]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *TestStreamPayload_Event) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*TestStreamPayload_Event) ProtoMessage() {}
+
+func (x *TestStreamPayload_Event) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_runner_api_proto_msgTypes[69]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use TestStreamPayload_Event.ProtoReflect.Descriptor instead.
+func (*TestStreamPayload_Event) Descriptor() ([]byte, []int) {
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{20, 0}
+}
+
+func (m *TestStreamPayload_Event) GetEvent() isTestStreamPayload_Event_Event {
+	if m != nil {
+		return m.Event
+	}
+	return nil
+}
+
+func (x *TestStreamPayload_Event) GetWatermarkEvent() *TestStreamPayload_Event_AdvanceWatermark {
+	if x, ok := x.GetEvent().(*TestStreamPayload_Event_WatermarkEvent); ok {
+		return x.WatermarkEvent
+	}
+	return nil
+}
+
+func (x *TestStreamPayload_Event) GetProcessingTimeEvent() *TestStreamPayload_Event_AdvanceProcessingTime {
+	if x, ok := x.GetEvent().(*TestStreamPayload_Event_ProcessingTimeEvent); ok {
+		return x.ProcessingTimeEvent
+	}
+	return nil
+}
+
+func (x *TestStreamPayload_Event) GetElementEvent() *TestStreamPayload_Event_AddElements {
+	if x, ok := x.GetEvent().(*TestStreamPayload_Event_ElementEvent); ok {
+		return x.ElementEvent
+	}
+	return nil
+}
+
+type isTestStreamPayload_Event_Event interface {
+	isTestStreamPayload_Event_Event()
+}
+
+type TestStreamPayload_Event_WatermarkEvent struct {
+	WatermarkEvent *TestStreamPayload_Event_AdvanceWatermark `protobuf:"bytes,1,opt,name=watermark_event,json=watermarkEvent,proto3,oneof"`
+}
+
+type TestStreamPayload_Event_ProcessingTimeEvent struct {
+	ProcessingTimeEvent *TestStreamPayload_Event_AdvanceProcessingTime `protobuf:"bytes,2,opt,name=processing_time_event,json=processingTimeEvent,proto3,oneof"`
+}
+
+type TestStreamPayload_Event_ElementEvent struct {
+	ElementEvent *TestStreamPayload_Event_AddElements `protobuf:"bytes,3,opt,name=element_event,json=elementEvent,proto3,oneof"`
+}
+
+func (*TestStreamPayload_Event_WatermarkEvent) isTestStreamPayload_Event_Event() {}
+
+func (*TestStreamPayload_Event_ProcessingTimeEvent) isTestStreamPayload_Event_Event() {}
+
+func (*TestStreamPayload_Event_ElementEvent) isTestStreamPayload_Event_Event() {}
+
+// A single element inside of the TestStream.
+type TestStreamPayload_TimestampedElement struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	// (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"`
+}
+
+func (x *TestStreamPayload_TimestampedElement) Reset() {
+	*x = TestStreamPayload_TimestampedElement{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_runner_api_proto_msgTypes[70]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *TestStreamPayload_TimestampedElement) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*TestStreamPayload_TimestampedElement) ProtoMessage() {}
+
+func (x *TestStreamPayload_TimestampedElement) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_runner_api_proto_msgTypes[70]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use TestStreamPayload_TimestampedElement.ProtoReflect.Descriptor instead.
+func (*TestStreamPayload_TimestampedElement) Descriptor() ([]byte, []int) {
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{20, 1}
+}
+
+func (x *TestStreamPayload_TimestampedElement) GetEncodedElement() []byte {
+	if x != nil {
+		return x.EncodedElement
+	}
+	return nil
+}
+
+func (x *TestStreamPayload_TimestampedElement) GetTimestamp() int64 {
+	if x != nil {
+		return x.Timestamp
+	}
+	return 0
+}
+
+// Advances the watermark to the specified timestamp.
+type TestStreamPayload_Event_AdvanceWatermark struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	// (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"`
+}
+
+func (x *TestStreamPayload_Event_AdvanceWatermark) Reset() {
+	*x = TestStreamPayload_Event_AdvanceWatermark{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_runner_api_proto_msgTypes[71]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *TestStreamPayload_Event_AdvanceWatermark) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*TestStreamPayload_Event_AdvanceWatermark) ProtoMessage() {}
+
+func (x *TestStreamPayload_Event_AdvanceWatermark) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_runner_api_proto_msgTypes[71]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use TestStreamPayload_Event_AdvanceWatermark.ProtoReflect.Descriptor instead.
+func (*TestStreamPayload_Event_AdvanceWatermark) Descriptor() ([]byte, []int) {
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{20, 0, 0}
+}
+
+func (x *TestStreamPayload_Event_AdvanceWatermark) GetNewWatermark() int64 {
+	if x != nil {
+		return x.NewWatermark
+	}
+	return 0
+}
+
+func (x *TestStreamPayload_Event_AdvanceWatermark) GetTag() string {
+	if x != nil {
+		return x.Tag
+	}
+	return ""
+}
+
+// Advances the processing time clock by the specified amount.
+type TestStreamPayload_Event_AdvanceProcessingTime struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	// (Required) The duration to advance by.
+	AdvanceDuration int64 `protobuf:"varint,1,opt,name=advance_duration,json=advanceDuration,proto3" json:"advance_duration,omitempty"`
+}
+
+func (x *TestStreamPayload_Event_AdvanceProcessingTime) Reset() {
+	*x = TestStreamPayload_Event_AdvanceProcessingTime{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_runner_api_proto_msgTypes[72]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *TestStreamPayload_Event_AdvanceProcessingTime) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*TestStreamPayload_Event_AdvanceProcessingTime) ProtoMessage() {}
+
+func (x *TestStreamPayload_Event_AdvanceProcessingTime) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_runner_api_proto_msgTypes[72]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use TestStreamPayload_Event_AdvanceProcessingTime.ProtoReflect.Descriptor instead.
+func (*TestStreamPayload_Event_AdvanceProcessingTime) Descriptor() ([]byte, []int) {
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{20, 0, 1}
+}
+
+func (x *TestStreamPayload_Event_AdvanceProcessingTime) GetAdvanceDuration() int64 {
+	if x != nil {
+		return x.AdvanceDuration
+	}
+	return 0
+}
+
+// Adds elements to the stream to be emitted.
+type TestStreamPayload_Event_AddElements struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	// (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"`
+}
+
+func (x *TestStreamPayload_Event_AddElements) Reset() {
+	*x = TestStreamPayload_Event_AddElements{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_runner_api_proto_msgTypes[73]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *TestStreamPayload_Event_AddElements) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*TestStreamPayload_Event_AddElements) ProtoMessage() {}
+
+func (x *TestStreamPayload_Event_AddElements) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_runner_api_proto_msgTypes[73]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use TestStreamPayload_Event_AddElements.ProtoReflect.Descriptor instead.
+func (*TestStreamPayload_Event_AddElements) Descriptor() ([]byte, []int) {
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{20, 0, 2}
+}
+
+func (x *TestStreamPayload_Event_AddElements) GetElements() []*TestStreamPayload_TimestampedElement {
+	if x != nil {
+		return x.Elements
+	}
+	return nil
+}
+
+func (x *TestStreamPayload_Event_AddElements) GetTag() string {
+	if x != nil {
+		return x.Tag
+	}
+	return ""
+}
+
+// Ready when all subtriggers are ready.
+type Trigger_AfterAll struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	Subtriggers []*Trigger `protobuf:"bytes,1,rep,name=subtriggers,proto3" json:"subtriggers,omitempty"`
+}
+
+func (x *Trigger_AfterAll) Reset() {
+	*x = Trigger_AfterAll{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_runner_api_proto_msgTypes[75]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *Trigger_AfterAll) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*Trigger_AfterAll) ProtoMessage() {}
+
+func (x *Trigger_AfterAll) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_runner_api_proto_msgTypes[75]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use Trigger_AfterAll.ProtoReflect.Descriptor instead.
+func (*Trigger_AfterAll) Descriptor() ([]byte, []int) {
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{34, 0}
+}
+
+func (x *Trigger_AfterAll) GetSubtriggers() []*Trigger {
+	if x != nil {
+		return x.Subtriggers
+	}
+	return nil
+}
+
+// Ready when any subtrigger is ready.
+type Trigger_AfterAny struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	Subtriggers []*Trigger `protobuf:"bytes,1,rep,name=subtriggers,proto3" json:"subtriggers,omitempty"`
+}
+
+func (x *Trigger_AfterAny) Reset() {
+	*x = Trigger_AfterAny{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_runner_api_proto_msgTypes[76]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *Trigger_AfterAny) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*Trigger_AfterAny) ProtoMessage() {}
+
+func (x *Trigger_AfterAny) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_runner_api_proto_msgTypes[76]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use Trigger_AfterAny.ProtoReflect.Descriptor instead.
+func (*Trigger_AfterAny) Descriptor() ([]byte, []int) {
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{34, 1}
+}
+
+func (x *Trigger_AfterAny) GetSubtriggers() []*Trigger {
+	if x != nil {
+		return x.Subtriggers
+	}
+	return nil
+}
+
+// Starting with the first subtrigger, ready when the _current_ subtrigger
+// is ready. After output, advances the current trigger by one.
+type Trigger_AfterEach struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	Subtriggers []*Trigger `protobuf:"bytes,1,rep,name=subtriggers,proto3" json:"subtriggers,omitempty"`
+}
+
+func (x *Trigger_AfterEach) Reset() {
+	*x = Trigger_AfterEach{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_runner_api_proto_msgTypes[77]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *Trigger_AfterEach) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*Trigger_AfterEach) ProtoMessage() {}
+
+func (x *Trigger_AfterEach) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_runner_api_proto_msgTypes[77]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use Trigger_AfterEach.ProtoReflect.Descriptor instead.
+func (*Trigger_AfterEach) Descriptor() ([]byte, []int) {
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{34, 2}
+}
+
+func (x *Trigger_AfterEach) GetSubtriggers() []*Trigger {
+	if x != nil {
+		return x.Subtriggers
+	}
+	return nil
+}
+
+// Ready after the input watermark is past the end of the window.
+//
+// May have implicitly-repeated subtriggers for early and late firings.
+// When the end of the window is reached, the trigger transitions between
+// the subtriggers.
+type Trigger_AfterEndOfWindow struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	// (Optional) A trigger governing output prior to the end of the window.
+	EarlyFirings *Trigger `protobuf:"bytes,1,opt,name=early_firings,json=earlyFirings,proto3" json:"early_firings,omitempty"`
+	// (Optional) A trigger governing output after the end of the window.
+	LateFirings *Trigger `protobuf:"bytes,2,opt,name=late_firings,json=lateFirings,proto3" json:"late_firings,omitempty"`
+}
+
+func (x *Trigger_AfterEndOfWindow) Reset() {
+	*x = Trigger_AfterEndOfWindow{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_runner_api_proto_msgTypes[78]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *Trigger_AfterEndOfWindow) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*Trigger_AfterEndOfWindow) ProtoMessage() {}
+
+func (x *Trigger_AfterEndOfWindow) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_runner_api_proto_msgTypes[78]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use Trigger_AfterEndOfWindow.ProtoReflect.Descriptor instead.
+func (*Trigger_AfterEndOfWindow) Descriptor() ([]byte, []int) {
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{34, 3}
+}
+
+func (x *Trigger_AfterEndOfWindow) GetEarlyFirings() *Trigger {
+	if x != nil {
+		return x.EarlyFirings
+	}
+	return nil
+}
+
+func (x *Trigger_AfterEndOfWindow) GetLateFirings() *Trigger {
+	if x != nil {
+		return x.LateFirings
+	}
+	return nil
+}
+
+// After input arrives, ready when the specified delay has passed.
+type Trigger_AfterProcessingTime struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	// (Required) The transforms to apply to an arriving element's timestamp,
+	// in order
+	TimestampTransforms []*TimestampTransform `protobuf:"bytes,1,rep,name=timestamp_transforms,json=timestampTransforms,proto3" json:"timestamp_transforms,omitempty"`
+}
+
+func (x *Trigger_AfterProcessingTime) Reset() {
+	*x = Trigger_AfterProcessingTime{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_runner_api_proto_msgTypes[79]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *Trigger_AfterProcessingTime) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*Trigger_AfterProcessingTime) ProtoMessage() {}
+
+func (x *Trigger_AfterProcessingTime) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_runner_api_proto_msgTypes[79]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use Trigger_AfterProcessingTime.ProtoReflect.Descriptor instead.
+func (*Trigger_AfterProcessingTime) Descriptor() ([]byte, []int) {
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{34, 4}
+}
+
+func (x *Trigger_AfterProcessingTime) GetTimestampTransforms() []*TimestampTransform {
+	if x != nil {
+		return x.TimestampTransforms
+	}
+	return nil
+}
+
+// Ready whenever upstream processing time has all caught up with
+// the arrival time of an input element
+type Trigger_AfterSynchronizedProcessingTime struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+}
+
+func (x *Trigger_AfterSynchronizedProcessingTime) Reset() {
+	*x = Trigger_AfterSynchronizedProcessingTime{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_runner_api_proto_msgTypes[80]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *Trigger_AfterSynchronizedProcessingTime) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*Trigger_AfterSynchronizedProcessingTime) ProtoMessage() {}
+
+func (x *Trigger_AfterSynchronizedProcessingTime) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_runner_api_proto_msgTypes[80]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use Trigger_AfterSynchronizedProcessingTime.ProtoReflect.Descriptor instead.
+func (*Trigger_AfterSynchronizedProcessingTime) Descriptor() ([]byte, []int) {
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{34, 5}
+}
+
+// The default trigger. Equivalent to Repeat { AfterEndOfWindow } but
+// specially denoted to indicate the user did not alter the triggering.
+type Trigger_Default struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+}
+
+func (x *Trigger_Default) Reset() {
+	*x = Trigger_Default{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_runner_api_proto_msgTypes[81]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *Trigger_Default) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*Trigger_Default) ProtoMessage() {}
+
+func (x *Trigger_Default) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_runner_api_proto_msgTypes[81]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use Trigger_Default.ProtoReflect.Descriptor instead.
+func (*Trigger_Default) Descriptor() ([]byte, []int) {
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{34, 6}
+}
+
+// Ready whenever the requisite number of input elements have arrived
+type Trigger_ElementCount struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	ElementCount int32 `protobuf:"varint,1,opt,name=element_count,json=elementCount,proto3" json:"element_count,omitempty"`
+}
+
+func (x *Trigger_ElementCount) Reset() {
+	*x = Trigger_ElementCount{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_runner_api_proto_msgTypes[82]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *Trigger_ElementCount) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*Trigger_ElementCount) ProtoMessage() {}
+
+func (x *Trigger_ElementCount) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_runner_api_proto_msgTypes[82]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use Trigger_ElementCount.ProtoReflect.Descriptor instead.
+func (*Trigger_ElementCount) Descriptor() ([]byte, []int) {
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{34, 7}
+}
+
+func (x *Trigger_ElementCount) GetElementCount() int32 {
+	if x != nil {
+		return x.ElementCount
+	}
+	return 0
+}
+
+// Never ready. There will only be an ON_TIME output and a final
+// output at window expiration.
+type Trigger_Never struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+}
+
+func (x *Trigger_Never) Reset() {
+	*x = Trigger_Never{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_runner_api_proto_msgTypes[83]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *Trigger_Never) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*Trigger_Never) ProtoMessage() {}
+
+func (x *Trigger_Never) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_runner_api_proto_msgTypes[83]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use Trigger_Never.ProtoReflect.Descriptor instead.
+func (*Trigger_Never) Descriptor() ([]byte, []int) {
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{34, 8}
+}
+
+// Always ready. This can also be expressed as ElementCount(1) but
+// is more explicit.
+type Trigger_Always struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+}
+
+func (x *Trigger_Always) Reset() {
+	*x = Trigger_Always{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_runner_api_proto_msgTypes[84]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *Trigger_Always) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*Trigger_Always) ProtoMessage() {}
+
+func (x *Trigger_Always) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_runner_api_proto_msgTypes[84]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use Trigger_Always.ProtoReflect.Descriptor instead.
+func (*Trigger_Always) Descriptor() ([]byte, []int) {
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{34, 9}
+}
+
+// Ready whenever either of its subtriggers are ready, but finishes output
+// when the finally subtrigger fires.
+type Trigger_OrFinally struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	// (Required) Trigger governing main output; may fire repeatedly.
+	Main *Trigger `protobuf:"bytes,1,opt,name=main,proto3" json:"main,omitempty"`
+	// (Required) Trigger governing termination of output.
+	Finally *Trigger `protobuf:"bytes,2,opt,name=finally,proto3" json:"finally,omitempty"`
+}
+
+func (x *Trigger_OrFinally) Reset() {
+	*x = Trigger_OrFinally{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_runner_api_proto_msgTypes[85]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *Trigger_OrFinally) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*Trigger_OrFinally) ProtoMessage() {}
+
+func (x *Trigger_OrFinally) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_runner_api_proto_msgTypes[85]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use Trigger_OrFinally.ProtoReflect.Descriptor instead.
+func (*Trigger_OrFinally) Descriptor() ([]byte, []int) {
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{34, 10}
+}
+
+func (x *Trigger_OrFinally) GetMain() *Trigger {
+	if x != nil {
+		return x.Main
+	}
+	return nil
+}
+
+func (x *Trigger_OrFinally) GetFinally() *Trigger {
+	if x != nil {
+		return x.Finally
+	}
+	return nil
+}
+
+// Ready whenever the subtrigger is ready; resets state when the subtrigger
+// completes.
+type Trigger_Repeat struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	// (Require) Trigger that is run repeatedly.
+	Subtrigger *Trigger `protobuf:"bytes,1,opt,name=subtrigger,proto3" json:"subtrigger,omitempty"`
+}
+
+func (x *Trigger_Repeat) Reset() {
+	*x = Trigger_Repeat{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_runner_api_proto_msgTypes[86]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *Trigger_Repeat) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*Trigger_Repeat) ProtoMessage() {}
+
+func (x *Trigger_Repeat) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_runner_api_proto_msgTypes[86]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use Trigger_Repeat.ProtoReflect.Descriptor instead.
+func (*Trigger_Repeat) Descriptor() ([]byte, []int) {
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{34, 11}
+}
+
+func (x *Trigger_Repeat) GetSubtrigger() *Trigger {
+	if x != nil {
+		return x.Subtrigger
+	}
+	return nil
+}
+
+type TimestampTransform_Delay struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	// (Required) The delay, in milliseconds.
+	DelayMillis int64 `protobuf:"varint,1,opt,name=delay_millis,json=delayMillis,proto3" json:"delay_millis,omitempty"`
+}
+
+func (x *TimestampTransform_Delay) Reset() {
+	*x = TimestampTransform_Delay{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_runner_api_proto_msgTypes[87]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *TimestampTransform_Delay) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*TimestampTransform_Delay) ProtoMessage() {}
+
+func (x *TimestampTransform_Delay) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_runner_api_proto_msgTypes[87]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use TimestampTransform_Delay.ProtoReflect.Descriptor instead.
+func (*TimestampTransform_Delay) Descriptor() ([]byte, []int) {
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{35, 0}
+}
+
+func (x *TimestampTransform_Delay) GetDelayMillis() int64 {
+	if x != nil {
+		return x.DelayMillis
+	}
+	return 0
+}
+
+type TimestampTransform_AlignTo struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	// (Required) A duration to which delays should be quantized
+	// in milliseconds.
+	Period int64 `protobuf:"varint,3,opt,name=period,proto3" json:"period,omitempty"`
+	// (Required) An offset from 0 for the quantization specified by
+	// alignment_size, in milliseconds
+	Offset int64 `protobuf:"varint,4,opt,name=offset,proto3" json:"offset,omitempty"`
+}
+
+func (x *TimestampTransform_AlignTo) Reset() {
+	*x = TimestampTransform_AlignTo{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_runner_api_proto_msgTypes[88]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *TimestampTransform_AlignTo) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*TimestampTransform_AlignTo) ProtoMessage() {}
+
+func (x *TimestampTransform_AlignTo) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_runner_api_proto_msgTypes[88]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use TimestampTransform_AlignTo.ProtoReflect.Descriptor instead.
+func (*TimestampTransform_AlignTo) Descriptor() ([]byte, []int) {
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{35, 1}
+}
+
+func (x *TimestampTransform_AlignTo) GetPeriod() int64 {
+	if x != nil {
+		return x.Period
+	}
+	return 0
+}
+
+func (x *TimestampTransform_AlignTo) GetOffset() int64 {
+	if x != nil {
+		return x.Offset
+	}
+	return 0
+}
+
 // A reference to a side input. Side inputs are uniquely identified by PTransform id and
 // local name.
 type ExecutableStagePayload_SideInputId struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
 	// (Required) The id of the PTransform that references this side input.
 	TransformId string `protobuf:"bytes,1,opt,name=transform_id,json=transformId,proto3" json:"transform_id,omitempty"`
 	// (Required) The local name of this side input from the PTransform that references it.
-	LocalName            string   `protobuf:"bytes,2,opt,name=local_name,json=localName,proto3" json:"local_name,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
+	LocalName string `protobuf:"bytes,2,opt,name=local_name,json=localName,proto3" json:"local_name,omitempty"`
 }
 
-func (m *ExecutableStagePayload_SideInputId) Reset()         { *m = ExecutableStagePayload_SideInputId{} }
-func (m *ExecutableStagePayload_SideInputId) String() string { return proto.CompactTextString(m) }
-func (*ExecutableStagePayload_SideInputId) ProtoMessage()    {}
+func (x *ExecutableStagePayload_SideInputId) Reset() {
+	*x = ExecutableStagePayload_SideInputId{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_runner_api_proto_msgTypes[91]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *ExecutableStagePayload_SideInputId) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*ExecutableStagePayload_SideInputId) ProtoMessage() {}
+
+func (x *ExecutableStagePayload_SideInputId) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_runner_api_proto_msgTypes[91]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use ExecutableStagePayload_SideInputId.ProtoReflect.Descriptor instead.
 func (*ExecutableStagePayload_SideInputId) Descriptor() ([]byte, []int) {
-	return fileDescriptor_cf57597c3a9659a9, []int{55, 0}
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{58, 0}
 }
 
-func (m *ExecutableStagePayload_SideInputId) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_ExecutableStagePayload_SideInputId.Unmarshal(m, b)
-}
-func (m *ExecutableStagePayload_SideInputId) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_ExecutableStagePayload_SideInputId.Marshal(b, m, deterministic)
-}
-func (m *ExecutableStagePayload_SideInputId) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ExecutableStagePayload_SideInputId.Merge(m, src)
-}
-func (m *ExecutableStagePayload_SideInputId) XXX_Size() int {
-	return xxx_messageInfo_ExecutableStagePayload_SideInputId.Size(m)
-}
-func (m *ExecutableStagePayload_SideInputId) XXX_DiscardUnknown() {
-	xxx_messageInfo_ExecutableStagePayload_SideInputId.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_ExecutableStagePayload_SideInputId proto.InternalMessageInfo
-
-func (m *ExecutableStagePayload_SideInputId) GetTransformId() string {
-	if m != nil {
-		return m.TransformId
+func (x *ExecutableStagePayload_SideInputId) GetTransformId() string {
+	if x != nil {
+		return x.TransformId
 	}
 	return ""
 }
 
-func (m *ExecutableStagePayload_SideInputId) GetLocalName() string {
-	if m != nil {
-		return m.LocalName
+func (x *ExecutableStagePayload_SideInputId) GetLocalName() string {
+	if x != nil {
+		return x.LocalName
 	}
 	return ""
 }
@@ -5517,50 +6741,58 @@
 // A reference to user state. User states are uniquely identified by PTransform id and
 // local name.
 type ExecutableStagePayload_UserStateId struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
 	// (Required) The id of the PTransform that references this user state.
 	TransformId string `protobuf:"bytes,1,opt,name=transform_id,json=transformId,proto3" json:"transform_id,omitempty"`
 	// (Required) The local name of this user state for the PTransform that references it.
-	LocalName            string   `protobuf:"bytes,2,opt,name=local_name,json=localName,proto3" json:"local_name,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
+	LocalName string `protobuf:"bytes,2,opt,name=local_name,json=localName,proto3" json:"local_name,omitempty"`
 }
 
-func (m *ExecutableStagePayload_UserStateId) Reset()         { *m = ExecutableStagePayload_UserStateId{} }
-func (m *ExecutableStagePayload_UserStateId) String() string { return proto.CompactTextString(m) }
-func (*ExecutableStagePayload_UserStateId) ProtoMessage()    {}
+func (x *ExecutableStagePayload_UserStateId) Reset() {
+	*x = ExecutableStagePayload_UserStateId{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_runner_api_proto_msgTypes[92]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *ExecutableStagePayload_UserStateId) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*ExecutableStagePayload_UserStateId) ProtoMessage() {}
+
+func (x *ExecutableStagePayload_UserStateId) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_runner_api_proto_msgTypes[92]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use ExecutableStagePayload_UserStateId.ProtoReflect.Descriptor instead.
 func (*ExecutableStagePayload_UserStateId) Descriptor() ([]byte, []int) {
-	return fileDescriptor_cf57597c3a9659a9, []int{55, 1}
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{58, 1}
 }
 
-func (m *ExecutableStagePayload_UserStateId) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_ExecutableStagePayload_UserStateId.Unmarshal(m, b)
-}
-func (m *ExecutableStagePayload_UserStateId) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_ExecutableStagePayload_UserStateId.Marshal(b, m, deterministic)
-}
-func (m *ExecutableStagePayload_UserStateId) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ExecutableStagePayload_UserStateId.Merge(m, src)
-}
-func (m *ExecutableStagePayload_UserStateId) XXX_Size() int {
-	return xxx_messageInfo_ExecutableStagePayload_UserStateId.Size(m)
-}
-func (m *ExecutableStagePayload_UserStateId) XXX_DiscardUnknown() {
-	xxx_messageInfo_ExecutableStagePayload_UserStateId.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_ExecutableStagePayload_UserStateId proto.InternalMessageInfo
-
-func (m *ExecutableStagePayload_UserStateId) GetTransformId() string {
-	if m != nil {
-		return m.TransformId
+func (x *ExecutableStagePayload_UserStateId) GetTransformId() string {
+	if x != nil {
+		return x.TransformId
 	}
 	return ""
 }
 
-func (m *ExecutableStagePayload_UserStateId) GetLocalName() string {
-	if m != nil {
-		return m.LocalName
+func (x *ExecutableStagePayload_UserStateId) GetLocalName() string {
+	if x != nil {
+		return x.LocalName
 	}
 	return ""
 }
@@ -5568,50 +6800,58 @@
 // A reference to a timer. Timers are uniquely identified by PTransform id and
 // local name.
 type ExecutableStagePayload_TimerId struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
 	// (Required) The id of the PTransform that references this timer.
 	TransformId string `protobuf:"bytes,1,opt,name=transform_id,json=transformId,proto3" json:"transform_id,omitempty"`
 	// (Required) The local name of this timer for the PTransform that references it.
-	LocalName            string   `protobuf:"bytes,2,opt,name=local_name,json=localName,proto3" json:"local_name,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
+	LocalName string `protobuf:"bytes,2,opt,name=local_name,json=localName,proto3" json:"local_name,omitempty"`
 }
 
-func (m *ExecutableStagePayload_TimerId) Reset()         { *m = ExecutableStagePayload_TimerId{} }
-func (m *ExecutableStagePayload_TimerId) String() string { return proto.CompactTextString(m) }
-func (*ExecutableStagePayload_TimerId) ProtoMessage()    {}
+func (x *ExecutableStagePayload_TimerId) Reset() {
+	*x = ExecutableStagePayload_TimerId{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_runner_api_proto_msgTypes[93]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *ExecutableStagePayload_TimerId) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*ExecutableStagePayload_TimerId) ProtoMessage() {}
+
+func (x *ExecutableStagePayload_TimerId) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_runner_api_proto_msgTypes[93]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use ExecutableStagePayload_TimerId.ProtoReflect.Descriptor instead.
 func (*ExecutableStagePayload_TimerId) Descriptor() ([]byte, []int) {
-	return fileDescriptor_cf57597c3a9659a9, []int{55, 2}
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{58, 2}
 }
 
-func (m *ExecutableStagePayload_TimerId) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_ExecutableStagePayload_TimerId.Unmarshal(m, b)
-}
-func (m *ExecutableStagePayload_TimerId) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_ExecutableStagePayload_TimerId.Marshal(b, m, deterministic)
-}
-func (m *ExecutableStagePayload_TimerId) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ExecutableStagePayload_TimerId.Merge(m, src)
-}
-func (m *ExecutableStagePayload_TimerId) XXX_Size() int {
-	return xxx_messageInfo_ExecutableStagePayload_TimerId.Size(m)
-}
-func (m *ExecutableStagePayload_TimerId) XXX_DiscardUnknown() {
-	xxx_messageInfo_ExecutableStagePayload_TimerId.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_ExecutableStagePayload_TimerId proto.InternalMessageInfo
-
-func (m *ExecutableStagePayload_TimerId) GetTransformId() string {
-	if m != nil {
-		return m.TransformId
+func (x *ExecutableStagePayload_TimerId) GetTransformId() string {
+	if x != nil {
+		return x.TransformId
 	}
 	return ""
 }
 
-func (m *ExecutableStagePayload_TimerId) GetLocalName() string {
-	if m != nil {
-		return m.LocalName
+func (x *ExecutableStagePayload_TimerId) GetLocalName() string {
+	if x != nil {
+		return x.LocalName
 	}
 	return ""
 }
@@ -5619,56 +6859,68 @@
 // A reference to a timer. Timers are uniquely identified by PTransform id and
 // local name.
 type ExecutableStagePayload_TimerFamilyId struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
 	// (Required) The id of the PTransform that references this timer family.
 	TransformId string `protobuf:"bytes,1,opt,name=transform_id,json=transformId,proto3" json:"transform_id,omitempty"`
 	// (Required) The local name of this timer family for the PTransform that references it.
-	LocalName            string   `protobuf:"bytes,2,opt,name=local_name,json=localName,proto3" json:"local_name,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
+	LocalName string `protobuf:"bytes,2,opt,name=local_name,json=localName,proto3" json:"local_name,omitempty"`
 }
 
-func (m *ExecutableStagePayload_TimerFamilyId) Reset()         { *m = ExecutableStagePayload_TimerFamilyId{} }
-func (m *ExecutableStagePayload_TimerFamilyId) String() string { return proto.CompactTextString(m) }
-func (*ExecutableStagePayload_TimerFamilyId) ProtoMessage()    {}
+func (x *ExecutableStagePayload_TimerFamilyId) Reset() {
+	*x = ExecutableStagePayload_TimerFamilyId{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_runner_api_proto_msgTypes[94]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *ExecutableStagePayload_TimerFamilyId) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*ExecutableStagePayload_TimerFamilyId) ProtoMessage() {}
+
+func (x *ExecutableStagePayload_TimerFamilyId) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_runner_api_proto_msgTypes[94]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use ExecutableStagePayload_TimerFamilyId.ProtoReflect.Descriptor instead.
 func (*ExecutableStagePayload_TimerFamilyId) Descriptor() ([]byte, []int) {
-	return fileDescriptor_cf57597c3a9659a9, []int{55, 3}
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{58, 3}
 }
 
-func (m *ExecutableStagePayload_TimerFamilyId) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_ExecutableStagePayload_TimerFamilyId.Unmarshal(m, b)
-}
-func (m *ExecutableStagePayload_TimerFamilyId) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_ExecutableStagePayload_TimerFamilyId.Marshal(b, m, deterministic)
-}
-func (m *ExecutableStagePayload_TimerFamilyId) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ExecutableStagePayload_TimerFamilyId.Merge(m, src)
-}
-func (m *ExecutableStagePayload_TimerFamilyId) XXX_Size() int {
-	return xxx_messageInfo_ExecutableStagePayload_TimerFamilyId.Size(m)
-}
-func (m *ExecutableStagePayload_TimerFamilyId) XXX_DiscardUnknown() {
-	xxx_messageInfo_ExecutableStagePayload_TimerFamilyId.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_ExecutableStagePayload_TimerFamilyId proto.InternalMessageInfo
-
-func (m *ExecutableStagePayload_TimerFamilyId) GetTransformId() string {
-	if m != nil {
-		return m.TransformId
+func (x *ExecutableStagePayload_TimerFamilyId) GetTransformId() string {
+	if x != nil {
+		return x.TransformId
 	}
 	return ""
 }
 
-func (m *ExecutableStagePayload_TimerFamilyId) GetLocalName() string {
-	if m != nil {
-		return m.LocalName
+func (x *ExecutableStagePayload_TimerFamilyId) GetLocalName() string {
+	if x != nil {
+		return x.LocalName
 	}
 	return ""
 }
 
 // Settings that decide the coder type of wire coder.
 type ExecutableStagePayload_WireCoderSetting struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
 	// (Required) The URN of the wire coder.
 	// Note that only windowed value coder or parameterized windowed value coder are supported.
 	Urn string `protobuf:"bytes,1,opt,name=urn,proto3" json:"urn,omitempty"`
@@ -5681,74 +6933,58 @@
 	Payload []byte `protobuf:"bytes,2,opt,name=payload,proto3" json:"payload,omitempty"`
 	// (Required) The target(PCollection or Timer) this setting applies to.
 	//
-	// Types that are valid to be assigned to Target:
+	// Types that are assignable to Target:
 	//	*ExecutableStagePayload_WireCoderSetting_InputOrOutputId
 	//	*ExecutableStagePayload_WireCoderSetting_Timer
-	Target               isExecutableStagePayload_WireCoderSetting_Target `protobuf_oneof:"target"`
-	XXX_NoUnkeyedLiteral struct{}                                         `json:"-"`
-	XXX_unrecognized     []byte                                           `json:"-"`
-	XXX_sizecache        int32                                            `json:"-"`
+	Target isExecutableStagePayload_WireCoderSetting_Target `protobuf_oneof:"target"`
 }
 
-func (m *ExecutableStagePayload_WireCoderSetting) Reset() {
-	*m = ExecutableStagePayload_WireCoderSetting{}
+func (x *ExecutableStagePayload_WireCoderSetting) Reset() {
+	*x = ExecutableStagePayload_WireCoderSetting{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_beam_runner_api_proto_msgTypes[95]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
 }
-func (m *ExecutableStagePayload_WireCoderSetting) String() string { return proto.CompactTextString(m) }
-func (*ExecutableStagePayload_WireCoderSetting) ProtoMessage()    {}
+
+func (x *ExecutableStagePayload_WireCoderSetting) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*ExecutableStagePayload_WireCoderSetting) ProtoMessage() {}
+
+func (x *ExecutableStagePayload_WireCoderSetting) ProtoReflect() protoreflect.Message {
+	mi := &file_beam_runner_api_proto_msgTypes[95]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use ExecutableStagePayload_WireCoderSetting.ProtoReflect.Descriptor instead.
 func (*ExecutableStagePayload_WireCoderSetting) Descriptor() ([]byte, []int) {
-	return fileDescriptor_cf57597c3a9659a9, []int{55, 4}
+	return file_beam_runner_api_proto_rawDescGZIP(), []int{58, 4}
 }
 
-func (m *ExecutableStagePayload_WireCoderSetting) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_ExecutableStagePayload_WireCoderSetting.Unmarshal(m, b)
-}
-func (m *ExecutableStagePayload_WireCoderSetting) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_ExecutableStagePayload_WireCoderSetting.Marshal(b, m, deterministic)
-}
-func (m *ExecutableStagePayload_WireCoderSetting) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ExecutableStagePayload_WireCoderSetting.Merge(m, src)
-}
-func (m *ExecutableStagePayload_WireCoderSetting) XXX_Size() int {
-	return xxx_messageInfo_ExecutableStagePayload_WireCoderSetting.Size(m)
-}
-func (m *ExecutableStagePayload_WireCoderSetting) XXX_DiscardUnknown() {
-	xxx_messageInfo_ExecutableStagePayload_WireCoderSetting.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_ExecutableStagePayload_WireCoderSetting proto.InternalMessageInfo
-
-func (m *ExecutableStagePayload_WireCoderSetting) GetUrn() string {
-	if m != nil {
-		return m.Urn
+func (x *ExecutableStagePayload_WireCoderSetting) GetUrn() string {
+	if x != nil {
+		return x.Urn
 	}
 	return ""
 }
 
-func (m *ExecutableStagePayload_WireCoderSetting) GetPayload() []byte {
-	if m != nil {
-		return m.Payload
+func (x *ExecutableStagePayload_WireCoderSetting) GetPayload() []byte {
+	if x != nil {
+		return x.Payload
 	}
 	return nil
 }
 
-type isExecutableStagePayload_WireCoderSetting_Target interface {
-	isExecutableStagePayload_WireCoderSetting_Target()
-}
-
-type ExecutableStagePayload_WireCoderSetting_InputOrOutputId struct {
-	InputOrOutputId string `protobuf:"bytes,3,opt,name=input_or_output_id,json=inputOrOutputId,proto3,oneof"`
-}
-
-type ExecutableStagePayload_WireCoderSetting_Timer struct {
-	Timer *ExecutableStagePayload_TimerId `protobuf:"bytes,4,opt,name=timer,proto3,oneof"`
-}
-
-func (*ExecutableStagePayload_WireCoderSetting_InputOrOutputId) isExecutableStagePayload_WireCoderSetting_Target() {
-}
-
-func (*ExecutableStagePayload_WireCoderSetting_Timer) isExecutableStagePayload_WireCoderSetting_Target() {
-}
-
 func (m *ExecutableStagePayload_WireCoderSetting) GetTarget() isExecutableStagePayload_WireCoderSetting_Target {
 	if m != nil {
 		return m.Target
@@ -5756,551 +6992,2659 @@
 	return nil
 }
 
-func (m *ExecutableStagePayload_WireCoderSetting) GetInputOrOutputId() string {
-	if x, ok := m.GetTarget().(*ExecutableStagePayload_WireCoderSetting_InputOrOutputId); ok {
+func (x *ExecutableStagePayload_WireCoderSetting) GetInputOrOutputId() string {
+	if x, ok := x.GetTarget().(*ExecutableStagePayload_WireCoderSetting_InputOrOutputId); ok {
 		return x.InputOrOutputId
 	}
 	return ""
 }
 
-func (m *ExecutableStagePayload_WireCoderSetting) GetTimer() *ExecutableStagePayload_TimerId {
-	if x, ok := m.GetTarget().(*ExecutableStagePayload_WireCoderSetting_Timer); ok {
+func (x *ExecutableStagePayload_WireCoderSetting) GetTimer() *ExecutableStagePayload_TimerId {
+	if x, ok := x.GetTarget().(*ExecutableStagePayload_WireCoderSetting_Timer); ok {
 		return x.Timer
 	}
 	return nil
 }
 
-// XXX_OneofWrappers is for the internal use of the proto package.
-func (*ExecutableStagePayload_WireCoderSetting) XXX_OneofWrappers() []interface{} {
-	return []interface{}{
+type isExecutableStagePayload_WireCoderSetting_Target interface {
+	isExecutableStagePayload_WireCoderSetting_Target()
+}
+
+type ExecutableStagePayload_WireCoderSetting_InputOrOutputId struct {
+	// The input or output PCollection id this setting applies to.
+	InputOrOutputId string `protobuf:"bytes,3,opt,name=input_or_output_id,json=inputOrOutputId,proto3,oneof"`
+}
+
+type ExecutableStagePayload_WireCoderSetting_Timer struct {
+	// The timer id this setting applies to.
+	Timer *ExecutableStagePayload_TimerId `protobuf:"bytes,4,opt,name=timer,proto3,oneof"`
+}
+
+func (*ExecutableStagePayload_WireCoderSetting_InputOrOutputId) isExecutableStagePayload_WireCoderSetting_Target() {
+}
+
+func (*ExecutableStagePayload_WireCoderSetting_Timer) isExecutableStagePayload_WireCoderSetting_Target() {
+}
+
+var file_beam_runner_api_proto_extTypes = []protoimpl.ExtensionInfo{
+	{
+		ExtendedType:  (*descriptor.EnumValueOptions)(nil),
+		ExtensionType: (*string)(nil),
+		Field:         185324356,
+		Name:          "org.apache.beam.model.pipeline.v1.beam_urn",
+		Tag:           "bytes,185324356,opt,name=beam_urn",
+		Filename:      "beam_runner_api.proto",
+	},
+	{
+		ExtendedType:  (*descriptor.EnumValueOptions)(nil),
+		ExtensionType: (*string)(nil),
+		Field:         185324357,
+		Name:          "org.apache.beam.model.pipeline.v1.beam_constant",
+		Tag:           "bytes,185324357,opt,name=beam_constant",
+		Filename:      "beam_runner_api.proto",
+	},
+}
+
+// Extension fields to descriptor.EnumValueOptions.
+var (
+	// An extension to be used for specifying the standard URN of various
+	// pipeline entities, e.g. transforms, functions, coders etc.
+	// Code should refer to the URNs of those entities by extracting
+	// it from the (beam_urn) extension, rather than by hard-coding
+	// the URN.
+	//
+	// The recommended pattern for declaring it is (exemplified by coders):
+	//
+	// message StandardCoders {
+	//   enum Enum {
+	//     BYTES = 0 [(beam_urn) = "beam:coder:bytes:v1"];
+	//     ...
+	//   }
+	// }
+	//
+	// If there are multiple categories of entities of this type, use the
+	// following pattern (exemplified by PTransforms):
+	//
+	// message StandardPTransforms {
+	//   enum Primitives {
+	//     ...
+	//   }
+	//   enum Composites {
+	//     ...
+	//   }
+	// }
+	//
+	// optional string beam_urn = 185324356;
+	E_BeamUrn = &file_beam_runner_api_proto_extTypes[0]
+	// A value to store other constants
+	//
+	// optional string beam_constant = 185324357;
+	E_BeamConstant = &file_beam_runner_api_proto_extTypes[1]
+)
+
+var File_beam_runner_api_proto protoreflect.FileDescriptor
+
+var file_beam_runner_api_proto_rawDesc = []byte{
+	0x0a, 0x15, 0x62, 0x65, 0x61, 0x6d, 0x5f, 0x72, 0x75, 0x6e, 0x6e, 0x65, 0x72, 0x5f, 0x61, 0x70,
+	0x69, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x21, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61,
+	0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70,
+	0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x1a, 0x0f, 0x65, 0x6e, 0x64, 0x70,
+	0x6f, 0x69, 0x6e, 0x74, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x20, 0x67, 0x6f, 0x6f,
+	0x67, 0x6c, 0x65, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x64, 0x65, 0x73,
+	0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0xc2, 0x01,
+	0x0a, 0x0d, 0x42, 0x65, 0x61, 0x6d, 0x43, 0x6f, 0x6e, 0x73, 0x74, 0x61, 0x6e, 0x74, 0x73, 0x22,
+	0xb0, 0x01, 0x0a, 0x09, 0x43, 0x6f, 0x6e, 0x73, 0x74, 0x61, 0x6e, 0x74, 0x73, 0x12, 0x31, 0x0a,
+	0x14, 0x4d, 0x49, 0x4e, 0x5f, 0x54, 0x49, 0x4d, 0x45, 0x53, 0x54, 0x41, 0x4d, 0x50, 0x5f, 0x4d,
+	0x49, 0x4c, 0x4c, 0x49, 0x53, 0x10, 0x00, 0x1a, 0x17, 0xaa, 0xb4, 0xfa, 0xc2, 0x05, 0x11, 0x2d,
+	0x39, 0x32, 0x32, 0x33, 0x33, 0x37, 0x32, 0x30, 0x33, 0x36, 0x38, 0x35, 0x34, 0x37, 0x37, 0x35,
+	0x12, 0x30, 0x0a, 0x14, 0x4d, 0x41, 0x58, 0x5f, 0x54, 0x49, 0x4d, 0x45, 0x53, 0x54, 0x41, 0x4d,
+	0x50, 0x5f, 0x4d, 0x49, 0x4c, 0x4c, 0x49, 0x53, 0x10, 0x01, 0x1a, 0x16, 0xaa, 0xb4, 0xfa, 0xc2,
+	0x05, 0x10, 0x39, 0x32, 0x32, 0x33, 0x33, 0x37, 0x32, 0x30, 0x33, 0x36, 0x38, 0x35, 0x34, 0x37,
+	0x37, 0x35, 0x12, 0x3e, 0x0a, 0x22, 0x47, 0x4c, 0x4f, 0x42, 0x41, 0x4c, 0x5f, 0x57, 0x49, 0x4e,
+	0x44, 0x4f, 0x57, 0x5f, 0x4d, 0x41, 0x58, 0x5f, 0x54, 0x49, 0x4d, 0x45, 0x53, 0x54, 0x41, 0x4d,
+	0x50, 0x5f, 0x4d, 0x49, 0x4c, 0x4c, 0x49, 0x53, 0x10, 0x02, 0x1a, 0x16, 0xaa, 0xb4, 0xfa, 0xc2,
+	0x05, 0x10, 0x39, 0x32, 0x32, 0x33, 0x33, 0x37, 0x31, 0x39, 0x35, 0x30, 0x34, 0x35, 0x34, 0x37,
+	0x37, 0x35, 0x22, 0xb6, 0x08, 0x0a, 0x0a, 0x43, 0x6f, 0x6d, 0x70, 0x6f, 0x6e, 0x65, 0x6e, 0x74,
+	0x73, 0x12, 0x5d, 0x0a, 0x0a, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x73, 0x18,
+	0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x3d, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63,
+	0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69,
+	0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x6f, 0x6d, 0x70, 0x6f, 0x6e,
+	0x65, 0x6e, 0x74, 0x73, 0x2e, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x73, 0x45,
+	0x6e, 0x74, 0x72, 0x79, 0x52, 0x0a, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x73,
+	0x12, 0x63, 0x0a, 0x0c, 0x70, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73,
+	0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x3f, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61,
+	0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70,
+	0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x6f, 0x6d, 0x70, 0x6f,
+	0x6e, 0x65, 0x6e, 0x74, 0x73, 0x2e, 0x50, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f,
+	0x6e, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0c, 0x70, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63,
+	0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12, 0x79, 0x0a, 0x14, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x69,
+	0x6e, 0x67, 0x5f, 0x73, 0x74, 0x72, 0x61, 0x74, 0x65, 0x67, 0x69, 0x65, 0x73, 0x18, 0x03, 0x20,
+	0x03, 0x28, 0x0b, 0x32, 0x46, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65,
+	0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65,
+	0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x6f, 0x6d, 0x70, 0x6f, 0x6e, 0x65, 0x6e,
+	0x74, 0x73, 0x2e, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x69, 0x6e, 0x67, 0x53, 0x74, 0x72, 0x61,
+	0x74, 0x65, 0x67, 0x69, 0x65, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x13, 0x77, 0x69, 0x6e,
+	0x64, 0x6f, 0x77, 0x69, 0x6e, 0x67, 0x53, 0x74, 0x72, 0x61, 0x74, 0x65, 0x67, 0x69, 0x65, 0x73,
+	0x12, 0x51, 0x0a, 0x06, 0x63, 0x6f, 0x64, 0x65, 0x72, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b,
+	0x32, 0x39, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65,
+	0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e,
+	0x65, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x6f, 0x6d, 0x70, 0x6f, 0x6e, 0x65, 0x6e, 0x74, 0x73, 0x2e,
+	0x43, 0x6f, 0x64, 0x65, 0x72, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x06, 0x63, 0x6f, 0x64,
+	0x65, 0x72, 0x73, 0x12, 0x63, 0x0a, 0x0c, 0x65, 0x6e, 0x76, 0x69, 0x72, 0x6f, 0x6e, 0x6d, 0x65,
+	0x6e, 0x74, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x3f, 0x2e, 0x6f, 0x72, 0x67, 0x2e,
+	0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65,
+	0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x6f,
+	0x6d, 0x70, 0x6f, 0x6e, 0x65, 0x6e, 0x74, 0x73, 0x2e, 0x45, 0x6e, 0x76, 0x69, 0x72, 0x6f, 0x6e,
+	0x6d, 0x65, 0x6e, 0x74, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0c, 0x65, 0x6e, 0x76, 0x69,
+	0x72, 0x6f, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x1a, 0x6c, 0x0a, 0x0f, 0x54, 0x72, 0x61, 0x6e,
+	0x73, 0x66, 0x6f, 0x72, 0x6d, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b,
+	0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x43, 0x0a,
+	0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2d, 0x2e, 0x6f,
+	0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d,
+	0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31,
+	0x2e, 0x50, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x52, 0x05, 0x76, 0x61, 0x6c,
+	0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x1a, 0x6f, 0x0a, 0x11, 0x50, 0x63, 0x6f, 0x6c, 0x6c, 0x65,
+	0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b,
+	0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x44, 0x0a,
+	0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2e, 0x2e, 0x6f,
+	0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d,
+	0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31,
+	0x2e, 0x50, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x05, 0x76, 0x61,
+	0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x1a, 0x7c, 0x0a, 0x18, 0x57, 0x69, 0x6e, 0x64, 0x6f,
+	0x77, 0x69, 0x6e, 0x67, 0x53, 0x74, 0x72, 0x61, 0x74, 0x65, 0x67, 0x69, 0x65, 0x73, 0x45, 0x6e,
+	0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09,
+	0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x4a, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02,
+	0x20, 0x01, 0x28, 0x0b, 0x32, 0x34, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68,
+	0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70,
+	0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x69,
+	0x6e, 0x67, 0x53, 0x74, 0x72, 0x61, 0x74, 0x65, 0x67, 0x79, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75,
+	0x65, 0x3a, 0x02, 0x38, 0x01, 0x1a, 0x63, 0x0a, 0x0b, 0x43, 0x6f, 0x64, 0x65, 0x72, 0x73, 0x45,
+	0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28,
+	0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x3e, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18,
+	0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x28, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63,
+	0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69,
+	0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x6f, 0x64, 0x65, 0x72, 0x52,
+	0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x1a, 0x6f, 0x0a, 0x11, 0x45, 0x6e,
+	0x76, 0x69, 0x72, 0x6f, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12,
+	0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65,
+	0x79, 0x12, 0x44, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b,
+	0x32, 0x2e, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65,
+	0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e,
+	0x65, 0x2e, 0x76, 0x31, 0x2e, 0x45, 0x6e, 0x76, 0x69, 0x72, 0x6f, 0x6e, 0x6d, 0x65, 0x6e, 0x74,
+	0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0xfe, 0x01, 0x0a, 0x08,
+	0x50, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x12, 0x4d, 0x0a, 0x0a, 0x63, 0x6f, 0x6d, 0x70,
+	0x6f, 0x6e, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2d, 0x2e, 0x6f,
+	0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d,
+	0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31,
+	0x2e, 0x43, 0x6f, 0x6d, 0x70, 0x6f, 0x6e, 0x65, 0x6e, 0x74, 0x73, 0x52, 0x0a, 0x63, 0x6f, 0x6d,
+	0x70, 0x6f, 0x6e, 0x65, 0x6e, 0x74, 0x73, 0x12, 0x2c, 0x0a, 0x12, 0x72, 0x6f, 0x6f, 0x74, 0x5f,
+	0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x5f, 0x69, 0x64, 0x73, 0x18, 0x02, 0x20,
+	0x03, 0x28, 0x09, 0x52, 0x10, 0x72, 0x6f, 0x6f, 0x74, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f,
+	0x72, 0x6d, 0x49, 0x64, 0x73, 0x12, 0x51, 0x0a, 0x0c, 0x64, 0x69, 0x73, 0x70, 0x6c, 0x61, 0x79,
+	0x5f, 0x64, 0x61, 0x74, 0x61, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x2e, 0x2e, 0x6f, 0x72,
+	0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f,
+	0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e,
+	0x44, 0x69, 0x73, 0x70, 0x6c, 0x61, 0x79, 0x44, 0x61, 0x74, 0x61, 0x52, 0x0b, 0x64, 0x69, 0x73,
+	0x70, 0x6c, 0x61, 0x79, 0x44, 0x61, 0x74, 0x61, 0x12, 0x22, 0x0a, 0x0c, 0x72, 0x65, 0x71, 0x75,
+	0x69, 0x72, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0c,
+	0x72, 0x65, 0x71, 0x75, 0x69, 0x72, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x22, 0xb2, 0x04, 0x0a,
+	0x0a, 0x50, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x12, 0x1f, 0x0a, 0x0b, 0x75,
+	0x6e, 0x69, 0x71, 0x75, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09,
+	0x52, 0x0a, 0x75, 0x6e, 0x69, 0x71, 0x75, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x43, 0x0a, 0x04,
+	0x73, 0x70, 0x65, 0x63, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2f, 0x2e, 0x6f, 0x72, 0x67,
+	0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64,
+	0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x46,
+	0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x70, 0x65, 0x63, 0x52, 0x04, 0x73, 0x70, 0x65,
+	0x63, 0x12, 0x24, 0x0a, 0x0d, 0x73, 0x75, 0x62, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72,
+	0x6d, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0d, 0x73, 0x75, 0x62, 0x74, 0x72, 0x61,
+	0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x73, 0x12, 0x51, 0x0a, 0x06, 0x69, 0x6e, 0x70, 0x75, 0x74,
+	0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x39, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70,
+	0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e,
+	0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x54, 0x72, 0x61,
+	0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x2e, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x73, 0x45, 0x6e, 0x74,
+	0x72, 0x79, 0x52, 0x06, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x73, 0x12, 0x54, 0x0a, 0x07, 0x6f, 0x75,
+	0x74, 0x70, 0x75, 0x74, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x3a, 0x2e, 0x6f, 0x72,
+	0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f,
+	0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e,
+	0x50, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x2e, 0x4f, 0x75, 0x74, 0x70, 0x75,
+	0x74, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x07, 0x6f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x73,
+	0x12, 0x51, 0x0a, 0x0c, 0x64, 0x69, 0x73, 0x70, 0x6c, 0x61, 0x79, 0x5f, 0x64, 0x61, 0x74, 0x61,
+	0x18, 0x06, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x2e, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61,
+	0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70,
+	0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x44, 0x69, 0x73, 0x70, 0x6c,
+	0x61, 0x79, 0x44, 0x61, 0x74, 0x61, 0x52, 0x0b, 0x64, 0x69, 0x73, 0x70, 0x6c, 0x61, 0x79, 0x44,
+	0x61, 0x74, 0x61, 0x12, 0x25, 0x0a, 0x0e, 0x65, 0x6e, 0x76, 0x69, 0x72, 0x6f, 0x6e, 0x6d, 0x65,
+	0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x65, 0x6e, 0x76,
+	0x69, 0x72, 0x6f, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x49, 0x64, 0x1a, 0x39, 0x0a, 0x0b, 0x49, 0x6e,
+	0x70, 0x75, 0x74, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79,
+	0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76,
+	0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75,
+	0x65, 0x3a, 0x02, 0x38, 0x01, 0x1a, 0x3a, 0x0a, 0x0c, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x73,
+	0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01,
+	0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65,
+	0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38,
+	0x01, 0x22, 0x90, 0x0e, 0x0a, 0x13, 0x53, 0x74, 0x61, 0x6e, 0x64, 0x61, 0x72, 0x64, 0x50, 0x54,
+	0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x73, 0x22, 0xa9, 0x03, 0x0a, 0x0a, 0x50, 0x72,
+	0x69, 0x6d, 0x69, 0x74, 0x69, 0x76, 0x65, 0x73, 0x12, 0x29, 0x0a, 0x06, 0x50, 0x41, 0x52, 0x5f,
+	0x44, 0x4f, 0x10, 0x00, 0x1a, 0x1d, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x17, 0x62, 0x65, 0x61, 0x6d,
+	0x3a, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x3a, 0x70, 0x61, 0x72, 0x64, 0x6f,
+	0x3a, 0x76, 0x31, 0x12, 0x2c, 0x0a, 0x07, 0x46, 0x4c, 0x41, 0x54, 0x54, 0x45, 0x4e, 0x10, 0x01,
+	0x1a, 0x1f, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x19, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x74, 0x72, 0x61,
+	0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x3a, 0x66, 0x6c, 0x61, 0x74, 0x74, 0x65, 0x6e, 0x3a, 0x76,
+	0x31, 0x12, 0x36, 0x0a, 0x0c, 0x47, 0x52, 0x4f, 0x55, 0x50, 0x5f, 0x42, 0x59, 0x5f, 0x4b, 0x45,
+	0x59, 0x10, 0x02, 0x1a, 0x24, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x1e, 0x62, 0x65, 0x61, 0x6d, 0x3a,
+	0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x3a, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x5f,
+	0x62, 0x79, 0x5f, 0x6b, 0x65, 0x79, 0x3a, 0x76, 0x31, 0x12, 0x2c, 0x0a, 0x07, 0x49, 0x4d, 0x50,
+	0x55, 0x4c, 0x53, 0x45, 0x10, 0x03, 0x1a, 0x1f, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x19, 0x62, 0x65,
+	0x61, 0x6d, 0x3a, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x3a, 0x69, 0x6d, 0x70,
+	0x75, 0x6c, 0x73, 0x65, 0x3a, 0x76, 0x31, 0x12, 0x37, 0x0a, 0x0e, 0x41, 0x53, 0x53, 0x49, 0x47,
+	0x4e, 0x5f, 0x57, 0x49, 0x4e, 0x44, 0x4f, 0x57, 0x53, 0x10, 0x04, 0x1a, 0x23, 0xa2, 0xb4, 0xfa,
+	0xc2, 0x05, 0x1d, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72,
+	0x6d, 0x3a, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x5f, 0x69, 0x6e, 0x74, 0x6f, 0x3a, 0x76, 0x31,
+	0x12, 0x33, 0x0a, 0x0b, 0x54, 0x45, 0x53, 0x54, 0x5f, 0x53, 0x54, 0x52, 0x45, 0x41, 0x4d, 0x10,
+	0x05, 0x1a, 0x22, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x1c, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x74, 0x72,
+	0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x3a, 0x74, 0x65, 0x73, 0x74, 0x73, 0x74, 0x72, 0x65,
+	0x61, 0x6d, 0x3a, 0x76, 0x31, 0x12, 0x34, 0x0a, 0x0b, 0x4d, 0x41, 0x50, 0x5f, 0x57, 0x49, 0x4e,
+	0x44, 0x4f, 0x57, 0x53, 0x10, 0x06, 0x1a, 0x23, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x1d, 0x62, 0x65,
+	0x61, 0x6d, 0x3a, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x3a, 0x6d, 0x61, 0x70,
+	0x5f, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x73, 0x3a, 0x76, 0x31, 0x12, 0x38, 0x0a, 0x0d, 0x4d,
+	0x45, 0x52, 0x47, 0x45, 0x5f, 0x57, 0x49, 0x4e, 0x44, 0x4f, 0x57, 0x53, 0x10, 0x07, 0x1a, 0x25,
+	0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x1f, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x74, 0x72, 0x61, 0x6e, 0x73,
+	0x66, 0x6f, 0x72, 0x6d, 0x3a, 0x6d, 0x65, 0x72, 0x67, 0x65, 0x5f, 0x77, 0x69, 0x6e, 0x64, 0x6f,
+	0x77, 0x73, 0x3a, 0x76, 0x31, 0x22, 0x74, 0x0a, 0x14, 0x44, 0x65, 0x70, 0x72, 0x65, 0x63, 0x61,
+	0x74, 0x65, 0x64, 0x50, 0x72, 0x69, 0x6d, 0x69, 0x74, 0x69, 0x76, 0x65, 0x73, 0x12, 0x26, 0x0a,
+	0x04, 0x52, 0x45, 0x41, 0x44, 0x10, 0x00, 0x1a, 0x1c, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x16, 0x62,
+	0x65, 0x61, 0x6d, 0x3a, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x3a, 0x72, 0x65,
+	0x61, 0x64, 0x3a, 0x76, 0x31, 0x12, 0x34, 0x0a, 0x0b, 0x43, 0x52, 0x45, 0x41, 0x54, 0x45, 0x5f,
+	0x56, 0x49, 0x45, 0x57, 0x10, 0x01, 0x1a, 0x23, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x1d, 0x62, 0x65,
+	0x61, 0x6d, 0x3a, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x3a, 0x63, 0x72, 0x65,
+	0x61, 0x74, 0x65, 0x5f, 0x76, 0x69, 0x65, 0x77, 0x3a, 0x76, 0x31, 0x22, 0xe0, 0x02, 0x0a, 0x0a,
+	0x43, 0x6f, 0x6d, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x65, 0x73, 0x12, 0x3c, 0x0a, 0x0f, 0x43, 0x4f,
+	0x4d, 0x42, 0x49, 0x4e, 0x45, 0x5f, 0x50, 0x45, 0x52, 0x5f, 0x4b, 0x45, 0x59, 0x10, 0x00, 0x1a,
+	0x27, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x21, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x74, 0x72, 0x61, 0x6e,
+	0x73, 0x66, 0x6f, 0x72, 0x6d, 0x3a, 0x63, 0x6f, 0x6d, 0x62, 0x69, 0x6e, 0x65, 0x5f, 0x70, 0x65,
+	0x72, 0x5f, 0x6b, 0x65, 0x79, 0x3a, 0x76, 0x31, 0x12, 0x3e, 0x0a, 0x10, 0x43, 0x4f, 0x4d, 0x42,
+	0x49, 0x4e, 0x45, 0x5f, 0x47, 0x4c, 0x4f, 0x42, 0x41, 0x4c, 0x4c, 0x59, 0x10, 0x01, 0x1a, 0x28,
+	0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x22, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x74, 0x72, 0x61, 0x6e, 0x73,
+	0x66, 0x6f, 0x72, 0x6d, 0x3a, 0x63, 0x6f, 0x6d, 0x62, 0x69, 0x6e, 0x65, 0x5f, 0x67, 0x6c, 0x6f,
+	0x62, 0x61, 0x6c, 0x6c, 0x79, 0x3a, 0x76, 0x31, 0x12, 0x30, 0x0a, 0x09, 0x52, 0x45, 0x53, 0x48,
+	0x55, 0x46, 0x46, 0x4c, 0x45, 0x10, 0x02, 0x1a, 0x21, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x1b, 0x62,
+	0x65, 0x61, 0x6d, 0x3a, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x3a, 0x72, 0x65,
+	0x73, 0x68, 0x75, 0x66, 0x66, 0x6c, 0x65, 0x3a, 0x76, 0x31, 0x12, 0x34, 0x0a, 0x0b, 0x57, 0x52,
+	0x49, 0x54, 0x45, 0x5f, 0x46, 0x49, 0x4c, 0x45, 0x53, 0x10, 0x03, 0x1a, 0x23, 0xa2, 0xb4, 0xfa,
+	0xc2, 0x05, 0x1d, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72,
+	0x6d, 0x3a, 0x77, 0x72, 0x69, 0x74, 0x65, 0x5f, 0x66, 0x69, 0x6c, 0x65, 0x73, 0x3a, 0x76, 0x31,
+	0x12, 0x34, 0x0a, 0x0b, 0x50, 0x55, 0x42, 0x53, 0x55, 0x42, 0x5f, 0x52, 0x45, 0x41, 0x44, 0x10,
+	0x04, 0x1a, 0x23, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x1d, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x74, 0x72,
+	0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x3a, 0x70, 0x75, 0x62, 0x73, 0x75, 0x62, 0x5f, 0x72,
+	0x65, 0x61, 0x64, 0x3a, 0x76, 0x31, 0x12, 0x36, 0x0a, 0x0c, 0x50, 0x55, 0x42, 0x53, 0x55, 0x42,
+	0x5f, 0x57, 0x52, 0x49, 0x54, 0x45, 0x10, 0x05, 0x1a, 0x24, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x1e,
+	0x62, 0x65, 0x61, 0x6d, 0x3a, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x3a, 0x70,
+	0x75, 0x62, 0x73, 0x75, 0x62, 0x5f, 0x77, 0x72, 0x69, 0x74, 0x65, 0x3a, 0x76, 0x31, 0x22, 0xe3,
+	0x03, 0x0a, 0x11, 0x43, 0x6f, 0x6d, 0x62, 0x69, 0x6e, 0x65, 0x43, 0x6f, 0x6d, 0x70, 0x6f, 0x6e,
+	0x65, 0x6e, 0x74, 0x73, 0x12, 0x52, 0x0a, 0x1a, 0x43, 0x4f, 0x4d, 0x42, 0x49, 0x4e, 0x45, 0x5f,
+	0x50, 0x45, 0x52, 0x5f, 0x4b, 0x45, 0x59, 0x5f, 0x50, 0x52, 0x45, 0x43, 0x4f, 0x4d, 0x42, 0x49,
+	0x4e, 0x45, 0x10, 0x00, 0x1a, 0x32, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x2c, 0x62, 0x65, 0x61, 0x6d,
+	0x3a, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x3a, 0x63, 0x6f, 0x6d, 0x62, 0x69,
+	0x6e, 0x65, 0x5f, 0x70, 0x65, 0x72, 0x5f, 0x6b, 0x65, 0x79, 0x5f, 0x70, 0x72, 0x65, 0x63, 0x6f,
+	0x6d, 0x62, 0x69, 0x6e, 0x65, 0x3a, 0x76, 0x31, 0x12, 0x62, 0x0a, 0x22, 0x43, 0x4f, 0x4d, 0x42,
+	0x49, 0x4e, 0x45, 0x5f, 0x50, 0x45, 0x52, 0x5f, 0x4b, 0x45, 0x59, 0x5f, 0x4d, 0x45, 0x52, 0x47,
+	0x45, 0x5f, 0x41, 0x43, 0x43, 0x55, 0x4d, 0x55, 0x4c, 0x41, 0x54, 0x4f, 0x52, 0x53, 0x10, 0x01,
+	0x1a, 0x3a, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x34, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x74, 0x72, 0x61,
+	0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x3a, 0x63, 0x6f, 0x6d, 0x62, 0x69, 0x6e, 0x65, 0x5f, 0x70,
+	0x65, 0x72, 0x5f, 0x6b, 0x65, 0x79, 0x5f, 0x6d, 0x65, 0x72, 0x67, 0x65, 0x5f, 0x61, 0x63, 0x63,
+	0x75, 0x6d, 0x75, 0x6c, 0x61, 0x74, 0x6f, 0x72, 0x73, 0x3a, 0x76, 0x31, 0x12, 0x5c, 0x0a, 0x1f,
+	0x43, 0x4f, 0x4d, 0x42, 0x49, 0x4e, 0x45, 0x5f, 0x50, 0x45, 0x52, 0x5f, 0x4b, 0x45, 0x59, 0x5f,
+	0x45, 0x58, 0x54, 0x52, 0x41, 0x43, 0x54, 0x5f, 0x4f, 0x55, 0x54, 0x50, 0x55, 0x54, 0x53, 0x10,
+	0x02, 0x1a, 0x37, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x31, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x74, 0x72,
+	0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x3a, 0x63, 0x6f, 0x6d, 0x62, 0x69, 0x6e, 0x65, 0x5f,
+	0x70, 0x65, 0x72, 0x5f, 0x6b, 0x65, 0x79, 0x5f, 0x65, 0x78, 0x74, 0x72, 0x61, 0x63, 0x74, 0x5f,
+	0x6f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x73, 0x3a, 0x76, 0x31, 0x12, 0x4a, 0x0a, 0x16, 0x43, 0x4f,
+	0x4d, 0x42, 0x49, 0x4e, 0x45, 0x5f, 0x47, 0x52, 0x4f, 0x55, 0x50, 0x45, 0x44, 0x5f, 0x56, 0x41,
+	0x4c, 0x55, 0x45, 0x53, 0x10, 0x03, 0x1a, 0x2e, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x28, 0x62, 0x65,
+	0x61, 0x6d, 0x3a, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x3a, 0x63, 0x6f, 0x6d,
+	0x62, 0x69, 0x6e, 0x65, 0x5f, 0x67, 0x72, 0x6f, 0x75, 0x70, 0x65, 0x64, 0x5f, 0x76, 0x61, 0x6c,
+	0x75, 0x65, 0x73, 0x3a, 0x76, 0x31, 0x12, 0x6c, 0x0a, 0x27, 0x43, 0x4f, 0x4d, 0x42, 0x49, 0x4e,
+	0x45, 0x5f, 0x50, 0x45, 0x52, 0x5f, 0x4b, 0x45, 0x59, 0x5f, 0x43, 0x4f, 0x4e, 0x56, 0x45, 0x52,
+	0x54, 0x5f, 0x54, 0x4f, 0x5f, 0x41, 0x43, 0x43, 0x55, 0x4d, 0x55, 0x4c, 0x41, 0x54, 0x4f, 0x52,
+	0x53, 0x10, 0x04, 0x1a, 0x3f, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x39, 0x62, 0x65, 0x61, 0x6d, 0x3a,
+	0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x3a, 0x63, 0x6f, 0x6d, 0x62, 0x69, 0x6e,
+	0x65, 0x5f, 0x70, 0x65, 0x72, 0x5f, 0x6b, 0x65, 0x79, 0x5f, 0x63, 0x6f, 0x6e, 0x76, 0x65, 0x72,
+	0x74, 0x5f, 0x74, 0x6f, 0x5f, 0x61, 0x63, 0x63, 0x75, 0x6d, 0x75, 0x6c, 0x61, 0x74, 0x6f, 0x72,
+	0x73, 0x3a, 0x76, 0x31, 0x22, 0x8d, 0x03, 0x0a, 0x19, 0x53, 0x70, 0x6c, 0x69, 0x74, 0x74, 0x61,
+	0x62, 0x6c, 0x65, 0x50, 0x61, 0x72, 0x44, 0x6f, 0x43, 0x6f, 0x6d, 0x70, 0x6f, 0x6e, 0x65, 0x6e,
+	0x74, 0x73, 0x12, 0x4c, 0x0a, 0x15, 0x50, 0x41, 0x49, 0x52, 0x5f, 0x57, 0x49, 0x54, 0x48, 0x5f,
+	0x52, 0x45, 0x53, 0x54, 0x52, 0x49, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x10, 0x00, 0x1a, 0x31, 0xa2,
+	0xb4, 0xfa, 0xc2, 0x05, 0x2b, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66,
+	0x6f, 0x72, 0x6d, 0x3a, 0x73, 0x64, 0x66, 0x5f, 0x70, 0x61, 0x69, 0x72, 0x5f, 0x77, 0x69, 0x74,
+	0x68, 0x5f, 0x72, 0x65, 0x73, 0x74, 0x72, 0x69, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x3a, 0x76, 0x31,
+	0x12, 0x58, 0x0a, 0x1b, 0x53, 0x50, 0x4c, 0x49, 0x54, 0x5f, 0x41, 0x4e, 0x44, 0x5f, 0x53, 0x49,
+	0x5a, 0x45, 0x5f, 0x52, 0x45, 0x53, 0x54, 0x52, 0x49, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x53, 0x10,
+	0x01, 0x1a, 0x37, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x31, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x74, 0x72,
+	0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x3a, 0x73, 0x64, 0x66, 0x5f, 0x73, 0x70, 0x6c, 0x69,
+	0x74, 0x5f, 0x61, 0x6e, 0x64, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x5f, 0x72, 0x65, 0x73, 0x74, 0x72,
+	0x69, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x3a, 0x76, 0x31, 0x12, 0x6f, 0x0a, 0x27, 0x50, 0x52,
+	0x4f, 0x43, 0x45, 0x53, 0x53, 0x5f, 0x53, 0x49, 0x5a, 0x45, 0x44, 0x5f, 0x45, 0x4c, 0x45, 0x4d,
+	0x45, 0x4e, 0x54, 0x53, 0x5f, 0x41, 0x4e, 0x44, 0x5f, 0x52, 0x45, 0x53, 0x54, 0x52, 0x49, 0x43,
+	0x54, 0x49, 0x4f, 0x4e, 0x53, 0x10, 0x02, 0x1a, 0x42, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x3c, 0x62,
+	0x65, 0x61, 0x6d, 0x3a, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x3a, 0x73, 0x64,
+	0x66, 0x5f, 0x70, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x64, 0x5f,
+	0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x61, 0x6e, 0x64, 0x5f, 0x72, 0x65, 0x73, 0x74,
+	0x72, 0x69, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x3a, 0x76, 0x31, 0x12, 0x57, 0x0a, 0x1a, 0x54,
+	0x52, 0x55, 0x4e, 0x43, 0x41, 0x54, 0x45, 0x5f, 0x53, 0x49, 0x5a, 0x45, 0x44, 0x5f, 0x52, 0x45,
+	0x53, 0x54, 0x52, 0x49, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x10, 0x03, 0x1a, 0x37, 0xa2, 0xb4, 0xfa,
+	0xc2, 0x05, 0x31, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72,
+	0x6d, 0x3a, 0x73, 0x64, 0x66, 0x5f, 0x74, 0x72, 0x75, 0x6e, 0x63, 0x61, 0x74, 0x65, 0x5f, 0x73,
+	0x69, 0x7a, 0x65, 0x64, 0x5f, 0x72, 0x65, 0x73, 0x74, 0x72, 0x69, 0x63, 0x74, 0x69, 0x6f, 0x6e,
+	0x73, 0x3a, 0x76, 0x31, 0x22, 0x82, 0x01, 0x0a, 0x16, 0x53, 0x74, 0x61, 0x6e, 0x64, 0x61, 0x72,
+	0x64, 0x53, 0x69, 0x64, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x54, 0x79, 0x70, 0x65, 0x73, 0x22,
+	0x68, 0x0a, 0x04, 0x45, 0x6e, 0x75, 0x6d, 0x12, 0x2f, 0x0a, 0x08, 0x49, 0x54, 0x45, 0x52, 0x41,
+	0x42, 0x4c, 0x45, 0x10, 0x00, 0x1a, 0x21, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x1b, 0x62, 0x65, 0x61,
+	0x6d, 0x3a, 0x73, 0x69, 0x64, 0x65, 0x5f, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x3a, 0x69, 0x74, 0x65,
+	0x72, 0x61, 0x62, 0x6c, 0x65, 0x3a, 0x76, 0x31, 0x12, 0x2f, 0x0a, 0x08, 0x4d, 0x55, 0x4c, 0x54,
+	0x49, 0x4d, 0x41, 0x50, 0x10, 0x01, 0x1a, 0x21, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x1b, 0x62, 0x65,
+	0x61, 0x6d, 0x3a, 0x73, 0x69, 0x64, 0x65, 0x5f, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x3a, 0x6d, 0x75,
+	0x6c, 0x74, 0x69, 0x6d, 0x61, 0x70, 0x3a, 0x76, 0x31, 0x22, 0xa2, 0x02, 0x0a, 0x0b, 0x50, 0x43,
+	0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x1f, 0x0a, 0x0b, 0x75, 0x6e, 0x69,
+	0x71, 0x75, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a,
+	0x75, 0x6e, 0x69, 0x71, 0x75, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x19, 0x0a, 0x08, 0x63, 0x6f,
+	0x64, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x63, 0x6f,
+	0x64, 0x65, 0x72, 0x49, 0x64, 0x12, 0x50, 0x0a, 0x0a, 0x69, 0x73, 0x5f, 0x62, 0x6f, 0x75, 0x6e,
+	0x64, 0x65, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x31, 0x2e, 0x6f, 0x72, 0x67, 0x2e,
+	0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65,
+	0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x49, 0x73,
+	0x42, 0x6f, 0x75, 0x6e, 0x64, 0x65, 0x64, 0x2e, 0x45, 0x6e, 0x75, 0x6d, 0x52, 0x09, 0x69, 0x73,
+	0x42, 0x6f, 0x75, 0x6e, 0x64, 0x65, 0x64, 0x12, 0x32, 0x0a, 0x15, 0x77, 0x69, 0x6e, 0x64, 0x6f,
+	0x77, 0x69, 0x6e, 0x67, 0x5f, 0x73, 0x74, 0x72, 0x61, 0x74, 0x65, 0x67, 0x79, 0x5f, 0x69, 0x64,
+	0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x13, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x69, 0x6e,
+	0x67, 0x53, 0x74, 0x72, 0x61, 0x74, 0x65, 0x67, 0x79, 0x49, 0x64, 0x12, 0x51, 0x0a, 0x0c, 0x64,
+	0x69, 0x73, 0x70, 0x6c, 0x61, 0x79, 0x5f, 0x64, 0x61, 0x74, 0x61, 0x18, 0x05, 0x20, 0x03, 0x28,
+	0x0b, 0x32, 0x2e, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62,
+	0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69,
+	0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x44, 0x69, 0x73, 0x70, 0x6c, 0x61, 0x79, 0x44, 0x61, 0x74,
+	0x61, 0x52, 0x0b, 0x64, 0x69, 0x73, 0x70, 0x6c, 0x61, 0x79, 0x44, 0x61, 0x74, 0x61, 0x22, 0xbe,
+	0x07, 0x0a, 0x0c, 0x50, 0x61, 0x72, 0x44, 0x6f, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12,
+	0x44, 0x0a, 0x05, 0x64, 0x6f, 0x5f, 0x66, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2f,
+	0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d,
+	0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e,
+	0x76, 0x31, 0x2e, 0x46, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x70, 0x65, 0x63, 0x52,
+	0x04, 0x64, 0x6f, 0x46, 0x6e, 0x12, 0x60, 0x0a, 0x0b, 0x73, 0x69, 0x64, 0x65, 0x5f, 0x69, 0x6e,
+	0x70, 0x75, 0x74, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x3f, 0x2e, 0x6f, 0x72, 0x67,
+	0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64,
+	0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x50,
+	0x61, 0x72, 0x44, 0x6f, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x2e, 0x53, 0x69, 0x64, 0x65,
+	0x49, 0x6e, 0x70, 0x75, 0x74, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0a, 0x73, 0x69, 0x64,
+	0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x73, 0x12, 0x60, 0x0a, 0x0b, 0x73, 0x74, 0x61, 0x74, 0x65,
+	0x5f, 0x73, 0x70, 0x65, 0x63, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x3f, 0x2e, 0x6f,
+	0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d,
+	0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31,
+	0x2e, 0x50, 0x61, 0x72, 0x44, 0x6f, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x2e, 0x53, 0x74,
+	0x61, 0x74, 0x65, 0x53, 0x70, 0x65, 0x63, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x0a, 0x73,
+	0x74, 0x61, 0x74, 0x65, 0x53, 0x70, 0x65, 0x63, 0x73, 0x12, 0x73, 0x0a, 0x12, 0x74, 0x69, 0x6d,
+	0x65, 0x72, 0x5f, 0x66, 0x61, 0x6d, 0x69, 0x6c, 0x79, 0x5f, 0x73, 0x70, 0x65, 0x63, 0x73, 0x18,
+	0x09, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x45, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63,
+	0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69,
+	0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x61, 0x72, 0x44, 0x6f, 0x50,
+	0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x72, 0x46, 0x61, 0x6d, 0x69,
+	0x6c, 0x79, 0x53, 0x70, 0x65, 0x63, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x10, 0x74, 0x69,
+	0x6d, 0x65, 0x72, 0x46, 0x61, 0x6d, 0x69, 0x6c, 0x79, 0x53, 0x70, 0x65, 0x63, 0x73, 0x12, 0x30,
+	0x0a, 0x14, 0x72, 0x65, 0x73, 0x74, 0x72, 0x69, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x63, 0x6f,
+	0x64, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x12, 0x72, 0x65,
+	0x73, 0x74, 0x72, 0x69, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x43, 0x6f, 0x64, 0x65, 0x72, 0x49, 0x64,
+	0x12, 0x33, 0x0a, 0x15, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x73, 0x5f, 0x66, 0x69, 0x6e,
+	0x61, 0x6c, 0x69, 0x7a, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x08, 0x20, 0x01, 0x28, 0x08, 0x52,
+	0x14, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x73, 0x46, 0x69, 0x6e, 0x61, 0x6c, 0x69, 0x7a,
+	0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x3b, 0x0a, 0x1a, 0x72, 0x65, 0x71, 0x75, 0x69, 0x72, 0x65,
+	0x73, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x5f, 0x73, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x5f, 0x69, 0x6e,
+	0x70, 0x75, 0x74, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x08, 0x52, 0x17, 0x72, 0x65, 0x71, 0x75, 0x69,
+	0x72, 0x65, 0x73, 0x54, 0x69, 0x6d, 0x65, 0x53, 0x6f, 0x72, 0x74, 0x65, 0x64, 0x49, 0x6e, 0x70,
+	0x75, 0x74, 0x12, 0x32, 0x0a, 0x15, 0x72, 0x65, 0x71, 0x75, 0x69, 0x72, 0x65, 0x73, 0x5f, 0x73,
+	0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x18, 0x0b, 0x20, 0x01, 0x28,
+	0x08, 0x52, 0x13, 0x72, 0x65, 0x71, 0x75, 0x69, 0x72, 0x65, 0x73, 0x53, 0x74, 0x61, 0x62, 0x6c,
+	0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x1a, 0x6b, 0x0a, 0x0f, 0x53, 0x69, 0x64, 0x65, 0x49, 0x6e,
+	0x70, 0x75, 0x74, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79,
+	0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x42, 0x0a, 0x05, 0x76,
+	0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2c, 0x2e, 0x6f, 0x72, 0x67,
+	0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64,
+	0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x53,
+	0x69, 0x64, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a,
+	0x02, 0x38, 0x01, 0x1a, 0x6b, 0x0a, 0x0f, 0x53, 0x74, 0x61, 0x74, 0x65, 0x53, 0x70, 0x65, 0x63,
+	0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20,
+	0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x42, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75,
+	0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2c, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70,
+	0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e,
+	0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x74, 0x61, 0x74,
+	0x65, 0x53, 0x70, 0x65, 0x63, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01,
+	0x1a, 0x77, 0x0a, 0x15, 0x54, 0x69, 0x6d, 0x65, 0x72, 0x46, 0x61, 0x6d, 0x69, 0x6c, 0x79, 0x53,
+	0x70, 0x65, 0x63, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79,
+	0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x48, 0x0a, 0x05, 0x76,
+	0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x32, 0x2e, 0x6f, 0x72, 0x67,
+	0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64,
+	0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x54,
+	0x69, 0x6d, 0x65, 0x72, 0x46, 0x61, 0x6d, 0x69, 0x6c, 0x79, 0x53, 0x70, 0x65, 0x63, 0x52, 0x05,
+	0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x4a, 0x04, 0x08, 0x06, 0x10, 0x07, 0x22,
+	0xb8, 0x04, 0x0a, 0x09, 0x53, 0x74, 0x61, 0x74, 0x65, 0x53, 0x70, 0x65, 0x63, 0x12, 0x72, 0x0a,
+	0x16, 0x72, 0x65, 0x61, 0x64, 0x5f, 0x6d, 0x6f, 0x64, 0x69, 0x66, 0x79, 0x5f, 0x77, 0x72, 0x69,
+	0x74, 0x65, 0x5f, 0x73, 0x70, 0x65, 0x63, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3b, 0x2e,
+	0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e,
+	0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76,
+	0x31, 0x2e, 0x52, 0x65, 0x61, 0x64, 0x4d, 0x6f, 0x64, 0x69, 0x66, 0x79, 0x57, 0x72, 0x69, 0x74,
+	0x65, 0x53, 0x74, 0x61, 0x74, 0x65, 0x53, 0x70, 0x65, 0x63, 0x48, 0x00, 0x52, 0x13, 0x72, 0x65,
+	0x61, 0x64, 0x4d, 0x6f, 0x64, 0x69, 0x66, 0x79, 0x57, 0x72, 0x69, 0x74, 0x65, 0x53, 0x70, 0x65,
+	0x63, 0x12, 0x4c, 0x0a, 0x08, 0x62, 0x61, 0x67, 0x5f, 0x73, 0x70, 0x65, 0x63, 0x18, 0x02, 0x20,
+	0x01, 0x28, 0x0b, 0x32, 0x2f, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65,
+	0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65,
+	0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x42, 0x61, 0x67, 0x53, 0x74, 0x61, 0x74, 0x65,
+	0x53, 0x70, 0x65, 0x63, 0x48, 0x00, 0x52, 0x07, 0x62, 0x61, 0x67, 0x53, 0x70, 0x65, 0x63, 0x12,
+	0x5e, 0x0a, 0x0e, 0x63, 0x6f, 0x6d, 0x62, 0x69, 0x6e, 0x69, 0x6e, 0x67, 0x5f, 0x73, 0x70, 0x65,
+	0x63, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x35, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70,
+	0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e,
+	0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x6f, 0x6d, 0x62,
+	0x69, 0x6e, 0x69, 0x6e, 0x67, 0x53, 0x74, 0x61, 0x74, 0x65, 0x53, 0x70, 0x65, 0x63, 0x48, 0x00,
+	0x52, 0x0d, 0x63, 0x6f, 0x6d, 0x62, 0x69, 0x6e, 0x69, 0x6e, 0x67, 0x53, 0x70, 0x65, 0x63, 0x12,
+	0x4c, 0x0a, 0x08, 0x6d, 0x61, 0x70, 0x5f, 0x73, 0x70, 0x65, 0x63, 0x18, 0x04, 0x20, 0x01, 0x28,
+	0x0b, 0x32, 0x2f, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62,
+	0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69,
+	0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x4d, 0x61, 0x70, 0x53, 0x74, 0x61, 0x74, 0x65, 0x53, 0x70,
+	0x65, 0x63, 0x48, 0x00, 0x52, 0x07, 0x6d, 0x61, 0x70, 0x53, 0x70, 0x65, 0x63, 0x12, 0x4c, 0x0a,
+	0x08, 0x73, 0x65, 0x74, 0x5f, 0x73, 0x70, 0x65, 0x63, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32,
+	0x2f, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61,
+	0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65,
+	0x2e, 0x76, 0x31, 0x2e, 0x53, 0x65, 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, 0x53, 0x70, 0x65, 0x63,
+	0x48, 0x00, 0x52, 0x07, 0x73, 0x65, 0x74, 0x53, 0x70, 0x65, 0x63, 0x12, 0x65, 0x0a, 0x11, 0x6f,
+	0x72, 0x64, 0x65, 0x72, 0x65, 0x64, 0x5f, 0x6c, 0x69, 0x73, 0x74, 0x5f, 0x73, 0x70, 0x65, 0x63,
+	0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x37, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61,
+	0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70,
+	0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x4f, 0x72, 0x64, 0x65, 0x72,
+	0x65, 0x64, 0x4c, 0x69, 0x73, 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, 0x53, 0x70, 0x65, 0x63, 0x48,
+	0x00, 0x52, 0x0f, 0x6f, 0x72, 0x64, 0x65, 0x72, 0x65, 0x64, 0x4c, 0x69, 0x73, 0x74, 0x53, 0x70,
+	0x65, 0x63, 0x42, 0x06, 0x0a, 0x04, 0x73, 0x70, 0x65, 0x63, 0x22, 0x35, 0x0a, 0x18, 0x52, 0x65,
+	0x61, 0x64, 0x4d, 0x6f, 0x64, 0x69, 0x66, 0x79, 0x57, 0x72, 0x69, 0x74, 0x65, 0x53, 0x74, 0x61,
+	0x74, 0x65, 0x53, 0x70, 0x65, 0x63, 0x12, 0x19, 0x0a, 0x08, 0x63, 0x6f, 0x64, 0x65, 0x72, 0x5f,
+	0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x63, 0x6f, 0x64, 0x65, 0x72, 0x49,
+	0x64, 0x22, 0x38, 0x0a, 0x0c, 0x42, 0x61, 0x67, 0x53, 0x74, 0x61, 0x74, 0x65, 0x53, 0x70, 0x65,
+	0x63, 0x12, 0x28, 0x0a, 0x10, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x63, 0x6f, 0x64,
+	0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x65, 0x6c, 0x65,
+	0x6d, 0x65, 0x6e, 0x74, 0x43, 0x6f, 0x64, 0x65, 0x72, 0x49, 0x64, 0x22, 0x40, 0x0a, 0x14, 0x4f,
+	0x72, 0x64, 0x65, 0x72, 0x65, 0x64, 0x4c, 0x69, 0x73, 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, 0x53,
+	0x70, 0x65, 0x63, 0x12, 0x28, 0x0a, 0x10, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x63,
+	0x6f, 0x64, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x65,
+	0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x43, 0x6f, 0x64, 0x65, 0x72, 0x49, 0x64, 0x22, 0x96, 0x01,
+	0x0a, 0x12, 0x43, 0x6f, 0x6d, 0x62, 0x69, 0x6e, 0x69, 0x6e, 0x67, 0x53, 0x74, 0x61, 0x74, 0x65,
+	0x53, 0x70, 0x65, 0x63, 0x12, 0x30, 0x0a, 0x14, 0x61, 0x63, 0x63, 0x75, 0x6d, 0x75, 0x6c, 0x61,
+	0x74, 0x6f, 0x72, 0x5f, 0x63, 0x6f, 0x64, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01,
+	0x28, 0x09, 0x52, 0x12, 0x61, 0x63, 0x63, 0x75, 0x6d, 0x75, 0x6c, 0x61, 0x74, 0x6f, 0x72, 0x43,
+	0x6f, 0x64, 0x65, 0x72, 0x49, 0x64, 0x12, 0x4e, 0x0a, 0x0a, 0x63, 0x6f, 0x6d, 0x62, 0x69, 0x6e,
+	0x65, 0x5f, 0x66, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2f, 0x2e, 0x6f, 0x72, 0x67,
+	0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64,
+	0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x46,
+	0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x70, 0x65, 0x63, 0x52, 0x09, 0x63, 0x6f, 0x6d,
+	0x62, 0x69, 0x6e, 0x65, 0x46, 0x6e, 0x22, 0x56, 0x0a, 0x0c, 0x4d, 0x61, 0x70, 0x53, 0x74, 0x61,
+	0x74, 0x65, 0x53, 0x70, 0x65, 0x63, 0x12, 0x20, 0x0a, 0x0c, 0x6b, 0x65, 0x79, 0x5f, 0x63, 0x6f,
+	0x64, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0a, 0x6b, 0x65,
+	0x79, 0x43, 0x6f, 0x64, 0x65, 0x72, 0x49, 0x64, 0x12, 0x24, 0x0a, 0x0e, 0x76, 0x61, 0x6c, 0x75,
+	0x65, 0x5f, 0x63, 0x6f, 0x64, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09,
+	0x52, 0x0c, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x43, 0x6f, 0x64, 0x65, 0x72, 0x49, 0x64, 0x22, 0x38,
+	0x0a, 0x0c, 0x53, 0x65, 0x74, 0x53, 0x74, 0x61, 0x74, 0x65, 0x53, 0x70, 0x65, 0x63, 0x12, 0x28,
+	0x0a, 0x10, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x63, 0x6f, 0x64, 0x65, 0x72, 0x5f,
+	0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e,
+	0x74, 0x43, 0x6f, 0x64, 0x65, 0x72, 0x49, 0x64, 0x22, 0x99, 0x01, 0x0a, 0x0f, 0x54, 0x69, 0x6d,
+	0x65, 0x72, 0x46, 0x61, 0x6d, 0x69, 0x6c, 0x79, 0x53, 0x70, 0x65, 0x63, 0x12, 0x53, 0x0a, 0x0b,
+	0x74, 0x69, 0x6d, 0x65, 0x5f, 0x64, 0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28,
+	0x0e, 0x32, 0x32, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62,
+	0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69,
+	0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x44, 0x6f, 0x6d, 0x61, 0x69, 0x6e,
+	0x2e, 0x45, 0x6e, 0x75, 0x6d, 0x52, 0x0a, 0x74, 0x69, 0x6d, 0x65, 0x44, 0x6f, 0x6d, 0x61, 0x69,
+	0x6e, 0x12, 0x31, 0x0a, 0x15, 0x74, 0x69, 0x6d, 0x65, 0x72, 0x5f, 0x66, 0x61, 0x6d, 0x69, 0x6c,
+	0x79, 0x5f, 0x63, 0x6f, 0x64, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09,
+	0x52, 0x12, 0x74, 0x69, 0x6d, 0x65, 0x72, 0x46, 0x61, 0x6d, 0x69, 0x6c, 0x79, 0x43, 0x6f, 0x64,
+	0x65, 0x72, 0x49, 0x64, 0x22, 0x40, 0x0a, 0x09, 0x49, 0x73, 0x42, 0x6f, 0x75, 0x6e, 0x64, 0x65,
+	0x64, 0x22, 0x33, 0x0a, 0x04, 0x45, 0x6e, 0x75, 0x6d, 0x12, 0x0f, 0x0a, 0x0b, 0x55, 0x4e, 0x53,
+	0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, 0x0d, 0x0a, 0x09, 0x55, 0x4e,
+	0x42, 0x4f, 0x55, 0x4e, 0x44, 0x45, 0x44, 0x10, 0x01, 0x12, 0x0b, 0x0a, 0x07, 0x42, 0x4f, 0x55,
+	0x4e, 0x44, 0x45, 0x44, 0x10, 0x02, 0x22, 0xa8, 0x01, 0x0a, 0x0b, 0x52, 0x65, 0x61, 0x64, 0x50,
+	0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x47, 0x0a, 0x06, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65,
+	0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2f, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61,
+	0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70,
+	0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x46, 0x75, 0x6e, 0x63, 0x74,
+	0x69, 0x6f, 0x6e, 0x53, 0x70, 0x65, 0x63, 0x52, 0x06, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x12,
+	0x50, 0x0a, 0x0a, 0x69, 0x73, 0x5f, 0x62, 0x6f, 0x75, 0x6e, 0x64, 0x65, 0x64, 0x18, 0x02, 0x20,
+	0x01, 0x28, 0x0e, 0x32, 0x31, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65,
+	0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65,
+	0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x49, 0x73, 0x42, 0x6f, 0x75, 0x6e, 0x64, 0x65,
+	0x64, 0x2e, 0x45, 0x6e, 0x75, 0x6d, 0x52, 0x09, 0x69, 0x73, 0x42, 0x6f, 0x75, 0x6e, 0x64, 0x65,
+	0x64, 0x22, 0x61, 0x0a, 0x11, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x49, 0x6e, 0x74, 0x6f, 0x50,
+	0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x4c, 0x0a, 0x09, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77,
+	0x5f, 0x66, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2f, 0x2e, 0x6f, 0x72, 0x67, 0x2e,
+	0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65,
+	0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x46, 0x75,
+	0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x70, 0x65, 0x63, 0x52, 0x08, 0x77, 0x69, 0x6e, 0x64,
+	0x6f, 0x77, 0x46, 0x6e, 0x22, 0x92, 0x01, 0x0a, 0x0e, 0x43, 0x6f, 0x6d, 0x62, 0x69, 0x6e, 0x65,
+	0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x4e, 0x0a, 0x0a, 0x63, 0x6f, 0x6d, 0x62, 0x69,
+	0x6e, 0x65, 0x5f, 0x66, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2f, 0x2e, 0x6f, 0x72,
+	0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f,
+	0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e,
+	0x46, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x70, 0x65, 0x63, 0x52, 0x09, 0x63, 0x6f,
+	0x6d, 0x62, 0x69, 0x6e, 0x65, 0x46, 0x6e, 0x12, 0x30, 0x0a, 0x14, 0x61, 0x63, 0x63, 0x75, 0x6d,
+	0x75, 0x6c, 0x61, 0x74, 0x6f, 0x72, 0x5f, 0x63, 0x6f, 0x64, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18,
+	0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x12, 0x61, 0x63, 0x63, 0x75, 0x6d, 0x75, 0x6c, 0x61, 0x74,
+	0x6f, 0x72, 0x43, 0x6f, 0x64, 0x65, 0x72, 0x49, 0x64, 0x22, 0xcd, 0x07, 0x0a, 0x11, 0x54, 0x65,
+	0x73, 0x74, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12,
+	0x19, 0x0a, 0x08, 0x63, 0x6f, 0x64, 0x65, 0x72, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28,
+	0x09, 0x52, 0x07, 0x63, 0x6f, 0x64, 0x65, 0x72, 0x49, 0x64, 0x12, 0x52, 0x0a, 0x06, 0x65, 0x76,
+	0x65, 0x6e, 0x74, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x3a, 0x2e, 0x6f, 0x72, 0x67,
+	0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64,
+	0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x54,
+	0x65, 0x73, 0x74, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64,
+	0x2e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x52, 0x06, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x12, 0x53,
+	0x0a, 0x08, 0x65, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b,
+	0x32, 0x37, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65,
+	0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e,
+	0x65, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x70, 0x69, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x44,
+	0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x52, 0x08, 0x65, 0x6e, 0x64, 0x70, 0x6f,
+	0x69, 0x6e, 0x74, 0x1a, 0x96, 0x05, 0x0a, 0x05, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x12, 0x76, 0x0a,
+	0x0f, 0x77, 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, 0x6b, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74,
+	0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x4b, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61,
+	0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70,
+	0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x54, 0x65, 0x73, 0x74, 0x53,
+	0x74, 0x72, 0x65, 0x61, 0x6d, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x2e, 0x45, 0x76, 0x65,
+	0x6e, 0x74, 0x2e, 0x41, 0x64, 0x76, 0x61, 0x6e, 0x63, 0x65, 0x57, 0x61, 0x74, 0x65, 0x72, 0x6d,
+	0x61, 0x72, 0x6b, 0x48, 0x00, 0x52, 0x0e, 0x77, 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, 0x6b,
+	0x45, 0x76, 0x65, 0x6e, 0x74, 0x12, 0x86, 0x01, 0x0a, 0x15, 0x70, 0x72, 0x6f, 0x63, 0x65, 0x73,
+	0x73, 0x69, 0x6e, 0x67, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x18,
+	0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x50, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63,
+	0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69,
+	0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x54, 0x65, 0x73, 0x74, 0x53, 0x74,
+	0x72, 0x65, 0x61, 0x6d, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x2e, 0x45, 0x76, 0x65, 0x6e,
+	0x74, 0x2e, 0x41, 0x64, 0x76, 0x61, 0x6e, 0x63, 0x65, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73,
+	0x69, 0x6e, 0x67, 0x54, 0x69, 0x6d, 0x65, 0x48, 0x00, 0x52, 0x13, 0x70, 0x72, 0x6f, 0x63, 0x65,
+	0x73, 0x73, 0x69, 0x6e, 0x67, 0x54, 0x69, 0x6d, 0x65, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x12, 0x6d,
+	0x0a, 0x0d, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x18,
+	0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x46, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63,
+	0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69,
+	0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x54, 0x65, 0x73, 0x74, 0x53, 0x74,
+	0x72, 0x65, 0x61, 0x6d, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x2e, 0x45, 0x76, 0x65, 0x6e,
+	0x74, 0x2e, 0x41, 0x64, 0x64, 0x45, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x48, 0x00, 0x52,
+	0x0c, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x1a, 0x49, 0x0a,
+	0x10, 0x41, 0x64, 0x76, 0x61, 0x6e, 0x63, 0x65, 0x57, 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72,
+	0x6b, 0x12, 0x23, 0x0a, 0x0d, 0x6e, 0x65, 0x77, 0x5f, 0x77, 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61,
+	0x72, 0x6b, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0c, 0x6e, 0x65, 0x77, 0x57, 0x61, 0x74,
+	0x65, 0x72, 0x6d, 0x61, 0x72, 0x6b, 0x12, 0x10, 0x0a, 0x03, 0x74, 0x61, 0x67, 0x18, 0x02, 0x20,
+	0x01, 0x28, 0x09, 0x52, 0x03, 0x74, 0x61, 0x67, 0x1a, 0x42, 0x0a, 0x15, 0x41, 0x64, 0x76, 0x61,
+	0x6e, 0x63, 0x65, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x69, 0x6e, 0x67, 0x54, 0x69, 0x6d,
+	0x65, 0x12, 0x29, 0x0a, 0x10, 0x61, 0x64, 0x76, 0x61, 0x6e, 0x63, 0x65, 0x5f, 0x64, 0x75, 0x72,
+	0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0f, 0x61, 0x64, 0x76,
+	0x61, 0x6e, 0x63, 0x65, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x1a, 0x84, 0x01, 0x0a,
+	0x0b, 0x41, 0x64, 0x64, 0x45, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x12, 0x63, 0x0a, 0x08,
+	0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x47,
+	0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d,
+	0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e,
+	0x76, 0x31, 0x2e, 0x54, 0x65, 0x73, 0x74, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x50, 0x61, 0x79,
+	0x6c, 0x6f, 0x61, 0x64, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x65, 0x64,
+	0x45, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x08, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74,
+	0x73, 0x12, 0x10, 0x0a, 0x03, 0x74, 0x61, 0x67, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03,
+	0x74, 0x61, 0x67, 0x42, 0x07, 0x0a, 0x05, 0x65, 0x76, 0x65, 0x6e, 0x74, 0x1a, 0x5b, 0x0a, 0x12,
+	0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x65, 0x64, 0x45, 0x6c, 0x65, 0x6d, 0x65,
+	0x6e, 0x74, 0x12, 0x27, 0x0a, 0x0f, 0x65, 0x6e, 0x63, 0x6f, 0x64, 0x65, 0x64, 0x5f, 0x65, 0x6c,
+	0x65, 0x6d, 0x65, 0x6e, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0e, 0x65, 0x6e, 0x63,
+	0x6f, 0x64, 0x65, 0x64, 0x45, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x12, 0x1c, 0x0a, 0x09, 0x74,
+	0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09,
+	0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x22, 0x2e, 0x0a, 0x0d, 0x45, 0x76, 0x65,
+	0x6e, 0x74, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1d, 0x0a, 0x0a, 0x6f, 0x75,
+	0x74, 0x70, 0x75, 0x74, 0x5f, 0x69, 0x64, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x09, 0x52, 0x09,
+	0x6f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x49, 0x64, 0x73, 0x22, 0xed, 0x03, 0x0a, 0x11, 0x57, 0x72,
+	0x69, 0x74, 0x65, 0x46, 0x69, 0x6c, 0x65, 0x73, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12,
+	0x43, 0x0a, 0x04, 0x73, 0x69, 0x6e, 0x6b, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2f, 0x2e,
+	0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e,
+	0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76,
+	0x31, 0x2e, 0x46, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x70, 0x65, 0x63, 0x52, 0x04,
+	0x73, 0x69, 0x6e, 0x6b, 0x12, 0x58, 0x0a, 0x0f, 0x66, 0x6f, 0x72, 0x6d, 0x61, 0x74, 0x5f, 0x66,
+	0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2f, 0x2e,
+	0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e,
+	0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76,
+	0x31, 0x2e, 0x46, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x70, 0x65, 0x63, 0x52, 0x0e,
+	0x66, 0x6f, 0x72, 0x6d, 0x61, 0x74, 0x46, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x27,
+	0x0a, 0x0f, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x65, 0x64, 0x5f, 0x77, 0x72, 0x69, 0x74, 0x65,
+	0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0e, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x65,
+	0x64, 0x57, 0x72, 0x69, 0x74, 0x65, 0x73, 0x12, 0x3c, 0x0a, 0x1a, 0x72, 0x75, 0x6e, 0x6e, 0x65,
+	0x72, 0x5f, 0x64, 0x65, 0x74, 0x65, 0x72, 0x6d, 0x69, 0x6e, 0x65, 0x64, 0x5f, 0x73, 0x68, 0x61,
+	0x72, 0x64, 0x69, 0x6e, 0x67, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08, 0x52, 0x18, 0x72, 0x75, 0x6e,
+	0x6e, 0x65, 0x72, 0x44, 0x65, 0x74, 0x65, 0x72, 0x6d, 0x69, 0x6e, 0x65, 0x64, 0x53, 0x68, 0x61,
+	0x72, 0x64, 0x69, 0x6e, 0x67, 0x12, 0x65, 0x0a, 0x0b, 0x73, 0x69, 0x64, 0x65, 0x5f, 0x69, 0x6e,
+	0x70, 0x75, 0x74, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x44, 0x2e, 0x6f, 0x72, 0x67,
+	0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64,
+	0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x57,
+	0x72, 0x69, 0x74, 0x65, 0x46, 0x69, 0x6c, 0x65, 0x73, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64,
+	0x2e, 0x53, 0x69, 0x64, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79,
+	0x52, 0x0a, 0x73, 0x69, 0x64, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x73, 0x1a, 0x6b, 0x0a, 0x0f,
+	0x53, 0x69, 0x64, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12,
+	0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65,
+	0x79, 0x12, 0x42, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b,
+	0x32, 0x2c, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65,
+	0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e,
+	0x65, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x69, 0x64, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x52, 0x05,
+	0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0xca, 0x01, 0x0a, 0x11, 0x50, 0x75,
+	0x62, 0x53, 0x75, 0x62, 0x52, 0x65, 0x61, 0x64, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12,
+	0x14, 0x0a, 0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05,
+	0x74, 0x6f, 0x70, 0x69, 0x63, 0x12, 0x22, 0x0a, 0x0c, 0x73, 0x75, 0x62, 0x73, 0x63, 0x72, 0x69,
+	0x70, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x73, 0x75, 0x62,
+	0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x2f, 0x0a, 0x13, 0x74, 0x69, 0x6d,
+	0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x5f, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65,
+	0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x12, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d,
+	0x70, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x12, 0x21, 0x0a, 0x0c, 0x69, 0x64,
+	0x5f, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09,
+	0x52, 0x0b, 0x69, 0x64, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x12, 0x27, 0x0a,
+	0x0f, 0x77, 0x69, 0x74, 0x68, 0x5f, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73,
+	0x18, 0x05, 0x20, 0x01, 0x28, 0x08, 0x52, 0x0e, 0x77, 0x69, 0x74, 0x68, 0x41, 0x74, 0x74, 0x72,
+	0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x22, 0xa7, 0x01, 0x0a, 0x12, 0x50, 0x75, 0x62, 0x53, 0x75,
+	0x62, 0x57, 0x72, 0x69, 0x74, 0x65, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x14, 0x0a,
+	0x05, 0x74, 0x6f, 0x70, 0x69, 0x63, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x74, 0x6f,
+	0x70, 0x69, 0x63, 0x12, 0x2f, 0x0a, 0x13, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70,
+	0x5f, 0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09,
+	0x52, 0x12, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x41, 0x74, 0x74, 0x72, 0x69,
+	0x62, 0x75, 0x74, 0x65, 0x12, 0x21, 0x0a, 0x0c, 0x69, 0x64, 0x5f, 0x61, 0x74, 0x74, 0x72, 0x69,
+	0x62, 0x75, 0x74, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x69, 0x64, 0x41, 0x74,
+	0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x12, 0x27, 0x0a, 0x0f, 0x77, 0x69, 0x74, 0x68, 0x5f,
+	0x61, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x08,
+	0x52, 0x0e, 0x77, 0x69, 0x74, 0x68, 0x41, 0x74, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x65, 0x73,
+	0x22, 0x7c, 0x0a, 0x05, 0x43, 0x6f, 0x64, 0x65, 0x72, 0x12, 0x43, 0x0a, 0x04, 0x73, 0x70, 0x65,
+	0x63, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2f, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70,
+	0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e,
+	0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x46, 0x75, 0x6e, 0x63,
+	0x74, 0x69, 0x6f, 0x6e, 0x53, 0x70, 0x65, 0x63, 0x52, 0x04, 0x73, 0x70, 0x65, 0x63, 0x12, 0x2e,
+	0x0a, 0x13, 0x63, 0x6f, 0x6d, 0x70, 0x6f, 0x6e, 0x65, 0x6e, 0x74, 0x5f, 0x63, 0x6f, 0x64, 0x65,
+	0x72, 0x5f, 0x69, 0x64, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28, 0x09, 0x52, 0x11, 0x63, 0x6f, 0x6d,
+	0x70, 0x6f, 0x6e, 0x65, 0x6e, 0x74, 0x43, 0x6f, 0x64, 0x65, 0x72, 0x49, 0x64, 0x73, 0x22, 0xe1,
+	0x05, 0x0a, 0x0e, 0x53, 0x74, 0x61, 0x6e, 0x64, 0x61, 0x72, 0x64, 0x43, 0x6f, 0x64, 0x65, 0x72,
+	0x73, 0x22, 0xce, 0x05, 0x0a, 0x04, 0x45, 0x6e, 0x75, 0x6d, 0x12, 0x24, 0x0a, 0x05, 0x42, 0x59,
+	0x54, 0x45, 0x53, 0x10, 0x00, 0x1a, 0x19, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x13, 0x62, 0x65, 0x61,
+	0x6d, 0x3a, 0x63, 0x6f, 0x64, 0x65, 0x72, 0x3a, 0x62, 0x79, 0x74, 0x65, 0x73, 0x3a, 0x76, 0x31,
+	0x12, 0x30, 0x0a, 0x0b, 0x53, 0x54, 0x52, 0x49, 0x4e, 0x47, 0x5f, 0x55, 0x54, 0x46, 0x38, 0x10,
+	0x0a, 0x1a, 0x1f, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x19, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x63, 0x6f,
+	0x64, 0x65, 0x72, 0x3a, 0x73, 0x74, 0x72, 0x69, 0x6e, 0x67, 0x5f, 0x75, 0x74, 0x66, 0x38, 0x3a,
+	0x76, 0x31, 0x12, 0x1e, 0x0a, 0x02, 0x4b, 0x56, 0x10, 0x01, 0x1a, 0x16, 0xa2, 0xb4, 0xfa, 0xc2,
+	0x05, 0x10, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x63, 0x6f, 0x64, 0x65, 0x72, 0x3a, 0x6b, 0x76, 0x3a,
+	0x76, 0x31, 0x12, 0x22, 0x0a, 0x04, 0x42, 0x4f, 0x4f, 0x4c, 0x10, 0x0c, 0x1a, 0x18, 0xa2, 0xb4,
+	0xfa, 0xc2, 0x05, 0x12, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x63, 0x6f, 0x64, 0x65, 0x72, 0x3a, 0x62,
+	0x6f, 0x6f, 0x6c, 0x3a, 0x76, 0x31, 0x12, 0x26, 0x0a, 0x06, 0x56, 0x41, 0x52, 0x49, 0x4e, 0x54,
+	0x10, 0x02, 0x1a, 0x1a, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x14, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x63,
+	0x6f, 0x64, 0x65, 0x72, 0x3a, 0x76, 0x61, 0x72, 0x69, 0x6e, 0x74, 0x3a, 0x76, 0x31, 0x12, 0x26,
+	0x0a, 0x06, 0x44, 0x4f, 0x55, 0x42, 0x4c, 0x45, 0x10, 0x0b, 0x1a, 0x1a, 0xa2, 0xb4, 0xfa, 0xc2,
+	0x05, 0x14, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x63, 0x6f, 0x64, 0x65, 0x72, 0x3a, 0x64, 0x6f, 0x75,
+	0x62, 0x6c, 0x65, 0x3a, 0x76, 0x31, 0x12, 0x2a, 0x0a, 0x08, 0x49, 0x54, 0x45, 0x52, 0x41, 0x42,
+	0x4c, 0x45, 0x10, 0x03, 0x1a, 0x1c, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x16, 0x62, 0x65, 0x61, 0x6d,
+	0x3a, 0x63, 0x6f, 0x64, 0x65, 0x72, 0x3a, 0x69, 0x74, 0x65, 0x72, 0x61, 0x62, 0x6c, 0x65, 0x3a,
+	0x76, 0x31, 0x12, 0x24, 0x0a, 0x05, 0x54, 0x49, 0x4d, 0x45, 0x52, 0x10, 0x04, 0x1a, 0x19, 0xa2,
+	0xb4, 0xfa, 0xc2, 0x05, 0x13, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x63, 0x6f, 0x64, 0x65, 0x72, 0x3a,
+	0x74, 0x69, 0x6d, 0x65, 0x72, 0x3a, 0x76, 0x31, 0x12, 0x38, 0x0a, 0x0f, 0x49, 0x4e, 0x54, 0x45,
+	0x52, 0x56, 0x41, 0x4c, 0x5f, 0x57, 0x49, 0x4e, 0x44, 0x4f, 0x57, 0x10, 0x05, 0x1a, 0x23, 0xa2,
+	0xb4, 0xfa, 0xc2, 0x05, 0x1d, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x63, 0x6f, 0x64, 0x65, 0x72, 0x3a,
+	0x69, 0x6e, 0x74, 0x65, 0x72, 0x76, 0x61, 0x6c, 0x5f, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x3a,
+	0x76, 0x31, 0x12, 0x34, 0x0a, 0x0d, 0x4c, 0x45, 0x4e, 0x47, 0x54, 0x48, 0x5f, 0x50, 0x52, 0x45,
+	0x46, 0x49, 0x58, 0x10, 0x06, 0x1a, 0x21, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x1b, 0x62, 0x65, 0x61,
+	0x6d, 0x3a, 0x63, 0x6f, 0x64, 0x65, 0x72, 0x3a, 0x6c, 0x65, 0x6e, 0x67, 0x74, 0x68, 0x5f, 0x70,
+	0x72, 0x65, 0x66, 0x69, 0x78, 0x3a, 0x76, 0x31, 0x12, 0x34, 0x0a, 0x0d, 0x47, 0x4c, 0x4f, 0x42,
+	0x41, 0x4c, 0x5f, 0x57, 0x49, 0x4e, 0x44, 0x4f, 0x57, 0x10, 0x07, 0x1a, 0x21, 0xa2, 0xb4, 0xfa,
+	0xc2, 0x05, 0x1b, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x63, 0x6f, 0x64, 0x65, 0x72, 0x3a, 0x67, 0x6c,
+	0x6f, 0x62, 0x61, 0x6c, 0x5f, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x3a, 0x76, 0x31, 0x12, 0x36,
+	0x0a, 0x0e, 0x57, 0x49, 0x4e, 0x44, 0x4f, 0x57, 0x45, 0x44, 0x5f, 0x56, 0x41, 0x4c, 0x55, 0x45,
+	0x10, 0x08, 0x1a, 0x22, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x1c, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x63,
+	0x6f, 0x64, 0x65, 0x72, 0x3a, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x65, 0x64, 0x5f, 0x76, 0x61,
+	0x6c, 0x75, 0x65, 0x3a, 0x76, 0x31, 0x12, 0x42, 0x0a, 0x14, 0x50, 0x41, 0x52, 0x41, 0x4d, 0x5f,
+	0x57, 0x49, 0x4e, 0x44, 0x4f, 0x57, 0x45, 0x44, 0x5f, 0x56, 0x41, 0x4c, 0x55, 0x45, 0x10, 0x0e,
+	0x1a, 0x28, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x22, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x63, 0x6f, 0x64,
+	0x65, 0x72, 0x3a, 0x70, 0x61, 0x72, 0x61, 0x6d, 0x5f, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x65,
+	0x64, 0x5f, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x76, 0x31, 0x12, 0x44, 0x0a, 0x15, 0x53, 0x54,
+	0x41, 0x54, 0x45, 0x5f, 0x42, 0x41, 0x43, 0x4b, 0x45, 0x44, 0x5f, 0x49, 0x54, 0x45, 0x52, 0x41,
+	0x42, 0x4c, 0x45, 0x10, 0x09, 0x1a, 0x29, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x23, 0x62, 0x65, 0x61,
+	0x6d, 0x3a, 0x63, 0x6f, 0x64, 0x65, 0x72, 0x3a, 0x73, 0x74, 0x61, 0x74, 0x65, 0x5f, 0x62, 0x61,
+	0x63, 0x6b, 0x65, 0x64, 0x5f, 0x69, 0x74, 0x65, 0x72, 0x61, 0x62, 0x6c, 0x65, 0x3a, 0x76, 0x31,
+	0x12, 0x20, 0x0a, 0x03, 0x52, 0x4f, 0x57, 0x10, 0x0d, 0x1a, 0x17, 0xa2, 0xb4, 0xfa, 0xc2, 0x05,
+	0x11, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x63, 0x6f, 0x64, 0x65, 0x72, 0x3a, 0x72, 0x6f, 0x77, 0x3a,
+	0x76, 0x31, 0x22, 0xac, 0x06, 0x0a, 0x11, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x69, 0x6e, 0x67,
+	0x53, 0x74, 0x72, 0x61, 0x74, 0x65, 0x67, 0x79, 0x12, 0x4c, 0x0a, 0x09, 0x77, 0x69, 0x6e, 0x64,
+	0x6f, 0x77, 0x5f, 0x66, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2f, 0x2e, 0x6f, 0x72,
+	0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f,
+	0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e,
+	0x46, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x70, 0x65, 0x63, 0x52, 0x08, 0x77, 0x69,
+	0x6e, 0x64, 0x6f, 0x77, 0x46, 0x6e, 0x12, 0x56, 0x0a, 0x0c, 0x6d, 0x65, 0x72, 0x67, 0x65, 0x5f,
+	0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x33, 0x2e, 0x6f,
+	0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d,
+	0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31,
+	0x2e, 0x4d, 0x65, 0x72, 0x67, 0x65, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x2e, 0x45, 0x6e, 0x75,
+	0x6d, 0x52, 0x0b, 0x6d, 0x65, 0x72, 0x67, 0x65, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x26,
+	0x0a, 0x0f, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x5f, 0x63, 0x6f, 0x64, 0x65, 0x72, 0x5f, 0x69,
+	0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x43,
+	0x6f, 0x64, 0x65, 0x72, 0x49, 0x64, 0x12, 0x44, 0x0a, 0x07, 0x74, 0x72, 0x69, 0x67, 0x67, 0x65,
+	0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2a, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70,
+	0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e,
+	0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x54, 0x72, 0x69, 0x67,
+	0x67, 0x65, 0x72, 0x52, 0x07, 0x74, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x12, 0x65, 0x0a, 0x11,
+	0x61, 0x63, 0x63, 0x75, 0x6d, 0x75, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x6d, 0x6f, 0x64,
+	0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x38, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70,
+	0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e,
+	0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x63, 0x63, 0x75,
+	0x6d, 0x75, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4d, 0x6f, 0x64, 0x65, 0x2e, 0x45, 0x6e, 0x75,
+	0x6d, 0x52, 0x10, 0x61, 0x63, 0x63, 0x75, 0x6d, 0x75, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4d,
+	0x6f, 0x64, 0x65, 0x12, 0x53, 0x0a, 0x0b, 0x6f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x5f, 0x74, 0x69,
+	0x6d, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x32, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61,
+	0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c,
+	0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x4f, 0x75, 0x74,
+	0x70, 0x75, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x2e, 0x45, 0x6e, 0x75, 0x6d, 0x52, 0x0a, 0x6f, 0x75,
+	0x74, 0x70, 0x75, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x12, 0x62, 0x0a, 0x10, 0x63, 0x6c, 0x6f, 0x73,
+	0x69, 0x6e, 0x67, 0x5f, 0x62, 0x65, 0x68, 0x61, 0x76, 0x69, 0x6f, 0x72, 0x18, 0x07, 0x20, 0x01,
+	0x28, 0x0e, 0x32, 0x37, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e,
+	0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c,
+	0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x6c, 0x6f, 0x73, 0x69, 0x6e, 0x67, 0x42, 0x65,
+	0x68, 0x61, 0x76, 0x69, 0x6f, 0x72, 0x2e, 0x45, 0x6e, 0x75, 0x6d, 0x52, 0x0f, 0x63, 0x6c, 0x6f,
+	0x73, 0x69, 0x6e, 0x67, 0x42, 0x65, 0x68, 0x61, 0x76, 0x69, 0x6f, 0x72, 0x12, 0x29, 0x0a, 0x10,
+	0x61, 0x6c, 0x6c, 0x6f, 0x77, 0x65, 0x64, 0x5f, 0x6c, 0x61, 0x74, 0x65, 0x6e, 0x65, 0x73, 0x73,
+	0x18, 0x08, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0f, 0x61, 0x6c, 0x6c, 0x6f, 0x77, 0x65, 0x64, 0x4c,
+	0x61, 0x74, 0x65, 0x6e, 0x65, 0x73, 0x73, 0x12, 0x5e, 0x0a, 0x0e, 0x4f, 0x6e, 0x54, 0x69, 0x6d,
+	0x65, 0x42, 0x65, 0x68, 0x61, 0x76, 0x69, 0x6f, 0x72, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0e, 0x32,
+	0x36, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61,
+	0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65,
+	0x2e, 0x76, 0x31, 0x2e, 0x4f, 0x6e, 0x54, 0x69, 0x6d, 0x65, 0x42, 0x65, 0x68, 0x61, 0x76, 0x69,
+	0x6f, 0x72, 0x2e, 0x45, 0x6e, 0x75, 0x6d, 0x52, 0x0e, 0x4f, 0x6e, 0x54, 0x69, 0x6d, 0x65, 0x42,
+	0x65, 0x68, 0x61, 0x76, 0x69, 0x6f, 0x72, 0x12, 0x31, 0x0a, 0x15, 0x61, 0x73, 0x73, 0x69, 0x67,
+	0x6e, 0x73, 0x5f, 0x74, 0x6f, 0x5f, 0x6f, 0x6e, 0x65, 0x5f, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77,
+	0x18, 0x0a, 0x20, 0x01, 0x28, 0x08, 0x52, 0x12, 0x61, 0x73, 0x73, 0x69, 0x67, 0x6e, 0x73, 0x54,
+	0x6f, 0x4f, 0x6e, 0x65, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x12, 0x25, 0x0a, 0x0e, 0x65, 0x6e,
+	0x76, 0x69, 0x72, 0x6f, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x0b, 0x20, 0x01,
+	0x28, 0x09, 0x52, 0x0d, 0x65, 0x6e, 0x76, 0x69, 0x72, 0x6f, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x49,
+	0x64, 0x22, 0x5c, 0x0a, 0x0b, 0x4d, 0x65, 0x72, 0x67, 0x65, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73,
+	0x22, 0x4d, 0x0a, 0x04, 0x45, 0x6e, 0x75, 0x6d, 0x12, 0x0f, 0x0a, 0x0b, 0x55, 0x4e, 0x53, 0x50,
+	0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, 0x0f, 0x0a, 0x0b, 0x4e, 0x4f, 0x4e,
+	0x5f, 0x4d, 0x45, 0x52, 0x47, 0x49, 0x4e, 0x47, 0x10, 0x01, 0x12, 0x0f, 0x0a, 0x0b, 0x4e, 0x45,
+	0x45, 0x44, 0x53, 0x5f, 0x4d, 0x45, 0x52, 0x47, 0x45, 0x10, 0x02, 0x12, 0x12, 0x0a, 0x0e, 0x41,
+	0x4c, 0x52, 0x45, 0x41, 0x44, 0x59, 0x5f, 0x4d, 0x45, 0x52, 0x47, 0x45, 0x44, 0x10, 0x03, 0x22,
+	0x5d, 0x0a, 0x10, 0x41, 0x63, 0x63, 0x75, 0x6d, 0x75, 0x6c, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x4d,
+	0x6f, 0x64, 0x65, 0x22, 0x49, 0x0a, 0x04, 0x45, 0x6e, 0x75, 0x6d, 0x12, 0x0f, 0x0a, 0x0b, 0x55,
+	0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, 0x0e, 0x0a, 0x0a,
+	0x44, 0x49, 0x53, 0x43, 0x41, 0x52, 0x44, 0x49, 0x4e, 0x47, 0x10, 0x01, 0x12, 0x10, 0x0a, 0x0c,
+	0x41, 0x43, 0x43, 0x55, 0x4d, 0x55, 0x4c, 0x41, 0x54, 0x49, 0x4e, 0x47, 0x10, 0x02, 0x12, 0x0e,
+	0x0a, 0x0a, 0x52, 0x45, 0x54, 0x52, 0x41, 0x43, 0x54, 0x49, 0x4e, 0x47, 0x10, 0x03, 0x22, 0x51,
+	0x0a, 0x0f, 0x43, 0x6c, 0x6f, 0x73, 0x69, 0x6e, 0x67, 0x42, 0x65, 0x68, 0x61, 0x76, 0x69, 0x6f,
+	0x72, 0x22, 0x3e, 0x0a, 0x04, 0x45, 0x6e, 0x75, 0x6d, 0x12, 0x0f, 0x0a, 0x0b, 0x55, 0x4e, 0x53,
+	0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, 0x0f, 0x0a, 0x0b, 0x45, 0x4d,
+	0x49, 0x54, 0x5f, 0x41, 0x4c, 0x57, 0x41, 0x59, 0x53, 0x10, 0x01, 0x12, 0x14, 0x0a, 0x10, 0x45,
+	0x4d, 0x49, 0x54, 0x5f, 0x49, 0x46, 0x5f, 0x4e, 0x4f, 0x4e, 0x45, 0x4d, 0x50, 0x54, 0x59, 0x10,
+	0x02, 0x22, 0x50, 0x0a, 0x0e, 0x4f, 0x6e, 0x54, 0x69, 0x6d, 0x65, 0x42, 0x65, 0x68, 0x61, 0x76,
+	0x69, 0x6f, 0x72, 0x22, 0x3e, 0x0a, 0x04, 0x45, 0x6e, 0x75, 0x6d, 0x12, 0x0f, 0x0a, 0x0b, 0x55,
+	0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, 0x0f, 0x0a, 0x0b,
+	0x46, 0x49, 0x52, 0x45, 0x5f, 0x41, 0x4c, 0x57, 0x41, 0x59, 0x53, 0x10, 0x01, 0x12, 0x14, 0x0a,
+	0x10, 0x46, 0x49, 0x52, 0x45, 0x5f, 0x49, 0x46, 0x5f, 0x4e, 0x4f, 0x4e, 0x45, 0x4d, 0x50, 0x54,
+	0x59, 0x10, 0x02, 0x22, 0x62, 0x0a, 0x0a, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x54, 0x69, 0x6d,
+	0x65, 0x22, 0x54, 0x0a, 0x04, 0x45, 0x6e, 0x75, 0x6d, 0x12, 0x0f, 0x0a, 0x0b, 0x55, 0x4e, 0x53,
+	0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, 0x11, 0x0a, 0x0d, 0x45, 0x4e,
+	0x44, 0x5f, 0x4f, 0x46, 0x5f, 0x57, 0x49, 0x4e, 0x44, 0x4f, 0x57, 0x10, 0x01, 0x12, 0x12, 0x0a,
+	0x0e, 0x4c, 0x41, 0x54, 0x45, 0x53, 0x54, 0x5f, 0x49, 0x4e, 0x5f, 0x50, 0x41, 0x4e, 0x45, 0x10,
+	0x02, 0x12, 0x14, 0x0a, 0x10, 0x45, 0x41, 0x52, 0x4c, 0x49, 0x45, 0x53, 0x54, 0x5f, 0x49, 0x4e,
+	0x5f, 0x50, 0x41, 0x4e, 0x45, 0x10, 0x03, 0x22, 0x6c, 0x0a, 0x0a, 0x54, 0x69, 0x6d, 0x65, 0x44,
+	0x6f, 0x6d, 0x61, 0x69, 0x6e, 0x22, 0x5e, 0x0a, 0x04, 0x45, 0x6e, 0x75, 0x6d, 0x12, 0x0f, 0x0a,
+	0x0b, 0x55, 0x4e, 0x53, 0x50, 0x45, 0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, 0x0e,
+	0x0a, 0x0a, 0x45, 0x56, 0x45, 0x4e, 0x54, 0x5f, 0x54, 0x49, 0x4d, 0x45, 0x10, 0x01, 0x12, 0x13,
+	0x0a, 0x0f, 0x50, 0x52, 0x4f, 0x43, 0x45, 0x53, 0x53, 0x49, 0x4e, 0x47, 0x5f, 0x54, 0x49, 0x4d,
+	0x45, 0x10, 0x02, 0x12, 0x20, 0x0a, 0x1c, 0x53, 0x59, 0x4e, 0x43, 0x48, 0x52, 0x4f, 0x4e, 0x49,
+	0x5a, 0x45, 0x44, 0x5f, 0x50, 0x52, 0x4f, 0x43, 0x45, 0x53, 0x53, 0x49, 0x4e, 0x47, 0x5f, 0x54,
+	0x49, 0x4d, 0x45, 0x10, 0x03, 0x22, 0xa3, 0x10, 0x0a, 0x07, 0x54, 0x72, 0x69, 0x67, 0x67, 0x65,
+	0x72, 0x12, 0x52, 0x0a, 0x09, 0x61, 0x66, 0x74, 0x65, 0x72, 0x5f, 0x61, 0x6c, 0x6c, 0x18, 0x01,
+	0x20, 0x01, 0x28, 0x0b, 0x32, 0x33, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68,
+	0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70,
+	0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x54, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72,
+	0x2e, 0x41, 0x66, 0x74, 0x65, 0x72, 0x41, 0x6c, 0x6c, 0x48, 0x00, 0x52, 0x08, 0x61, 0x66, 0x74,
+	0x65, 0x72, 0x41, 0x6c, 0x6c, 0x12, 0x52, 0x0a, 0x09, 0x61, 0x66, 0x74, 0x65, 0x72, 0x5f, 0x61,
+	0x6e, 0x79, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x33, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61,
+	0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c,
+	0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x54, 0x72, 0x69,
+	0x67, 0x67, 0x65, 0x72, 0x2e, 0x41, 0x66, 0x74, 0x65, 0x72, 0x41, 0x6e, 0x79, 0x48, 0x00, 0x52,
+	0x08, 0x61, 0x66, 0x74, 0x65, 0x72, 0x41, 0x6e, 0x79, 0x12, 0x55, 0x0a, 0x0a, 0x61, 0x66, 0x74,
+	0x65, 0x72, 0x5f, 0x65, 0x61, 0x63, 0x68, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x34, 0x2e,
+	0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e,
+	0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76,
+	0x31, 0x2e, 0x54, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x2e, 0x41, 0x66, 0x74, 0x65, 0x72, 0x45,
+	0x61, 0x63, 0x68, 0x48, 0x00, 0x52, 0x09, 0x61, 0x66, 0x74, 0x65, 0x72, 0x45, 0x61, 0x63, 0x68,
+	0x12, 0x6c, 0x0a, 0x13, 0x61, 0x66, 0x74, 0x65, 0x72, 0x5f, 0x65, 0x6e, 0x64, 0x5f, 0x6f, 0x66,
+	0x5f, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3b, 0x2e,
+	0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e,
+	0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76,
+	0x31, 0x2e, 0x54, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x2e, 0x41, 0x66, 0x74, 0x65, 0x72, 0x45,
+	0x6e, 0x64, 0x4f, 0x66, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x48, 0x00, 0x52, 0x10, 0x61, 0x66,
+	0x74, 0x65, 0x72, 0x45, 0x6e, 0x64, 0x4f, 0x66, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x12, 0x74,
+	0x0a, 0x15, 0x61, 0x66, 0x74, 0x65, 0x72, 0x5f, 0x70, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x69,
+	0x6e, 0x67, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3e, 0x2e,
+	0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e,
+	0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76,
+	0x31, 0x2e, 0x54, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x2e, 0x41, 0x66, 0x74, 0x65, 0x72, 0x50,
+	0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x69, 0x6e, 0x67, 0x54, 0x69, 0x6d, 0x65, 0x48, 0x00, 0x52,
+	0x13, 0x61, 0x66, 0x74, 0x65, 0x72, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x69, 0x6e, 0x67,
+	0x54, 0x69, 0x6d, 0x65, 0x12, 0x99, 0x01, 0x0a, 0x22, 0x61, 0x66, 0x74, 0x65, 0x72, 0x5f, 0x73,
+	0x79, 0x6e, 0x63, 0x68, 0x72, 0x6f, 0x6e, 0x69, 0x7a, 0x65, 0x64, 0x5f, 0x70, 0x72, 0x6f, 0x63,
+	0x65, 0x73, 0x73, 0x69, 0x6e, 0x67, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28,
+	0x0b, 0x32, 0x4a, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62,
+	0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69,
+	0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x54, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x2e, 0x41, 0x66,
+	0x74, 0x65, 0x72, 0x53, 0x79, 0x6e, 0x63, 0x68, 0x72, 0x6f, 0x6e, 0x69, 0x7a, 0x65, 0x64, 0x50,
+	0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x69, 0x6e, 0x67, 0x54, 0x69, 0x6d, 0x65, 0x48, 0x00, 0x52,
+	0x1f, 0x61, 0x66, 0x74, 0x65, 0x72, 0x53, 0x79, 0x6e, 0x63, 0x68, 0x72, 0x6f, 0x6e, 0x69, 0x7a,
+	0x65, 0x64, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x69, 0x6e, 0x67, 0x54, 0x69, 0x6d, 0x65,
+	0x12, 0x4b, 0x0a, 0x06, 0x61, 0x6c, 0x77, 0x61, 0x79, 0x73, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x0b,
+	0x32, 0x31, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65,
+	0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e,
+	0x65, 0x2e, 0x76, 0x31, 0x2e, 0x54, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x2e, 0x41, 0x6c, 0x77,
+	0x61, 0x79, 0x73, 0x48, 0x00, 0x52, 0x06, 0x61, 0x6c, 0x77, 0x61, 0x79, 0x73, 0x12, 0x4e, 0x0a,
+	0x07, 0x64, 0x65, 0x66, 0x61, 0x75, 0x6c, 0x74, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x32,
+	0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d,
+	0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e,
+	0x76, 0x31, 0x2e, 0x54, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x2e, 0x44, 0x65, 0x66, 0x61, 0x75,
+	0x6c, 0x74, 0x48, 0x00, 0x52, 0x07, 0x64, 0x65, 0x66, 0x61, 0x75, 0x6c, 0x74, 0x12, 0x5e, 0x0a,
+	0x0d, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x08,
+	0x20, 0x01, 0x28, 0x0b, 0x32, 0x37, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68,
+	0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70,
+	0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x54, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72,
+	0x2e, 0x45, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x48, 0x00, 0x52,
+	0x0c, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x12, 0x48, 0x0a,
+	0x05, 0x6e, 0x65, 0x76, 0x65, 0x72, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x30, 0x2e, 0x6f,
+	0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d,
+	0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31,
+	0x2e, 0x54, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x2e, 0x4e, 0x65, 0x76, 0x65, 0x72, 0x48, 0x00,
+	0x52, 0x05, 0x6e, 0x65, 0x76, 0x65, 0x72, 0x12, 0x55, 0x0a, 0x0a, 0x6f, 0x72, 0x5f, 0x66, 0x69,
+	0x6e, 0x61, 0x6c, 0x6c, 0x79, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x34, 0x2e, 0x6f, 0x72,
+	0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f,
+	0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e,
+	0x54, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x2e, 0x4f, 0x72, 0x46, 0x69, 0x6e, 0x61, 0x6c, 0x6c,
+	0x79, 0x48, 0x00, 0x52, 0x09, 0x6f, 0x72, 0x46, 0x69, 0x6e, 0x61, 0x6c, 0x6c, 0x79, 0x12, 0x4b,
+	0x0a, 0x06, 0x72, 0x65, 0x70, 0x65, 0x61, 0x74, 0x18, 0x0b, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x31,
+	0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d,
+	0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e,
+	0x76, 0x31, 0x2e, 0x54, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x2e, 0x52, 0x65, 0x70, 0x65, 0x61,
+	0x74, 0x48, 0x00, 0x52, 0x06, 0x72, 0x65, 0x70, 0x65, 0x61, 0x74, 0x1a, 0x58, 0x0a, 0x08, 0x41,
+	0x66, 0x74, 0x65, 0x72, 0x41, 0x6c, 0x6c, 0x12, 0x4c, 0x0a, 0x0b, 0x73, 0x75, 0x62, 0x74, 0x72,
+	0x69, 0x67, 0x67, 0x65, 0x72, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x2a, 0x2e, 0x6f,
+	0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d,
+	0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31,
+	0x2e, 0x54, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x52, 0x0b, 0x73, 0x75, 0x62, 0x74, 0x72, 0x69,
+	0x67, 0x67, 0x65, 0x72, 0x73, 0x1a, 0x58, 0x0a, 0x08, 0x41, 0x66, 0x74, 0x65, 0x72, 0x41, 0x6e,
+	0x79, 0x12, 0x4c, 0x0a, 0x0b, 0x73, 0x75, 0x62, 0x74, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x73,
+	0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x2a, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61,
+	0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70,
+	0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x54, 0x72, 0x69, 0x67, 0x67,
+	0x65, 0x72, 0x52, 0x0b, 0x73, 0x75, 0x62, 0x74, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x73, 0x1a,
+	0x59, 0x0a, 0x09, 0x41, 0x66, 0x74, 0x65, 0x72, 0x45, 0x61, 0x63, 0x68, 0x12, 0x4c, 0x0a, 0x0b,
+	0x73, 0x75, 0x62, 0x74, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28,
+	0x0b, 0x32, 0x2a, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62,
+	0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69,
+	0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x54, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x52, 0x0b, 0x73,
+	0x75, 0x62, 0x74, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x73, 0x1a, 0xb2, 0x01, 0x0a, 0x10, 0x41,
+	0x66, 0x74, 0x65, 0x72, 0x45, 0x6e, 0x64, 0x4f, 0x66, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x12,
+	0x4f, 0x0a, 0x0d, 0x65, 0x61, 0x72, 0x6c, 0x79, 0x5f, 0x66, 0x69, 0x72, 0x69, 0x6e, 0x67, 0x73,
+	0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2a, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61,
+	0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70,
+	0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x54, 0x72, 0x69, 0x67, 0x67,
+	0x65, 0x72, 0x52, 0x0c, 0x65, 0x61, 0x72, 0x6c, 0x79, 0x46, 0x69, 0x72, 0x69, 0x6e, 0x67, 0x73,
+	0x12, 0x4d, 0x0a, 0x0c, 0x6c, 0x61, 0x74, 0x65, 0x5f, 0x66, 0x69, 0x72, 0x69, 0x6e, 0x67, 0x73,
+	0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2a, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61,
+	0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70,
+	0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x54, 0x72, 0x69, 0x67, 0x67,
+	0x65, 0x72, 0x52, 0x0b, 0x6c, 0x61, 0x74, 0x65, 0x46, 0x69, 0x72, 0x69, 0x6e, 0x67, 0x73, 0x1a,
+	0x7f, 0x0a, 0x13, 0x41, 0x66, 0x74, 0x65, 0x72, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x69,
+	0x6e, 0x67, 0x54, 0x69, 0x6d, 0x65, 0x12, 0x68, 0x0a, 0x14, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74,
+	0x61, 0x6d, 0x70, 0x5f, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x73, 0x18, 0x01,
+	0x20, 0x03, 0x28, 0x0b, 0x32, 0x35, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68,
+	0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70,
+	0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61,
+	0x6d, 0x70, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x52, 0x13, 0x74, 0x69, 0x6d,
+	0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x73,
+	0x1a, 0x21, 0x0a, 0x1f, 0x41, 0x66, 0x74, 0x65, 0x72, 0x53, 0x79, 0x6e, 0x63, 0x68, 0x72, 0x6f,
+	0x6e, 0x69, 0x7a, 0x65, 0x64, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x69, 0x6e, 0x67, 0x54,
+	0x69, 0x6d, 0x65, 0x1a, 0x09, 0x0a, 0x07, 0x44, 0x65, 0x66, 0x61, 0x75, 0x6c, 0x74, 0x1a, 0x33,
+	0x0a, 0x0c, 0x45, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x12, 0x23,
+	0x0a, 0x0d, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18,
+	0x01, 0x20, 0x01, 0x28, 0x05, 0x52, 0x0c, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x43, 0x6f,
+	0x75, 0x6e, 0x74, 0x1a, 0x07, 0x0a, 0x05, 0x4e, 0x65, 0x76, 0x65, 0x72, 0x1a, 0x08, 0x0a, 0x06,
+	0x41, 0x6c, 0x77, 0x61, 0x79, 0x73, 0x1a, 0x91, 0x01, 0x0a, 0x09, 0x4f, 0x72, 0x46, 0x69, 0x6e,
+	0x61, 0x6c, 0x6c, 0x79, 0x12, 0x3e, 0x0a, 0x04, 0x6d, 0x61, 0x69, 0x6e, 0x18, 0x01, 0x20, 0x01,
+	0x28, 0x0b, 0x32, 0x2a, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e,
+	0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c,
+	0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x54, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x52, 0x04,
+	0x6d, 0x61, 0x69, 0x6e, 0x12, 0x44, 0x0a, 0x07, 0x66, 0x69, 0x6e, 0x61, 0x6c, 0x6c, 0x79, 0x18,
+	0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2a, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63,
+	0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69,
+	0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x54, 0x72, 0x69, 0x67, 0x67, 0x65,
+	0x72, 0x52, 0x07, 0x66, 0x69, 0x6e, 0x61, 0x6c, 0x6c, 0x79, 0x1a, 0x54, 0x0a, 0x06, 0x52, 0x65,
+	0x70, 0x65, 0x61, 0x74, 0x12, 0x4a, 0x0a, 0x0a, 0x73, 0x75, 0x62, 0x74, 0x72, 0x69, 0x67, 0x67,
+	0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2a, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61,
+	0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c,
+	0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x54, 0x72, 0x69,
+	0x67, 0x67, 0x65, 0x72, 0x52, 0x0a, 0x73, 0x75, 0x62, 0x74, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72,
+	0x42, 0x09, 0x0a, 0x07, 0x74, 0x72, 0x69, 0x67, 0x67, 0x65, 0x72, 0x22, 0xc3, 0x02, 0x0a, 0x12,
+	0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f,
+	0x72, 0x6d, 0x12, 0x53, 0x0a, 0x05, 0x64, 0x65, 0x6c, 0x61, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28,
+	0x0b, 0x32, 0x3b, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62,
+	0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69,
+	0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x54,
+	0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x2e, 0x44, 0x65, 0x6c, 0x61, 0x79, 0x48, 0x00,
+	0x52, 0x05, 0x64, 0x65, 0x6c, 0x61, 0x79, 0x12, 0x5a, 0x0a, 0x08, 0x61, 0x6c, 0x69, 0x67, 0x6e,
+	0x5f, 0x74, 0x6f, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3d, 0x2e, 0x6f, 0x72, 0x67, 0x2e,
+	0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65,
+	0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x54, 0x69,
+	0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d,
+	0x2e, 0x41, 0x6c, 0x69, 0x67, 0x6e, 0x54, 0x6f, 0x48, 0x00, 0x52, 0x07, 0x61, 0x6c, 0x69, 0x67,
+	0x6e, 0x54, 0x6f, 0x1a, 0x2a, 0x0a, 0x05, 0x44, 0x65, 0x6c, 0x61, 0x79, 0x12, 0x21, 0x0a, 0x0c,
+	0x64, 0x65, 0x6c, 0x61, 0x79, 0x5f, 0x6d, 0x69, 0x6c, 0x6c, 0x69, 0x73, 0x18, 0x01, 0x20, 0x01,
+	0x28, 0x03, 0x52, 0x0b, 0x64, 0x65, 0x6c, 0x61, 0x79, 0x4d, 0x69, 0x6c, 0x6c, 0x69, 0x73, 0x1a,
+	0x39, 0x0a, 0x07, 0x41, 0x6c, 0x69, 0x67, 0x6e, 0x54, 0x6f, 0x12, 0x16, 0x0a, 0x06, 0x70, 0x65,
+	0x72, 0x69, 0x6f, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x70, 0x65, 0x72, 0x69,
+	0x6f, 0x64, 0x12, 0x16, 0x0a, 0x06, 0x6f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x18, 0x04, 0x20, 0x01,
+	0x28, 0x03, 0x52, 0x06, 0x6f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x42, 0x15, 0x0a, 0x13, 0x74, 0x69,
+	0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x5f, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72,
+	0x6d, 0x22, 0x8a, 0x02, 0x0a, 0x09, 0x53, 0x69, 0x64, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12,
+	0x56, 0x0a, 0x0e, 0x61, 0x63, 0x63, 0x65, 0x73, 0x73, 0x5f, 0x70, 0x61, 0x74, 0x74, 0x65, 0x72,
+	0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2f, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70,
+	0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e,
+	0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x46, 0x75, 0x6e, 0x63,
+	0x74, 0x69, 0x6f, 0x6e, 0x53, 0x70, 0x65, 0x63, 0x52, 0x0d, 0x61, 0x63, 0x63, 0x65, 0x73, 0x73,
+	0x50, 0x61, 0x74, 0x74, 0x65, 0x72, 0x6e, 0x12, 0x48, 0x0a, 0x07, 0x76, 0x69, 0x65, 0x77, 0x5f,
+	0x66, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2f, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61,
+	0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c,
+	0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x46, 0x75, 0x6e,
+	0x63, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x70, 0x65, 0x63, 0x52, 0x06, 0x76, 0x69, 0x65, 0x77, 0x46,
+	0x6e, 0x12, 0x5b, 0x0a, 0x11, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x5f, 0x6d, 0x61, 0x70, 0x70,
+	0x69, 0x6e, 0x67, 0x5f, 0x66, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2f, 0x2e, 0x6f,
+	0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d,
+	0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31,
+	0x2e, 0x46, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x70, 0x65, 0x63, 0x52, 0x0f, 0x77,
+	0x69, 0x6e, 0x64, 0x6f, 0x77, 0x4d, 0x61, 0x70, 0x70, 0x69, 0x6e, 0x67, 0x46, 0x6e, 0x22, 0xf6,
+	0x02, 0x0a, 0x11, 0x53, 0x74, 0x61, 0x6e, 0x64, 0x61, 0x72, 0x64, 0x41, 0x72, 0x74, 0x69, 0x66,
+	0x61, 0x63, 0x74, 0x73, 0x22, 0x9f, 0x02, 0x0a, 0x05, 0x54, 0x79, 0x70, 0x65, 0x73, 0x12, 0x2a,
+	0x0a, 0x04, 0x46, 0x49, 0x4c, 0x45, 0x10, 0x00, 0x1a, 0x20, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x1a,
+	0x62, 0x65, 0x61, 0x6d, 0x3a, 0x61, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x3a, 0x74, 0x79,
+	0x70, 0x65, 0x3a, 0x66, 0x69, 0x6c, 0x65, 0x3a, 0x76, 0x31, 0x12, 0x28, 0x0a, 0x03, 0x55, 0x52,
+	0x4c, 0x10, 0x01, 0x1a, 0x1f, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x19, 0x62, 0x65, 0x61, 0x6d, 0x3a,
+	0x61, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x3a, 0x74, 0x79, 0x70, 0x65, 0x3a, 0x75, 0x72,
+	0x6c, 0x3a, 0x76, 0x31, 0x12, 0x32, 0x0a, 0x08, 0x45, 0x4d, 0x42, 0x45, 0x44, 0x44, 0x45, 0x44,
+	0x10, 0x02, 0x1a, 0x24, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x1e, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x61,
+	0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x3a, 0x74, 0x79, 0x70, 0x65, 0x3a, 0x65, 0x6d, 0x62,
+	0x65, 0x64, 0x64, 0x65, 0x64, 0x3a, 0x76, 0x31, 0x12, 0x2a, 0x0a, 0x04, 0x50, 0x59, 0x50, 0x49,
+	0x10, 0x03, 0x1a, 0x20, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x1a, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x61,
+	0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x3a, 0x74, 0x79, 0x70, 0x65, 0x3a, 0x70, 0x79, 0x70,
+	0x69, 0x3a, 0x76, 0x31, 0x12, 0x2c, 0x0a, 0x05, 0x4d, 0x41, 0x56, 0x45, 0x4e, 0x10, 0x04, 0x1a,
+	0x21, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x1b, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x61, 0x72, 0x74, 0x69,
+	0x66, 0x61, 0x63, 0x74, 0x3a, 0x74, 0x79, 0x70, 0x65, 0x3a, 0x6d, 0x61, 0x76, 0x65, 0x6e, 0x3a,
+	0x76, 0x31, 0x12, 0x32, 0x0a, 0x08, 0x44, 0x45, 0x46, 0x45, 0x52, 0x52, 0x45, 0x44, 0x10, 0x05,
+	0x1a, 0x24, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x1e, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x61, 0x72, 0x74,
+	0x69, 0x66, 0x61, 0x63, 0x74, 0x3a, 0x74, 0x79, 0x70, 0x65, 0x3a, 0x64, 0x65, 0x66, 0x65, 0x72,
+	0x72, 0x65, 0x64, 0x3a, 0x76, 0x31, 0x22, 0x3f, 0x0a, 0x05, 0x52, 0x6f, 0x6c, 0x65, 0x73, 0x12,
+	0x36, 0x0a, 0x0a, 0x53, 0x54, 0x41, 0x47, 0x49, 0x4e, 0x47, 0x5f, 0x54, 0x4f, 0x10, 0x00, 0x1a,
+	0x26, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x20, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x61, 0x72, 0x74, 0x69,
+	0x66, 0x61, 0x63, 0x74, 0x3a, 0x72, 0x6f, 0x6c, 0x65, 0x3a, 0x73, 0x74, 0x61, 0x67, 0x69, 0x6e,
+	0x67, 0x5f, 0x74, 0x6f, 0x3a, 0x76, 0x31, 0x22, 0x41, 0x0a, 0x13, 0x41, 0x72, 0x74, 0x69, 0x66,
+	0x61, 0x63, 0x74, 0x46, 0x69, 0x6c, 0x65, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x12,
+	0x0a, 0x04, 0x70, 0x61, 0x74, 0x68, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x70, 0x61,
+	0x74, 0x68, 0x12, 0x16, 0x0a, 0x06, 0x73, 0x68, 0x61, 0x32, 0x35, 0x36, 0x18, 0x02, 0x20, 0x01,
+	0x28, 0x09, 0x52, 0x06, 0x73, 0x68, 0x61, 0x32, 0x35, 0x36, 0x22, 0x26, 0x0a, 0x12, 0x41, 0x72,
+	0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x55, 0x72, 0x6c, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64,
+	0x12, 0x10, 0x0a, 0x03, 0x75, 0x72, 0x6c, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x75,
+	0x72, 0x6c, 0x22, 0x29, 0x0a, 0x13, 0x45, 0x6d, 0x62, 0x65, 0x64, 0x64, 0x65, 0x64, 0x46, 0x69,
+	0x6c, 0x65, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x12, 0x0a, 0x04, 0x64, 0x61, 0x74,
+	0x61, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x04, 0x64, 0x61, 0x74, 0x61, 0x22, 0x48, 0x0a,
+	0x0b, 0x50, 0x79, 0x50, 0x49, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x1f, 0x0a, 0x0b,
+	0x61, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28,
+	0x09, 0x52, 0x0a, 0x61, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x49, 0x64, 0x12, 0x18, 0x0a,
+	0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07,
+	0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x22, 0x51, 0x0a, 0x0c, 0x4d, 0x61, 0x76, 0x65, 0x6e,
+	0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x1a, 0x0a, 0x08, 0x61, 0x72, 0x74, 0x69, 0x66,
+	0x61, 0x63, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x61, 0x72, 0x74, 0x69, 0x66,
+	0x61, 0x63, 0x74, 0x12, 0x25, 0x0a, 0x0e, 0x72, 0x65, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72,
+	0x79, 0x5f, 0x75, 0x72, 0x6c, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x72, 0x65, 0x70,
+	0x6f, 0x73, 0x69, 0x74, 0x6f, 0x72, 0x79, 0x55, 0x72, 0x6c, 0x22, 0x3f, 0x0a, 0x17, 0x44, 0x65,
+	0x66, 0x65, 0x72, 0x72, 0x65, 0x64, 0x41, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x50, 0x61,
+	0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01,
+	0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x12, 0x0a, 0x04, 0x64, 0x61, 0x74, 0x61, 0x18,
+	0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x04, 0x64, 0x61, 0x74, 0x61, 0x22, 0x3f, 0x0a, 0x1c, 0x41,
+	0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x53, 0x74, 0x61, 0x67, 0x69, 0x6e, 0x67, 0x54, 0x6f,
+	0x52, 0x6f, 0x6c, 0x65, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x1f, 0x0a, 0x0b, 0x73,
+	0x74, 0x61, 0x67, 0x65, 0x64, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09,
+	0x52, 0x0a, 0x73, 0x74, 0x61, 0x67, 0x65, 0x64, 0x4e, 0x61, 0x6d, 0x65, 0x22, 0x91, 0x01, 0x0a,
+	0x13, 0x41, 0x72, 0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x72, 0x6d, 0x61,
+	0x74, 0x69, 0x6f, 0x6e, 0x12, 0x19, 0x0a, 0x08, 0x74, 0x79, 0x70, 0x65, 0x5f, 0x75, 0x72, 0x6e,
+	0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x74, 0x79, 0x70, 0x65, 0x55, 0x72, 0x6e, 0x12,
+	0x21, 0x0a, 0x0c, 0x74, 0x79, 0x70, 0x65, 0x5f, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x18,
+	0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0b, 0x74, 0x79, 0x70, 0x65, 0x50, 0x61, 0x79, 0x6c, 0x6f,
+	0x61, 0x64, 0x12, 0x19, 0x0a, 0x08, 0x72, 0x6f, 0x6c, 0x65, 0x5f, 0x75, 0x72, 0x6e, 0x18, 0x03,
+	0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x72, 0x6f, 0x6c, 0x65, 0x55, 0x72, 0x6e, 0x12, 0x21, 0x0a,
+	0x0c, 0x72, 0x6f, 0x6c, 0x65, 0x5f, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x18, 0x04, 0x20,
+	0x01, 0x28, 0x0c, 0x52, 0x0b, 0x72, 0x6f, 0x6c, 0x65, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64,
+	0x22, 0x92, 0x02, 0x0a, 0x0b, 0x45, 0x6e, 0x76, 0x69, 0x72, 0x6f, 0x6e, 0x6d, 0x65, 0x6e, 0x74,
+	0x12, 0x10, 0x0a, 0x03, 0x75, 0x72, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x75,
+	0x72, 0x6e, 0x12, 0x18, 0x0a, 0x07, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x18, 0x03, 0x20,
+	0x01, 0x28, 0x0c, 0x52, 0x07, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x51, 0x0a, 0x0c,
+	0x64, 0x69, 0x73, 0x70, 0x6c, 0x61, 0x79, 0x5f, 0x64, 0x61, 0x74, 0x61, 0x18, 0x04, 0x20, 0x03,
+	0x28, 0x0b, 0x32, 0x2e, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e,
+	0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c,
+	0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x44, 0x69, 0x73, 0x70, 0x6c, 0x61, 0x79, 0x44, 0x61,
+	0x74, 0x61, 0x52, 0x0b, 0x64, 0x69, 0x73, 0x70, 0x6c, 0x61, 0x79, 0x44, 0x61, 0x74, 0x61, 0x12,
+	0x22, 0x0a, 0x0c, 0x63, 0x61, 0x70, 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74, 0x69, 0x65, 0x73, 0x18,
+	0x05, 0x20, 0x03, 0x28, 0x09, 0x52, 0x0c, 0x63, 0x61, 0x70, 0x61, 0x62, 0x69, 0x6c, 0x69, 0x74,
+	0x69, 0x65, 0x73, 0x12, 0x5a, 0x0a, 0x0c, 0x64, 0x65, 0x70, 0x65, 0x6e, 0x64, 0x65, 0x6e, 0x63,
+	0x69, 0x65, 0x73, 0x18, 0x06, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x36, 0x2e, 0x6f, 0x72, 0x67, 0x2e,
+	0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65,
+	0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x72,
+	0x74, 0x69, 0x66, 0x61, 0x63, 0x74, 0x49, 0x6e, 0x66, 0x6f, 0x72, 0x6d, 0x61, 0x74, 0x69, 0x6f,
+	0x6e, 0x52, 0x0c, 0x64, 0x65, 0x70, 0x65, 0x6e, 0x64, 0x65, 0x6e, 0x63, 0x69, 0x65, 0x73, 0x4a,
+	0x04, 0x08, 0x01, 0x10, 0x02, 0x22, 0x9f, 0x01, 0x0a, 0x14, 0x53, 0x74, 0x61, 0x6e, 0x64, 0x61,
+	0x72, 0x64, 0x45, 0x6e, 0x76, 0x69, 0x72, 0x6f, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x22, 0x86,
+	0x01, 0x0a, 0x0c, 0x45, 0x6e, 0x76, 0x69, 0x72, 0x6f, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x12,
+	0x24, 0x0a, 0x06, 0x44, 0x4f, 0x43, 0x4b, 0x45, 0x52, 0x10, 0x00, 0x1a, 0x18, 0xa2, 0xb4, 0xfa,
+	0xc2, 0x05, 0x12, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x65, 0x6e, 0x76, 0x3a, 0x64, 0x6f, 0x63, 0x6b,
+	0x65, 0x72, 0x3a, 0x76, 0x31, 0x12, 0x26, 0x0a, 0x07, 0x50, 0x52, 0x4f, 0x43, 0x45, 0x53, 0x53,
+	0x10, 0x01, 0x1a, 0x19, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x13, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x65,
+	0x6e, 0x76, 0x3a, 0x70, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x3a, 0x76, 0x31, 0x12, 0x28, 0x0a,
+	0x08, 0x45, 0x58, 0x54, 0x45, 0x52, 0x4e, 0x41, 0x4c, 0x10, 0x02, 0x1a, 0x1a, 0xa2, 0xb4, 0xfa,
+	0xc2, 0x05, 0x14, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x65, 0x6e, 0x76, 0x3a, 0x65, 0x78, 0x74, 0x65,
+	0x72, 0x6e, 0x61, 0x6c, 0x3a, 0x76, 0x31, 0x22, 0x38, 0x0a, 0x0d, 0x44, 0x6f, 0x63, 0x6b, 0x65,
+	0x72, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x27, 0x0a, 0x0f, 0x63, 0x6f, 0x6e, 0x74,
+	0x61, 0x69, 0x6e, 0x65, 0x72, 0x5f, 0x69, 0x6d, 0x61, 0x67, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28,
+	0x09, 0x52, 0x0e, 0x63, 0x6f, 0x6e, 0x74, 0x61, 0x69, 0x6e, 0x65, 0x72, 0x49, 0x6d, 0x61, 0x67,
+	0x65, 0x22, 0xd4, 0x01, 0x0a, 0x0e, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x50, 0x61, 0x79,
+	0x6c, 0x6f, 0x61, 0x64, 0x12, 0x0e, 0x0a, 0x02, 0x6f, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09,
+	0x52, 0x02, 0x6f, 0x73, 0x12, 0x12, 0x0a, 0x04, 0x61, 0x72, 0x63, 0x68, 0x18, 0x02, 0x20, 0x01,
+	0x28, 0x09, 0x52, 0x04, 0x61, 0x72, 0x63, 0x68, 0x12, 0x18, 0x0a, 0x07, 0x63, 0x6f, 0x6d, 0x6d,
+	0x61, 0x6e, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x63, 0x6f, 0x6d, 0x6d, 0x61,
+	0x6e, 0x64, 0x12, 0x4c, 0x0a, 0x03, 0x65, 0x6e, 0x76, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32,
+	0x3a, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61,
+	0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65,
+	0x2e, 0x76, 0x31, 0x2e, 0x50, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x50, 0x61, 0x79, 0x6c, 0x6f,
+	0x61, 0x64, 0x2e, 0x45, 0x6e, 0x76, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x03, 0x65, 0x6e, 0x76,
+	0x1a, 0x36, 0x0a, 0x08, 0x45, 0x6e, 0x76, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03,
+	0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14,
+	0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76,
+	0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0xf9, 0x01, 0x0a, 0x0f, 0x45, 0x78, 0x74,
+	0x65, 0x72, 0x6e, 0x61, 0x6c, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x53, 0x0a, 0x08,
+	0x65, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x37,
+	0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d,
+	0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e,
+	0x76, 0x31, 0x2e, 0x41, 0x70, 0x69, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x44, 0x65, 0x73,
+	0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x52, 0x08, 0x65, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e,
+	0x74, 0x12, 0x56, 0x0a, 0x06, 0x70, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x18, 0x02, 0x20, 0x03, 0x28,
+	0x0b, 0x32, 0x3e, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62,
+	0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69,
+	0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x45, 0x78, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x50, 0x61,
+	0x79, 0x6c, 0x6f, 0x61, 0x64, 0x2e, 0x50, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x45, 0x6e, 0x74, 0x72,
+	0x79, 0x52, 0x06, 0x70, 0x61, 0x72, 0x61, 0x6d, 0x73, 0x1a, 0x39, 0x0a, 0x0b, 0x50, 0x61, 0x72,
+	0x61, 0x6d, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18,
+	0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61,
+	0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65,
+	0x3a, 0x02, 0x38, 0x01, 0x22, 0xb9, 0x02, 0x0a, 0x11, 0x53, 0x74, 0x61, 0x6e, 0x64, 0x61, 0x72,
+	0x64, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x73, 0x22, 0xa3, 0x02, 0x0a, 0x04, 0x45,
+	0x6e, 0x75, 0x6d, 0x12, 0x48, 0x0a, 0x19, 0x4c, 0x45, 0x47, 0x41, 0x43, 0x59, 0x5f, 0x50, 0x52,
+	0x4f, 0x47, 0x52, 0x45, 0x53, 0x53, 0x5f, 0x52, 0x45, 0x50, 0x4f, 0x52, 0x54, 0x49, 0x4e, 0x47,
+	0x10, 0x00, 0x1a, 0x29, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x23, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x70,
+	0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x3a, 0x70, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73,
+	0x5f, 0x72, 0x65, 0x70, 0x6f, 0x72, 0x74, 0x69, 0x6e, 0x67, 0x3a, 0x76, 0x30, 0x12, 0x41, 0x0a,
+	0x12, 0x50, 0x52, 0x4f, 0x47, 0x52, 0x45, 0x53, 0x53, 0x5f, 0x52, 0x45, 0x50, 0x4f, 0x52, 0x54,
+	0x49, 0x4e, 0x47, 0x10, 0x01, 0x1a, 0x29, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x23, 0x62, 0x65, 0x61,
+	0x6d, 0x3a, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x3a, 0x70, 0x72, 0x6f, 0x67, 0x72,
+	0x65, 0x73, 0x73, 0x5f, 0x72, 0x65, 0x70, 0x6f, 0x72, 0x74, 0x69, 0x6e, 0x67, 0x3a, 0x76, 0x31,
+	0x12, 0x37, 0x0a, 0x0d, 0x57, 0x4f, 0x52, 0x4b, 0x45, 0x52, 0x5f, 0x53, 0x54, 0x41, 0x54, 0x55,
+	0x53, 0x10, 0x02, 0x1a, 0x24, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x1e, 0x62, 0x65, 0x61, 0x6d, 0x3a,
+	0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x3a, 0x77, 0x6f, 0x72, 0x6b, 0x65, 0x72, 0x5f,
+	0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x3a, 0x76, 0x31, 0x12, 0x55, 0x0a, 0x1c, 0x4d, 0x55, 0x4c,
+	0x54, 0x49, 0x5f, 0x43, 0x4f, 0x52, 0x45, 0x5f, 0x42, 0x55, 0x4e, 0x44, 0x4c, 0x45, 0x5f, 0x50,
+	0x52, 0x4f, 0x43, 0x45, 0x53, 0x53, 0x49, 0x4e, 0x47, 0x10, 0x03, 0x1a, 0x33, 0xa2, 0xb4, 0xfa,
+	0xc2, 0x05, 0x2d, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c,
+	0x3a, 0x6d, 0x75, 0x6c, 0x74, 0x69, 0x5f, 0x63, 0x6f, 0x72, 0x65, 0x5f, 0x62, 0x75, 0x6e, 0x64,
+	0x6c, 0x65, 0x5f, 0x70, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x69, 0x6e, 0x67, 0x3a, 0x76, 0x31,
+	0x22, 0xa6, 0x03, 0x0a, 0x14, 0x53, 0x74, 0x61, 0x6e, 0x64, 0x61, 0x72, 0x64, 0x52, 0x65, 0x71,
+	0x75, 0x69, 0x72, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x22, 0x8d, 0x03, 0x0a, 0x04, 0x45, 0x6e,
+	0x75, 0x6d, 0x12, 0x4a, 0x0a, 0x1c, 0x52, 0x45, 0x51, 0x55, 0x49, 0x52, 0x45, 0x53, 0x5f, 0x53,
+	0x54, 0x41, 0x54, 0x45, 0x46, 0x55, 0x4c, 0x5f, 0x50, 0x52, 0x4f, 0x43, 0x45, 0x53, 0x53, 0x49,
+	0x4e, 0x47, 0x10, 0x00, 0x1a, 0x28, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x22, 0x62, 0x65, 0x61, 0x6d,
+	0x3a, 0x72, 0x65, 0x71, 0x75, 0x69, 0x72, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x3a, 0x70, 0x61, 0x72,
+	0x64, 0x6f, 0x3a, 0x73, 0x74, 0x61, 0x74, 0x65, 0x66, 0x75, 0x6c, 0x3a, 0x76, 0x31, 0x12, 0x4e,
+	0x0a, 0x1c, 0x52, 0x45, 0x51, 0x55, 0x49, 0x52, 0x45, 0x53, 0x5f, 0x42, 0x55, 0x4e, 0x44, 0x4c,
+	0x45, 0x5f, 0x46, 0x49, 0x4e, 0x41, 0x4c, 0x49, 0x5a, 0x41, 0x54, 0x49, 0x4f, 0x4e, 0x10, 0x01,
+	0x1a, 0x2c, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x26, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x72, 0x65, 0x71,
+	0x75, 0x69, 0x72, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x3a, 0x70, 0x61, 0x72, 0x64, 0x6f, 0x3a, 0x66,
+	0x69, 0x6e, 0x61, 0x6c, 0x69, 0x7a, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x3a, 0x76, 0x31, 0x12, 0x47,
+	0x0a, 0x15, 0x52, 0x45, 0x51, 0x55, 0x49, 0x52, 0x45, 0x53, 0x5f, 0x53, 0x54, 0x41, 0x42, 0x4c,
+	0x45, 0x5f, 0x49, 0x4e, 0x50, 0x55, 0x54, 0x10, 0x02, 0x1a, 0x2c, 0xa2, 0xb4, 0xfa, 0xc2, 0x05,
+	0x26, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x72, 0x65, 0x71, 0x75, 0x69, 0x72, 0x65, 0x6d, 0x65, 0x6e,
+	0x74, 0x3a, 0x70, 0x61, 0x72, 0x64, 0x6f, 0x3a, 0x73, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x69,
+	0x6e, 0x70, 0x75, 0x74, 0x3a, 0x76, 0x31, 0x12, 0x51, 0x0a, 0x1a, 0x52, 0x45, 0x51, 0x55, 0x49,
+	0x52, 0x45, 0x53, 0x5f, 0x54, 0x49, 0x4d, 0x45, 0x5f, 0x53, 0x4f, 0x52, 0x54, 0x45, 0x44, 0x5f,
+	0x49, 0x4e, 0x50, 0x55, 0x54, 0x10, 0x03, 0x1a, 0x31, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x2b, 0x62,
+	0x65, 0x61, 0x6d, 0x3a, 0x72, 0x65, 0x71, 0x75, 0x69, 0x72, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x3a,
+	0x70, 0x61, 0x72, 0x64, 0x6f, 0x3a, 0x74, 0x69, 0x6d, 0x65, 0x5f, 0x73, 0x6f, 0x72, 0x74, 0x65,
+	0x64, 0x5f, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x3a, 0x76, 0x31, 0x12, 0x4d, 0x0a, 0x18, 0x52, 0x45,
+	0x51, 0x55, 0x49, 0x52, 0x45, 0x53, 0x5f, 0x53, 0x50, 0x4c, 0x49, 0x54, 0x54, 0x41, 0x42, 0x4c,
+	0x45, 0x5f, 0x44, 0x4f, 0x46, 0x4e, 0x10, 0x04, 0x1a, 0x2f, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x29,
+	0x62, 0x65, 0x61, 0x6d, 0x3a, 0x72, 0x65, 0x71, 0x75, 0x69, 0x72, 0x65, 0x6d, 0x65, 0x6e, 0x74,
+	0x3a, 0x70, 0x61, 0x72, 0x64, 0x6f, 0x3a, 0x73, 0x70, 0x6c, 0x69, 0x74, 0x74, 0x61, 0x62, 0x6c,
+	0x65, 0x5f, 0x64, 0x6f, 0x66, 0x6e, 0x3a, 0x76, 0x31, 0x22, 0x3a, 0x0a, 0x0c, 0x46, 0x75, 0x6e,
+	0x63, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x70, 0x65, 0x63, 0x12, 0x10, 0x0a, 0x03, 0x75, 0x72, 0x6e,
+	0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x75, 0x72, 0x6e, 0x12, 0x18, 0x0a, 0x07, 0x70,
+	0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x07, 0x70, 0x61,
+	0x79, 0x6c, 0x6f, 0x61, 0x64, 0x22, 0x65, 0x0a, 0x13, 0x53, 0x74, 0x61, 0x6e, 0x64, 0x61, 0x72,
+	0x64, 0x44, 0x69, 0x73, 0x70, 0x6c, 0x61, 0x79, 0x44, 0x61, 0x74, 0x61, 0x22, 0x4e, 0x0a, 0x0b,
+	0x44, 0x69, 0x73, 0x70, 0x6c, 0x61, 0x79, 0x44, 0x61, 0x74, 0x61, 0x12, 0x3f, 0x0a, 0x0f, 0x4c,
+	0x41, 0x42, 0x45, 0x4c, 0x4c, 0x45, 0x44, 0x5f, 0x53, 0x54, 0x52, 0x49, 0x4e, 0x47, 0x10, 0x00,
+	0x1a, 0x2a, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x24, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x64, 0x69, 0x73,
+	0x70, 0x6c, 0x61, 0x79, 0x5f, 0x64, 0x61, 0x74, 0x61, 0x3a, 0x6c, 0x61, 0x62, 0x65, 0x6c, 0x6c,
+	0x65, 0x64, 0x5f, 0x73, 0x74, 0x72, 0x69, 0x6e, 0x67, 0x3a, 0x76, 0x31, 0x22, 0x43, 0x0a, 0x15,
+	0x4c, 0x61, 0x62, 0x65, 0x6c, 0x6c, 0x65, 0x64, 0x53, 0x74, 0x72, 0x69, 0x6e, 0x67, 0x50, 0x61,
+	0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x14, 0x0a, 0x05, 0x6c, 0x61, 0x62, 0x65, 0x6c, 0x18, 0x01,
+	0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x6c, 0x61, 0x62, 0x65, 0x6c, 0x12, 0x14, 0x0a, 0x05, 0x76,
+	0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75,
+	0x65, 0x22, 0x39, 0x0a, 0x0b, 0x44, 0x69, 0x73, 0x70, 0x6c, 0x61, 0x79, 0x44, 0x61, 0x74, 0x61,
+	0x12, 0x10, 0x0a, 0x03, 0x75, 0x72, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x75,
+	0x72, 0x6e, 0x12, 0x18, 0x0a, 0x07, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x18, 0x02, 0x20,
+	0x01, 0x28, 0x0c, 0x52, 0x07, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x22, 0xd7, 0x07, 0x0a,
+	0x15, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x57, 0x69, 0x74, 0x68, 0x43, 0x6f, 0x6d, 0x70,
+	0x6f, 0x6e, 0x65, 0x6e, 0x74, 0x73, 0x12, 0x4d, 0x0a, 0x0a, 0x63, 0x6f, 0x6d, 0x70, 0x6f, 0x6e,
+	0x65, 0x6e, 0x74, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2d, 0x2e, 0x6f, 0x72, 0x67,
+	0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64,
+	0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x43,
+	0x6f, 0x6d, 0x70, 0x6f, 0x6e, 0x65, 0x6e, 0x74, 0x73, 0x52, 0x0a, 0x63, 0x6f, 0x6d, 0x70, 0x6f,
+	0x6e, 0x65, 0x6e, 0x74, 0x73, 0x12, 0x40, 0x0a, 0x05, 0x63, 0x6f, 0x64, 0x65, 0x72, 0x18, 0x02,
+	0x20, 0x01, 0x28, 0x0b, 0x32, 0x28, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68,
+	0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70,
+	0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x6f, 0x64, 0x65, 0x72, 0x48, 0x00,
+	0x52, 0x05, 0x63, 0x6f, 0x64, 0x65, 0x72, 0x12, 0x5c, 0x0a, 0x0f, 0x63, 0x6f, 0x6d, 0x62, 0x69,
+	0x6e, 0x65, 0x5f, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b,
+	0x32, 0x31, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65,
+	0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e,
+	0x65, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x6f, 0x6d, 0x62, 0x69, 0x6e, 0x65, 0x50, 0x61, 0x79, 0x6c,
+	0x6f, 0x61, 0x64, 0x48, 0x00, 0x52, 0x0e, 0x63, 0x6f, 0x6d, 0x62, 0x69, 0x6e, 0x65, 0x50, 0x61,
+	0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x56, 0x0a, 0x0d, 0x66, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f,
+	0x6e, 0x5f, 0x73, 0x70, 0x65, 0x63, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2f, 0x2e, 0x6f,
+	0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d,
+	0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31,
+	0x2e, 0x46, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x70, 0x65, 0x63, 0x48, 0x00, 0x52,
+	0x0c, 0x66, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x70, 0x65, 0x63, 0x12, 0x57, 0x0a,
+	0x0e, 0x70, 0x61, 0x72, 0x5f, 0x64, 0x6f, 0x5f, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x18,
+	0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2f, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63,
+	0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69,
+	0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x50, 0x61, 0x72, 0x44, 0x6f, 0x50,
+	0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x48, 0x00, 0x52, 0x0c, 0x70, 0x61, 0x72, 0x44, 0x6f, 0x50,
+	0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x4f, 0x0a, 0x0a, 0x70, 0x74, 0x72, 0x61, 0x6e, 0x73,
+	0x66, 0x6f, 0x72, 0x6d, 0x18, 0x07, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2d, 0x2e, 0x6f, 0x72, 0x67,
+	0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64,
+	0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x50,
+	0x54, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x48, 0x00, 0x52, 0x0a, 0x70, 0x74, 0x72,
+	0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x12, 0x52, 0x0a, 0x0b, 0x70, 0x63, 0x6f, 0x6c, 0x6c,
+	0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x08, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2e, 0x2e, 0x6f,
+	0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d,
+	0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31,
+	0x2e, 0x50, 0x43, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x48, 0x00, 0x52, 0x0b,
+	0x70, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x53, 0x0a, 0x0c, 0x72,
+	0x65, 0x61, 0x64, 0x5f, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x18, 0x09, 0x20, 0x01, 0x28,
+	0x0b, 0x32, 0x2e, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62,
+	0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69,
+	0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x65, 0x61, 0x64, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61,
+	0x64, 0x48, 0x00, 0x52, 0x0b, 0x72, 0x65, 0x61, 0x64, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64,
+	0x12, 0x4d, 0x0a, 0x0a, 0x73, 0x69, 0x64, 0x65, 0x5f, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x18, 0x0b,
+	0x20, 0x01, 0x28, 0x0b, 0x32, 0x2c, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68,
+	0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70,
+	0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x69, 0x64, 0x65, 0x49, 0x6e, 0x70,
+	0x75, 0x74, 0x48, 0x00, 0x52, 0x09, 0x73, 0x69, 0x64, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x12,
+	0x66, 0x0a, 0x13, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x5f, 0x69, 0x6e, 0x74, 0x6f, 0x5f, 0x70,
+	0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x18, 0x0c, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x34, 0x2e, 0x6f,
+	0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d,
+	0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31,
+	0x2e, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x49, 0x6e, 0x74, 0x6f, 0x50, 0x61, 0x79, 0x6c, 0x6f,
+	0x61, 0x64, 0x48, 0x00, 0x52, 0x11, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x49, 0x6e, 0x74, 0x6f,
+	0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x65, 0x0a, 0x12, 0x77, 0x69, 0x6e, 0x64, 0x6f,
+	0x77, 0x69, 0x6e, 0x67, 0x5f, 0x73, 0x74, 0x72, 0x61, 0x74, 0x65, 0x67, 0x79, 0x18, 0x0d, 0x20,
+	0x01, 0x28, 0x0b, 0x32, 0x34, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65,
+	0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65,
+	0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x69, 0x6e,
+	0x67, 0x53, 0x74, 0x72, 0x61, 0x74, 0x65, 0x67, 0x79, 0x48, 0x00, 0x52, 0x11, 0x77, 0x69, 0x6e,
+	0x64, 0x6f, 0x77, 0x69, 0x6e, 0x67, 0x53, 0x74, 0x72, 0x61, 0x74, 0x65, 0x67, 0x79, 0x42, 0x06,
+	0x0a, 0x04, 0x72, 0x6f, 0x6f, 0x74, 0x22, 0xb6, 0x0a, 0x0a, 0x16, 0x45, 0x78, 0x65, 0x63, 0x75,
+	0x74, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x74, 0x61, 0x67, 0x65, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61,
+	0x64, 0x12, 0x50, 0x0a, 0x0b, 0x65, 0x6e, 0x76, 0x69, 0x72, 0x6f, 0x6e, 0x6d, 0x65, 0x6e, 0x74,
+	0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2e, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61,
+	0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70,
+	0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x45, 0x6e, 0x76, 0x69, 0x72,
+	0x6f, 0x6e, 0x6d, 0x65, 0x6e, 0x74, 0x52, 0x0b, 0x65, 0x6e, 0x76, 0x69, 0x72, 0x6f, 0x6e, 0x6d,
+	0x65, 0x6e, 0x74, 0x12, 0x7a, 0x0a, 0x13, 0x77, 0x69, 0x72, 0x65, 0x5f, 0x63, 0x6f, 0x64, 0x65,
+	0x72, 0x5f, 0x73, 0x65, 0x74, 0x74, 0x69, 0x6e, 0x67, 0x73, 0x18, 0x09, 0x20, 0x03, 0x28, 0x0b,
+	0x32, 0x4a, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65,
+	0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e,
+	0x65, 0x2e, 0x76, 0x31, 0x2e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x53,
+	0x74, 0x61, 0x67, 0x65, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x2e, 0x57, 0x69, 0x72, 0x65,
+	0x43, 0x6f, 0x64, 0x65, 0x72, 0x53, 0x65, 0x74, 0x74, 0x69, 0x6e, 0x67, 0x52, 0x11, 0x77, 0x69,
+	0x72, 0x65, 0x43, 0x6f, 0x64, 0x65, 0x72, 0x53, 0x65, 0x74, 0x74, 0x69, 0x6e, 0x67, 0x73, 0x12,
+	0x14, 0x0a, 0x05, 0x69, 0x6e, 0x70, 0x75, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x05,
+	0x69, 0x6e, 0x70, 0x75, 0x74, 0x12, 0x66, 0x0a, 0x0b, 0x73, 0x69, 0x64, 0x65, 0x5f, 0x69, 0x6e,
+	0x70, 0x75, 0x74, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x45, 0x2e, 0x6f, 0x72, 0x67,
+	0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64,
+	0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x45,
+	0x78, 0x65, 0x63, 0x75, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x74, 0x61, 0x67, 0x65, 0x50, 0x61,
+	0x79, 0x6c, 0x6f, 0x61, 0x64, 0x2e, 0x53, 0x69, 0x64, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x49,
+	0x64, 0x52, 0x0a, 0x73, 0x69, 0x64, 0x65, 0x49, 0x6e, 0x70, 0x75, 0x74, 0x73, 0x12, 0x1e, 0x0a,
+	0x0a, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28,
+	0x09, 0x52, 0x0a, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x73, 0x12, 0x18, 0x0a,
+	0x07, 0x6f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x09, 0x52, 0x07,
+	0x6f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x73, 0x12, 0x4d, 0x0a, 0x0a, 0x63, 0x6f, 0x6d, 0x70, 0x6f,
+	0x6e, 0x65, 0x6e, 0x74, 0x73, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2d, 0x2e, 0x6f, 0x72,
+	0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f,
+	0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e,
+	0x43, 0x6f, 0x6d, 0x70, 0x6f, 0x6e, 0x65, 0x6e, 0x74, 0x73, 0x52, 0x0a, 0x63, 0x6f, 0x6d, 0x70,
+	0x6f, 0x6e, 0x65, 0x6e, 0x74, 0x73, 0x12, 0x66, 0x0a, 0x0b, 0x75, 0x73, 0x65, 0x72, 0x5f, 0x73,
+	0x74, 0x61, 0x74, 0x65, 0x73, 0x18, 0x07, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x45, 0x2e, 0x6f, 0x72,
+	0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f,
+	0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e,
+	0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x74, 0x61, 0x67, 0x65, 0x50,
+	0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x2e, 0x55, 0x73, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x65,
+	0x49, 0x64, 0x52, 0x0a, 0x75, 0x73, 0x65, 0x72, 0x53, 0x74, 0x61, 0x74, 0x65, 0x73, 0x12, 0x59,
+	0x0a, 0x06, 0x74, 0x69, 0x6d, 0x65, 0x72, 0x73, 0x18, 0x08, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x41,
+	0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d,
+	0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e,
+	0x76, 0x31, 0x2e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x74, 0x61,
+	0x67, 0x65, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x72, 0x49,
+	0x64, 0x52, 0x06, 0x74, 0x69, 0x6d, 0x65, 0x72, 0x73, 0x12, 0x6d, 0x0a, 0x0d, 0x74, 0x69, 0x6d,
+	0x65, 0x72, 0x46, 0x61, 0x6d, 0x69, 0x6c, 0x69, 0x65, 0x73, 0x18, 0x0a, 0x20, 0x03, 0x28, 0x0b,
+	0x32, 0x47, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65,
+	0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e,
+	0x65, 0x2e, 0x76, 0x31, 0x2e, 0x45, 0x78, 0x65, 0x63, 0x75, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x53,
+	0x74, 0x61, 0x67, 0x65, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x2e, 0x54, 0x69, 0x6d, 0x65,
+	0x72, 0x46, 0x61, 0x6d, 0x69, 0x6c, 0x79, 0x49, 0x64, 0x52, 0x0d, 0x74, 0x69, 0x6d, 0x65, 0x72,
+	0x46, 0x61, 0x6d, 0x69, 0x6c, 0x69, 0x65, 0x73, 0x1a, 0x4f, 0x0a, 0x0b, 0x53, 0x69, 0x64, 0x65,
+	0x49, 0x6e, 0x70, 0x75, 0x74, 0x49, 0x64, 0x12, 0x21, 0x0a, 0x0c, 0x74, 0x72, 0x61, 0x6e, 0x73,
+	0x66, 0x6f, 0x72, 0x6d, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x74,
+	0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x49, 0x64, 0x12, 0x1d, 0x0a, 0x0a, 0x6c, 0x6f,
+	0x63, 0x61, 0x6c, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09,
+	0x6c, 0x6f, 0x63, 0x61, 0x6c, 0x4e, 0x61, 0x6d, 0x65, 0x1a, 0x4f, 0x0a, 0x0b, 0x55, 0x73, 0x65,
+	0x72, 0x53, 0x74, 0x61, 0x74, 0x65, 0x49, 0x64, 0x12, 0x21, 0x0a, 0x0c, 0x74, 0x72, 0x61, 0x6e,
+	0x73, 0x66, 0x6f, 0x72, 0x6d, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b,
+	0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x49, 0x64, 0x12, 0x1d, 0x0a, 0x0a, 0x6c,
+	0x6f, 0x63, 0x61, 0x6c, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52,
+	0x09, 0x6c, 0x6f, 0x63, 0x61, 0x6c, 0x4e, 0x61, 0x6d, 0x65, 0x1a, 0x4b, 0x0a, 0x07, 0x54, 0x69,
+	0x6d, 0x65, 0x72, 0x49, 0x64, 0x12, 0x21, 0x0a, 0x0c, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f,
+	0x72, 0x6d, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x74, 0x72, 0x61,
+	0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x49, 0x64, 0x12, 0x1d, 0x0a, 0x0a, 0x6c, 0x6f, 0x63, 0x61,
+	0x6c, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6c, 0x6f,
+	0x63, 0x61, 0x6c, 0x4e, 0x61, 0x6d, 0x65, 0x1a, 0x51, 0x0a, 0x0d, 0x54, 0x69, 0x6d, 0x65, 0x72,
+	0x46, 0x61, 0x6d, 0x69, 0x6c, 0x79, 0x49, 0x64, 0x12, 0x21, 0x0a, 0x0c, 0x74, 0x72, 0x61, 0x6e,
+	0x73, 0x66, 0x6f, 0x72, 0x6d, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b,
+	0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x49, 0x64, 0x12, 0x1d, 0x0a, 0x0a, 0x6c,
+	0x6f, 0x63, 0x61, 0x6c, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52,
+	0x09, 0x6c, 0x6f, 0x63, 0x61, 0x6c, 0x4e, 0x61, 0x6d, 0x65, 0x1a, 0xd2, 0x01, 0x0a, 0x10, 0x57,
+	0x69, 0x72, 0x65, 0x43, 0x6f, 0x64, 0x65, 0x72, 0x53, 0x65, 0x74, 0x74, 0x69, 0x6e, 0x67, 0x12,
+	0x10, 0x0a, 0x03, 0x75, 0x72, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x75, 0x72,
+	0x6e, 0x12, 0x18, 0x0a, 0x07, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x18, 0x02, 0x20, 0x01,
+	0x28, 0x0c, 0x52, 0x07, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x2d, 0x0a, 0x12, 0x69,
+	0x6e, 0x70, 0x75, 0x74, 0x5f, 0x6f, 0x72, 0x5f, 0x6f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x5f, 0x69,
+	0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x48, 0x00, 0x52, 0x0f, 0x69, 0x6e, 0x70, 0x75, 0x74,
+	0x4f, 0x72, 0x4f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x49, 0x64, 0x12, 0x59, 0x0a, 0x05, 0x74, 0x69,
+	0x6d, 0x65, 0x72, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x41, 0x2e, 0x6f, 0x72, 0x67, 0x2e,
+	0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65,
+	0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x45, 0x78,
+	0x65, 0x63, 0x75, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x74, 0x61, 0x67, 0x65, 0x50, 0x61, 0x79,
+	0x6c, 0x6f, 0x61, 0x64, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x72, 0x49, 0x64, 0x48, 0x00, 0x52, 0x05,
+	0x74, 0x69, 0x6d, 0x65, 0x72, 0x42, 0x08, 0x0a, 0x06, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x32,
+	0x8f, 0x01, 0x0a, 0x11, 0x54, 0x65, 0x73, 0x74, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x53, 0x65,
+	0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x7a, 0x0a, 0x06, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x12,
+	0x30, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61,
+	0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65,
+	0x2e, 0x76, 0x31, 0x2e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73,
+	0x74, 0x1a, 0x3a, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62,
+	0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69,
+	0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x54, 0x65, 0x73, 0x74, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d,
+	0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x2e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x22, 0x00, 0x30,
+	0x01, 0x3a, 0x3f, 0x0a, 0x08, 0x62, 0x65, 0x61, 0x6d, 0x5f, 0x75, 0x72, 0x6e, 0x12, 0x21, 0x2e,
+	0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e,
+	0x45, 0x6e, 0x75, 0x6d, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73,
+	0x18, 0xc4, 0xa6, 0xaf, 0x58, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x62, 0x65, 0x61, 0x6d, 0x55,
+	0x72, 0x6e, 0x3a, 0x49, 0x0a, 0x0d, 0x62, 0x65, 0x61, 0x6d, 0x5f, 0x63, 0x6f, 0x6e, 0x73, 0x74,
+	0x61, 0x6e, 0x74, 0x12, 0x21, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f,
+	0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6e, 0x75, 0x6d, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x4f,
+	0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0xc5, 0xa6, 0xaf, 0x58, 0x20, 0x01, 0x28, 0x09, 0x52,
+	0x0c, 0x62, 0x65, 0x61, 0x6d, 0x43, 0x6f, 0x6e, 0x73, 0x74, 0x61, 0x6e, 0x74, 0x42, 0x75, 0x0a,
+	0x21, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d,
+	0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e,
+	0x76, 0x31, 0x42, 0x09, 0x52, 0x75, 0x6e, 0x6e, 0x65, 0x72, 0x41, 0x70, 0x69, 0x5a, 0x45, 0x67,
+	0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65,
+	0x2f, 0x62, 0x65, 0x61, 0x6d, 0x2f, 0x73, 0x64, 0x6b, 0x73, 0x2f, 0x67, 0x6f, 0x2f, 0x70, 0x6b,
+	0x67, 0x2f, 0x62, 0x65, 0x61, 0x6d, 0x2f, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2f, 0x70, 0x69, 0x70,
+	0x65, 0x6c, 0x69, 0x6e, 0x65, 0x5f, 0x76, 0x31, 0x3b, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e,
+	0x65, 0x5f, 0x76, 0x31, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
+}
+
+var (
+	file_beam_runner_api_proto_rawDescOnce sync.Once
+	file_beam_runner_api_proto_rawDescData = file_beam_runner_api_proto_rawDesc
+)
+
+func file_beam_runner_api_proto_rawDescGZIP() []byte {
+	file_beam_runner_api_proto_rawDescOnce.Do(func() {
+		file_beam_runner_api_proto_rawDescData = protoimpl.X.CompressGZIP(file_beam_runner_api_proto_rawDescData)
+	})
+	return file_beam_runner_api_proto_rawDescData
+}
+
+var file_beam_runner_api_proto_enumTypes = make([]protoimpl.EnumInfo, 21)
+var file_beam_runner_api_proto_msgTypes = make([]protoimpl.MessageInfo, 96)
+var file_beam_runner_api_proto_goTypes = []interface{}{
+	(BeamConstants_Constants)(0),                       // 0: org.apache.beam.model.pipeline.v1.BeamConstants.Constants
+	(StandardPTransforms_Primitives)(0),                // 1: org.apache.beam.model.pipeline.v1.StandardPTransforms.Primitives
+	(StandardPTransforms_DeprecatedPrimitives)(0),      // 2: org.apache.beam.model.pipeline.v1.StandardPTransforms.DeprecatedPrimitives
+	(StandardPTransforms_Composites)(0),                // 3: org.apache.beam.model.pipeline.v1.StandardPTransforms.Composites
+	(StandardPTransforms_CombineComponents)(0),         // 4: org.apache.beam.model.pipeline.v1.StandardPTransforms.CombineComponents
+	(StandardPTransforms_SplittableParDoComponents)(0), // 5: org.apache.beam.model.pipeline.v1.StandardPTransforms.SplittableParDoComponents
+	(StandardSideInputTypes_Enum)(0),                   // 6: org.apache.beam.model.pipeline.v1.StandardSideInputTypes.Enum
+	(IsBounded_Enum)(0),                                // 7: org.apache.beam.model.pipeline.v1.IsBounded.Enum
+	(StandardCoders_Enum)(0),                           // 8: org.apache.beam.model.pipeline.v1.StandardCoders.Enum
+	(MergeStatus_Enum)(0),                              // 9: org.apache.beam.model.pipeline.v1.MergeStatus.Enum
+	(AccumulationMode_Enum)(0),                         // 10: org.apache.beam.model.pipeline.v1.AccumulationMode.Enum
+	(ClosingBehavior_Enum)(0),                          // 11: org.apache.beam.model.pipeline.v1.ClosingBehavior.Enum
+	(OnTimeBehavior_Enum)(0),                           // 12: org.apache.beam.model.pipeline.v1.OnTimeBehavior.Enum
+	(OutputTime_Enum)(0),                               // 13: org.apache.beam.model.pipeline.v1.OutputTime.Enum
+	(TimeDomain_Enum)(0),                               // 14: org.apache.beam.model.pipeline.v1.TimeDomain.Enum
+	(StandardArtifacts_Types)(0),                       // 15: org.apache.beam.model.pipeline.v1.StandardArtifacts.Types
+	(StandardArtifacts_Roles)(0),                       // 16: org.apache.beam.model.pipeline.v1.StandardArtifacts.Roles
+	(StandardEnvironments_Environments)(0),             // 17: org.apache.beam.model.pipeline.v1.StandardEnvironments.Environments
+	(StandardProtocols_Enum)(0),                        // 18: org.apache.beam.model.pipeline.v1.StandardProtocols.Enum
+	(StandardRequirements_Enum)(0),                     // 19: org.apache.beam.model.pipeline.v1.StandardRequirements.Enum
+	(StandardDisplayData_DisplayData)(0),               // 20: org.apache.beam.model.pipeline.v1.StandardDisplayData.DisplayData
+	(*BeamConstants)(nil),                              // 21: org.apache.beam.model.pipeline.v1.BeamConstants
+	(*Components)(nil),                                 // 22: org.apache.beam.model.pipeline.v1.Components
+	(*Pipeline)(nil),                                   // 23: org.apache.beam.model.pipeline.v1.Pipeline
+	(*PTransform)(nil),                                 // 24: org.apache.beam.model.pipeline.v1.PTransform
+	(*StandardPTransforms)(nil),                        // 25: org.apache.beam.model.pipeline.v1.StandardPTransforms
+	(*StandardSideInputTypes)(nil),                     // 26: org.apache.beam.model.pipeline.v1.StandardSideInputTypes
+	(*PCollection)(nil),                                // 27: org.apache.beam.model.pipeline.v1.PCollection
+	(*ParDoPayload)(nil),                               // 28: org.apache.beam.model.pipeline.v1.ParDoPayload
+	(*StateSpec)(nil),                                  // 29: org.apache.beam.model.pipeline.v1.StateSpec
+	(*ReadModifyWriteStateSpec)(nil),                   // 30: org.apache.beam.model.pipeline.v1.ReadModifyWriteStateSpec
+	(*BagStateSpec)(nil),                               // 31: org.apache.beam.model.pipeline.v1.BagStateSpec
+	(*OrderedListStateSpec)(nil),                       // 32: org.apache.beam.model.pipeline.v1.OrderedListStateSpec
+	(*CombiningStateSpec)(nil),                         // 33: org.apache.beam.model.pipeline.v1.CombiningStateSpec
+	(*MapStateSpec)(nil),                               // 34: org.apache.beam.model.pipeline.v1.MapStateSpec
+	(*SetStateSpec)(nil),                               // 35: org.apache.beam.model.pipeline.v1.SetStateSpec
+	(*TimerFamilySpec)(nil),                            // 36: org.apache.beam.model.pipeline.v1.TimerFamilySpec
+	(*IsBounded)(nil),                                  // 37: org.apache.beam.model.pipeline.v1.IsBounded
+	(*ReadPayload)(nil),                                // 38: org.apache.beam.model.pipeline.v1.ReadPayload
+	(*WindowIntoPayload)(nil),                          // 39: org.apache.beam.model.pipeline.v1.WindowIntoPayload
+	(*CombinePayload)(nil),                             // 40: org.apache.beam.model.pipeline.v1.CombinePayload
+	(*TestStreamPayload)(nil),                          // 41: org.apache.beam.model.pipeline.v1.TestStreamPayload
+	(*EventsRequest)(nil),                              // 42: org.apache.beam.model.pipeline.v1.EventsRequest
+	(*WriteFilesPayload)(nil),                          // 43: org.apache.beam.model.pipeline.v1.WriteFilesPayload
+	(*PubSubReadPayload)(nil),                          // 44: org.apache.beam.model.pipeline.v1.PubSubReadPayload
+	(*PubSubWritePayload)(nil),                         // 45: org.apache.beam.model.pipeline.v1.PubSubWritePayload
+	(*Coder)(nil),                                      // 46: org.apache.beam.model.pipeline.v1.Coder
+	(*StandardCoders)(nil),                             // 47: org.apache.beam.model.pipeline.v1.StandardCoders
+	(*WindowingStrategy)(nil),                          // 48: org.apache.beam.model.pipeline.v1.WindowingStrategy
+	(*MergeStatus)(nil),                                // 49: org.apache.beam.model.pipeline.v1.MergeStatus
+	(*AccumulationMode)(nil),                           // 50: org.apache.beam.model.pipeline.v1.AccumulationMode
+	(*ClosingBehavior)(nil),                            // 51: org.apache.beam.model.pipeline.v1.ClosingBehavior
+	(*OnTimeBehavior)(nil),                             // 52: org.apache.beam.model.pipeline.v1.OnTimeBehavior
+	(*OutputTime)(nil),                                 // 53: org.apache.beam.model.pipeline.v1.OutputTime
+	(*TimeDomain)(nil),                                 // 54: org.apache.beam.model.pipeline.v1.TimeDomain
+	(*Trigger)(nil),                                    // 55: org.apache.beam.model.pipeline.v1.Trigger
+	(*TimestampTransform)(nil),                         // 56: org.apache.beam.model.pipeline.v1.TimestampTransform
+	(*SideInput)(nil),                                  // 57: org.apache.beam.model.pipeline.v1.SideInput
+	(*StandardArtifacts)(nil),                          // 58: org.apache.beam.model.pipeline.v1.StandardArtifacts
+	(*ArtifactFilePayload)(nil),                        // 59: org.apache.beam.model.pipeline.v1.ArtifactFilePayload
+	(*ArtifactUrlPayload)(nil),                         // 60: org.apache.beam.model.pipeline.v1.ArtifactUrlPayload
+	(*EmbeddedFilePayload)(nil),                        // 61: org.apache.beam.model.pipeline.v1.EmbeddedFilePayload
+	(*PyPIPayload)(nil),                                // 62: org.apache.beam.model.pipeline.v1.PyPIPayload
+	(*MavenPayload)(nil),                               // 63: org.apache.beam.model.pipeline.v1.MavenPayload
+	(*DeferredArtifactPayload)(nil),                    // 64: org.apache.beam.model.pipeline.v1.DeferredArtifactPayload
+	(*ArtifactStagingToRolePayload)(nil),               // 65: org.apache.beam.model.pipeline.v1.ArtifactStagingToRolePayload
+	(*ArtifactInformation)(nil),                        // 66: org.apache.beam.model.pipeline.v1.ArtifactInformation
+	(*Environment)(nil),                                // 67: org.apache.beam.model.pipeline.v1.Environment
+	(*StandardEnvironments)(nil),                       // 68: org.apache.beam.model.pipeline.v1.StandardEnvironments
+	(*DockerPayload)(nil),                              // 69: org.apache.beam.model.pipeline.v1.DockerPayload
+	(*ProcessPayload)(nil),                             // 70: org.apache.beam.model.pipeline.v1.ProcessPayload
+	(*ExternalPayload)(nil),                            // 71: org.apache.beam.model.pipeline.v1.ExternalPayload
+	(*StandardProtocols)(nil),                          // 72: org.apache.beam.model.pipeline.v1.StandardProtocols
+	(*StandardRequirements)(nil),                       // 73: org.apache.beam.model.pipeline.v1.StandardRequirements
+	(*FunctionSpec)(nil),                               // 74: org.apache.beam.model.pipeline.v1.FunctionSpec
+	(*StandardDisplayData)(nil),                        // 75: org.apache.beam.model.pipeline.v1.StandardDisplayData
+	(*LabelledStringPayload)(nil),                      // 76: org.apache.beam.model.pipeline.v1.LabelledStringPayload
+	(*DisplayData)(nil),                                // 77: org.apache.beam.model.pipeline.v1.DisplayData
+	(*MessageWithComponents)(nil),                      // 78: org.apache.beam.model.pipeline.v1.MessageWithComponents
+	(*ExecutableStagePayload)(nil),                     // 79: org.apache.beam.model.pipeline.v1.ExecutableStagePayload
+	nil,                                                // 80: org.apache.beam.model.pipeline.v1.Components.TransformsEntry
+	nil,                                                // 81: org.apache.beam.model.pipeline.v1.Components.PcollectionsEntry
+	nil,                                                // 82: org.apache.beam.model.pipeline.v1.Components.WindowingStrategiesEntry
+	nil,                                                // 83: org.apache.beam.model.pipeline.v1.Components.CodersEntry
+	nil,                                                // 84: org.apache.beam.model.pipeline.v1.Components.EnvironmentsEntry
+	nil,                                                // 85: org.apache.beam.model.pipeline.v1.PTransform.InputsEntry
+	nil,                                                // 86: org.apache.beam.model.pipeline.v1.PTransform.OutputsEntry
+	nil,                                                // 87: org.apache.beam.model.pipeline.v1.ParDoPayload.SideInputsEntry
+	nil,                                                // 88: org.apache.beam.model.pipeline.v1.ParDoPayload.StateSpecsEntry
+	nil,                                                // 89: org.apache.beam.model.pipeline.v1.ParDoPayload.TimerFamilySpecsEntry
+	(*TestStreamPayload_Event)(nil),                    // 90: org.apache.beam.model.pipeline.v1.TestStreamPayload.Event
+	(*TestStreamPayload_TimestampedElement)(nil),       // 91: org.apache.beam.model.pipeline.v1.TestStreamPayload.TimestampedElement
+	(*TestStreamPayload_Event_AdvanceWatermark)(nil),   // 92: org.apache.beam.model.pipeline.v1.TestStreamPayload.Event.AdvanceWatermark
+	(*TestStreamPayload_Event_AdvanceProcessingTime)(nil), // 93: org.apache.beam.model.pipeline.v1.TestStreamPayload.Event.AdvanceProcessingTime
+	(*TestStreamPayload_Event_AddElements)(nil),           // 94: org.apache.beam.model.pipeline.v1.TestStreamPayload.Event.AddElements
+	nil,                                             // 95: org.apache.beam.model.pipeline.v1.WriteFilesPayload.SideInputsEntry
+	(*Trigger_AfterAll)(nil),                        // 96: org.apache.beam.model.pipeline.v1.Trigger.AfterAll
+	(*Trigger_AfterAny)(nil),                        // 97: org.apache.beam.model.pipeline.v1.Trigger.AfterAny
+	(*Trigger_AfterEach)(nil),                       // 98: org.apache.beam.model.pipeline.v1.Trigger.AfterEach
+	(*Trigger_AfterEndOfWindow)(nil),                // 99: org.apache.beam.model.pipeline.v1.Trigger.AfterEndOfWindow
+	(*Trigger_AfterProcessingTime)(nil),             // 100: org.apache.beam.model.pipeline.v1.Trigger.AfterProcessingTime
+	(*Trigger_AfterSynchronizedProcessingTime)(nil), // 101: org.apache.beam.model.pipeline.v1.Trigger.AfterSynchronizedProcessingTime
+	(*Trigger_Default)(nil),                         // 102: org.apache.beam.model.pipeline.v1.Trigger.Default
+	(*Trigger_ElementCount)(nil),                    // 103: org.apache.beam.model.pipeline.v1.Trigger.ElementCount
+	(*Trigger_Never)(nil),                           // 104: org.apache.beam.model.pipeline.v1.Trigger.Never
+	(*Trigger_Always)(nil),                          // 105: org.apache.beam.model.pipeline.v1.Trigger.Always
+	(*Trigger_OrFinally)(nil),                       // 106: org.apache.beam.model.pipeline.v1.Trigger.OrFinally
+	(*Trigger_Repeat)(nil),                          // 107: org.apache.beam.model.pipeline.v1.Trigger.Repeat
+	(*TimestampTransform_Delay)(nil),                // 108: org.apache.beam.model.pipeline.v1.TimestampTransform.Delay
+	(*TimestampTransform_AlignTo)(nil),              // 109: org.apache.beam.model.pipeline.v1.TimestampTransform.AlignTo
+	nil,                                             // 110: org.apache.beam.model.pipeline.v1.ProcessPayload.EnvEntry
+	nil,                                             // 111: org.apache.beam.model.pipeline.v1.ExternalPayload.ParamsEntry
+	(*ExecutableStagePayload_SideInputId)(nil),      // 112: org.apache.beam.model.pipeline.v1.ExecutableStagePayload.SideInputId
+	(*ExecutableStagePayload_UserStateId)(nil),      // 113: org.apache.beam.model.pipeline.v1.ExecutableStagePayload.UserStateId
+	(*ExecutableStagePayload_TimerId)(nil),          // 114: org.apache.beam.model.pipeline.v1.ExecutableStagePayload.TimerId
+	(*ExecutableStagePayload_TimerFamilyId)(nil),    // 115: org.apache.beam.model.pipeline.v1.ExecutableStagePayload.TimerFamilyId
+	(*ExecutableStagePayload_WireCoderSetting)(nil), // 116: org.apache.beam.model.pipeline.v1.ExecutableStagePayload.WireCoderSetting
+	(*ApiServiceDescriptor)(nil),                    // 117: org.apache.beam.model.pipeline.v1.ApiServiceDescriptor
+	(*descriptor.EnumValueOptions)(nil),             // 118: google.protobuf.EnumValueOptions
+}
+var file_beam_runner_api_proto_depIdxs = []int32{
+	80,  // 0: org.apache.beam.model.pipeline.v1.Components.transforms:type_name -> org.apache.beam.model.pipeline.v1.Components.TransformsEntry
+	81,  // 1: org.apache.beam.model.pipeline.v1.Components.pcollections:type_name -> org.apache.beam.model.pipeline.v1.Components.PcollectionsEntry
+	82,  // 2: org.apache.beam.model.pipeline.v1.Components.windowing_strategies:type_name -> org.apache.beam.model.pipeline.v1.Components.WindowingStrategiesEntry
+	83,  // 3: org.apache.beam.model.pipeline.v1.Components.coders:type_name -> org.apache.beam.model.pipeline.v1.Components.CodersEntry
+	84,  // 4: org.apache.beam.model.pipeline.v1.Components.environments:type_name -> org.apache.beam.model.pipeline.v1.Components.EnvironmentsEntry
+	22,  // 5: org.apache.beam.model.pipeline.v1.Pipeline.components:type_name -> org.apache.beam.model.pipeline.v1.Components
+	77,  // 6: org.apache.beam.model.pipeline.v1.Pipeline.display_data:type_name -> org.apache.beam.model.pipeline.v1.DisplayData
+	74,  // 7: org.apache.beam.model.pipeline.v1.PTransform.spec:type_name -> org.apache.beam.model.pipeline.v1.FunctionSpec
+	85,  // 8: org.apache.beam.model.pipeline.v1.PTransform.inputs:type_name -> org.apache.beam.model.pipeline.v1.PTransform.InputsEntry
+	86,  // 9: org.apache.beam.model.pipeline.v1.PTransform.outputs:type_name -> org.apache.beam.model.pipeline.v1.PTransform.OutputsEntry
+	77,  // 10: org.apache.beam.model.pipeline.v1.PTransform.display_data:type_name -> org.apache.beam.model.pipeline.v1.DisplayData
+	7,   // 11: org.apache.beam.model.pipeline.v1.PCollection.is_bounded:type_name -> org.apache.beam.model.pipeline.v1.IsBounded.Enum
+	77,  // 12: org.apache.beam.model.pipeline.v1.PCollection.display_data:type_name -> org.apache.beam.model.pipeline.v1.DisplayData
+	74,  // 13: org.apache.beam.model.pipeline.v1.ParDoPayload.do_fn:type_name -> org.apache.beam.model.pipeline.v1.FunctionSpec
+	87,  // 14: org.apache.beam.model.pipeline.v1.ParDoPayload.side_inputs:type_name -> org.apache.beam.model.pipeline.v1.ParDoPayload.SideInputsEntry
+	88,  // 15: org.apache.beam.model.pipeline.v1.ParDoPayload.state_specs:type_name -> org.apache.beam.model.pipeline.v1.ParDoPayload.StateSpecsEntry
+	89,  // 16: org.apache.beam.model.pipeline.v1.ParDoPayload.timer_family_specs:type_name -> org.apache.beam.model.pipeline.v1.ParDoPayload.TimerFamilySpecsEntry
+	30,  // 17: org.apache.beam.model.pipeline.v1.StateSpec.read_modify_write_spec:type_name -> org.apache.beam.model.pipeline.v1.ReadModifyWriteStateSpec
+	31,  // 18: org.apache.beam.model.pipeline.v1.StateSpec.bag_spec:type_name -> org.apache.beam.model.pipeline.v1.BagStateSpec
+	33,  // 19: org.apache.beam.model.pipeline.v1.StateSpec.combining_spec:type_name -> org.apache.beam.model.pipeline.v1.CombiningStateSpec
+	34,  // 20: org.apache.beam.model.pipeline.v1.StateSpec.map_spec:type_name -> org.apache.beam.model.pipeline.v1.MapStateSpec
+	35,  // 21: org.apache.beam.model.pipeline.v1.StateSpec.set_spec:type_name -> org.apache.beam.model.pipeline.v1.SetStateSpec
+	32,  // 22: org.apache.beam.model.pipeline.v1.StateSpec.ordered_list_spec:type_name -> org.apache.beam.model.pipeline.v1.OrderedListStateSpec
+	74,  // 23: org.apache.beam.model.pipeline.v1.CombiningStateSpec.combine_fn:type_name -> org.apache.beam.model.pipeline.v1.FunctionSpec
+	14,  // 24: org.apache.beam.model.pipeline.v1.TimerFamilySpec.time_domain:type_name -> org.apache.beam.model.pipeline.v1.TimeDomain.Enum
+	74,  // 25: org.apache.beam.model.pipeline.v1.ReadPayload.source:type_name -> org.apache.beam.model.pipeline.v1.FunctionSpec
+	7,   // 26: org.apache.beam.model.pipeline.v1.ReadPayload.is_bounded:type_name -> org.apache.beam.model.pipeline.v1.IsBounded.Enum
+	74,  // 27: org.apache.beam.model.pipeline.v1.WindowIntoPayload.window_fn:type_name -> org.apache.beam.model.pipeline.v1.FunctionSpec
+	74,  // 28: org.apache.beam.model.pipeline.v1.CombinePayload.combine_fn:type_name -> org.apache.beam.model.pipeline.v1.FunctionSpec
+	90,  // 29: org.apache.beam.model.pipeline.v1.TestStreamPayload.events:type_name -> org.apache.beam.model.pipeline.v1.TestStreamPayload.Event
+	117, // 30: org.apache.beam.model.pipeline.v1.TestStreamPayload.endpoint:type_name -> org.apache.beam.model.pipeline.v1.ApiServiceDescriptor
+	74,  // 31: org.apache.beam.model.pipeline.v1.WriteFilesPayload.sink:type_name -> org.apache.beam.model.pipeline.v1.FunctionSpec
+	74,  // 32: org.apache.beam.model.pipeline.v1.WriteFilesPayload.format_function:type_name -> org.apache.beam.model.pipeline.v1.FunctionSpec
+	95,  // 33: org.apache.beam.model.pipeline.v1.WriteFilesPayload.side_inputs:type_name -> org.apache.beam.model.pipeline.v1.WriteFilesPayload.SideInputsEntry
+	74,  // 34: org.apache.beam.model.pipeline.v1.Coder.spec:type_name -> org.apache.beam.model.pipeline.v1.FunctionSpec
+	74,  // 35: org.apache.beam.model.pipeline.v1.WindowingStrategy.window_fn:type_name -> org.apache.beam.model.pipeline.v1.FunctionSpec
+	9,   // 36: org.apache.beam.model.pipeline.v1.WindowingStrategy.merge_status:type_name -> org.apache.beam.model.pipeline.v1.MergeStatus.Enum
+	55,  // 37: org.apache.beam.model.pipeline.v1.WindowingStrategy.trigger:type_name -> org.apache.beam.model.pipeline.v1.Trigger
+	10,  // 38: org.apache.beam.model.pipeline.v1.WindowingStrategy.accumulation_mode:type_name -> org.apache.beam.model.pipeline.v1.AccumulationMode.Enum
+	13,  // 39: org.apache.beam.model.pipeline.v1.WindowingStrategy.output_time:type_name -> org.apache.beam.model.pipeline.v1.OutputTime.Enum
+	11,  // 40: org.apache.beam.model.pipeline.v1.WindowingStrategy.closing_behavior:type_name -> org.apache.beam.model.pipeline.v1.ClosingBehavior.Enum
+	12,  // 41: org.apache.beam.model.pipeline.v1.WindowingStrategy.OnTimeBehavior:type_name -> org.apache.beam.model.pipeline.v1.OnTimeBehavior.Enum
+	96,  // 42: org.apache.beam.model.pipeline.v1.Trigger.after_all:type_name -> org.apache.beam.model.pipeline.v1.Trigger.AfterAll
+	97,  // 43: org.apache.beam.model.pipeline.v1.Trigger.after_any:type_name -> org.apache.beam.model.pipeline.v1.Trigger.AfterAny
+	98,  // 44: org.apache.beam.model.pipeline.v1.Trigger.after_each:type_name -> org.apache.beam.model.pipeline.v1.Trigger.AfterEach
+	99,  // 45: org.apache.beam.model.pipeline.v1.Trigger.after_end_of_window:type_name -> org.apache.beam.model.pipeline.v1.Trigger.AfterEndOfWindow
+	100, // 46: org.apache.beam.model.pipeline.v1.Trigger.after_processing_time:type_name -> org.apache.beam.model.pipeline.v1.Trigger.AfterProcessingTime
+	101, // 47: org.apache.beam.model.pipeline.v1.Trigger.after_synchronized_processing_time:type_name -> org.apache.beam.model.pipeline.v1.Trigger.AfterSynchronizedProcessingTime
+	105, // 48: org.apache.beam.model.pipeline.v1.Trigger.always:type_name -> org.apache.beam.model.pipeline.v1.Trigger.Always
+	102, // 49: org.apache.beam.model.pipeline.v1.Trigger.default:type_name -> org.apache.beam.model.pipeline.v1.Trigger.Default
+	103, // 50: org.apache.beam.model.pipeline.v1.Trigger.element_count:type_name -> org.apache.beam.model.pipeline.v1.Trigger.ElementCount
+	104, // 51: org.apache.beam.model.pipeline.v1.Trigger.never:type_name -> org.apache.beam.model.pipeline.v1.Trigger.Never
+	106, // 52: org.apache.beam.model.pipeline.v1.Trigger.or_finally:type_name -> org.apache.beam.model.pipeline.v1.Trigger.OrFinally
+	107, // 53: org.apache.beam.model.pipeline.v1.Trigger.repeat:type_name -> org.apache.beam.model.pipeline.v1.Trigger.Repeat
+	108, // 54: org.apache.beam.model.pipeline.v1.TimestampTransform.delay:type_name -> org.apache.beam.model.pipeline.v1.TimestampTransform.Delay
+	109, // 55: org.apache.beam.model.pipeline.v1.TimestampTransform.align_to:type_name -> org.apache.beam.model.pipeline.v1.TimestampTransform.AlignTo
+	74,  // 56: org.apache.beam.model.pipeline.v1.SideInput.access_pattern:type_name -> org.apache.beam.model.pipeline.v1.FunctionSpec
+	74,  // 57: org.apache.beam.model.pipeline.v1.SideInput.view_fn:type_name -> org.apache.beam.model.pipeline.v1.FunctionSpec
+	74,  // 58: org.apache.beam.model.pipeline.v1.SideInput.window_mapping_fn:type_name -> org.apache.beam.model.pipeline.v1.FunctionSpec
+	77,  // 59: org.apache.beam.model.pipeline.v1.Environment.display_data:type_name -> org.apache.beam.model.pipeline.v1.DisplayData
+	66,  // 60: org.apache.beam.model.pipeline.v1.Environment.dependencies:type_name -> org.apache.beam.model.pipeline.v1.ArtifactInformation
+	110, // 61: org.apache.beam.model.pipeline.v1.ProcessPayload.env:type_name -> org.apache.beam.model.pipeline.v1.ProcessPayload.EnvEntry
+	117, // 62: org.apache.beam.model.pipeline.v1.ExternalPayload.endpoint:type_name -> org.apache.beam.model.pipeline.v1.ApiServiceDescriptor
+	111, // 63: org.apache.beam.model.pipeline.v1.ExternalPayload.params:type_name -> org.apache.beam.model.pipeline.v1.ExternalPayload.ParamsEntry
+	22,  // 64: org.apache.beam.model.pipeline.v1.MessageWithComponents.components:type_name -> org.apache.beam.model.pipeline.v1.Components
+	46,  // 65: org.apache.beam.model.pipeline.v1.MessageWithComponents.coder:type_name -> org.apache.beam.model.pipeline.v1.Coder
+	40,  // 66: org.apache.beam.model.pipeline.v1.MessageWithComponents.combine_payload:type_name -> org.apache.beam.model.pipeline.v1.CombinePayload
+	74,  // 67: org.apache.beam.model.pipeline.v1.MessageWithComponents.function_spec:type_name -> org.apache.beam.model.pipeline.v1.FunctionSpec
+	28,  // 68: org.apache.beam.model.pipeline.v1.MessageWithComponents.par_do_payload:type_name -> org.apache.beam.model.pipeline.v1.ParDoPayload
+	24,  // 69: org.apache.beam.model.pipeline.v1.MessageWithComponents.ptransform:type_name -> org.apache.beam.model.pipeline.v1.PTransform
+	27,  // 70: org.apache.beam.model.pipeline.v1.MessageWithComponents.pcollection:type_name -> org.apache.beam.model.pipeline.v1.PCollection
+	38,  // 71: org.apache.beam.model.pipeline.v1.MessageWithComponents.read_payload:type_name -> org.apache.beam.model.pipeline.v1.ReadPayload
+	57,  // 72: org.apache.beam.model.pipeline.v1.MessageWithComponents.side_input:type_name -> org.apache.beam.model.pipeline.v1.SideInput
+	39,  // 73: org.apache.beam.model.pipeline.v1.MessageWithComponents.window_into_payload:type_name -> org.apache.beam.model.pipeline.v1.WindowIntoPayload
+	48,  // 74: org.apache.beam.model.pipeline.v1.MessageWithComponents.windowing_strategy:type_name -> org.apache.beam.model.pipeline.v1.WindowingStrategy
+	67,  // 75: org.apache.beam.model.pipeline.v1.ExecutableStagePayload.environment:type_name -> org.apache.beam.model.pipeline.v1.Environment
+	116, // 76: org.apache.beam.model.pipeline.v1.ExecutableStagePayload.wire_coder_settings:type_name -> org.apache.beam.model.pipeline.v1.ExecutableStagePayload.WireCoderSetting
+	112, // 77: org.apache.beam.model.pipeline.v1.ExecutableStagePayload.side_inputs:type_name -> org.apache.beam.model.pipeline.v1.ExecutableStagePayload.SideInputId
+	22,  // 78: org.apache.beam.model.pipeline.v1.ExecutableStagePayload.components:type_name -> org.apache.beam.model.pipeline.v1.Components
+	113, // 79: org.apache.beam.model.pipeline.v1.ExecutableStagePayload.user_states:type_name -> org.apache.beam.model.pipeline.v1.ExecutableStagePayload.UserStateId
+	114, // 80: org.apache.beam.model.pipeline.v1.ExecutableStagePayload.timers:type_name -> org.apache.beam.model.pipeline.v1.ExecutableStagePayload.TimerId
+	115, // 81: org.apache.beam.model.pipeline.v1.ExecutableStagePayload.timerFamilies:type_name -> org.apache.beam.model.pipeline.v1.ExecutableStagePayload.TimerFamilyId
+	24,  // 82: org.apache.beam.model.pipeline.v1.Components.TransformsEntry.value:type_name -> org.apache.beam.model.pipeline.v1.PTransform
+	27,  // 83: org.apache.beam.model.pipeline.v1.Components.PcollectionsEntry.value:type_name -> org.apache.beam.model.pipeline.v1.PCollection
+	48,  // 84: org.apache.beam.model.pipeline.v1.Components.WindowingStrategiesEntry.value:type_name -> org.apache.beam.model.pipeline.v1.WindowingStrategy
+	46,  // 85: org.apache.beam.model.pipeline.v1.Components.CodersEntry.value:type_name -> org.apache.beam.model.pipeline.v1.Coder
+	67,  // 86: org.apache.beam.model.pipeline.v1.Components.EnvironmentsEntry.value:type_name -> org.apache.beam.model.pipeline.v1.Environment
+	57,  // 87: org.apache.beam.model.pipeline.v1.ParDoPayload.SideInputsEntry.value:type_name -> org.apache.beam.model.pipeline.v1.SideInput
+	29,  // 88: org.apache.beam.model.pipeline.v1.ParDoPayload.StateSpecsEntry.value:type_name -> org.apache.beam.model.pipeline.v1.StateSpec
+	36,  // 89: org.apache.beam.model.pipeline.v1.ParDoPayload.TimerFamilySpecsEntry.value:type_name -> org.apache.beam.model.pipeline.v1.TimerFamilySpec
+	92,  // 90: org.apache.beam.model.pipeline.v1.TestStreamPayload.Event.watermark_event:type_name -> org.apache.beam.model.pipeline.v1.TestStreamPayload.Event.AdvanceWatermark
+	93,  // 91: org.apache.beam.model.pipeline.v1.TestStreamPayload.Event.processing_time_event:type_name -> org.apache.beam.model.pipeline.v1.TestStreamPayload.Event.AdvanceProcessingTime
+	94,  // 92: org.apache.beam.model.pipeline.v1.TestStreamPayload.Event.element_event:type_name -> org.apache.beam.model.pipeline.v1.TestStreamPayload.Event.AddElements
+	91,  // 93: org.apache.beam.model.pipeline.v1.TestStreamPayload.Event.AddElements.elements:type_name -> org.apache.beam.model.pipeline.v1.TestStreamPayload.TimestampedElement
+	57,  // 94: org.apache.beam.model.pipeline.v1.WriteFilesPayload.SideInputsEntry.value:type_name -> org.apache.beam.model.pipeline.v1.SideInput
+	55,  // 95: org.apache.beam.model.pipeline.v1.Trigger.AfterAll.subtriggers:type_name -> org.apache.beam.model.pipeline.v1.Trigger
+	55,  // 96: org.apache.beam.model.pipeline.v1.Trigger.AfterAny.subtriggers:type_name -> org.apache.beam.model.pipeline.v1.Trigger
+	55,  // 97: org.apache.beam.model.pipeline.v1.Trigger.AfterEach.subtriggers:type_name -> org.apache.beam.model.pipeline.v1.Trigger
+	55,  // 98: org.apache.beam.model.pipeline.v1.Trigger.AfterEndOfWindow.early_firings:type_name -> org.apache.beam.model.pipeline.v1.Trigger
+	55,  // 99: org.apache.beam.model.pipeline.v1.Trigger.AfterEndOfWindow.late_firings:type_name -> org.apache.beam.model.pipeline.v1.Trigger
+	56,  // 100: org.apache.beam.model.pipeline.v1.Trigger.AfterProcessingTime.timestamp_transforms:type_name -> org.apache.beam.model.pipeline.v1.TimestampTransform
+	55,  // 101: org.apache.beam.model.pipeline.v1.Trigger.OrFinally.main:type_name -> org.apache.beam.model.pipeline.v1.Trigger
+	55,  // 102: org.apache.beam.model.pipeline.v1.Trigger.OrFinally.finally:type_name -> org.apache.beam.model.pipeline.v1.Trigger
+	55,  // 103: org.apache.beam.model.pipeline.v1.Trigger.Repeat.subtrigger:type_name -> org.apache.beam.model.pipeline.v1.Trigger
+	114, // 104: org.apache.beam.model.pipeline.v1.ExecutableStagePayload.WireCoderSetting.timer:type_name -> org.apache.beam.model.pipeline.v1.ExecutableStagePayload.TimerId
+	118, // 105: org.apache.beam.model.pipeline.v1.beam_urn:extendee -> google.protobuf.EnumValueOptions
+	118, // 106: org.apache.beam.model.pipeline.v1.beam_constant:extendee -> google.protobuf.EnumValueOptions
+	42,  // 107: org.apache.beam.model.pipeline.v1.TestStreamService.Events:input_type -> org.apache.beam.model.pipeline.v1.EventsRequest
+	90,  // 108: org.apache.beam.model.pipeline.v1.TestStreamService.Events:output_type -> org.apache.beam.model.pipeline.v1.TestStreamPayload.Event
+	108, // [108:109] is the sub-list for method output_type
+	107, // [107:108] is the sub-list for method input_type
+	107, // [107:107] is the sub-list for extension type_name
+	105, // [105:107] is the sub-list for extension extendee
+	0,   // [0:105] is the sub-list for field type_name
+}
+
+func init() { file_beam_runner_api_proto_init() }
+func file_beam_runner_api_proto_init() {
+	if File_beam_runner_api_proto != nil {
+		return
+	}
+	file_endpoints_proto_init()
+	if !protoimpl.UnsafeEnabled {
+		file_beam_runner_api_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*BeamConstants); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_runner_api_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*Components); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_runner_api_proto_msgTypes[2].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*Pipeline); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_runner_api_proto_msgTypes[3].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*PTransform); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_runner_api_proto_msgTypes[4].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*StandardPTransforms); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_runner_api_proto_msgTypes[5].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*StandardSideInputTypes); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_runner_api_proto_msgTypes[6].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*PCollection); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_runner_api_proto_msgTypes[7].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*ParDoPayload); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_runner_api_proto_msgTypes[8].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*StateSpec); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_runner_api_proto_msgTypes[9].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*ReadModifyWriteStateSpec); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_runner_api_proto_msgTypes[10].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*BagStateSpec); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_runner_api_proto_msgTypes[11].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*OrderedListStateSpec); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_runner_api_proto_msgTypes[12].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*CombiningStateSpec); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_runner_api_proto_msgTypes[13].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*MapStateSpec); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_runner_api_proto_msgTypes[14].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*SetStateSpec); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_runner_api_proto_msgTypes[15].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*TimerFamilySpec); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_runner_api_proto_msgTypes[16].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*IsBounded); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_runner_api_proto_msgTypes[17].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*ReadPayload); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_runner_api_proto_msgTypes[18].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*WindowIntoPayload); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_runner_api_proto_msgTypes[19].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*CombinePayload); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_runner_api_proto_msgTypes[20].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*TestStreamPayload); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_runner_api_proto_msgTypes[21].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*EventsRequest); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_runner_api_proto_msgTypes[22].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*WriteFilesPayload); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_runner_api_proto_msgTypes[23].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*PubSubReadPayload); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_runner_api_proto_msgTypes[24].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*PubSubWritePayload); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_runner_api_proto_msgTypes[25].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*Coder); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_runner_api_proto_msgTypes[26].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*StandardCoders); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_runner_api_proto_msgTypes[27].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*WindowingStrategy); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_runner_api_proto_msgTypes[28].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*MergeStatus); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_runner_api_proto_msgTypes[29].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*AccumulationMode); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_runner_api_proto_msgTypes[30].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*ClosingBehavior); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_runner_api_proto_msgTypes[31].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*OnTimeBehavior); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_runner_api_proto_msgTypes[32].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*OutputTime); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_runner_api_proto_msgTypes[33].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*TimeDomain); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_runner_api_proto_msgTypes[34].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*Trigger); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_runner_api_proto_msgTypes[35].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*TimestampTransform); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_runner_api_proto_msgTypes[36].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*SideInput); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_runner_api_proto_msgTypes[37].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*StandardArtifacts); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_runner_api_proto_msgTypes[38].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*ArtifactFilePayload); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_runner_api_proto_msgTypes[39].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*ArtifactUrlPayload); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_runner_api_proto_msgTypes[40].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*EmbeddedFilePayload); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_runner_api_proto_msgTypes[41].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*PyPIPayload); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_runner_api_proto_msgTypes[42].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*MavenPayload); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_runner_api_proto_msgTypes[43].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*DeferredArtifactPayload); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_runner_api_proto_msgTypes[44].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*ArtifactStagingToRolePayload); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_runner_api_proto_msgTypes[45].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*ArtifactInformation); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_runner_api_proto_msgTypes[46].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*Environment); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_runner_api_proto_msgTypes[47].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*StandardEnvironments); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_runner_api_proto_msgTypes[48].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*DockerPayload); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_runner_api_proto_msgTypes[49].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*ProcessPayload); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_runner_api_proto_msgTypes[50].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*ExternalPayload); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_runner_api_proto_msgTypes[51].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*StandardProtocols); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_runner_api_proto_msgTypes[52].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*StandardRequirements); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_runner_api_proto_msgTypes[53].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*FunctionSpec); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_runner_api_proto_msgTypes[54].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*StandardDisplayData); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_runner_api_proto_msgTypes[55].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*LabelledStringPayload); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_runner_api_proto_msgTypes[56].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*DisplayData); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_runner_api_proto_msgTypes[57].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*MessageWithComponents); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_runner_api_proto_msgTypes[58].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*ExecutableStagePayload); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_runner_api_proto_msgTypes[69].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*TestStreamPayload_Event); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_runner_api_proto_msgTypes[70].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*TestStreamPayload_TimestampedElement); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_runner_api_proto_msgTypes[71].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*TestStreamPayload_Event_AdvanceWatermark); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_runner_api_proto_msgTypes[72].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*TestStreamPayload_Event_AdvanceProcessingTime); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_runner_api_proto_msgTypes[73].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*TestStreamPayload_Event_AddElements); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_runner_api_proto_msgTypes[75].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*Trigger_AfterAll); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_runner_api_proto_msgTypes[76].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*Trigger_AfterAny); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_runner_api_proto_msgTypes[77].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*Trigger_AfterEach); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_runner_api_proto_msgTypes[78].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*Trigger_AfterEndOfWindow); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_runner_api_proto_msgTypes[79].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*Trigger_AfterProcessingTime); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_runner_api_proto_msgTypes[80].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*Trigger_AfterSynchronizedProcessingTime); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_runner_api_proto_msgTypes[81].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*Trigger_Default); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_runner_api_proto_msgTypes[82].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*Trigger_ElementCount); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_runner_api_proto_msgTypes[83].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*Trigger_Never); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_runner_api_proto_msgTypes[84].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*Trigger_Always); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_runner_api_proto_msgTypes[85].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*Trigger_OrFinally); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_runner_api_proto_msgTypes[86].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*Trigger_Repeat); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_runner_api_proto_msgTypes[87].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*TimestampTransform_Delay); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_runner_api_proto_msgTypes[88].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*TimestampTransform_AlignTo); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_runner_api_proto_msgTypes[91].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*ExecutableStagePayload_SideInputId); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_runner_api_proto_msgTypes[92].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*ExecutableStagePayload_UserStateId); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_runner_api_proto_msgTypes[93].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*ExecutableStagePayload_TimerId); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_runner_api_proto_msgTypes[94].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*ExecutableStagePayload_TimerFamilyId); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_beam_runner_api_proto_msgTypes[95].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*ExecutableStagePayload_WireCoderSetting); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+	}
+	file_beam_runner_api_proto_msgTypes[8].OneofWrappers = []interface{}{
+		(*StateSpec_ReadModifyWriteSpec)(nil),
+		(*StateSpec_BagSpec)(nil),
+		(*StateSpec_CombiningSpec)(nil),
+		(*StateSpec_MapSpec)(nil),
+		(*StateSpec_SetSpec)(nil),
+		(*StateSpec_OrderedListSpec)(nil),
+	}
+	file_beam_runner_api_proto_msgTypes[34].OneofWrappers = []interface{}{
+		(*Trigger_AfterAll_)(nil),
+		(*Trigger_AfterAny_)(nil),
+		(*Trigger_AfterEach_)(nil),
+		(*Trigger_AfterEndOfWindow_)(nil),
+		(*Trigger_AfterProcessingTime_)(nil),
+		(*Trigger_AfterSynchronizedProcessingTime_)(nil),
+		(*Trigger_Always_)(nil),
+		(*Trigger_Default_)(nil),
+		(*Trigger_ElementCount_)(nil),
+		(*Trigger_Never_)(nil),
+		(*Trigger_OrFinally_)(nil),
+		(*Trigger_Repeat_)(nil),
+	}
+	file_beam_runner_api_proto_msgTypes[35].OneofWrappers = []interface{}{
+		(*TimestampTransform_Delay_)(nil),
+		(*TimestampTransform_AlignTo_)(nil),
+	}
+	file_beam_runner_api_proto_msgTypes[57].OneofWrappers = []interface{}{
+		(*MessageWithComponents_Coder)(nil),
+		(*MessageWithComponents_CombinePayload)(nil),
+		(*MessageWithComponents_FunctionSpec)(nil),
+		(*MessageWithComponents_ParDoPayload)(nil),
+		(*MessageWithComponents_Ptransform)(nil),
+		(*MessageWithComponents_Pcollection)(nil),
+		(*MessageWithComponents_ReadPayload)(nil),
+		(*MessageWithComponents_SideInput)(nil),
+		(*MessageWithComponents_WindowIntoPayload)(nil),
+		(*MessageWithComponents_WindowingStrategy)(nil),
+	}
+	file_beam_runner_api_proto_msgTypes[69].OneofWrappers = []interface{}{
+		(*TestStreamPayload_Event_WatermarkEvent)(nil),
+		(*TestStreamPayload_Event_ProcessingTimeEvent)(nil),
+		(*TestStreamPayload_Event_ElementEvent)(nil),
+	}
+	file_beam_runner_api_proto_msgTypes[95].OneofWrappers = []interface{}{
 		(*ExecutableStagePayload_WireCoderSetting_InputOrOutputId)(nil),
 		(*ExecutableStagePayload_WireCoderSetting_Timer)(nil),
 	}
-}
-
-var E_BeamUrn = &proto.ExtensionDesc{
-	ExtendedType:  (*descriptor.EnumValueOptions)(nil),
-	ExtensionType: (*string)(nil),
-	Field:         185324356,
-	Name:          "org.apache.beam.model.pipeline.v1.beam_urn",
-	Tag:           "bytes,185324356,opt,name=beam_urn",
-	Filename:      "beam_runner_api.proto",
-}
-
-var E_BeamConstant = &proto.ExtensionDesc{
-	ExtendedType:  (*descriptor.EnumValueOptions)(nil),
-	ExtensionType: (*string)(nil),
-	Field:         185324357,
-	Name:          "org.apache.beam.model.pipeline.v1.beam_constant",
-	Tag:           "bytes,185324357,opt,name=beam_constant",
-	Filename:      "beam_runner_api.proto",
-}
-
-func init() {
-	proto.RegisterEnum("org.apache.beam.model.pipeline.v1.BeamConstants_Constants", BeamConstants_Constants_name, BeamConstants_Constants_value)
-	proto.RegisterEnum("org.apache.beam.model.pipeline.v1.StandardPTransforms_Primitives", StandardPTransforms_Primitives_name, StandardPTransforms_Primitives_value)
-	proto.RegisterEnum("org.apache.beam.model.pipeline.v1.StandardPTransforms_DeprecatedPrimitives", StandardPTransforms_DeprecatedPrimitives_name, StandardPTransforms_DeprecatedPrimitives_value)
-	proto.RegisterEnum("org.apache.beam.model.pipeline.v1.StandardPTransforms_Composites", StandardPTransforms_Composites_name, StandardPTransforms_Composites_value)
-	proto.RegisterEnum("org.apache.beam.model.pipeline.v1.StandardPTransforms_CombineComponents", StandardPTransforms_CombineComponents_name, StandardPTransforms_CombineComponents_value)
-	proto.RegisterEnum("org.apache.beam.model.pipeline.v1.StandardPTransforms_SplittableParDoComponents", StandardPTransforms_SplittableParDoComponents_name, StandardPTransforms_SplittableParDoComponents_value)
-	proto.RegisterEnum("org.apache.beam.model.pipeline.v1.StandardSideInputTypes_Enum", StandardSideInputTypes_Enum_name, StandardSideInputTypes_Enum_value)
-	proto.RegisterEnum("org.apache.beam.model.pipeline.v1.IsBounded_Enum", IsBounded_Enum_name, IsBounded_Enum_value)
-	proto.RegisterEnum("org.apache.beam.model.pipeline.v1.StandardCoders_Enum", StandardCoders_Enum_name, StandardCoders_Enum_value)
-	proto.RegisterEnum("org.apache.beam.model.pipeline.v1.MergeStatus_Enum", MergeStatus_Enum_name, MergeStatus_Enum_value)
-	proto.RegisterEnum("org.apache.beam.model.pipeline.v1.AccumulationMode_Enum", AccumulationMode_Enum_name, AccumulationMode_Enum_value)
-	proto.RegisterEnum("org.apache.beam.model.pipeline.v1.ClosingBehavior_Enum", ClosingBehavior_Enum_name, ClosingBehavior_Enum_value)
-	proto.RegisterEnum("org.apache.beam.model.pipeline.v1.OnTimeBehavior_Enum", OnTimeBehavior_Enum_name, OnTimeBehavior_Enum_value)
-	proto.RegisterEnum("org.apache.beam.model.pipeline.v1.OutputTime_Enum", OutputTime_Enum_name, OutputTime_Enum_value)
-	proto.RegisterEnum("org.apache.beam.model.pipeline.v1.TimeDomain_Enum", TimeDomain_Enum_name, TimeDomain_Enum_value)
-	proto.RegisterEnum("org.apache.beam.model.pipeline.v1.StandardArtifacts_Types", StandardArtifacts_Types_name, StandardArtifacts_Types_value)
-	proto.RegisterEnum("org.apache.beam.model.pipeline.v1.StandardArtifacts_Roles", StandardArtifacts_Roles_name, StandardArtifacts_Roles_value)
-	proto.RegisterEnum("org.apache.beam.model.pipeline.v1.StandardEnvironments_Environments", StandardEnvironments_Environments_name, StandardEnvironments_Environments_value)
-	proto.RegisterEnum("org.apache.beam.model.pipeline.v1.StandardProtocols_Enum", StandardProtocols_Enum_name, StandardProtocols_Enum_value)
-	proto.RegisterEnum("org.apache.beam.model.pipeline.v1.StandardRequirements_Enum", StandardRequirements_Enum_name, StandardRequirements_Enum_value)
-	proto.RegisterEnum("org.apache.beam.model.pipeline.v1.StandardDisplayData_DisplayData", StandardDisplayData_DisplayData_name, StandardDisplayData_DisplayData_value)
-	proto.RegisterType((*BeamConstants)(nil), "org.apache.beam.model.pipeline.v1.BeamConstants")
-	proto.RegisterType((*Components)(nil), "org.apache.beam.model.pipeline.v1.Components")
-	proto.RegisterMapType((map[string]*Coder)(nil), "org.apache.beam.model.pipeline.v1.Components.CodersEntry")
-	proto.RegisterMapType((map[string]*Environment)(nil), "org.apache.beam.model.pipeline.v1.Components.EnvironmentsEntry")
-	proto.RegisterMapType((map[string]*PCollection)(nil), "org.apache.beam.model.pipeline.v1.Components.PcollectionsEntry")
-	proto.RegisterMapType((map[string]*PTransform)(nil), "org.apache.beam.model.pipeline.v1.Components.TransformsEntry")
-	proto.RegisterMapType((map[string]*WindowingStrategy)(nil), "org.apache.beam.model.pipeline.v1.Components.WindowingStrategiesEntry")
-	proto.RegisterType((*Pipeline)(nil), "org.apache.beam.model.pipeline.v1.Pipeline")
-	proto.RegisterType((*PTransform)(nil), "org.apache.beam.model.pipeline.v1.PTransform")
-	proto.RegisterMapType((map[string]string)(nil), "org.apache.beam.model.pipeline.v1.PTransform.InputsEntry")
-	proto.RegisterMapType((map[string]string)(nil), "org.apache.beam.model.pipeline.v1.PTransform.OutputsEntry")
-	proto.RegisterType((*StandardPTransforms)(nil), "org.apache.beam.model.pipeline.v1.StandardPTransforms")
-	proto.RegisterType((*StandardSideInputTypes)(nil), "org.apache.beam.model.pipeline.v1.StandardSideInputTypes")
-	proto.RegisterType((*PCollection)(nil), "org.apache.beam.model.pipeline.v1.PCollection")
-	proto.RegisterType((*ParDoPayload)(nil), "org.apache.beam.model.pipeline.v1.ParDoPayload")
-	proto.RegisterMapType((map[string]*SideInput)(nil), "org.apache.beam.model.pipeline.v1.ParDoPayload.SideInputsEntry")
-	proto.RegisterMapType((map[string]*StateSpec)(nil), "org.apache.beam.model.pipeline.v1.ParDoPayload.StateSpecsEntry")
-	proto.RegisterMapType((map[string]*TimerFamilySpec)(nil), "org.apache.beam.model.pipeline.v1.ParDoPayload.TimerFamilySpecsEntry")
-	proto.RegisterType((*StateSpec)(nil), "org.apache.beam.model.pipeline.v1.StateSpec")
-	proto.RegisterType((*ReadModifyWriteStateSpec)(nil), "org.apache.beam.model.pipeline.v1.ReadModifyWriteStateSpec")
-	proto.RegisterType((*BagStateSpec)(nil), "org.apache.beam.model.pipeline.v1.BagStateSpec")
-	proto.RegisterType((*CombiningStateSpec)(nil), "org.apache.beam.model.pipeline.v1.CombiningStateSpec")
-	proto.RegisterType((*MapStateSpec)(nil), "org.apache.beam.model.pipeline.v1.MapStateSpec")
-	proto.RegisterType((*SetStateSpec)(nil), "org.apache.beam.model.pipeline.v1.SetStateSpec")
-	proto.RegisterType((*TimerFamilySpec)(nil), "org.apache.beam.model.pipeline.v1.TimerFamilySpec")
-	proto.RegisterType((*IsBounded)(nil), "org.apache.beam.model.pipeline.v1.IsBounded")
-	proto.RegisterType((*ReadPayload)(nil), "org.apache.beam.model.pipeline.v1.ReadPayload")
-	proto.RegisterType((*WindowIntoPayload)(nil), "org.apache.beam.model.pipeline.v1.WindowIntoPayload")
-	proto.RegisterType((*CombinePayload)(nil), "org.apache.beam.model.pipeline.v1.CombinePayload")
-	proto.RegisterType((*TestStreamPayload)(nil), "org.apache.beam.model.pipeline.v1.TestStreamPayload")
-	proto.RegisterType((*TestStreamPayload_Event)(nil), "org.apache.beam.model.pipeline.v1.TestStreamPayload.Event")
-	proto.RegisterType((*TestStreamPayload_Event_AdvanceWatermark)(nil), "org.apache.beam.model.pipeline.v1.TestStreamPayload.Event.AdvanceWatermark")
-	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")
-	proto.RegisterType((*StandardCoders)(nil), "org.apache.beam.model.pipeline.v1.StandardCoders")
-	proto.RegisterType((*WindowingStrategy)(nil), "org.apache.beam.model.pipeline.v1.WindowingStrategy")
-	proto.RegisterType((*MergeStatus)(nil), "org.apache.beam.model.pipeline.v1.MergeStatus")
-	proto.RegisterType((*AccumulationMode)(nil), "org.apache.beam.model.pipeline.v1.AccumulationMode")
-	proto.RegisterType((*ClosingBehavior)(nil), "org.apache.beam.model.pipeline.v1.ClosingBehavior")
-	proto.RegisterType((*OnTimeBehavior)(nil), "org.apache.beam.model.pipeline.v1.OnTimeBehavior")
-	proto.RegisterType((*OutputTime)(nil), "org.apache.beam.model.pipeline.v1.OutputTime")
-	proto.RegisterType((*TimeDomain)(nil), "org.apache.beam.model.pipeline.v1.TimeDomain")
-	proto.RegisterType((*Trigger)(nil), "org.apache.beam.model.pipeline.v1.Trigger")
-	proto.RegisterType((*Trigger_AfterAll)(nil), "org.apache.beam.model.pipeline.v1.Trigger.AfterAll")
-	proto.RegisterType((*Trigger_AfterAny)(nil), "org.apache.beam.model.pipeline.v1.Trigger.AfterAny")
-	proto.RegisterType((*Trigger_AfterEach)(nil), "org.apache.beam.model.pipeline.v1.Trigger.AfterEach")
-	proto.RegisterType((*Trigger_AfterEndOfWindow)(nil), "org.apache.beam.model.pipeline.v1.Trigger.AfterEndOfWindow")
-	proto.RegisterType((*Trigger_AfterProcessingTime)(nil), "org.apache.beam.model.pipeline.v1.Trigger.AfterProcessingTime")
-	proto.RegisterType((*Trigger_AfterSynchronizedProcessingTime)(nil), "org.apache.beam.model.pipeline.v1.Trigger.AfterSynchronizedProcessingTime")
-	proto.RegisterType((*Trigger_Default)(nil), "org.apache.beam.model.pipeline.v1.Trigger.Default")
-	proto.RegisterType((*Trigger_ElementCount)(nil), "org.apache.beam.model.pipeline.v1.Trigger.ElementCount")
-	proto.RegisterType((*Trigger_Never)(nil), "org.apache.beam.model.pipeline.v1.Trigger.Never")
-	proto.RegisterType((*Trigger_Always)(nil), "org.apache.beam.model.pipeline.v1.Trigger.Always")
-	proto.RegisterType((*Trigger_OrFinally)(nil), "org.apache.beam.model.pipeline.v1.Trigger.OrFinally")
-	proto.RegisterType((*Trigger_Repeat)(nil), "org.apache.beam.model.pipeline.v1.Trigger.Repeat")
-	proto.RegisterType((*TimestampTransform)(nil), "org.apache.beam.model.pipeline.v1.TimestampTransform")
-	proto.RegisterType((*TimestampTransform_Delay)(nil), "org.apache.beam.model.pipeline.v1.TimestampTransform.Delay")
-	proto.RegisterType((*TimestampTransform_AlignTo)(nil), "org.apache.beam.model.pipeline.v1.TimestampTransform.AlignTo")
-	proto.RegisterType((*SideInput)(nil), "org.apache.beam.model.pipeline.v1.SideInput")
-	proto.RegisterType((*StandardArtifacts)(nil), "org.apache.beam.model.pipeline.v1.StandardArtifacts")
-	proto.RegisterType((*ArtifactFilePayload)(nil), "org.apache.beam.model.pipeline.v1.ArtifactFilePayload")
-	proto.RegisterType((*ArtifactUrlPayload)(nil), "org.apache.beam.model.pipeline.v1.ArtifactUrlPayload")
-	proto.RegisterType((*EmbeddedFilePayload)(nil), "org.apache.beam.model.pipeline.v1.EmbeddedFilePayload")
-	proto.RegisterType((*PyPIPayload)(nil), "org.apache.beam.model.pipeline.v1.PyPIPayload")
-	proto.RegisterType((*MavenPayload)(nil), "org.apache.beam.model.pipeline.v1.MavenPayload")
-	proto.RegisterType((*DeferredArtifactPayload)(nil), "org.apache.beam.model.pipeline.v1.DeferredArtifactPayload")
-	proto.RegisterType((*ArtifactStagingToRolePayload)(nil), "org.apache.beam.model.pipeline.v1.ArtifactStagingToRolePayload")
-	proto.RegisterType((*ArtifactInformation)(nil), "org.apache.beam.model.pipeline.v1.ArtifactInformation")
-	proto.RegisterType((*Environment)(nil), "org.apache.beam.model.pipeline.v1.Environment")
-	proto.RegisterType((*StandardEnvironments)(nil), "org.apache.beam.model.pipeline.v1.StandardEnvironments")
-	proto.RegisterType((*DockerPayload)(nil), "org.apache.beam.model.pipeline.v1.DockerPayload")
-	proto.RegisterType((*ProcessPayload)(nil), "org.apache.beam.model.pipeline.v1.ProcessPayload")
-	proto.RegisterMapType((map[string]string)(nil), "org.apache.beam.model.pipeline.v1.ProcessPayload.EnvEntry")
-	proto.RegisterType((*ExternalPayload)(nil), "org.apache.beam.model.pipeline.v1.ExternalPayload")
-	proto.RegisterMapType((map[string]string)(nil), "org.apache.beam.model.pipeline.v1.ExternalPayload.ParamsEntry")
-	proto.RegisterType((*StandardProtocols)(nil), "org.apache.beam.model.pipeline.v1.StandardProtocols")
-	proto.RegisterType((*StandardRequirements)(nil), "org.apache.beam.model.pipeline.v1.StandardRequirements")
-	proto.RegisterType((*FunctionSpec)(nil), "org.apache.beam.model.pipeline.v1.FunctionSpec")
-	proto.RegisterType((*StandardDisplayData)(nil), "org.apache.beam.model.pipeline.v1.StandardDisplayData")
-	proto.RegisterType((*LabelledStringPayload)(nil), "org.apache.beam.model.pipeline.v1.LabelledStringPayload")
-	proto.RegisterType((*DisplayData)(nil), "org.apache.beam.model.pipeline.v1.DisplayData")
-	proto.RegisterType((*MessageWithComponents)(nil), "org.apache.beam.model.pipeline.v1.MessageWithComponents")
-	proto.RegisterType((*ExecutableStagePayload)(nil), "org.apache.beam.model.pipeline.v1.ExecutableStagePayload")
-	proto.RegisterType((*ExecutableStagePayload_SideInputId)(nil), "org.apache.beam.model.pipeline.v1.ExecutableStagePayload.SideInputId")
-	proto.RegisterType((*ExecutableStagePayload_UserStateId)(nil), "org.apache.beam.model.pipeline.v1.ExecutableStagePayload.UserStateId")
-	proto.RegisterType((*ExecutableStagePayload_TimerId)(nil), "org.apache.beam.model.pipeline.v1.ExecutableStagePayload.TimerId")
-	proto.RegisterType((*ExecutableStagePayload_TimerFamilyId)(nil), "org.apache.beam.model.pipeline.v1.ExecutableStagePayload.TimerFamilyId")
-	proto.RegisterType((*ExecutableStagePayload_WireCoderSetting)(nil), "org.apache.beam.model.pipeline.v1.ExecutableStagePayload.WireCoderSetting")
-	proto.RegisterExtension(E_BeamUrn)
-	proto.RegisterExtension(E_BeamConstant)
-}
-
-func init() { proto.RegisterFile("beam_runner_api.proto", fileDescriptor_cf57597c3a9659a9) }
-
-var fileDescriptor_cf57597c3a9659a9 = []byte{
-	// 5958 bytes of a gzipped FileDescriptorProto
-	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xbc, 0x7c, 0x4d, 0x6c, 0x1b, 0xe9,
-	0x79, 0x3f, 0x3f, 0x44, 0x89, 0x7c, 0x48, 0x49, 0xe3, 0xd7, 0x96, 0x57, 0x3b, 0xd9, 0x8d, 0xbd,
-	0xb3, 0x9b, 0x5d, 0xef, 0xfe, 0x77, 0xb5, 0xb6, 0xbc, 0xfe, 0x52, 0x92, 0x75, 0x48, 0x71, 0x68,
-	0x8d, 0xcd, 0xaf, 0x1d, 0x52, 0x92, 0xbd, 0x49, 0x76, 0xfe, 0x23, 0xce, 0x4b, 0x69, 0xe0, 0xe1,
-	0x0c, 0x33, 0x33, 0x94, 0x97, 0x8b, 0x00, 0x05, 0x8a, 0x22, 0x87, 0x02, 0x6d, 0x91, 0x1c, 0x02,
-	0xe4, 0x14, 0x20, 0x01, 0x82, 0xb6, 0x28, 0xd0, 0x22, 0x28, 0x02, 0xf4, 0xd2, 0x43, 0xda, 0x1e,
-	0x5b, 0xa0, 0x45, 0x81, 0x9e, 0x8b, 0x9e, 0x7b, 0xe9, 0xa1, 0x40, 0x7b, 0x28, 0x8a, 0xf7, 0x63,
-	0x86, 0xef, 0x50, 0x92, 0x4d, 0xca, 0x46, 0x6f, 0x9c, 0x67, 0xde, 0xe7, 0xf7, 0x3c, 0xef, 0xf7,
-	0xf3, 0x35, 0x84, 0xb5, 0x03, 0x6c, 0x0e, 0x0c, 0x7f, 0xe4, 0xba, 0xd8, 0x37, 0xcc, 0xa1, 0xbd,
-	0x31, 0xf4, 0xbd, 0xd0, 0x43, 0x6f, 0x79, 0xfe, 0xe1, 0x86, 0x39, 0x34, 0x7b, 0x47, 0x78, 0x83,
-	0xb4, 0xd8, 0x18, 0x78, 0x16, 0x76, 0x36, 0x86, 0xf6, 0x10, 0x3b, 0xb6, 0x8b, 0x37, 0x8e, 0x6f,
-	0xc8, 0xab, 0xd8, 0xb5, 0x86, 0x9e, 0xed, 0x86, 0x01, 0xe3, 0x91, 0x5f, 0x3f, 0xf4, 0xbc, 0x43,
-	0x07, 0x7f, 0x4c, 0x9f, 0x0e, 0x46, 0xfd, 0x8f, 0x4d, 0x77, 0xcc, 0x5f, 0x5d, 0x9d, 0x7e, 0x65,
-	0xe1, 0xa0, 0xe7, 0xdb, 0xc3, 0xd0, 0xf3, 0x79, 0x8b, 0x2b, 0xd3, 0x2d, 0x42, 0x7b, 0x80, 0x83,
-	0xd0, 0x1c, 0x0c, 0x59, 0x03, 0xe5, 0xb7, 0x69, 0x58, 0xae, 0x60, 0x73, 0xb0, 0xed, 0xb9, 0x41,
-	0x68, 0xba, 0x61, 0xa0, 0xfc, 0x45, 0x1a, 0x0a, 0xf1, 0x13, 0xba, 0x01, 0x97, 0x1a, 0x5a, 0xd3,
-	0xe8, 0x6a, 0x0d, 0xb5, 0xd3, 0x2d, 0x37, 0xda, 0x46, 0x43, 0xab, 0xd7, 0xb5, 0x8e, 0x94, 0x92,
-	0x5f, 0xfb, 0xd3, 0xbf, 0xfc, 0xef, 0xdf, 0xe6, 0x2e, 0x7c, 0x74, 0x6f, 0x73, 0xf3, 0xe6, 0xcd,
-	0x3b, 0x9b, 0xd7, 0x6f, 0xde, 0xbe, 0x7b, 0xeb, 0x93, 0x3b, 0x77, 0x6e, 0xa1, 0xeb, 0x70, 0xa9,
-	0x51, 0x7e, 0x7c, 0x92, 0x25, 0x2d, 0x5f, 0xa6, 0x2c, 0xd2, 0x09, 0x8e, 0x4f, 0x41, 0x79, 0x50,
-	0x6f, 0x55, 0xca, 0x75, 0x63, 0x5f, 0x6b, 0x56, 0x5b, 0xfb, 0xc6, 0xa9, 0xfc, 0x99, 0x24, 0xff,
-	0x8d, 0x7b, 0xb7, 0xae, 0x7f, 0x42, 0xf9, 0x95, 0xdf, 0xe4, 0x01, 0xb6, 0xbd, 0xc1, 0xd0, 0x73,
-	0x31, 0xd1, 0xf9, 0xfb, 0x00, 0xa1, 0x6f, 0xba, 0x41, 0xdf, 0xf3, 0x07, 0xc1, 0x7a, 0xfa, 0x6a,
-	0xf6, 0x5a, 0x71, 0xf3, 0xdb, 0x1b, 0x2f, 0x1c, 0xfa, 0x8d, 0x09, 0xc4, 0x46, 0x37, 0xe6, 0x57,
-	0xdd, 0xd0, 0x1f, 0xeb, 0x02, 0x20, 0xea, 0x41, 0x69, 0xd8, 0xf3, 0x1c, 0x07, 0xf7, 0x42, 0xdb,
-	0x73, 0x83, 0xf5, 0x0c, 0x15, 0x70, 0x7f, 0x3e, 0x01, 0x6d, 0x01, 0x81, 0x89, 0x48, 0x80, 0xa2,
-	0x31, 0x5c, 0x7a, 0x66, 0xbb, 0x96, 0xf7, 0xcc, 0x76, 0x0f, 0x8d, 0x20, 0xf4, 0xcd, 0x10, 0x1f,
-	0xda, 0x38, 0x58, 0xcf, 0x52, 0x61, 0xb5, 0xf9, 0x84, 0xed, 0x47, 0x48, 0x9d, 0x18, 0x88, 0xc9,
-	0xbc, 0xf8, 0xec, 0xe4, 0x1b, 0xf4, 0x19, 0x2c, 0xf6, 0x3c, 0x0b, 0xfb, 0xc1, 0xfa, 0x02, 0x15,
-	0x76, 0x6f, 0x3e, 0x61, 0xdb, 0x94, 0x97, 0xe1, 0x73, 0x20, 0x32, 0x64, 0xd8, 0x3d, 0xb6, 0x7d,
-	0xcf, 0x1d, 0x90, 0x36, 0xeb, 0xb9, 0xf3, 0x0c, 0x99, 0x2a, 0x20, 0xf0, 0x21, 0x13, 0x41, 0x65,
-	0x07, 0x56, 0xa7, 0xa6, 0x0d, 0x49, 0x90, 0x7d, 0x8a, 0xc7, 0xeb, 0xe9, 0xab, 0xe9, 0x6b, 0x05,
-	0x9d, 0xfc, 0x44, 0xdb, 0x90, 0x3b, 0x36, 0x9d, 0x11, 0x5e, 0xcf, 0x5c, 0x4d, 0x5f, 0x2b, 0x6e,
-	0x7e, 0x34, 0x83, 0x0a, 0xed, 0x18, 0x55, 0x67, 0xbc, 0x5b, 0x99, 0xbb, 0x69, 0xd9, 0x83, 0x0b,
-	0x27, 0xe6, 0xf0, 0x14, 0x79, 0xd5, 0xa4, 0xbc, 0x8d, 0x59, 0xe4, 0x6d, 0xc7, 0xb0, 0xa2, 0xc0,
-	0x1f, 0xc2, 0xfa, 0x59, 0xf3, 0x78, 0x8a, 0xdc, 0x87, 0x49, 0xb9, 0x9f, 0xcc, 0x20, 0x77, 0x1a,
-	0x7d, 0x2c, 0x4a, 0xef, 0x41, 0x51, 0x98, 0xd8, 0x53, 0x04, 0x7e, 0x9a, 0x14, 0x78, 0x6d, 0xa6,
-	0xb9, 0xb5, 0xb0, 0x3f, 0x35, 0xa6, 0x27, 0x26, 0xf9, 0xd5, 0x8c, 0xa9, 0x00, 0x2b, 0x08, 0x54,
-	0xfe, 0x27, 0x0d, 0xf9, 0x36, 0x6f, 0x86, 0x1a, 0x00, 0xbd, 0x78, 0xb5, 0x51, 0x79, 0xb3, 0xad,
-	0x8f, 0xc9, 0x12, 0xd5, 0x05, 0x00, 0xf4, 0x21, 0x20, 0xdf, 0xf3, 0x42, 0x23, 0x3e, 0x39, 0x0c,
-	0xdb, 0x62, 0x87, 0x45, 0x41, 0x97, 0xc8, 0x9b, 0x78, 0x59, 0x69, 0x16, 0xd9, 0x74, 0x25, 0xcb,
-	0x0e, 0x86, 0x8e, 0x39, 0x36, 0x2c, 0x33, 0x34, 0xf9, 0x3e, 0x9f, 0xa5, 0x6b, 0x55, 0xc6, 0x56,
-	0x35, 0x43, 0x53, 0x2f, 0x5a, 0x93, 0x07, 0xa4, 0x40, 0xc9, 0xc7, 0x3f, 0x18, 0xd9, 0x3e, 0x66,
-	0x9b, 0x6e, 0x81, 0x8a, 0x4e, 0xd0, 0x94, 0x5f, 0x2f, 0x00, 0x4c, 0xd6, 0x37, 0xba, 0x02, 0xc5,
-	0x91, 0x6b, 0xff, 0x60, 0x84, 0x0d, 0xd7, 0x1c, 0xe0, 0xf5, 0x1c, 0x1d, 0x73, 0x60, 0xa4, 0xa6,
-	0x39, 0xc0, 0x68, 0x1b, 0x16, 0x82, 0x21, 0xee, 0xf1, 0xd1, 0xf9, 0x78, 0x06, 0xf5, 0x6a, 0x23,
-	0x97, 0x2e, 0xe5, 0xce, 0x10, 0xf7, 0x74, 0xca, 0x8c, 0xde, 0x81, 0xe5, 0x60, 0x74, 0x20, 0x1c,
-	0xd1, 0x6c, 0x50, 0x92, 0x44, 0x72, 0x0c, 0xd9, 0xee, 0x70, 0x14, 0x46, 0x67, 0xde, 0xbd, 0xb9,
-	0xb6, 0xea, 0x86, 0x46, 0x79, 0xf9, 0x31, 0xc4, 0x80, 0x50, 0x17, 0x96, 0xbc, 0x51, 0x48, 0x31,
-	0xd9, 0xd1, 0xb6, 0x35, 0x1f, 0x66, 0x8b, 0x31, 0x33, 0xd0, 0x08, 0xea, 0xc4, 0xd4, 0x2d, 0xbe,
-	0xfc, 0xd4, 0x7d, 0x03, 0x56, 0x84, 0xa3, 0xcd, 0xb0, 0xad, 0xf5, 0x25, 0x3a, 0x15, 0xcb, 0x02,
-	0x55, 0xb3, 0xe4, 0x7b, 0x50, 0x14, 0xba, 0x79, 0xca, 0x4e, 0xb9, 0x24, 0xee, 0x94, 0x82, 0xb8,
-	0xd5, 0xb6, 0xa0, 0x24, 0xf6, 0x66, 0x1e, 0x5e, 0xe5, 0x97, 0x2b, 0x70, 0xb1, 0x13, 0x9a, 0xae,
-	0x65, 0xfa, 0xd6, 0x64, 0x74, 0x02, 0xe5, 0x4f, 0xb2, 0x00, 0x6d, 0xdf, 0x1e, 0xd8, 0xa1, 0x7d,
-	0x8c, 0x03, 0xf4, 0x3e, 0x2c, 0xb6, 0xcb, 0xba, 0x51, 0x6d, 0x49, 0x29, 0xf9, 0xcd, 0x5f, 0x90,
-	0x9b, 0xfb, 0x35, 0x32, 0x0e, 0x5b, 0xf1, 0x1c, 0x6f, 0x0d, 0x4d, 0xdf, 0xf2, 0xb6, 0x8e, 0x6f,
-	0xa0, 0x0f, 0x61, 0xa9, 0x56, 0x2f, 0x77, 0xbb, 0x6a, 0x53, 0x4a, 0xcb, 0x57, 0x68, 0xdb, 0xd7,
-	0xa7, 0xda, 0xf6, 0x1d, 0x33, 0x0c, 0xb1, 0x4b, 0x5a, 0xdf, 0x86, 0xd2, 0x03, 0xbd, 0xb5, 0xdb,
-	0x36, 0x2a, 0x4f, 0x8c, 0x47, 0xea, 0x13, 0x29, 0x23, 0xbf, 0x43, 0x59, 0xbe, 0x3e, 0xc5, 0x72,
-	0xe8, 0x7b, 0xa3, 0xa1, 0x71, 0x30, 0x36, 0x9e, 0xe2, 0x31, 0x97, 0xa2, 0x35, 0xda, 0xbb, 0xf5,
-	0x8e, 0x2a, 0x65, 0xcf, 0x90, 0x62, 0x0f, 0x86, 0x23, 0x27, 0xc0, 0xa4, 0xf5, 0x1d, 0x58, 0x29,
-	0x77, 0x3a, 0xda, 0x83, 0x26, 0x37, 0x4a, 0x3a, 0xd2, 0x82, 0xfc, 0x36, 0x65, 0x7a, 0x73, 0x8a,
-	0x89, 0x5d, 0xa2, 0x86, 0xed, 0x86, 0xb4, 0x33, 0x37, 0xa1, 0xd8, 0x55, 0x3b, 0x5d, 0xa3, 0xd3,
-	0xd5, 0xd5, 0x72, 0x43, 0xca, 0xc9, 0x0a, 0xe5, 0x7a, 0x63, 0x8a, 0x2b, 0xc4, 0x41, 0x18, 0x84,
-	0x3e, 0x21, 0x1e, 0xdf, 0x40, 0x9f, 0x40, 0xb1, 0x51, 0x6e, 0xc7, 0xa2, 0x16, 0xcf, 0x10, 0x35,
-	0x30, 0x87, 0x06, 0x13, 0x17, 0x10, 0xae, 0xbb, 0xb0, 0xdc, 0x50, 0xf5, 0x07, 0x6a, 0xcc, 0xb7,
-	0x24, 0x7f, 0x83, 0xf2, 0x5d, 0x99, 0xe6, 0xc3, 0xfe, 0x21, 0x16, 0x38, 0x95, 0x10, 0x2e, 0x55,
-	0xf1, 0xd0, 0xc7, 0x3d, 0x33, 0xc4, 0x96, 0x30, 0x69, 0xef, 0xc2, 0x82, 0xae, 0x96, 0xab, 0x52,
-	0x4a, 0x7e, 0x83, 0x02, 0x5d, 0x9e, 0x02, 0xf2, 0xb1, 0x69, 0x71, 0x7d, 0xb7, 0x75, 0xb5, 0xdc,
-	0x55, 0x8d, 0x3d, 0x4d, 0xdd, 0x97, 0xd2, 0x67, 0xe8, 0xdb, 0xf3, 0xb1, 0x19, 0x62, 0xe3, 0xd8,
-	0xc6, 0xcf, 0x88, 0xd4, 0xff, 0x48, 0x73, 0x43, 0x2d, 0xb0, 0x43, 0x1c, 0xa0, 0x6f, 0xc1, 0xea,
-	0x76, 0xab, 0x51, 0xd1, 0x9a, 0xaa, 0xd1, 0x56, 0x75, 0x3a, 0x97, 0x29, 0xf9, 0x3d, 0x0a, 0xf4,
-	0xd6, 0x34, 0x90, 0x37, 0x38, 0xb0, 0x5d, 0x6c, 0x0c, 0xb1, 0x1f, 0x4d, 0xe7, 0xa7, 0x20, 0x45,
-	0xdc, 0xcc, 0x7a, 0xac, 0x3f, 0x91, 0xd2, 0xf2, 0x35, 0xca, 0xae, 0x9c, 0xc1, 0x7e, 0xe8, 0x78,
-	0x07, 0xa6, 0xe3, 0x50, 0xfe, 0xeb, 0x50, 0xd0, 0xd5, 0xce, 0xce, 0x6e, 0xad, 0x56, 0x57, 0xa5,
-	0x8c, 0xfc, 0x16, 0x65, 0xfc, 0xda, 0x89, 0xfe, 0x06, 0x47, 0xa3, 0x7e, 0xdf, 0xc1, 0xbc, 0xd3,
-	0xfb, 0xba, 0xd6, 0x55, 0x8d, 0x9a, 0x56, 0x57, 0x3b, 0x52, 0xf6, 0xac, 0xf5, 0xe0, 0xdb, 0x21,
-	0x36, 0xfa, 0xb6, 0x83, 0xe9, 0x50, 0xff, 0x5b, 0x16, 0x2e, 0x6c, 0x33, 0xf9, 0x82, 0x91, 0xaa,
-	0x83, 0x3c, 0xd5, 0x77, 0xa3, 0xad, 0xab, 0x9c, 0x24, 0xa5, 0xe4, 0x4d, 0x0a, 0xfd, 0xe1, 0xf3,
-	0x87, 0xc1, 0x20, 0x33, 0xc8, 0x48, 0x44, 0xbf, 0x03, 0x50, 0xa6, 0x31, 0xd9, 0xf2, 0x28, 0x6f,
-	0x6f, 0xef, 0x36, 0x76, 0xeb, 0xe5, 0x6e, 0x4b, 0x27, 0x76, 0xf8, 0x16, 0xc5, 0xfe, 0xe4, 0x05,
-	0xd8, 0x6c, 0xcd, 0x98, 0xbd, 0xde, 0x68, 0x30, 0x72, 0xcc, 0xd0, 0xf3, 0xe9, 0x92, 0xfb, 0x1e,
-	0x5c, 0x99, 0x96, 0xa1, 0x3e, 0xee, 0xea, 0xe5, 0xed, 0xae, 0xd1, 0xda, 0xed, 0xb6, 0x77, 0xbb,
-	0xc4, 0x50, 0xbf, 0x43, 0x05, 0xdc, 0x78, 0x81, 0x00, 0xfc, 0x65, 0xe8, 0x9b, 0xbd, 0xd0, 0xe0,
-	0x07, 0x29, 0x41, 0x7f, 0x08, 0x97, 0xe3, 0x39, 0x25, 0x5b, 0x5c, 0xad, 0x1a, 0x7b, 0xe5, 0xfa,
-	0x2e, 0x1d, 0xec, 0x0d, 0x0a, 0x7a, 0xed, 0xac, 0x99, 0x25, 0x9b, 0x1d, 0x5b, 0x06, 0x3d, 0xa6,
-	0x28, 0x96, 0x03, 0xef, 0x4d, 0x6b, 0xba, 0xdd, 0x6a, 0xee, 0xa9, 0x7a, 0xd7, 0xe8, 0xb6, 0x92,
-	0x43, 0xb2, 0x20, 0xdf, 0xa7, 0xe0, 0xf7, 0x5e, 0xa0, 0x71, 0xcf, 0x73, 0x8f, 0xb1, 0x1f, 0x1a,
-	0xa1, 0x37, 0x3d, 0x2e, 0xca, 0x1f, 0x2e, 0xc0, 0xeb, 0x9d, 0xa1, 0x63, 0x87, 0xa1, 0x79, 0xe0,
-	0xe0, 0xb6, 0xe9, 0x57, 0x3d, 0x61, 0xb6, 0xeb, 0xb0, 0xd6, 0x2e, 0x6b, 0xba, 0xb1, 0xaf, 0x75,
-	0x77, 0x0c, 0x5d, 0xed, 0x74, 0x75, 0x6d, 0xbb, 0xab, 0xb5, 0x9a, 0x52, 0x4a, 0xbe, 0x41, 0x25,
-	0xff, 0xbf, 0x29, 0xc9, 0x81, 0xd5, 0x37, 0x86, 0xa6, 0xed, 0x1b, 0xcf, 0xec, 0xf0, 0xc8, 0xf0,
-	0x71, 0x10, 0xfa, 0x36, 0xbd, 0x47, 0x49, 0xcf, 0xaa, 0x70, 0xa1, 0xd3, 0xae, 0x6b, 0xdd, 0x04,
-	0x52, 0x5a, 0xfe, 0x88, 0x22, 0xbd, 0x77, 0x0a, 0x52, 0x40, 0x14, 0x9b, 0x46, 0x69, 0xc2, 0xe5,
-	0xb6, 0xde, 0xda, 0x56, 0x3b, 0x1d, 0x32, 0x36, 0x6a, 0xd5, 0x50, 0xeb, 0x6a, 0x43, 0x6d, 0xd2,
-	0x09, 0x3c, 0x7d, 0xf5, 0x51, 0xa5, 0x7c, 0xaf, 0x87, 0x83, 0x80, 0x0c, 0x07, 0xb6, 0x0c, 0xec,
-	0x30, 0x43, 0x82, 0xe0, 0x55, 0x40, 0x8a, 0xf0, 0x62, 0xa4, 0xac, 0xfc, 0x21, 0x45, 0x7a, 0xf7,
-	0x39, 0x48, 0x22, 0xc6, 0x63, 0xf8, 0x1a, 0xeb, 0x59, 0xb9, 0x59, 0x35, 0x3a, 0xda, 0xe7, 0xaa,
-	0xd8, 0x45, 0x32, 0x4f, 0xa7, 0xaf, 0xac, 0x49, 0x1f, 0x4d, 0xd7, 0x32, 0x02, 0xfb, 0x2b, 0x2c,
-	0x76, 0x96, 0x22, 0x7b, 0xf0, 0x5e, 0xa4, 0x1d, 0xc1, 0x9d, 0xf4, 0x96, 0x8a, 0x4a, 0x48, 0xc9,
-	0xc9, 0x15, 0x2a, 0xe5, 0x5b, 0xcf, 0x51, 0x9a, 0xc8, 0x88, 0xbb, 0x4f, 0xa5, 0x4e, 0x09, 0x54,
-	0x7e, 0x37, 0x0d, 0x97, 0xa3, 0x5b, 0xb2, 0x63, 0x5b, 0x98, 0xde, 0xd4, 0xdd, 0xf1, 0x10, 0x07,
-	0xca, 0x11, 0x2c, 0xa8, 0xee, 0x68, 0x80, 0x3e, 0x86, 0xbc, 0xd6, 0x55, 0xf5, 0x72, 0xa5, 0x4e,
-	0x76, 0xbc, 0x78, 0x00, 0x05, 0xb6, 0x85, 0x0d, 0x6a, 0xb5, 0x6c, 0xd9, 0x21, 0xf6, 0xc9, 0x92,
-	0x22, 0x9d, 0xf8, 0x18, 0xf2, 0x8d, 0xdd, 0x7a, 0x57, 0x6b, 0x94, 0xdb, 0x52, 0xfa, 0x2c, 0x86,
-	0xc1, 0xc8, 0x09, 0xed, 0x81, 0x39, 0x24, 0x4a, 0xfc, 0x22, 0x03, 0x45, 0xc1, 0x9f, 0x98, 0x36,
-	0xf0, 0xd2, 0x27, 0x0c, 0xbc, 0xd7, 0x21, 0x4f, 0x7d, 0x36, 0x62, 0x73, 0xb0, 0x8b, 0x7f, 0x89,
-	0x3e, 0x6b, 0x16, 0x6a, 0x03, 0xd8, 0x81, 0x71, 0xe0, 0x8d, 0x5c, 0x0b, 0x5b, 0xeb, 0xd9, 0xab,
-	0xe9, 0x6b, 0x2b, 0x9b, 0x37, 0x66, 0xb0, 0x72, 0xb4, 0xa0, 0xc2, 0x78, 0x36, 0x48, 0xa7, 0xf5,
-	0x82, 0x1d, 0x3d, 0xa3, 0x4d, 0x58, 0x3b, 0xe1, 0xe4, 0x8e, 0x89, 0xe4, 0x05, 0x2a, 0xf9, 0x84,
-	0x77, 0x3a, 0xd6, 0xac, 0x13, 0xd6, 0x56, 0xee, 0xa5, 0xad, 0x2d, 0xe5, 0xaf, 0x97, 0xa0, 0x44,
-	0x37, 0x6c, 0xdb, 0x1c, 0x3b, 0x9e, 0x69, 0x11, 0x07, 0xc3, 0xf2, 0x8c, 0xbe, 0x7b, 0x6e, 0x33,
-	0xd7, 0xf2, 0x6a, 0x2e, 0xfa, 0xff, 0x50, 0x9c, 0xcc, 0x4a, 0x64, 0xc5, 0xce, 0xe2, 0xf3, 0x8a,
-	0xba, 0x6c, 0xc4, 0x4b, 0x27, 0x8a, 0x44, 0x04, 0x31, 0x81, 0x4a, 0x08, 0xc9, 0xfd, 0x4a, 0xcc,
-	0xea, 0xc8, 0xa6, 0x9d, 0x5f, 0x02, 0x81, 0x20, 0x7a, 0xc7, 0x12, 0x62, 0x02, 0x0a, 0x00, 0x85,
-	0xf6, 0x00, 0xfb, 0x46, 0xdf, 0x1c, 0xd8, 0xce, 0x98, 0x0b, 0x2a, 0x50, 0x41, 0xea, 0xbc, 0x82,
-	0xba, 0x04, 0xa9, 0x46, 0x81, 0x04, 0x71, 0x52, 0x38, 0x45, 0x46, 0xd7, 0xe1, 0x92, 0xb0, 0x99,
-	0x8c, 0x78, 0x3d, 0x32, 0x1b, 0x18, 0x09, 0xef, 0xb6, 0xf9, 0xd2, 0xbc, 0x09, 0x6b, 0xc4, 0xad,
-	0x21, 0x16, 0x95, 0xd1, 0xb7, 0x5d, 0xd3, 0xb1, 0xbf, 0x32, 0xc9, 0xfb, 0xf5, 0xfc, 0xd5, 0xf4,
-	0xb5, 0xbc, 0x7e, 0x29, 0x7a, 0x59, 0x13, 0xde, 0xa1, 0x6f, 0x82, 0xcc, 0x7d, 0xa1, 0xc0, 0x20,
-	0x3a, 0x18, 0x81, 0xe7, 0x87, 0xd8, 0x62, 0xf3, 0xb5, 0x0e, 0x94, 0xf3, 0xb5, 0xa8, 0x05, 0xd1,
-	0xbd, 0x43, 0xdf, 0xd3, 0xb1, 0x27, 0x4b, 0x37, 0x66, 0x0e, 0xe8, 0x91, 0xcf, 0xf9, 0x8a, 0x94,
-	0xef, 0x62, 0xf4, 0xb2, 0x43, 0xdf, 0x51, 0x1e, 0xf9, 0x29, 0xac, 0x4e, 0xcd, 0xe6, 0x29, 0x66,
-	0x77, 0x25, 0xe9, 0xdc, 0x7e, 0x38, 0xc3, 0x20, 0xc7, 0xa0, 0xa2, 0x81, 0x4f, 0x84, 0x25, 0x27,
-	0xf6, 0x15, 0x09, 0x8b, 0x40, 0x45, 0x61, 0xcf, 0x60, 0xed, 0xd4, 0xc9, 0x3d, 0x45, 0xe4, 0x4e,
-	0x52, 0xe4, 0xe6, 0x0c, 0x22, 0xa7, 0xa0, 0x05, 0xc1, 0x0f, 0x17, 0xf2, 0x8b, 0xd2, 0x92, 0xf2,
-	0x8f, 0x59, 0x28, 0xc4, 0x7a, 0x21, 0x1f, 0x2e, 0x13, 0x2b, 0xd5, 0x18, 0x78, 0x96, 0xdd, 0x1f,
-	0x1b, 0xcc, 0x1a, 0x13, 0xbc, 0xd6, 0x6f, 0xce, 0x20, 0x52, 0xc7, 0xa6, 0xd5, 0xa0, 0xfc, 0xfb,
-	0x84, 0x3d, 0x06, 0xdf, 0x49, 0x91, 0xa9, 0x4d, 0xbe, 0x23, 0x32, 0xeb, 0x90, 0x3f, 0x30, 0x0f,
-	0x99, 0x94, 0xcc, 0xcc, 0x87, 0x46, 0xc5, 0x3c, 0x14, 0x91, 0x97, 0x0e, 0xcc, 0x43, 0x8a, 0xf6,
-	0x05, 0xac, 0x30, 0xcb, 0x83, 0x9e, 0x8b, 0x04, 0x33, 0x4b, 0x31, 0x6f, 0xcd, 0x16, 0x8d, 0x60,
-	0x8c, 0x22, 0xf2, 0x72, 0x0c, 0x17, 0x69, 0x4b, 0x1c, 0x09, 0x8a, 0xbc, 0x30, 0xb3, 0xb6, 0x0d,
-	0x73, 0x98, 0xd0, 0x76, 0x60, 0x0e, 0x23, 0xb4, 0x00, 0x87, 0x0c, 0x2d, 0x37, 0x33, 0x5a, 0x07,
-	0x87, 0x09, 0xb4, 0x00, 0x87, 0xe4, 0x67, 0x65, 0x91, 0x45, 0x18, 0x94, 0x5b, 0xb0, 0x7e, 0xd6,
-	0x24, 0x24, 0x2e, 0xa9, 0x74, 0xe2, 0x92, 0x52, 0xee, 0x42, 0x49, 0x1c, 0x55, 0x74, 0x0d, 0xa4,
-	0xe8, 0x92, 0x9e, 0x62, 0x59, 0xe1, 0x74, 0x7e, 0x86, 0x28, 0x3f, 0x4d, 0x03, 0x3a, 0x39, 0x78,
-	0xe4, 0x30, 0x12, 0x4c, 0xbd, 0x69, 0x10, 0x24, 0xbc, 0x8b, 0x0e, 0xa3, 0x26, 0x8d, 0x23, 0x51,
-	0xbb, 0xb1, 0xef, 0xce, 0xb1, 0x1a, 0x12, 0x57, 0x48, 0x81, 0x43, 0xd4, 0x5c, 0x65, 0x0f, 0x4a,
-	0xe2, 0xd0, 0xa3, 0xab, 0x50, 0x62, 0xb6, 0x68, 0x42, 0x13, 0x78, 0x8a, 0xc7, 0x91, 0x06, 0xef,
-	0xc0, 0x0a, 0xdd, 0x22, 0xc6, 0xd4, 0x55, 0x5e, 0xa2, 0xd4, 0xed, 0xc9, 0x50, 0x89, 0x93, 0x30,
-	0xc7, 0x50, 0xfd, 0x2c, 0x0d, 0xab, 0x53, 0x9b, 0x12, 0x75, 0xa0, 0x48, 0x0f, 0x51, 0xcb, 0x1b,
-	0x98, 0x36, 0xbb, 0x39, 0x57, 0x66, 0xde, 0xdd, 0x55, 0xca, 0xc4, 0xec, 0x03, 0x08, 0x63, 0x02,
-	0xba, 0x01, 0x6b, 0x89, 0xeb, 0x67, 0xaa, 0x3f, 0x48, 0xb8, 0x3a, 0x22, 0xdd, 0xbe, 0x03, 0x85,
-	0xd8, 0xe0, 0x50, 0x6e, 0x72, 0x43, 0x6b, 0x15, 0x8a, 0xbb, 0xcd, 0x4e, 0x5b, 0xdd, 0xd6, 0x6a,
-	0x9a, 0x5a, 0x95, 0x52, 0x68, 0x19, 0x0a, 0xbb, 0xcd, 0x4a, 0x6b, 0xb7, 0x59, 0x55, 0xab, 0x52,
-	0x1a, 0x15, 0x61, 0x29, 0x7a, 0xc8, 0x28, 0x7f, 0x9c, 0x86, 0x22, 0x59, 0x7a, 0x91, 0x35, 0xf0,
-	0x00, 0x16, 0x03, 0x6f, 0xe4, 0xf7, 0xf0, 0x79, 0xcd, 0x01, 0xce, 0x3e, 0x65, 0x40, 0x65, 0x5e,
-	0xde, 0x80, 0x52, 0x4c, 0xb8, 0xc0, 0xa2, 0xb6, 0x9a, 0x1b, 0xc6, 0xd6, 0x4b, 0x1d, 0x0a, 0x3c,
-	0x22, 0x71, 0x7e, 0x0b, 0x26, 0xcf, 0x10, 0x6a, 0xae, 0xf2, 0x93, 0x34, 0xac, 0x70, 0xef, 0x35,
-	0x12, 0x90, 0x5c, 0xe0, 0xe9, 0x97, 0x5d, 0xe0, 0x67, 0x6e, 0xb1, 0xcc, 0x59, 0x5b, 0x4c, 0xf9,
-	0xfb, 0x25, 0xb8, 0xd0, 0xc5, 0x41, 0xd8, 0xa1, 0xf1, 0x93, 0x48, 0xaf, 0xb3, 0x8f, 0x05, 0xa4,
-	0xc3, 0x22, 0x3e, 0xa6, 0x51, 0xd0, 0xcc, 0xcc, 0x81, 0xbf, 0x13, 0x02, 0x36, 0x54, 0x02, 0xa1,
-	0x73, 0x24, 0xd4, 0x81, 0x7c, 0x94, 0xab, 0xe3, 0xe7, 0xf3, 0x9d, 0x19, 0x50, 0xcb, 0x43, 0xbb,
-	0x83, 0xfd, 0x63, 0xbb, 0x87, 0xab, 0x71, 0xb2, 0x4e, 0x8f, 0x81, 0xe4, 0x9f, 0xe6, 0x20, 0x47,
-	0xc5, 0xa0, 0x63, 0x58, 0x7d, 0x66, 0x86, 0xd8, 0x1f, 0x98, 0xfe, 0x53, 0x83, 0x8a, 0xe4, 0x43,
-	0xfd, 0xe8, 0xfc, 0xba, 0x6f, 0x94, 0xad, 0x63, 0xd3, 0xed, 0xe1, 0xfd, 0x08, 0x78, 0x27, 0xa5,
-	0xaf, 0xc4, 0x52, 0x98, 0xdc, 0x1f, 0xa5, 0x61, 0x8d, 0x7b, 0x39, 0xe4, 0xfa, 0xa1, 0x9b, 0x9a,
-	0x89, 0x67, 0x47, 0x59, 0xfb, 0xe5, 0xc5, 0xb7, 0x63, 0x78, 0xb2, 0xf9, 0xc9, 0x9d, 0x3a, 0x4c,
-	0x50, 0x98, 0x22, 0x03, 0x58, 0x8e, 0xce, 0x23, 0x26, 0x9f, 0x0d, 0x72, 0xed, 0xa5, 0xe4, 0x5b,
-	0x2a, 0xf7, 0x36, 0x77, 0x52, 0x7a, 0x89, 0xc3, 0xd3, 0x77, 0xb2, 0x06, 0xd2, 0xf4, 0xe8, 0xa0,
-	0xb7, 0x61, 0xd9, 0xc5, 0xcf, 0x8c, 0x78, 0x84, 0xe8, 0x0c, 0x64, 0xf5, 0x92, 0x8b, 0x9f, 0x4d,
-	0x1a, 0x49, 0x90, 0x0d, 0xcd, 0x43, 0xbe, 0x5a, 0xc9, 0x4f, 0xb9, 0x02, 0x6b, 0xa7, 0xf6, 0x14,
-	0xbd, 0x0f, 0x92, 0xc9, 0x5e, 0x18, 0xd6, 0xc8, 0x67, 0x26, 0x2a, 0x83, 0x5c, 0xe5, 0xf4, 0x2a,
-	0x27, 0xcb, 0xbf, 0x97, 0x86, 0xa2, 0xa0, 0x2e, 0xea, 0x41, 0x3e, 0x72, 0x94, 0x79, 0x4a, 0xf3,
-	0xc1, 0xb9, 0x06, 0xa2, 0x1b, 0x25, 0x80, 0x71, 0x84, 0xad, 0xc7, 0xc0, 0x51, 0x57, 0xb2, 0x71,
-	0x57, 0x2a, 0x4b, 0x90, 0xa3, 0x83, 0x2f, 0x7f, 0x17, 0xd0, 0x49, 0x56, 0xf4, 0x1e, 0xac, 0x62,
-	0x97, 0x6c, 0xb2, 0xd8, 0x17, 0xa6, 0xfd, 0x29, 0xe9, 0x2b, 0x9c, 0x1c, 0x35, 0x7c, 0x03, 0x0a,
-	0x71, 0xea, 0x99, 0x0e, 0x55, 0x56, 0x9f, 0x10, 0x94, 0x0d, 0x58, 0xa6, 0x93, 0x10, 0xe8, 0xcc,
-	0x50, 0x47, 0x6f, 0x02, 0xb0, 0xa8, 0x10, 0x4d, 0x9a, 0xa4, 0x69, 0x7e, 0xa0, 0xc0, 0x28, 0x9a,
-	0x15, 0x28, 0xff, 0x9e, 0x85, 0x0b, 0xd4, 0x26, 0xa8, 0xd9, 0x0e, 0x0e, 0xa2, 0xfd, 0xbf, 0x0d,
-	0x0b, 0x81, 0xed, 0x3e, 0x3d, 0x7f, 0x72, 0xc2, 0x76, 0x9f, 0xa2, 0xc7, 0xb0, 0x4a, 0x9c, 0x7d,
-	0x33, 0x34, 0xfa, 0xfc, 0xe5, 0x79, 0xaf, 0xf0, 0x15, 0x86, 0x13, 0xd1, 0xc8, 0x58, 0xb1, 0x53,
-	0x15, 0x5b, 0xcc, 0x28, 0x0d, 0xe8, 0x40, 0xe7, 0xf5, 0x95, 0x88, 0x4c, 0xbb, 0x14, 0xa0, 0x6f,
-	0x81, 0xcc, 0x2b, 0x07, 0x2c, 0x4c, 0x16, 0x99, 0xed, 0x62, 0xcb, 0x08, 0x8e, 0x4c, 0xdf, 0xb2,
-	0xdd, 0x43, 0x6a, 0xb0, 0xe5, 0xf5, 0x75, 0xd6, 0xa2, 0x1a, 0x37, 0xe8, 0xf0, 0xf7, 0x08, 0x27,
-	0xdd, 0x4e, 0xe6, 0x1f, 0x57, 0x67, 0xc9, 0xff, 0x4d, 0x0f, 0xe8, 0xf3, 0x7c, 0xcf, 0xff, 0x53,
-	0x67, 0x46, 0xf9, 0x21, 0xe4, 0xe8, 0xd1, 0xff, 0x6a, 0xf2, 0x4f, 0x1b, 0x70, 0x31, 0xce, 0xd3,
-	0xc5, 0xb7, 0x4d, 0x94, 0x85, 0xba, 0x10, 0xbf, 0xe2, 0x97, 0x4d, 0xa0, 0xfc, 0x6b, 0x0e, 0x56,
-	0xa2, 0x48, 0x0e, 0x4b, 0x82, 0x2a, 0xff, 0x90, 0xe3, 0x96, 0xc5, 0x3b, 0x90, 0xab, 0x3c, 0xe9,
-	0xaa, 0x1d, 0x29, 0x25, 0xbf, 0x4e, 0xc3, 0x31, 0x17, 0x69, 0x38, 0x86, 0xa2, 0x6e, 0x1d, 0x8c,
-	0x43, 0x16, 0x8a, 0xbc, 0x0e, 0xc5, 0x4e, 0x57, 0xd7, 0x9a, 0x0f, 0x8c, 0xdd, 0x6e, 0xed, 0xae,
-	0x04, 0x89, 0xec, 0x03, 0x6b, 0x4b, 0x7c, 0x5a, 0xf7, 0xd0, 0x18, 0x85, 0xfd, 0xbb, 0x84, 0xe3,
-	0xeb, 0x90, 0x79, 0xb4, 0x27, 0xa5, 0xe5, 0xcb, 0xb4, 0xa1, 0x24, 0x34, 0x7c, 0x7a, 0x4c, 0xde,
-	0x2b, 0xb0, 0x50, 0x69, 0xb5, 0xea, 0x52, 0x49, 0x5e, 0xa7, 0x2d, 0x90, 0x28, 0xd6, 0xf3, 0x1c,
-	0xd2, 0xe6, 0x5d, 0x58, 0xdc, 0x2b, 0xeb, 0x5a, 0xb3, 0x2b, 0x65, 0x64, 0x99, 0xb6, 0xba, 0x24,
-	0xb4, 0x3a, 0x36, 0x7d, 0xdb, 0x0d, 0x79, 0xbb, 0x6a, 0x6b, 0xb7, 0x52, 0x57, 0xa5, 0xe2, 0x29,
-	0xed, 0x2c, 0x6f, 0xc4, 0xa3, 0x4f, 0x1f, 0x08, 0xe1, 0xaa, 0x6c, 0x22, 0x3f, 0xc0, 0x5a, 0x8a,
-	0x91, 0xaa, 0x77, 0x20, 0xd7, 0xd5, 0x1a, 0xaa, 0x2e, 0x2d, 0x9c, 0x32, 0x2e, 0xd4, 0x6c, 0x63,
-	0xf9, 0x8b, 0x55, 0xad, 0xd9, 0x55, 0xf5, 0xbd, 0xb8, 0xf4, 0x43, 0xca, 0x25, 0x82, 0xea, 0x1c,
-	0xd8, 0x0d, 0xb1, 0x7f, 0x6c, 0x3a, 0x3c, 0x81, 0xc1, 0x42, 0xf1, 0xcb, 0x75, 0xb5, 0xf9, 0xa0,
-	0xbb, 0x63, 0xb4, 0x75, 0xb5, 0xa6, 0x3d, 0x96, 0x16, 0x13, 0xe1, 0x30, 0xc6, 0xe7, 0x60, 0xf7,
-	0x30, 0x3c, 0x32, 0x86, 0x3e, 0xee, 0xdb, 0x5f, 0x72, 0xae, 0x44, 0xa1, 0x89, 0xb4, 0x74, 0x0a,
-	0x17, 0xcb, 0x11, 0x08, 0xb2, 0x6e, 0xc3, 0x0a, 0x6b, 0x1e, 0x45, 0xa3, 0xa5, 0x7c, 0x22, 0xa7,
-	0xc3, 0xd8, 0xe2, 0xbd, 0xcd, 0x96, 0x2d, 0x0d, 0x88, 0x5e, 0x6a, 0x97, 0xf5, 0x72, 0xc3, 0x98,
-	0xe2, 0x5e, 0x49, 0x24, 0x29, 0x18, 0xf7, 0xd0, 0xf4, 0xcd, 0x81, 0x71, 0x12, 0xa3, 0x0a, 0x6b,
-	0x9d, 0x6e, 0xb9, 0xab, 0x1a, 0x95, 0xf2, 0xf6, 0x23, 0xb5, 0x6a, 0xc4, 0x13, 0x50, 0x90, 0xdf,
-	0xa7, 0x20, 0x6f, 0x27, 0xd6, 0x90, 0x19, 0x62, 0xe3, 0xc0, 0xec, 0x3d, 0xc5, 0x96, 0x21, 0xce,
-	0xc6, 0x55, 0xc8, 0xea, 0xad, 0x7d, 0x69, 0x59, 0x7e, 0x8d, 0xf2, 0x5c, 0x10, 0x78, 0x7c, 0xda,
-	0x47, 0xe5, 0xcf, 0x16, 0x23, 0x53, 0x52, 0x08, 0xb7, 0xbd, 0x5a, 0x53, 0x12, 0xed, 0x41, 0x89,
-	0xe5, 0x14, 0x88, 0x9e, 0xa3, 0x80, 0x5b, 0xc0, 0x37, 0x67, 0x71, 0x3d, 0x09, 0x5b, 0x87, 0x72,
-	0x31, 0x1b, 0xb8, 0x38, 0x98, 0x50, 0xd0, 0xbb, 0xd1, 0xc1, 0x3a, 0x31, 0x1d, 0xd9, 0x0d, 0xb6,
-	0xcc, 0xc8, 0x91, 0x5b, 0x54, 0x85, 0xa5, 0xd0, 0xb7, 0x0f, 0x0f, 0xb1, 0xcf, 0xbd, 0xde, 0x0f,
-	0x66, 0xb9, 0x41, 0x19, 0x87, 0x1e, 0xb1, 0x22, 0x0c, 0x17, 0x62, 0x8b, 0xd4, 0xf6, 0x5c, 0x83,
-	0xb0, 0x50, 0xbf, 0x77, 0x65, 0xf3, 0xee, 0x2c, 0xf6, 0x9f, 0xc0, 0xdb, 0xf0, 0x2c, 0xcc, 0xfa,
-	0x23, 0x99, 0x53, 0x64, 0xe2, 0x4f, 0xf1, 0x1b, 0x90, 0xec, 0x96, 0xf5, 0xc5, 0x99, 0xfd, 0x29,
-	0x96, 0xd6, 0x25, 0x77, 0x35, 0xf7, 0xa7, 0xbc, 0x98, 0x80, 0x0e, 0x40, 0xea, 0x39, 0x1e, 0xb5,
-	0xeb, 0x0e, 0xf0, 0x91, 0x79, 0x6c, 0x7b, 0x3e, 0x8d, 0xaa, 0xad, 0xcc, 0x64, 0xba, 0x6e, 0x33,
-	0xd6, 0x0a, 0xe7, 0x64, 0xf0, 0xab, 0xbd, 0x24, 0x95, 0xda, 0x38, 0x8e, 0x43, 0x57, 0xb1, 0x63,
-	0x86, 0xd8, 0xc5, 0x41, 0x40, 0xc3, 0x70, 0xc4, 0xc6, 0x61, 0xf4, 0x3a, 0x27, 0xa3, 0x2f, 0x60,
-	0xa5, 0xe5, 0x12, 0xc5, 0x22, 0xe6, 0xf5, 0x02, 0x55, 0xe6, 0xf6, 0x2c, 0xdd, 0x4c, 0x30, 0x32,
-	0x5d, 0xa6, 0xd0, 0x88, 0xfb, 0x68, 0x06, 0x81, 0x7d, 0xe8, 0x06, 0x46, 0xe8, 0x19, 0x9e, 0x1b,
-	0x65, 0x40, 0x79, 0x70, 0x0f, 0xf1, 0x97, 0x5d, 0xaf, 0xe5, 0x62, 0xb6, 0xf8, 0x4f, 0xc9, 0xbc,
-	0x17, 0x4f, 0xc9, 0xbc, 0x2b, 0xdf, 0x83, 0xa2, 0xb0, 0x26, 0x95, 0xc6, 0x59, 0x7e, 0xe6, 0x2a,
-	0x14, 0x9b, 0xad, 0x26, 0xcd, 0xc2, 0x69, 0xcd, 0x07, 0x52, 0x9a, 0x12, 0x54, 0xb5, 0xda, 0x61,
-	0x89, 0x39, 0x29, 0x83, 0x10, 0xac, 0x94, 0xeb, 0xba, 0x5a, 0xae, 0xf2, 0x5c, 0x5d, 0x55, 0xca,
-	0x2a, 0xdf, 0x07, 0x69, 0x7a, 0x99, 0x28, 0xda, 0x59, 0x22, 0x56, 0x00, 0xaa, 0x5a, 0x67, 0xbb,
-	0xac, 0x57, 0x99, 0x04, 0x09, 0x4a, 0x71, 0x6e, 0x8b, 0x50, 0x32, 0xa4, 0x85, 0xae, 0xd2, 0x14,
-	0x1d, 0x79, 0xce, 0x2a, 0x9f, 0xc1, 0xea, 0xd4, 0x54, 0x2a, 0x9f, 0x3e, 0xa7, 0x03, 0x6a, 0x43,
-	0xeb, 0x1a, 0xe5, 0xfa, 0x7e, 0xf9, 0x49, 0x47, 0x4a, 0xa3, 0x4b, 0x20, 0x51, 0x82, 0x56, 0x33,
-	0x9a, 0xad, 0xa6, 0xda, 0x68, 0x77, 0x9f, 0x48, 0x19, 0xa5, 0x3d, 0x3d, 0x93, 0xcf, 0x45, 0xac,
-	0x69, 0xba, 0x9a, 0x40, 0xa4, 0x84, 0x24, 0xe2, 0x01, 0xc0, 0x64, 0x25, 0x2b, 0xdd, 0xb3, 0xd0,
-	0x2e, 0xc0, 0xb2, 0xda, 0xac, 0x1a, 0xad, 0x5a, 0x74, 0xa2, 0xa7, 0xc9, 0x88, 0xd6, 0xcb, 0x34,
-	0x03, 0xaf, 0x35, 0x8d, 0x76, 0xb9, 0x49, 0x46, 0x99, 0x68, 0x5d, 0xd6, 0xeb, 0x9a, 0x48, 0xcd,
-	0x2a, 0x0e, 0xc0, 0x24, 0xfa, 0xa0, 0x7c, 0xf1, 0x9c, 0x11, 0x56, 0xf7, 0xd4, 0x66, 0x97, 0x96,
-	0x24, 0x4a, 0x69, 0x74, 0x11, 0x56, 0x79, 0x2a, 0x89, 0xdc, 0xe8, 0x94, 0x98, 0x41, 0x57, 0xe1,
-	0x8d, 0xce, 0x93, 0xe6, 0xf6, 0x8e, 0xde, 0x6a, 0xd2, 0xf4, 0xd2, 0x74, 0x8b, 0xac, 0xf2, 0x4b,
-	0x09, 0x96, 0xf8, 0x69, 0x82, 0x74, 0x28, 0x98, 0xfd, 0x10, 0xfb, 0x86, 0xe9, 0x38, 0xfc, 0x60,
-	0xbd, 0x39, 0xfb, 0x61, 0xb4, 0x51, 0x26, 0xbc, 0x65, 0xc7, 0xd9, 0x49, 0xe9, 0x79, 0x93, 0xff,
-	0x16, 0x30, 0xdd, 0x31, 0x37, 0xb8, 0xe6, 0xc7, 0x74, 0xc7, 0x13, 0x4c, 0x77, 0x8c, 0x76, 0x01,
-	0x18, 0x26, 0x36, 0x7b, 0x47, 0xdc, 0x01, 0xfb, 0x64, 0x5e, 0x50, 0xd5, 0xec, 0x1d, 0xed, 0xa4,
-	0x74, 0xa6, 0x1d, 0x79, 0x40, 0x0e, 0x5c, 0xe4, 0xb0, 0xae, 0x65, 0x78, 0xfd, 0x68, 0x5b, 0x2e,
-	0xcc, 0x1c, 0x9f, 0x4d, 0xe2, 0xbb, 0x56, 0xab, 0xcf, 0xf6, 0xef, 0x4e, 0x4a, 0x97, 0xcc, 0x29,
-	0x1a, 0x0a, 0x61, 0x8d, 0x49, 0x9b, 0x72, 0x6b, 0x79, 0xb4, 0xf2, 0xd3, 0x79, 0xe5, 0x9d, 0x74,
-	0x5f, 0xcd, 0x93, 0x64, 0xf4, 0xb3, 0x34, 0x28, 0x4c, 0x6c, 0x30, 0x76, 0x7b, 0x47, 0xbe, 0xe7,
-	0xd2, 0x94, 0xe1, 0xb4, 0x0e, 0x8b, 0x54, 0x87, 0x87, 0xf3, 0xea, 0xd0, 0x11, 0x30, 0x4f, 0xe8,
-	0x73, 0xc5, 0x7c, 0x7e, 0x13, 0xf4, 0x08, 0x16, 0x4d, 0xe7, 0x99, 0x39, 0x0e, 0xd6, 0x4b, 0x54,
-	0xfc, 0x8d, 0x79, 0xc4, 0x53, 0xc6, 0x9d, 0x94, 0xce, 0x21, 0x50, 0x13, 0x96, 0x2c, 0xdc, 0x37,
-	0x47, 0x4e, 0x48, 0xef, 0x92, 0x19, 0x63, 0xfa, 0x1c, 0xad, 0xca, 0x38, 0x77, 0x52, 0x7a, 0x04,
-	0x82, 0xbe, 0x98, 0xf8, 0xfd, 0x3d, 0x6f, 0xe4, 0x86, 0xf4, 0xf6, 0x98, 0x2d, 0xb8, 0x12, 0xa1,
-	0xaa, 0x51, 0xbc, 0x72, 0xe4, 0x86, 0x82, 0xa3, 0x4f, 0x9f, 0xd1, 0x0e, 0xe4, 0x5c, 0x7c, 0x8c,
-	0xd9, 0x65, 0x53, 0xdc, 0xbc, 0x3e, 0x07, 0x6e, 0x93, 0xf0, 0xed, 0xa4, 0x74, 0x06, 0x40, 0x76,
-	0x87, 0xe7, 0xb3, 0x84, 0x93, 0x33, 0xa6, 0x97, 0xca, 0x7c, 0xbb, 0xa3, 0xe5, 0xd7, 0x18, 0x2f,
-	0xd9, 0x1d, 0x5e, 0xf4, 0x40, 0x66, 0xc7, 0xc7, 0x43, 0x6c, 0xb2, 0x64, 0xd2, 0x7c, 0xb3, 0xa3,
-	0x53, 0x46, 0x32, 0x3b, 0x0c, 0x42, 0x7e, 0x0c, 0xf9, 0xe8, 0xb4, 0x40, 0x75, 0x28, 0xd2, 0x1a,
-	0x3b, 0xda, 0x34, 0x0a, 0x23, 0xcc, 0x63, 0x04, 0x89, 0xec, 0x13, 0x64, 0x77, 0xfc, 0x8a, 0x91,
-	0x9f, 0x40, 0x21, 0x3e, 0x38, 0x5e, 0x31, 0xf4, 0xaf, 0xd3, 0x20, 0x4d, 0x1f, 0x1a, 0xa8, 0x05,
-	0xcb, 0xd8, 0xf4, 0x9d, 0xb1, 0xd1, 0xb7, 0x89, 0x13, 0x16, 0x15, 0x7f, 0xce, 0x23, 0xa4, 0x44,
-	0x01, 0x6a, 0x8c, 0x1f, 0x35, 0xa0, 0x44, 0x6c, 0x9f, 0x18, 0x2f, 0x33, 0x37, 0x5e, 0x91, 0xf0,
-	0x73, 0x38, 0xf9, 0x77, 0xe0, 0xe2, 0x29, 0x07, 0x0f, 0x3a, 0x82, 0x4b, 0x71, 0x08, 0xc5, 0x38,
-	0x51, 0xf1, 0x7e, 0x6b, 0xc6, 0xd8, 0x3b, 0x65, 0x9f, 0x94, 0x38, 0x5f, 0x0c, 0x4f, 0xd0, 0x02,
-	0xf9, 0x2d, 0xb8, 0xf2, 0x82, 0x53, 0x47, 0x2e, 0xc0, 0x12, 0xdf, 0xcb, 0xf2, 0x4d, 0x28, 0x89,
-	0x1b, 0x10, 0xbd, 0x3d, 0xbd, 0xa1, 0xc9, 0xf0, 0xe6, 0x92, 0xbb, 0x52, 0x5e, 0x82, 0x1c, 0xdd,
-	0x5d, 0x72, 0x1e, 0x16, 0xd9, 0x11, 0x23, 0xff, 0x38, 0x0d, 0x85, 0x78, 0x8b, 0xa0, 0x4f, 0x61,
-	0x21, 0xce, 0x2c, 0xcc, 0x37, 0x96, 0x94, 0x8f, 0x58, 0xff, 0xd1, 0x4e, 0x9d, 0x7f, 0x3a, 0x22,
-	0x56, 0xb9, 0x0b, 0x8b, 0x6c, 0x8b, 0xa1, 0x87, 0x00, 0x93, 0x85, 0x75, 0x0e, 0xad, 0x04, 0xee,
-	0x4a, 0x21, 0xf6, 0x4c, 0x94, 0xbf, 0xc9, 0x08, 0x81, 0xb6, 0x49, 0x65, 0x6e, 0x07, 0x72, 0x16,
-	0x76, 0xcc, 0xf1, 0x1c, 0x39, 0xcc, 0x93, 0x28, 0x1b, 0x55, 0x02, 0x41, 0xce, 0x2f, 0x8a, 0x85,
-	0x3e, 0x87, 0xbc, 0xe9, 0xd8, 0x87, 0xae, 0x11, 0x7a, 0x7c, 0x4c, 0xbe, 0x7d, 0x3e, 0xdc, 0x32,
-	0x41, 0xe9, 0x7a, 0xe4, 0x14, 0x37, 0xd9, 0x4f, 0xf9, 0x03, 0xc8, 0x51, 0x69, 0xe8, 0x2d, 0x28,
-	0x51, 0x69, 0xc6, 0xc0, 0x76, 0x1c, 0x3b, 0xe0, 0xf1, 0xce, 0x22, 0xa5, 0x35, 0x28, 0x49, 0xbe,
-	0x07, 0x4b, 0x1c, 0x01, 0x5d, 0x86, 0xc5, 0x21, 0xf6, 0x6d, 0x8f, 0xb9, 0x70, 0x59, 0x9d, 0x3f,
-	0x11, 0xba, 0xd7, 0xef, 0x07, 0x38, 0xa4, 0x46, 0x42, 0x56, 0xe7, 0x4f, 0x95, 0x35, 0xb8, 0x78,
-	0xca, 0x1e, 0x50, 0x7e, 0x3f, 0x03, 0x85, 0x38, 0x92, 0x84, 0xf6, 0x60, 0xc5, 0xec, 0xd1, 0xb2,
-	0x9d, 0xa1, 0x19, 0x86, 0xd8, 0x3f, 0xb7, 0x2f, 0xbb, 0xcc, 0x60, 0xda, 0x0c, 0x05, 0xed, 0xc0,
-	0xd2, 0xb1, 0x8d, 0x9f, 0xbd, 0x44, 0x9a, 0x6f, 0x91, 0xf0, 0xd7, 0x5c, 0xf4, 0x5d, 0xb8, 0xc0,
-	0x5d, 0xd8, 0x81, 0x39, 0x1c, 0x12, 0xe3, 0xa0, 0xef, 0x72, 0x73, 0x6b, 0x6e, 0x4c, 0xee, 0x0c,
-	0x37, 0x18, 0x50, 0xcd, 0x55, 0xfe, 0x33, 0x03, 0x17, 0xa2, 0xf8, 0x55, 0xd9, 0x0f, 0xed, 0xbe,
-	0xd9, 0x0b, 0x03, 0xe5, 0xe7, 0x19, 0xc8, 0xd1, 0x72, 0x24, 0xf4, 0x01, 0x2c, 0xd4, 0x34, 0x5a,
-	0x82, 0x74, 0x95, 0x86, 0x07, 0x64, 0x1a, 0x1e, 0x30, 0x79, 0xcb, 0xad, 0x70, 0x3c, 0xc4, 0x5b,
-	0x7d, 0x9b, 0x45, 0x12, 0xae, 0x41, 0x76, 0x57, 0xaf, 0x4f, 0x55, 0xe9, 0x26, 0x9b, 0x8e, 0x7c,
-	0x1a, 0x7d, 0xda, 0x84, 0xbc, 0xda, 0xa8, 0xa8, 0x55, 0x9a, 0x54, 0x4b, 0x54, 0xe8, 0x26, 0x9b,
-	0xe3, 0xc1, 0x01, 0xb6, 0x2c, 0x6c, 0xb1, 0x08, 0xd3, 0x42, 0xfb, 0x49, 0x5b, 0x93, 0xb2, 0xcf,
-	0xd1, 0x64, 0x38, 0x1e, 0xda, 0xac, 0x9a, 0x37, 0xd7, 0x28, 0xef, 0xa9, 0x4d, 0x69, 0x21, 0x11,
-	0xc3, 0x49, 0x36, 0x1e, 0x98, 0xc7, 0xac, 0x66, 0x78, 0x13, 0xf2, 0x55, 0xb5, 0xa6, 0xea, 0xba,
-	0x5a, 0x95, 0x72, 0xcf, 0xd1, 0xc6, 0xc2, 0x7d, 0xec, 0xfb, 0x54, 0x1b, 0xe5, 0x3e, 0xe4, 0x74,
-	0xcf, 0xc1, 0x01, 0xba, 0x0d, 0xd0, 0xe9, 0x96, 0x1f, 0x50, 0x5b, 0xbe, 0x25, 0xa5, 0xe4, 0x77,
-	0x29, 0xfb, 0xd5, 0x24, 0xbb, 0xef, 0x39, 0x78, 0x2b, 0x08, 0xcd, 0x43, 0x6a, 0xda, 0x79, 0x04,
-	0xa0, 0x0c, 0x17, 0xa3, 0xf1, 0xae, 0xd9, 0x4e, 0x9c, 0x3f, 0x43, 0xb0, 0x30, 0x34, 0xc3, 0x23,
-	0x1e, 0x28, 0xa5, 0xbf, 0xc9, 0xfa, 0x0e, 0x8e, 0xcc, 0xcd, 0x5b, 0xb7, 0x79, 0x1e, 0x81, 0x3f,
-	0x29, 0xef, 0x02, 0x8a, 0x20, 0x76, 0x7d, 0x27, 0x42, 0x90, 0x20, 0x3b, 0xf2, 0x9d, 0x28, 0xd2,
-	0x3a, 0xf2, 0x1d, 0xe5, 0x7d, 0xb8, 0xa8, 0xf2, 0x81, 0x9c, 0x12, 0x45, 0xab, 0xa4, 0x58, 0x50,
-	0x9e, 0xfe, 0x56, 0x76, 0xa0, 0xd8, 0x1e, 0xb7, 0xb5, 0xa8, 0xc9, 0x15, 0x28, 0x46, 0x7d, 0x10,
-	0xb2, 0xc9, 0x11, 0x49, 0xb3, 0xd0, 0x3a, 0x2c, 0x1d, 0x63, 0x3f, 0x88, 0x22, 0xe1, 0x05, 0x3d,
-	0x7a, 0x54, 0x3e, 0x83, 0x52, 0x83, 0x0c, 0x70, 0x04, 0x25, 0x43, 0x3e, 0xe2, 0xe3, 0x38, 0xf1,
-	0x33, 0x71, 0xac, 0x7d, 0x4c, 0x0b, 0x7f, 0x3d, 0x7f, 0x6c, 0x10, 0xed, 0x19, 0xd8, 0xf2, 0x84,
-	0xba, 0xeb, 0x3b, 0xca, 0x7d, 0x78, 0xad, 0xca, 0xa7, 0x20, 0xea, 0xb7, 0xd0, 0xe9, 0xa9, 0xf0,
-	0x72, 0xd4, 0xbb, 0x8c, 0xd0, 0xbb, 0xfb, 0xf0, 0x46, 0xc4, 0xd8, 0x61, 0x93, 0xd1, 0xf5, 0xc8,
-	0x2c, 0x0a, 0xdd, 0x25, 0x93, 0x84, 0xad, 0x44, 0x05, 0x1c, 0x23, 0x35, 0xcd, 0x01, 0x56, 0x7e,
-	0x9c, 0x9e, 0xcc, 0x9a, 0xe6, 0xb2, 0x18, 0xbe, 0xed, 0xb9, 0xe8, 0x75, 0xc8, 0x93, 0x15, 0x62,
-	0x8c, 0xf8, 0xf1, 0x51, 0xd0, 0x97, 0xc8, 0xf3, 0xae, 0xef, 0x92, 0x23, 0x8e, 0xbe, 0x1a, 0x32,
-	0x19, 0x5c, 0x9f, 0x22, 0xa1, 0x09, 0xb9, 0x49, 0xb2, 0x40, 0x28, 0x37, 0x0b, 0x4e, 0x2d, 0x91,
-	0x67, 0xce, 0x4d, 0x5f, 0x45, 0xdc, 0x0b, 0x8c, 0xdb, 0x9f, 0x28, 0xad, 0xfc, 0x24, 0x03, 0x45,
-	0xe1, 0x13, 0x16, 0x36, 0xff, 0xd1, 0x74, 0x90, 0x9f, 0x64, 0x92, 0x22, 0xfe, 0x2c, 0xe5, 0x8f,
-	0x1e, 0x4f, 0x14, 0xcc, 0x2d, 0xbc, 0x92, 0x2f, 0x4b, 0x7a, 0xe6, 0xd0, 0x3c, 0xb0, 0x1d, 0x3b,
-	0xb4, 0x31, 0xcb, 0x31, 0x14, 0xf4, 0x04, 0x0d, 0x7d, 0x4e, 0x8e, 0xfd, 0x21, 0x76, 0x2d, 0xec,
-	0xf6, 0x48, 0x1b, 0xf6, 0x55, 0xc4, 0x2c, 0x91, 0x9d, 0x53, 0x06, 0x5f, 0x4f, 0x60, 0x3d, 0x5c,
-	0xc8, 0xa7, 0xa5, 0x8c, 0xf2, 0xf3, 0x34, 0x5c, 0x8a, 0x8e, 0x35, 0xf1, 0xb3, 0x21, 0xe5, 0x47,
-	0x69, 0x28, 0x89, 0x04, 0xf4, 0x0e, 0x2c, 0x56, 0x5b, 0xdb, 0x8f, 0x54, 0x5d, 0x4a, 0x25, 0xc2,
-	0xe5, 0xd8, 0x3d, 0xde, 0xb2, 0xbc, 0xde, 0x53, 0x16, 0x8c, 0x7e, 0x17, 0x96, 0xb8, 0xd3, 0x2e,
-	0xa5, 0x13, 0x41, 0x6b, 0xd2, 0x8c, 0xbb, 0x6d, 0xec, 0x08, 0xcc, 0xab, 0x8f, 0xbb, 0xaa, 0xde,
-	0x2c, 0xd7, 0xa7, 0x02, 0xeb, 0xa4, 0x21, 0xfe, 0x92, 0x5c, 0x0d, 0x26, 0x39, 0x02, 0x95, 0xbb,
-	0xb0, 0x5c, 0xa5, 0xf0, 0xd1, 0x2a, 0x78, 0x0f, 0x56, 0x7b, 0x9e, 0x1b, 0x9a, 0xb6, 0x8b, 0x7d,
-	0xc3, 0x1e, 0x98, 0x87, 0xd1, 0x02, 0x5c, 0x89, 0xc9, 0x1a, 0xa1, 0x2a, 0xff, 0x9c, 0x86, 0x15,
-	0x6e, 0x60, 0x45, 0xbc, 0x2b, 0x90, 0xf1, 0x02, 0xde, 0x3c, 0xe3, 0x05, 0x64, 0xf1, 0x9b, 0x7e,
-	0xef, 0x88, 0x2f, 0x02, 0xfa, 0x9b, 0xac, 0x82, 0x9e, 0x37, 0x18, 0x98, 0x6e, 0x14, 0x01, 0x8d,
-	0x1e, 0x51, 0x1d, 0xb2, 0xd8, 0x3d, 0x9e, 0xe7, 0xb3, 0x97, 0x84, 0xf4, 0x0d, 0xd5, 0x3d, 0x66,
-	0x39, 0x20, 0x02, 0x23, 0xdf, 0x86, 0x7c, 0x44, 0x98, 0xeb, 0xcb, 0x91, 0xff, 0x4a, 0xc3, 0xaa,
-	0xca, 0x07, 0x28, 0xea, 0x97, 0x98, 0x46, 0x4f, 0xbf, 0xa2, 0x34, 0x3a, 0xda, 0x83, 0x45, 0x1a,
-	0x51, 0x8f, 0xf2, 0xfd, 0xb3, 0xf8, 0xf8, 0x53, 0x8a, 0x6d, 0xb4, 0x29, 0x00, 0xff, 0x82, 0x88,
-	0xa1, 0xc9, 0xf7, 0xa0, 0x28, 0x90, 0xe7, 0xea, 0xfb, 0x5f, 0x09, 0xb7, 0x70, 0xdb, 0xf7, 0x42,
-	0xaf, 0xe7, 0x39, 0x81, 0xf2, 0xcb, 0x0c, 0x8f, 0x3a, 0xed, 0xc0, 0xeb, 0x75, 0xf5, 0x41, 0x79,
-	0xfb, 0x89, 0xd1, 0xd6, 0x5b, 0x0f, 0x74, 0xb5, 0xd3, 0x31, 0x74, 0xb5, 0xdd, 0xd2, 0x69, 0xcc,
-	0x2e, 0x95, 0x08, 0xf6, 0x0f, 0x39, 0x37, 0xf9, 0x71, 0xe8, 0x13, 0xb3, 0x86, 0x9c, 0xa1, 0x7e,
-	0x68, 0xbb, 0x87, 0x5b, 0xc7, 0xd7, 0x51, 0x19, 0xd0, 0x29, 0x10, 0xe9, 0xd9, 0x21, 0x6e, 0xa0,
-	0x3b, 0xb0, 0xbc, 0xdf, 0xd2, 0x1f, 0xa9, 0xba, 0xd1, 0xe9, 0x96, 0xbb, 0xbb, 0x9d, 0xa9, 0x0b,
-	0x3c, 0xe6, 0x7e, 0xe6, 0xf9, 0x4f, 0xb1, 0xcf, 0xe3, 0xf9, 0x84, 0x71, 0x17, 0xde, 0xa0, 0x05,
-	0xca, 0xc6, 0x76, 0x4b, 0x57, 0x8d, 0xca, 0x6e, 0xb3, 0x5a, 0x57, 0x85, 0x50, 0x98, 0x94, 0x95,
-	0x6f, 0x52, 0x9c, 0x8f, 0x92, 0x38, 0xb4, 0x64, 0xd9, 0xe8, 0x79, 0x3e, 0x36, 0x0e, 0x46, 0xae,
-	0x45, 0x0e, 0xc3, 0xd8, 0xbb, 0x20, 0x1b, 0xe9, 0x57, 0xd9, 0xc9, 0x56, 0xd7, 0xc5, 0xef, 0xd7,
-	0xfe, 0x20, 0xcb, 0x87, 0xef, 0x21, 0xbc, 0xa1, 0xab, 0x9f, 0xed, 0x6a, 0xba, 0xda, 0xa1, 0x3a,
-	0xab, 0xb5, 0xdd, 0xba, 0x28, 0x38, 0x95, 0xc8, 0xb9, 0x08, 0xdf, 0xc0, 0xf1, 0x8f, 0x90, 0x68,
-	0xea, 0xa4, 0x3f, 0x72, 0x58, 0x61, 0xfc, 0x04, 0x8b, 0x77, 0xa1, 0xa6, 0x35, 0xcb, 0x75, 0xed,
-	0xf3, 0x32, 0xaf, 0xb4, 0x17, 0x8b, 0xda, 0x4f, 0x62, 0x89, 0x15, 0xa8, 0x04, 0xef, 0x01, 0xac,
-	0x89, 0xba, 0x55, 0xea, 0xaa, 0xa1, 0x35, 0xdb, 0xbb, 0x5d, 0x29, 0xf3, 0x02, 0x20, 0xb1, 0xb0,
-	0x94, 0x00, 0x7d, 0x06, 0x72, 0x0c, 0xd4, 0xd5, 0x1a, 0xaa, 0xd1, 0x69, 0xe9, 0x5d, 0xb5, 0xca,
-	0xd1, 0xb2, 0x89, 0x4f, 0x09, 0x4e, 0xa2, 0x9d, 0xa8, 0x71, 0x25, 0x90, 0x0d, 0x58, 0x9f, 0xe8,
-	0xd6, 0xae, 0x6b, 0x5d, 0xa6, 0x5f, 0xb5, 0x55, 0x23, 0x86, 0xd5, 0xc7, 0x14, 0xf0, 0xfd, 0xb3,
-	0xd4, 0x8b, 0x3f, 0x77, 0x30, 0x2c, 0xaf, 0x4f, 0xba, 0xaa, 0x6c, 0x41, 0x49, 0xb4, 0x45, 0xa3,
-	0x8b, 0x2a, 0x3d, 0xc3, 0x45, 0xa5, 0xe0, 0xc9, 0x57, 0x65, 0xc2, 0xcd, 0xa3, 0x34, 0xa1, 0x28,
-	0x3c, 0xa2, 0xfb, 0xb0, 0x5a, 0x2f, 0x57, 0xd4, 0x7a, 0x5d, 0xad, 0x1a, 0x2c, 0xa7, 0x2a, 0xa5,
-	0xe4, 0x0f, 0xa8, 0x9e, 0xef, 0x50, 0x3d, 0xc5, 0xcb, 0x6e, 0xcb, 0x31, 0x0f, 0xb0, 0xe3, 0x60,
-	0xcb, 0x60, 0xe9, 0x55, 0xa2, 0xe2, 0x36, 0xac, 0xd5, 0x39, 0xb5, 0x43, 0x89, 0xd1, 0x41, 0x74,
-	0x09, 0x72, 0xb4, 0x39, 0xd7, 0x96, 0x3d, 0x9c, 0xbe, 0xa1, 0x95, 0x7b, 0x49, 0xa5, 0x9e, 0xdb,
-	0xcd, 0x4c, 0xb2, 0x9b, 0xff, 0xb2, 0x04, 0x6b, 0x0d, 0x1c, 0x04, 0xe6, 0x21, 0xde, 0xb7, 0xc3,
-	0x23, 0xe1, 0x23, 0x91, 0x57, 0xfc, 0x01, 0xea, 0x77, 0x20, 0x47, 0xf3, 0x61, 0xf3, 0x7e, 0x91,
-	0x4b, 0xfc, 0x43, 0xca, 0x88, 0xbe, 0x47, 0xae, 0x2b, 0xfe, 0x05, 0x8c, 0x30, 0x67, 0xb3, 0x45,
-	0xa4, 0x92, 0x45, 0x63, 0x3b, 0x29, 0x9d, 0xd7, 0xb4, 0xc6, 0x26, 0xd1, 0x1e, 0x2c, 0x47, 0x25,
-	0x16, 0xf3, 0x96, 0xa2, 0x8a, 0x6b, 0x6c, 0x27, 0xa5, 0x97, 0xfa, 0xe2, 0x9a, 0xdb, 0x87, 0x95,
-	0xa1, 0xe9, 0x1b, 0x96, 0x17, 0x2b, 0xbd, 0x38, 0x33, 0xb0, 0x58, 0xaf, 0x4e, 0x80, 0x87, 0xe2,
-	0x67, 0x01, 0x2d, 0x80, 0x61, 0xec, 0x62, 0xf2, 0x58, 0xe7, 0x7c, 0x1f, 0x90, 0xef, 0xa4, 0x74,
-	0x01, 0x02, 0xe9, 0x50, 0x14, 0x3e, 0xfa, 0xe7, 0x71, 0xce, 0x39, 0x3f, 0x11, 0xdf, 0x49, 0xe9,
-	0x22, 0x08, 0xea, 0x40, 0x89, 0x56, 0x3f, 0x47, 0x7d, 0x2f, 0xcc, 0x0c, 0x2a, 0xd4, 0x3c, 0x12,
-	0x50, 0x5f, 0x28, 0x81, 0x6c, 0x00, 0x4c, 0x6a, 0x4a, 0x78, 0x54, 0x72, 0xae, 0x62, 0x8e, 0x9d,
-	0x94, 0x5e, 0x88, 0x8b, 0x47, 0x50, 0x1f, 0x2e, 0x0a, 0xdf, 0x4c, 0xc6, 0xaa, 0x96, 0xe6, 0xfc,
-	0x54, 0x5d, 0x28, 0x7a, 0xdc, 0x49, 0xe9, 0xdc, 0x81, 0x16, 0x2b, 0x21, 0x31, 0xa0, 0x93, 0xdf,
-	0x97, 0xac, 0x2f, 0x9f, 0xff, 0x8b, 0xf8, 0x89, 0x18, 0x81, 0x58, 0x59, 0x84, 0x05, 0xdf, 0xf3,
-	0x42, 0xe5, 0x37, 0x00, 0x97, 0xd5, 0x2f, 0x71, 0x6f, 0x44, 0xcf, 0x44, 0xe2, 0x7d, 0xc4, 0x6b,
-	0xbd, 0x0d, 0x45, 0x21, 0x81, 0xc8, 0xf7, 0xf6, 0xbc, 0x1f, 0xae, 0x8b, 0x10, 0xe8, 0x2b, 0x32,
-	0x86, 0x7e, 0x54, 0xe2, 0x1b, 0xe0, 0x30, 0xa4, 0x91, 0x46, 0xf6, 0x69, 0xc6, 0xc3, 0x99, 0xcc,
-	0x9d, 0xd3, 0x34, 0xdd, 0xd8, 0xb7, 0x7d, 0x56, 0x21, 0xdc, 0x61, 0x90, 0xa4, 0xc3, 0x49, 0x4a,
-	0x40, 0xce, 0x44, 0xb6, 0x12, 0xf8, 0x99, 0x68, 0xf3, 0x59, 0x3d, 0xe5, 0x7b, 0x17, 0xf5, 0xfc,
-	0x9a, 0xc4, 0x8b, 0x47, 0xb3, 0x12, 0x5f, 0xbd, 0x7c, 0x3d, 0xf1, 0xf7, 0x1e, 0xec, 0xab, 0x76,
-	0xf1, 0xff, 0x39, 0xd6, 0x27, 0x5f, 0x79, 0x33, 0xc7, 0x24, 0xfe, 0x52, 0x3b, 0x79, 0xc0, 0x2e,
-	0xbe, 0xec, 0x01, 0xdb, 0x87, 0xe2, 0x28, 0xe0, 0xe6, 0x0f, 0x0e, 0xd6, 0x97, 0x5e, 0xb6, 0xc3,
-	0xbb, 0x01, 0xf6, 0x69, 0x39, 0x36, 0xe9, 0xf0, 0x28, 0x7a, 0x08, 0xd0, 0x13, 0x58, 0xa4, 0x15,
-	0x33, 0xc1, 0x7a, 0x9e, 0x8a, 0x28, 0x9f, 0x5f, 0x04, 0xad, 0xda, 0xd6, 0x2c, 0x9d, 0x03, 0xa2,
-	0x01, 0x2c, 0x4f, 0x6a, 0xa8, 0x89, 0x9b, 0x06, 0x33, 0x97, 0x16, 0x3e, 0x4f, 0x02, 0x2b, 0xc9,
-	0xd6, 0x2c, 0x3d, 0x89, 0x2e, 0xb7, 0xa0, 0x28, 0xcc, 0x2a, 0xf5, 0x9b, 0x85, 0x7f, 0x47, 0xe0,
-	0xf7, 0x67, 0x31, 0x9c, 0xfc, 0x31, 0x02, 0x7a, 0x13, 0xc0, 0xf1, 0x7a, 0xa6, 0xc3, 0xbc, 0x75,
-	0xb6, 0xde, 0x0a, 0x94, 0x42, 0x9c, 0x75, 0x02, 0x28, 0x8c, 0xda, 0x2b, 0x00, 0x7c, 0x04, 0x4b,
-	0x7c, 0x8c, 0x5e, 0x01, 0xd8, 0x67, 0xb0, 0x9c, 0x18, 0x8e, 0x57, 0x00, 0xf9, 0x4f, 0x69, 0x90,
-	0xa6, 0xb7, 0xe8, 0x3c, 0xe6, 0x07, 0xfa, 0x08, 0x10, 0xdd, 0xa0, 0x86, 0xe7, 0x1b, 0x71, 0x89,
-	0x25, 0xf3, 0x16, 0x77, 0x52, 0xfa, 0x2a, 0x7d, 0xd7, 0xf2, 0x5b, 0xbc, 0xd4, 0x12, 0x3d, 0x81,
-	0x1c, 0x9d, 0x42, 0x7e, 0x39, 0xbf, 0xfc, 0xd2, 0x23, 0xd6, 0x05, 0x45, 0xac, 0xe4, 0x61, 0x31,
-	0x34, 0xfd, 0x43, 0x1c, 0x6e, 0xfe, 0x51, 0x5a, 0x2c, 0xe7, 0xe6, 0x7e, 0x1d, 0xfa, 0x0a, 0x16,
-	0x59, 0x51, 0x28, 0x9a, 0x25, 0x45, 0x97, 0xa8, 0x1f, 0x95, 0x5f, 0xa2, 0xbe, 0x5b, 0x49, 0x5d,
-	0x4f, 0x6f, 0xdd, 0x87, 0x3c, 0xfd, 0x07, 0x27, 0x32, 0x96, 0x6f, 0x6d, 0xb0, 0x3f, 0x51, 0xda,
-	0x88, 0xfe, 0x44, 0x89, 0x16, 0x9b, 0xec, 0x11, 0x3b, 0xb0, 0x35, 0x64, 0x7f, 0x01, 0xf4, 0xb7,
-	0xbf, 0xfa, 0xf3, 0xc7, 0xcc, 0xdf, 0x26, 0x5c, 0xbb, 0xbe, 0xbb, 0xa5, 0xc1, 0x32, 0x05, 0xe8,
-	0xf1, 0x7f, 0x52, 0x9a, 0x05, 0xe5, 0xef, 0x22, 0x94, 0xd2, 0x81, 0xf0, 0x8f, 0x4c, 0x95, 0x6f,
-	0xc2, 0x8b, 0xff, 0x36, 0xaa, 0x52, 0xd0, 0x69, 0x3d, 0x68, 0x79, 0x68, 0x7f, 0x5e, 0x8c, 0xe8,
-	0xc6, 0xf1, 0x8d, 0x83, 0x45, 0x2a, 0xee, 0xe6, 0xff, 0x06, 0x00, 0x00, 0xff, 0xff, 0x8b, 0xfc,
-	0x1e, 0xa0, 0x91, 0x4a, 0x00, 0x00,
+	type x struct{}
+	out := protoimpl.TypeBuilder{
+		File: protoimpl.DescBuilder{
+			GoPackagePath: reflect.TypeOf(x{}).PkgPath(),
+			RawDescriptor: file_beam_runner_api_proto_rawDesc,
+			NumEnums:      21,
+			NumMessages:   96,
+			NumExtensions: 2,
+			NumServices:   1,
+		},
+		GoTypes:           file_beam_runner_api_proto_goTypes,
+		DependencyIndexes: file_beam_runner_api_proto_depIdxs,
+		EnumInfos:         file_beam_runner_api_proto_enumTypes,
+		MessageInfos:      file_beam_runner_api_proto_msgTypes,
+		ExtensionInfos:    file_beam_runner_api_proto_extTypes,
+	}.Build()
+	File_beam_runner_api_proto = out.File
+	file_beam_runner_api_proto_rawDesc = nil
+	file_beam_runner_api_proto_goTypes = nil
+	file_beam_runner_api_proto_depIdxs = nil
 }
 
 // Reference imports to suppress errors if they are not otherwise used.
 var _ context.Context
-var _ grpc.ClientConn
+var _ grpc.ClientConnInterface
 
 // 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
+const _ = grpc.SupportPackageIsVersion6
 
 // TestStreamServiceClient is the client API for TestStreamService service.
 //
@@ -6311,10 +9655,10 @@
 }
 
 type testStreamServiceClient struct {
-	cc *grpc.ClientConn
+	cc grpc.ClientConnInterface
 }
 
-func NewTestStreamServiceClient(cc *grpc.ClientConn) TestStreamServiceClient {
+func NewTestStreamServiceClient(cc grpc.ClientConnInterface) TestStreamServiceClient {
 	return &testStreamServiceClient{cc}
 }
 
@@ -6360,7 +9704,7 @@
 type UnimplementedTestStreamServiceServer struct {
 }
 
-func (*UnimplementedTestStreamServiceServer) Events(req *EventsRequest, srv TestStreamService_EventsServer) error {
+func (*UnimplementedTestStreamServiceServer) Events(*EventsRequest, TestStreamService_EventsServer) error {
 	return status.Errorf(codes.Unimplemented, "method Events not implemented")
 }
 
diff --git a/sdks/go/pkg/beam/model/pipeline_v1/endpoints.pb.go b/sdks/go/pkg/beam/model/pipeline_v1/endpoints.pb.go
index ae6684b..af1a0dd 100644
--- a/sdks/go/pkg/beam/model/pipeline_v1/endpoints.pb.go
+++ b/sdks/go/pkg/beam/model/pipeline_v1/endpoints.pb.go
@@ -1,78 +1,110 @@
+//
+// 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.
+
+//
+// Protocol Buffers describing endpoints containing a service.
+
 // Code generated by protoc-gen-go. DO NOT EDIT.
+// versions:
+// 	protoc-gen-go v1.25.0-devel
+// 	protoc        v3.13.0
 // source: endpoints.proto
 
 package pipeline_v1
 
 import (
-	fmt "fmt"
-	proto "github.com/golang/protobuf/proto"
-	math "math"
+	protoreflect "google.golang.org/protobuf/reflect/protoreflect"
+	protoimpl "google.golang.org/protobuf/runtime/protoimpl"
+	reflect "reflect"
+	sync "sync"
 )
 
-// Reference imports to suppress errors if they are not otherwise used.
-var _ = proto.Marshal
-var _ = fmt.Errorf
-var _ = math.Inf
-
-// This is a compile-time assertion to ensure that this generated file
-// is compatible with the proto package it is being compiled against.
-// A compilation error at this line likely means your copy of the
-// proto package needs to be updated.
-const _ = proto.ProtoPackageIsVersion3 // please upgrade the proto package
+const (
+	// Verify that this generated code is sufficiently up-to-date.
+	_ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion)
+	// Verify that runtime/protoimpl is sufficiently up-to-date.
+	_ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20)
+)
 
 // A description of how to connect to a Beam API endpoint.
 type ApiServiceDescriptor struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
 	// (Required) The URL to connect to.
 	Url string `protobuf:"bytes,1,opt,name=url,proto3" json:"url,omitempty"`
 	// (Optional) The method for authentication. If unspecified, access to the
 	// url is already being performed in a trusted context (e.g. localhost,
 	// private network).
-	Authentication       *AuthenticationSpec `protobuf:"bytes,2,opt,name=authentication,proto3" json:"authentication,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}            `json:"-"`
-	XXX_unrecognized     []byte              `json:"-"`
-	XXX_sizecache        int32               `json:"-"`
+	Authentication *AuthenticationSpec `protobuf:"bytes,2,opt,name=authentication,proto3" json:"authentication,omitempty"`
 }
 
-func (m *ApiServiceDescriptor) Reset()         { *m = ApiServiceDescriptor{} }
-func (m *ApiServiceDescriptor) String() string { return proto.CompactTextString(m) }
-func (*ApiServiceDescriptor) ProtoMessage()    {}
+func (x *ApiServiceDescriptor) Reset() {
+	*x = ApiServiceDescriptor{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_endpoints_proto_msgTypes[0]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *ApiServiceDescriptor) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*ApiServiceDescriptor) ProtoMessage() {}
+
+func (x *ApiServiceDescriptor) ProtoReflect() protoreflect.Message {
+	mi := &file_endpoints_proto_msgTypes[0]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use ApiServiceDescriptor.ProtoReflect.Descriptor instead.
 func (*ApiServiceDescriptor) Descriptor() ([]byte, []int) {
-	return fileDescriptor_6445e0c85107719d, []int{0}
+	return file_endpoints_proto_rawDescGZIP(), []int{0}
 }
 
-func (m *ApiServiceDescriptor) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_ApiServiceDescriptor.Unmarshal(m, b)
-}
-func (m *ApiServiceDescriptor) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_ApiServiceDescriptor.Marshal(b, m, deterministic)
-}
-func (m *ApiServiceDescriptor) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ApiServiceDescriptor.Merge(m, src)
-}
-func (m *ApiServiceDescriptor) XXX_Size() int {
-	return xxx_messageInfo_ApiServiceDescriptor.Size(m)
-}
-func (m *ApiServiceDescriptor) XXX_DiscardUnknown() {
-	xxx_messageInfo_ApiServiceDescriptor.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_ApiServiceDescriptor proto.InternalMessageInfo
-
-func (m *ApiServiceDescriptor) GetUrl() string {
-	if m != nil {
-		return m.Url
+func (x *ApiServiceDescriptor) GetUrl() string {
+	if x != nil {
+		return x.Url
 	}
 	return ""
 }
 
-func (m *ApiServiceDescriptor) GetAuthentication() *AuthenticationSpec {
-	if m != nil {
-		return m.Authentication
+func (x *ApiServiceDescriptor) GetAuthentication() *AuthenticationSpec {
+	if x != nil {
+		return x.Authentication
 	}
 	return nil
 }
 
 type AuthenticationSpec struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
 	// (Required) A URN that describes the accompanying payload.
 	// For any URN that is not recognized (by whomever is inspecting
 	// it) the parameter payload should be treated as opaque and
@@ -80,72 +112,157 @@
 	Urn string `protobuf:"bytes,1,opt,name=urn,proto3" json:"urn,omitempty"`
 	// (Optional) The data specifying any parameters to the URN. If
 	// the URN does not require any arguments, this may be omitted.
-	Payload              []byte   `protobuf:"bytes,2,opt,name=payload,proto3" json:"payload,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
+	Payload []byte `protobuf:"bytes,2,opt,name=payload,proto3" json:"payload,omitempty"`
 }
 
-func (m *AuthenticationSpec) Reset()         { *m = AuthenticationSpec{} }
-func (m *AuthenticationSpec) String() string { return proto.CompactTextString(m) }
-func (*AuthenticationSpec) ProtoMessage()    {}
+func (x *AuthenticationSpec) Reset() {
+	*x = AuthenticationSpec{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_endpoints_proto_msgTypes[1]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *AuthenticationSpec) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*AuthenticationSpec) ProtoMessage() {}
+
+func (x *AuthenticationSpec) ProtoReflect() protoreflect.Message {
+	mi := &file_endpoints_proto_msgTypes[1]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use AuthenticationSpec.ProtoReflect.Descriptor instead.
 func (*AuthenticationSpec) Descriptor() ([]byte, []int) {
-	return fileDescriptor_6445e0c85107719d, []int{1}
+	return file_endpoints_proto_rawDescGZIP(), []int{1}
 }
 
-func (m *AuthenticationSpec) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_AuthenticationSpec.Unmarshal(m, b)
-}
-func (m *AuthenticationSpec) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_AuthenticationSpec.Marshal(b, m, deterministic)
-}
-func (m *AuthenticationSpec) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_AuthenticationSpec.Merge(m, src)
-}
-func (m *AuthenticationSpec) XXX_Size() int {
-	return xxx_messageInfo_AuthenticationSpec.Size(m)
-}
-func (m *AuthenticationSpec) XXX_DiscardUnknown() {
-	xxx_messageInfo_AuthenticationSpec.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_AuthenticationSpec proto.InternalMessageInfo
-
-func (m *AuthenticationSpec) GetUrn() string {
-	if m != nil {
-		return m.Urn
+func (x *AuthenticationSpec) GetUrn() string {
+	if x != nil {
+		return x.Urn
 	}
 	return ""
 }
 
-func (m *AuthenticationSpec) GetPayload() []byte {
-	if m != nil {
-		return m.Payload
+func (x *AuthenticationSpec) GetPayload() []byte {
+	if x != nil {
+		return x.Payload
 	}
 	return nil
 }
 
-func init() {
-	proto.RegisterType((*ApiServiceDescriptor)(nil), "org.apache.beam.model.pipeline.v1.ApiServiceDescriptor")
-	proto.RegisterType((*AuthenticationSpec)(nil), "org.apache.beam.model.pipeline.v1.AuthenticationSpec")
+var File_endpoints_proto protoreflect.FileDescriptor
+
+var file_endpoints_proto_rawDesc = []byte{
+	0x0a, 0x0f, 0x65, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74,
+	0x6f, 0x12, 0x21, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65,
+	0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e,
+	0x65, 0x2e, 0x76, 0x31, 0x22, 0x87, 0x01, 0x0a, 0x14, 0x41, 0x70, 0x69, 0x53, 0x65, 0x72, 0x76,
+	0x69, 0x63, 0x65, 0x44, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x6f, 0x72, 0x12, 0x10, 0x0a,
+	0x03, 0x75, 0x72, 0x6c, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x75, 0x72, 0x6c, 0x12,
+	0x5d, 0x0a, 0x0e, 0x61, 0x75, 0x74, 0x68, 0x65, 0x6e, 0x74, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f,
+	0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x35, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70,
+	0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e,
+	0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x75, 0x74, 0x68,
+	0x65, 0x6e, 0x74, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x70, 0x65, 0x63, 0x52, 0x0e,
+	0x61, 0x75, 0x74, 0x68, 0x65, 0x6e, 0x74, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x22, 0x40,
+	0x0a, 0x12, 0x41, 0x75, 0x74, 0x68, 0x65, 0x6e, 0x74, 0x69, 0x63, 0x61, 0x74, 0x69, 0x6f, 0x6e,
+	0x53, 0x70, 0x65, 0x63, 0x12, 0x10, 0x0a, 0x03, 0x75, 0x72, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28,
+	0x09, 0x52, 0x03, 0x75, 0x72, 0x6e, 0x12, 0x18, 0x0a, 0x07, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61,
+	0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x07, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64,
+	0x42, 0x75, 0x0a, 0x21, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62,
+	0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69,
+	0x6e, 0x65, 0x2e, 0x76, 0x31, 0x42, 0x09, 0x45, 0x6e, 0x64, 0x70, 0x6f, 0x69, 0x6e, 0x74, 0x73,
+	0x5a, 0x45, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x61, 0x70, 0x61,
+	0x63, 0x68, 0x65, 0x2f, 0x62, 0x65, 0x61, 0x6d, 0x2f, 0x73, 0x64, 0x6b, 0x73, 0x2f, 0x67, 0x6f,
+	0x2f, 0x70, 0x6b, 0x67, 0x2f, 0x62, 0x65, 0x61, 0x6d, 0x2f, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2f,
+	0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x5f, 0x76, 0x31, 0x3b, 0x70, 0x69, 0x70, 0x65,
+	0x6c, 0x69, 0x6e, 0x65, 0x5f, 0x76, 0x31, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
 }
 
-func init() { proto.RegisterFile("endpoints.proto", fileDescriptor_6445e0c85107719d) }
+var (
+	file_endpoints_proto_rawDescOnce sync.Once
+	file_endpoints_proto_rawDescData = file_endpoints_proto_rawDesc
+)
 
-var fileDescriptor_6445e0c85107719d = []byte{
-	// 210 bytes of a gzipped FileDescriptorProto
-	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe2, 0xe2, 0x4f, 0xcd, 0x4b, 0x29,
-	0xc8, 0xcf, 0xcc, 0x2b, 0x29, 0xd6, 0x2b, 0x28, 0xca, 0x2f, 0xc9, 0x17, 0x52, 0xcc, 0x2f, 0x4a,
-	0xd7, 0x4b, 0x2c, 0x48, 0x4c, 0xce, 0x48, 0xd5, 0x4b, 0x4a, 0x4d, 0xcc, 0xd5, 0xcb, 0xcd, 0x4f,
-	0x49, 0xcd, 0xd1, 0x2b, 0xc8, 0x2c, 0x48, 0xcd, 0xc9, 0xcc, 0x4b, 0xd5, 0x2b, 0x33, 0x54, 0x6a,
-	0x67, 0xe4, 0x12, 0x71, 0x2c, 0xc8, 0x0c, 0x4e, 0x2d, 0x2a, 0xcb, 0x4c, 0x4e, 0x75, 0x49, 0x2d,
-	0x4e, 0x2e, 0xca, 0x2c, 0x28, 0xc9, 0x2f, 0x12, 0x12, 0xe0, 0x62, 0x2e, 0x2d, 0xca, 0x91, 0x60,
-	0x54, 0x60, 0xd4, 0xe0, 0x0c, 0x02, 0x31, 0x85, 0x62, 0xb9, 0xf8, 0x12, 0x4b, 0x4b, 0x32, 0x52,
-	0xf3, 0x4a, 0x32, 0x93, 0x13, 0x4b, 0x32, 0xf3, 0xf3, 0x24, 0x98, 0x14, 0x18, 0x35, 0xb8, 0x8d,
-	0x4c, 0xf5, 0x08, 0x5a, 0xa3, 0xe7, 0x88, 0xa2, 0x31, 0xb8, 0x20, 0x35, 0x39, 0x08, 0xcd, 0x30,
-	0x25, 0x07, 0x2e, 0x21, 0x4c, 0x55, 0x10, 0x67, 0xe4, 0x21, 0x9c, 0x91, 0x27, 0x24, 0xc1, 0xc5,
-	0x5e, 0x90, 0x58, 0x99, 0x93, 0x9f, 0x98, 0x02, 0xb6, 0x9f, 0x27, 0x08, 0xc6, 0x75, 0xb2, 0xe6,
-	0x22, 0xec, 0x61, 0x27, 0x4e, 0x57, 0x58, 0x20, 0x45, 0x71, 0xc3, 0xc4, 0xe3, 0xcb, 0x0c, 0x93,
-	0xd8, 0xc0, 0x41, 0x66, 0x0c, 0x08, 0x00, 0x00, 0xff, 0xff, 0x1a, 0x49, 0x57, 0x74, 0x45, 0x01,
-	0x00, 0x00,
+func file_endpoints_proto_rawDescGZIP() []byte {
+	file_endpoints_proto_rawDescOnce.Do(func() {
+		file_endpoints_proto_rawDescData = protoimpl.X.CompressGZIP(file_endpoints_proto_rawDescData)
+	})
+	return file_endpoints_proto_rawDescData
+}
+
+var file_endpoints_proto_msgTypes = make([]protoimpl.MessageInfo, 2)
+var file_endpoints_proto_goTypes = []interface{}{
+	(*ApiServiceDescriptor)(nil), // 0: org.apache.beam.model.pipeline.v1.ApiServiceDescriptor
+	(*AuthenticationSpec)(nil),   // 1: org.apache.beam.model.pipeline.v1.AuthenticationSpec
+}
+var file_endpoints_proto_depIdxs = []int32{
+	1, // 0: org.apache.beam.model.pipeline.v1.ApiServiceDescriptor.authentication:type_name -> org.apache.beam.model.pipeline.v1.AuthenticationSpec
+	1, // [1:1] is the sub-list for method output_type
+	1, // [1:1] is the sub-list for method input_type
+	1, // [1:1] is the sub-list for extension type_name
+	1, // [1:1] is the sub-list for extension extendee
+	0, // [0:1] is the sub-list for field type_name
+}
+
+func init() { file_endpoints_proto_init() }
+func file_endpoints_proto_init() {
+	if File_endpoints_proto != nil {
+		return
+	}
+	if !protoimpl.UnsafeEnabled {
+		file_endpoints_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*ApiServiceDescriptor); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_endpoints_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*AuthenticationSpec); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+	}
+	type x struct{}
+	out := protoimpl.TypeBuilder{
+		File: protoimpl.DescBuilder{
+			GoPackagePath: reflect.TypeOf(x{}).PkgPath(),
+			RawDescriptor: file_endpoints_proto_rawDesc,
+			NumEnums:      0,
+			NumMessages:   2,
+			NumExtensions: 0,
+			NumServices:   0,
+		},
+		GoTypes:           file_endpoints_proto_goTypes,
+		DependencyIndexes: file_endpoints_proto_depIdxs,
+		MessageInfos:      file_endpoints_proto_msgTypes,
+	}.Build()
+	File_endpoints_proto = out.File
+	file_endpoints_proto_rawDesc = nil
+	file_endpoints_proto_goTypes = nil
+	file_endpoints_proto_depIdxs = nil
 }
diff --git a/sdks/go/pkg/beam/model/pipeline_v1/external_transforms.pb.go b/sdks/go/pkg/beam/model/pipeline_v1/external_transforms.pb.go
index 5cdf157..d27d0f9 100644
--- a/sdks/go/pkg/beam/model/pipeline_v1/external_transforms.pb.go
+++ b/sdks/go/pkg/beam/model/pipeline_v1/external_transforms.pb.go
@@ -1,142 +1,194 @@
+//
+// 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.
+
+//
+// Protocol Buffers describing the external transforms available.
+
 // Code generated by protoc-gen-go. DO NOT EDIT.
+// versions:
+// 	protoc-gen-go v1.25.0-devel
+// 	protoc        v3.13.0
 // source: external_transforms.proto
 
 package pipeline_v1
 
 import (
-	fmt "fmt"
-	proto "github.com/golang/protobuf/proto"
-	math "math"
+	protoreflect "google.golang.org/protobuf/reflect/protoreflect"
+	protoimpl "google.golang.org/protobuf/runtime/protoimpl"
+	reflect "reflect"
+	sync "sync"
 )
 
-// Reference imports to suppress errors if they are not otherwise used.
-var _ = proto.Marshal
-var _ = fmt.Errorf
-var _ = math.Inf
-
-// This is a compile-time assertion to ensure that this generated file
-// is compatible with the proto package it is being compiled against.
-// A compilation error at this line likely means your copy of the
-// proto package needs to be updated.
-const _ = proto.ProtoPackageIsVersion3 // please upgrade the proto package
-
-type ConfigValue struct {
-	// Coder and its components (in case of a compound Coder)
-	CoderUrn []string `protobuf:"bytes,1,rep,name=coder_urn,json=coderUrn,proto3" json:"coder_urn,omitempty"`
-	// The Payload which is decoded using the coder_urn
-	Payload              []byte   `protobuf:"bytes,2,opt,name=payload,proto3" json:"payload,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
-}
-
-func (m *ConfigValue) Reset()         { *m = ConfigValue{} }
-func (m *ConfigValue) String() string { return proto.CompactTextString(m) }
-func (*ConfigValue) ProtoMessage()    {}
-func (*ConfigValue) Descriptor() ([]byte, []int) {
-	return fileDescriptor_d0efcf8cb92c1e82, []int{0}
-}
-
-func (m *ConfigValue) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_ConfigValue.Unmarshal(m, b)
-}
-func (m *ConfigValue) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_ConfigValue.Marshal(b, m, deterministic)
-}
-func (m *ConfigValue) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ConfigValue.Merge(m, src)
-}
-func (m *ConfigValue) XXX_Size() int {
-	return xxx_messageInfo_ConfigValue.Size(m)
-}
-func (m *ConfigValue) XXX_DiscardUnknown() {
-	xxx_messageInfo_ConfigValue.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_ConfigValue proto.InternalMessageInfo
-
-func (m *ConfigValue) GetCoderUrn() []string {
-	if m != nil {
-		return m.CoderUrn
-	}
-	return nil
-}
-
-func (m *ConfigValue) GetPayload() []byte {
-	if m != nil {
-		return m.Payload
-	}
-	return nil
-}
+const (
+	// Verify that this generated code is sufficiently up-to-date.
+	_ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion)
+	// Verify that runtime/protoimpl is sufficiently up-to-date.
+	_ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20)
+)
 
 // A configuration payload for an external transform.
 // Used as the payload of ExternalTransform as part of an ExpansionRequest.
 type ExternalConfigurationPayload struct {
-	// Configuration key => value
-	Configuration        map[string]*ConfigValue `protobuf:"bytes,1,rep,name=configuration,proto3" json:"configuration,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"`
-	XXX_NoUnkeyedLiteral struct{}                `json:"-"`
-	XXX_unrecognized     []byte                  `json:"-"`
-	XXX_sizecache        int32                   `json:"-"`
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	// A schema for use in beam:coder:row:v1
+	Schema *Schema `protobuf:"bytes,1,opt,name=schema,proto3" json:"schema,omitempty"`
+	// A payload which can be decoded using beam:coder:row:v1 and the given
+	// schema.
+	Payload []byte `protobuf:"bytes,2,opt,name=payload,proto3" json:"payload,omitempty"`
 }
 
-func (m *ExternalConfigurationPayload) Reset()         { *m = ExternalConfigurationPayload{} }
-func (m *ExternalConfigurationPayload) String() string { return proto.CompactTextString(m) }
-func (*ExternalConfigurationPayload) ProtoMessage()    {}
+func (x *ExternalConfigurationPayload) Reset() {
+	*x = ExternalConfigurationPayload{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_external_transforms_proto_msgTypes[0]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *ExternalConfigurationPayload) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*ExternalConfigurationPayload) ProtoMessage() {}
+
+func (x *ExternalConfigurationPayload) ProtoReflect() protoreflect.Message {
+	mi := &file_external_transforms_proto_msgTypes[0]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use ExternalConfigurationPayload.ProtoReflect.Descriptor instead.
 func (*ExternalConfigurationPayload) Descriptor() ([]byte, []int) {
-	return fileDescriptor_d0efcf8cb92c1e82, []int{1}
+	return file_external_transforms_proto_rawDescGZIP(), []int{0}
 }
 
-func (m *ExternalConfigurationPayload) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_ExternalConfigurationPayload.Unmarshal(m, b)
-}
-func (m *ExternalConfigurationPayload) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_ExternalConfigurationPayload.Marshal(b, m, deterministic)
-}
-func (m *ExternalConfigurationPayload) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ExternalConfigurationPayload.Merge(m, src)
-}
-func (m *ExternalConfigurationPayload) XXX_Size() int {
-	return xxx_messageInfo_ExternalConfigurationPayload.Size(m)
-}
-func (m *ExternalConfigurationPayload) XXX_DiscardUnknown() {
-	xxx_messageInfo_ExternalConfigurationPayload.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_ExternalConfigurationPayload proto.InternalMessageInfo
-
-func (m *ExternalConfigurationPayload) GetConfiguration() map[string]*ConfigValue {
-	if m != nil {
-		return m.Configuration
+func (x *ExternalConfigurationPayload) GetSchema() *Schema {
+	if x != nil {
+		return x.Schema
 	}
 	return nil
 }
 
-func init() {
-	proto.RegisterType((*ConfigValue)(nil), "org.apache.beam.model.pipeline.v1.ConfigValue")
-	proto.RegisterType((*ExternalConfigurationPayload)(nil), "org.apache.beam.model.pipeline.v1.ExternalConfigurationPayload")
-	proto.RegisterMapType((map[string]*ConfigValue)(nil), "org.apache.beam.model.pipeline.v1.ExternalConfigurationPayload.ConfigurationEntry")
+func (x *ExternalConfigurationPayload) GetPayload() []byte {
+	if x != nil {
+		return x.Payload
+	}
+	return nil
 }
 
-func init() { proto.RegisterFile("external_transforms.proto", fileDescriptor_d0efcf8cb92c1e82) }
+var File_external_transforms_proto protoreflect.FileDescriptor
 
-var fileDescriptor_d0efcf8cb92c1e82 = []byte{
-	// 278 bytes of a gzipped FileDescriptorProto
-	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x8c, 0x51, 0x4d, 0x4b, 0xc3, 0x40,
-	0x10, 0x25, 0x29, 0x7e, 0x64, 0xa3, 0x20, 0x0b, 0x42, 0xac, 0x1e, 0x62, 0x4f, 0x39, 0x2d, 0xb4,
-	0x5e, 0xc4, 0x63, 0x4d, 0xef, 0xb2, 0xa8, 0x07, 0x2f, 0x61, 0x9b, 0x4c, 0x6b, 0x30, 0x99, 0x5d,
-	0xa6, 0x9b, 0xd0, 0xfc, 0x05, 0x7f, 0xb5, 0x24, 0x31, 0xd2, 0x22, 0x58, 0x6f, 0xbb, 0x8f, 0x37,
-	0x6f, 0xde, 0xbc, 0xc7, 0xae, 0x60, 0x6b, 0x81, 0x50, 0x15, 0x89, 0x25, 0x85, 0x9b, 0x95, 0xa6,
-	0x72, 0x23, 0x0c, 0x69, 0xab, 0xf9, 0xad, 0xa6, 0xb5, 0x50, 0x46, 0xa5, 0xef, 0x20, 0x96, 0xa0,
-	0x4a, 0x51, 0xea, 0x0c, 0x0a, 0x61, 0x72, 0x03, 0x45, 0x8e, 0x20, 0xea, 0xe9, 0xf8, 0xb2, 0xc5,
-	0x13, 0xaa, 0x10, 0x81, 0x12, 0x65, 0xf2, 0x7e, 0x72, 0x12, 0x33, 0xff, 0x51, 0xe3, 0x2a, 0x5f,
-	0xbf, 0xaa, 0xa2, 0x02, 0x7e, 0xcd, 0xbc, 0x54, 0x67, 0x40, 0x49, 0x45, 0x18, 0x38, 0xe1, 0x28,
-	0xf2, 0xe4, 0x69, 0x07, 0xbc, 0x10, 0xf2, 0x80, 0x9d, 0x18, 0xd5, 0x14, 0x5a, 0x65, 0x81, 0x1b,
-	0x3a, 0xd1, 0x99, 0x1c, 0xbe, 0x93, 0x4f, 0x97, 0xdd, 0x2c, 0xbe, 0xdd, 0xf5, 0x72, 0x15, 0x29,
-	0x9b, 0x6b, 0x7c, 0xea, 0x09, 0x7c, 0xcb, 0xce, 0xd3, 0x5d, 0xbc, 0xd3, 0xf6, 0x67, 0x52, 0x1c,
-	0x34, 0x2e, 0xfe, 0xd2, 0x15, 0x7b, 0xe0, 0x02, 0x2d, 0x35, 0x72, 0x7f, 0xd1, 0xd8, 0x30, 0xfe,
-	0x9b, 0xc4, 0x2f, 0xd8, 0xe8, 0x03, 0x9a, 0xc0, 0x09, 0x9d, 0xc8, 0x93, 0xed, 0x93, 0xc7, 0xec,
-	0xa8, 0x6e, 0x23, 0xe8, 0x4e, 0xf3, 0x67, 0xe2, 0x1f, 0xce, 0x76, 0x82, 0x93, 0xfd, 0xf0, 0x83,
-	0x7b, 0xef, 0xcc, 0x63, 0x76, 0xb8, 0x8e, 0x39, 0x1f, 0xce, 0x7a, 0xfe, 0xe9, 0xf2, 0xcd, 0x1f,
-	0x08, 0x49, 0x3d, 0x5d, 0x1e, 0x77, 0xfd, 0xdc, 0x7d, 0x05, 0x00, 0x00, 0xff, 0xff, 0x2e, 0x44,
-	0x94, 0xc3, 0xf6, 0x01, 0x00, 0x00,
+var file_external_transforms_proto_rawDesc = []byte{
+	0x0a, 0x19, 0x65, 0x78, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x5f, 0x74, 0x72, 0x61, 0x6e, 0x73,
+	0x66, 0x6f, 0x72, 0x6d, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x21, 0x6f, 0x72, 0x67,
+	0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64,
+	0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x1a, 0x0c,
+	0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0x7b, 0x0a, 0x1c,
+	0x45, 0x78, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x75, 0x72,
+	0x61, 0x74, 0x69, 0x6f, 0x6e, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x41, 0x0a, 0x06,
+	0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x6f,
+	0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d,
+	0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31,
+	0x2e, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12,
+	0x18, 0x0a, 0x07, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c,
+	0x52, 0x07, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x42, 0x7e, 0x0a, 0x21, 0x6f, 0x72, 0x67,
+	0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64,
+	0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x42, 0x12,
+	0x45, 0x78, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72,
+	0x6d, 0x73, 0x5a, 0x45, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x61,
+	0x70, 0x61, 0x63, 0x68, 0x65, 0x2f, 0x62, 0x65, 0x61, 0x6d, 0x2f, 0x73, 0x64, 0x6b, 0x73, 0x2f,
+	0x67, 0x6f, 0x2f, 0x70, 0x6b, 0x67, 0x2f, 0x62, 0x65, 0x61, 0x6d, 0x2f, 0x6d, 0x6f, 0x64, 0x65,
+	0x6c, 0x2f, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x5f, 0x76, 0x31, 0x3b, 0x70, 0x69,
+	0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x5f, 0x76, 0x31, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f,
+	0x33,
+}
+
+var (
+	file_external_transforms_proto_rawDescOnce sync.Once
+	file_external_transforms_proto_rawDescData = file_external_transforms_proto_rawDesc
+)
+
+func file_external_transforms_proto_rawDescGZIP() []byte {
+	file_external_transforms_proto_rawDescOnce.Do(func() {
+		file_external_transforms_proto_rawDescData = protoimpl.X.CompressGZIP(file_external_transforms_proto_rawDescData)
+	})
+	return file_external_transforms_proto_rawDescData
+}
+
+var file_external_transforms_proto_msgTypes = make([]protoimpl.MessageInfo, 1)
+var file_external_transforms_proto_goTypes = []interface{}{
+	(*ExternalConfigurationPayload)(nil), // 0: org.apache.beam.model.pipeline.v1.ExternalConfigurationPayload
+	(*Schema)(nil),                       // 1: org.apache.beam.model.pipeline.v1.Schema
+}
+var file_external_transforms_proto_depIdxs = []int32{
+	1, // 0: org.apache.beam.model.pipeline.v1.ExternalConfigurationPayload.schema:type_name -> org.apache.beam.model.pipeline.v1.Schema
+	1, // [1:1] is the sub-list for method output_type
+	1, // [1:1] is the sub-list for method input_type
+	1, // [1:1] is the sub-list for extension type_name
+	1, // [1:1] is the sub-list for extension extendee
+	0, // [0:1] is the sub-list for field type_name
+}
+
+func init() { file_external_transforms_proto_init() }
+func file_external_transforms_proto_init() {
+	if File_external_transforms_proto != nil {
+		return
+	}
+	file_schema_proto_init()
+	if !protoimpl.UnsafeEnabled {
+		file_external_transforms_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*ExternalConfigurationPayload); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+	}
+	type x struct{}
+	out := protoimpl.TypeBuilder{
+		File: protoimpl.DescBuilder{
+			GoPackagePath: reflect.TypeOf(x{}).PkgPath(),
+			RawDescriptor: file_external_transforms_proto_rawDesc,
+			NumEnums:      0,
+			NumMessages:   1,
+			NumExtensions: 0,
+			NumServices:   0,
+		},
+		GoTypes:           file_external_transforms_proto_goTypes,
+		DependencyIndexes: file_external_transforms_proto_depIdxs,
+		MessageInfos:      file_external_transforms_proto_msgTypes,
+	}.Build()
+	File_external_transforms_proto = out.File
+	file_external_transforms_proto_rawDesc = nil
+	file_external_transforms_proto_goTypes = nil
+	file_external_transforms_proto_depIdxs = nil
 }
diff --git a/sdks/go/pkg/beam/model/pipeline_v1/metrics.pb.go b/sdks/go/pkg/beam/model/pipeline_v1/metrics.pb.go
index ff5391c..16d4fcb 100644
--- a/sdks/go/pkg/beam/model/pipeline_v1/metrics.pb.go
+++ b/sdks/go/pkg/beam/model/pipeline_v1/metrics.pb.go
@@ -1,26 +1,45 @@
+//
+// 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.
+
+//
+// Protocol Buffers for metrics classes, used in the Fn API, Job API, and by SDKs.
+
 // Code generated by protoc-gen-go. DO NOT EDIT.
+// versions:
+// 	protoc-gen-go v1.25.0-devel
+// 	protoc        v3.13.0
 // source: metrics.proto
 
 package pipeline_v1
 
 import (
-	fmt "fmt"
-	proto "github.com/golang/protobuf/proto"
 	descriptor "github.com/golang/protobuf/protoc-gen-go/descriptor"
-	_ "github.com/golang/protobuf/ptypes/timestamp"
-	math "math"
+	protoreflect "google.golang.org/protobuf/reflect/protoreflect"
+	protoimpl "google.golang.org/protobuf/runtime/protoimpl"
+	reflect "reflect"
+	sync "sync"
 )
 
-// Reference imports to suppress errors if they are not otherwise used.
-var _ = proto.Marshal
-var _ = fmt.Errorf
-var _ = math.Inf
-
-// This is a compile-time assertion to ensure that this generated file
-// is compatible with the proto package it is being compiled against.
-// A compilation error at this line likely means your copy of the
-// proto package needs to be updated.
-const _ = proto.ProtoPackageIsVersion3 // please upgrade the proto package
+const (
+	// Verify that this generated code is sufficiently up-to-date.
+	_ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion)
+	// Verify that runtime/protoimpl is sufficiently up-to-date.
+	_ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20)
+)
 
 type MonitoringInfoSpecs_Enum int32
 
@@ -77,56 +96,77 @@
 	MonitoringInfoSpecs_DATA_CHANNEL_READ_INDEX MonitoringInfoSpecs_Enum = 18
 )
 
-var MonitoringInfoSpecs_Enum_name = map[int32]string{
-	0:  "USER_SUM_INT64",
-	1:  "USER_SUM_DOUBLE",
-	2:  "USER_DISTRIBUTION_INT64",
-	3:  "USER_DISTRIBUTION_DOUBLE",
-	4:  "USER_LATEST_INT64",
-	5:  "USER_LATEST_DOUBLE",
-	6:  "USER_TOP_N_INT64",
-	7:  "USER_TOP_N_DOUBLE",
-	8:  "USER_BOTTOM_N_INT64",
-	9:  "USER_BOTTOM_N_DOUBLE",
-	10: "ELEMENT_COUNT",
-	11: "SAMPLED_BYTE_SIZE",
-	12: "START_BUNDLE_MSECS",
-	13: "PROCESS_BUNDLE_MSECS",
-	14: "FINISH_BUNDLE_MSECS",
-	15: "TOTAL_MSECS",
-	16: "WORK_REMAINING",
-	17: "WORK_COMPLETED",
-	18: "DATA_CHANNEL_READ_INDEX",
-}
+// Enum value maps for MonitoringInfoSpecs_Enum.
+var (
+	MonitoringInfoSpecs_Enum_name = map[int32]string{
+		0:  "USER_SUM_INT64",
+		1:  "USER_SUM_DOUBLE",
+		2:  "USER_DISTRIBUTION_INT64",
+		3:  "USER_DISTRIBUTION_DOUBLE",
+		4:  "USER_LATEST_INT64",
+		5:  "USER_LATEST_DOUBLE",
+		6:  "USER_TOP_N_INT64",
+		7:  "USER_TOP_N_DOUBLE",
+		8:  "USER_BOTTOM_N_INT64",
+		9:  "USER_BOTTOM_N_DOUBLE",
+		10: "ELEMENT_COUNT",
+		11: "SAMPLED_BYTE_SIZE",
+		12: "START_BUNDLE_MSECS",
+		13: "PROCESS_BUNDLE_MSECS",
+		14: "FINISH_BUNDLE_MSECS",
+		15: "TOTAL_MSECS",
+		16: "WORK_REMAINING",
+		17: "WORK_COMPLETED",
+		18: "DATA_CHANNEL_READ_INDEX",
+	}
+	MonitoringInfoSpecs_Enum_value = map[string]int32{
+		"USER_SUM_INT64":           0,
+		"USER_SUM_DOUBLE":          1,
+		"USER_DISTRIBUTION_INT64":  2,
+		"USER_DISTRIBUTION_DOUBLE": 3,
+		"USER_LATEST_INT64":        4,
+		"USER_LATEST_DOUBLE":       5,
+		"USER_TOP_N_INT64":         6,
+		"USER_TOP_N_DOUBLE":        7,
+		"USER_BOTTOM_N_INT64":      8,
+		"USER_BOTTOM_N_DOUBLE":     9,
+		"ELEMENT_COUNT":            10,
+		"SAMPLED_BYTE_SIZE":        11,
+		"START_BUNDLE_MSECS":       12,
+		"PROCESS_BUNDLE_MSECS":     13,
+		"FINISH_BUNDLE_MSECS":      14,
+		"TOTAL_MSECS":              15,
+		"WORK_REMAINING":           16,
+		"WORK_COMPLETED":           17,
+		"DATA_CHANNEL_READ_INDEX":  18,
+	}
+)
 
-var MonitoringInfoSpecs_Enum_value = map[string]int32{
-	"USER_SUM_INT64":           0,
-	"USER_SUM_DOUBLE":          1,
-	"USER_DISTRIBUTION_INT64":  2,
-	"USER_DISTRIBUTION_DOUBLE": 3,
-	"USER_LATEST_INT64":        4,
-	"USER_LATEST_DOUBLE":       5,
-	"USER_TOP_N_INT64":         6,
-	"USER_TOP_N_DOUBLE":        7,
-	"USER_BOTTOM_N_INT64":      8,
-	"USER_BOTTOM_N_DOUBLE":     9,
-	"ELEMENT_COUNT":            10,
-	"SAMPLED_BYTE_SIZE":        11,
-	"START_BUNDLE_MSECS":       12,
-	"PROCESS_BUNDLE_MSECS":     13,
-	"FINISH_BUNDLE_MSECS":      14,
-	"TOTAL_MSECS":              15,
-	"WORK_REMAINING":           16,
-	"WORK_COMPLETED":           17,
-	"DATA_CHANNEL_READ_INDEX":  18,
+func (x MonitoringInfoSpecs_Enum) Enum() *MonitoringInfoSpecs_Enum {
+	p := new(MonitoringInfoSpecs_Enum)
+	*p = x
+	return p
 }
 
 func (x MonitoringInfoSpecs_Enum) String() string {
-	return proto.EnumName(MonitoringInfoSpecs_Enum_name, int32(x))
+	return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x))
 }
 
+func (MonitoringInfoSpecs_Enum) Descriptor() protoreflect.EnumDescriptor {
+	return file_metrics_proto_enumTypes[0].Descriptor()
+}
+
+func (MonitoringInfoSpecs_Enum) Type() protoreflect.EnumType {
+	return &file_metrics_proto_enumTypes[0]
+}
+
+func (x MonitoringInfoSpecs_Enum) Number() protoreflect.EnumNumber {
+	return protoreflect.EnumNumber(x)
+}
+
+// Deprecated: Use MonitoringInfoSpecs_Enum.Descriptor instead.
 func (MonitoringInfoSpecs_Enum) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_6039342a2ba47b72, []int{2, 0}
+	return file_metrics_proto_rawDescGZIP(), []int{2, 0}
 }
 
 type MonitoringInfo_MonitoringInfoLabels int32
@@ -146,32 +186,53 @@
 	MonitoringInfo_NAME               MonitoringInfo_MonitoringInfoLabels = 6
 )
 
-var MonitoringInfo_MonitoringInfoLabels_name = map[int32]string{
-	0: "TRANSFORM",
-	1: "PCOLLECTION",
-	2: "WINDOWING_STRATEGY",
-	3: "CODER",
-	4: "ENVIRONMENT",
-	5: "NAMESPACE",
-	6: "NAME",
-}
+// Enum value maps for MonitoringInfo_MonitoringInfoLabels.
+var (
+	MonitoringInfo_MonitoringInfoLabels_name = map[int32]string{
+		0: "TRANSFORM",
+		1: "PCOLLECTION",
+		2: "WINDOWING_STRATEGY",
+		3: "CODER",
+		4: "ENVIRONMENT",
+		5: "NAMESPACE",
+		6: "NAME",
+	}
+	MonitoringInfo_MonitoringInfoLabels_value = map[string]int32{
+		"TRANSFORM":          0,
+		"PCOLLECTION":        1,
+		"WINDOWING_STRATEGY": 2,
+		"CODER":              3,
+		"ENVIRONMENT":        4,
+		"NAMESPACE":          5,
+		"NAME":               6,
+	}
+)
 
-var MonitoringInfo_MonitoringInfoLabels_value = map[string]int32{
-	"TRANSFORM":          0,
-	"PCOLLECTION":        1,
-	"WINDOWING_STRATEGY": 2,
-	"CODER":              3,
-	"ENVIRONMENT":        4,
-	"NAMESPACE":          5,
-	"NAME":               6,
+func (x MonitoringInfo_MonitoringInfoLabels) Enum() *MonitoringInfo_MonitoringInfoLabels {
+	p := new(MonitoringInfo_MonitoringInfoLabels)
+	*p = x
+	return p
 }
 
 func (x MonitoringInfo_MonitoringInfoLabels) String() string {
-	return proto.EnumName(MonitoringInfo_MonitoringInfoLabels_name, int32(x))
+	return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x))
 }
 
+func (MonitoringInfo_MonitoringInfoLabels) Descriptor() protoreflect.EnumDescriptor {
+	return file_metrics_proto_enumTypes[1].Descriptor()
+}
+
+func (MonitoringInfo_MonitoringInfoLabels) Type() protoreflect.EnumType {
+	return &file_metrics_proto_enumTypes[1]
+}
+
+func (x MonitoringInfo_MonitoringInfoLabels) Number() protoreflect.EnumNumber {
+	return protoreflect.EnumNumber(x)
+}
+
+// Deprecated: Use MonitoringInfo_MonitoringInfoLabels.Descriptor instead.
 func (MonitoringInfo_MonitoringInfoLabels) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_6039342a2ba47b72, []int{4, 0}
+	return file_metrics_proto_rawDescGZIP(), []int{4, 0}
 }
 
 type MonitoringInfoTypeUrns_Enum int32
@@ -257,46 +318,71 @@
 	MonitoringInfoTypeUrns_PROGRESS_TYPE MonitoringInfoTypeUrns_Enum = 10
 )
 
-var MonitoringInfoTypeUrns_Enum_name = map[int32]string{
-	0:  "SUM_INT64_TYPE",
-	1:  "SUM_DOUBLE_TYPE",
-	2:  "DISTRIBUTION_INT64_TYPE",
-	3:  "DISTRIBUTION_DOUBLE_TYPE",
-	4:  "LATEST_INT64_TYPE",
-	5:  "LATEST_DOUBLE_TYPE",
-	6:  "TOP_N_INT64_TYPE",
-	7:  "TOP_N_DOUBLE_TYPE",
-	8:  "BOTTOM_N_INT64_TYPE",
-	9:  "BOTTOM_N_DOUBLE_TYPE",
-	10: "PROGRESS_TYPE",
-}
+// Enum value maps for MonitoringInfoTypeUrns_Enum.
+var (
+	MonitoringInfoTypeUrns_Enum_name = map[int32]string{
+		0:  "SUM_INT64_TYPE",
+		1:  "SUM_DOUBLE_TYPE",
+		2:  "DISTRIBUTION_INT64_TYPE",
+		3:  "DISTRIBUTION_DOUBLE_TYPE",
+		4:  "LATEST_INT64_TYPE",
+		5:  "LATEST_DOUBLE_TYPE",
+		6:  "TOP_N_INT64_TYPE",
+		7:  "TOP_N_DOUBLE_TYPE",
+		8:  "BOTTOM_N_INT64_TYPE",
+		9:  "BOTTOM_N_DOUBLE_TYPE",
+		10: "PROGRESS_TYPE",
+	}
+	MonitoringInfoTypeUrns_Enum_value = map[string]int32{
+		"SUM_INT64_TYPE":           0,
+		"SUM_DOUBLE_TYPE":          1,
+		"DISTRIBUTION_INT64_TYPE":  2,
+		"DISTRIBUTION_DOUBLE_TYPE": 3,
+		"LATEST_INT64_TYPE":        4,
+		"LATEST_DOUBLE_TYPE":       5,
+		"TOP_N_INT64_TYPE":         6,
+		"TOP_N_DOUBLE_TYPE":        7,
+		"BOTTOM_N_INT64_TYPE":      8,
+		"BOTTOM_N_DOUBLE_TYPE":     9,
+		"PROGRESS_TYPE":            10,
+	}
+)
 
-var MonitoringInfoTypeUrns_Enum_value = map[string]int32{
-	"SUM_INT64_TYPE":           0,
-	"SUM_DOUBLE_TYPE":          1,
-	"DISTRIBUTION_INT64_TYPE":  2,
-	"DISTRIBUTION_DOUBLE_TYPE": 3,
-	"LATEST_INT64_TYPE":        4,
-	"LATEST_DOUBLE_TYPE":       5,
-	"TOP_N_INT64_TYPE":         6,
-	"TOP_N_DOUBLE_TYPE":        7,
-	"BOTTOM_N_INT64_TYPE":      8,
-	"BOTTOM_N_DOUBLE_TYPE":     9,
-	"PROGRESS_TYPE":            10,
+func (x MonitoringInfoTypeUrns_Enum) Enum() *MonitoringInfoTypeUrns_Enum {
+	p := new(MonitoringInfoTypeUrns_Enum)
+	*p = x
+	return p
 }
 
 func (x MonitoringInfoTypeUrns_Enum) String() string {
-	return proto.EnumName(MonitoringInfoTypeUrns_Enum_name, int32(x))
+	return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x))
 }
 
+func (MonitoringInfoTypeUrns_Enum) Descriptor() protoreflect.EnumDescriptor {
+	return file_metrics_proto_enumTypes[2].Descriptor()
+}
+
+func (MonitoringInfoTypeUrns_Enum) Type() protoreflect.EnumType {
+	return &file_metrics_proto_enumTypes[2]
+}
+
+func (x MonitoringInfoTypeUrns_Enum) Number() protoreflect.EnumNumber {
+	return protoreflect.EnumNumber(x)
+}
+
+// Deprecated: Use MonitoringInfoTypeUrns_Enum.Descriptor instead.
 func (MonitoringInfoTypeUrns_Enum) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_6039342a2ba47b72, []int{5, 0}
+	return file_metrics_proto_rawDescGZIP(), []int{5, 0}
 }
 
 // A specification for describing a well known MonitoringInfo.
 //
 // All specifications are uniquely identified by the urn.
 type MonitoringInfoSpec struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
 	// Defines the semantic meaning of the metric or monitored state.
 	//
 	// See MonitoringInfoSpecs.Enum for the set of well known metrics/monitored
@@ -310,188 +396,219 @@
 	RequiredLabels []string `protobuf:"bytes,3,rep,name=required_labels,json=requiredLabels,proto3" json:"required_labels,omitempty"`
 	// Extra non functional parts of the spec for descriptive purposes.
 	// i.e. description, units, etc.
-	Annotations          []*Annotation `protobuf:"bytes,4,rep,name=annotations,proto3" json:"annotations,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}      `json:"-"`
-	XXX_unrecognized     []byte        `json:"-"`
-	XXX_sizecache        int32         `json:"-"`
+	Annotations []*Annotation `protobuf:"bytes,4,rep,name=annotations,proto3" json:"annotations,omitempty"`
 }
 
-func (m *MonitoringInfoSpec) Reset()         { *m = MonitoringInfoSpec{} }
-func (m *MonitoringInfoSpec) String() string { return proto.CompactTextString(m) }
-func (*MonitoringInfoSpec) ProtoMessage()    {}
+func (x *MonitoringInfoSpec) Reset() {
+	*x = MonitoringInfoSpec{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_metrics_proto_msgTypes[0]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *MonitoringInfoSpec) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*MonitoringInfoSpec) ProtoMessage() {}
+
+func (x *MonitoringInfoSpec) ProtoReflect() protoreflect.Message {
+	mi := &file_metrics_proto_msgTypes[0]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use MonitoringInfoSpec.ProtoReflect.Descriptor instead.
 func (*MonitoringInfoSpec) Descriptor() ([]byte, []int) {
-	return fileDescriptor_6039342a2ba47b72, []int{0}
+	return file_metrics_proto_rawDescGZIP(), []int{0}
 }
 
-func (m *MonitoringInfoSpec) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_MonitoringInfoSpec.Unmarshal(m, b)
-}
-func (m *MonitoringInfoSpec) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_MonitoringInfoSpec.Marshal(b, m, deterministic)
-}
-func (m *MonitoringInfoSpec) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_MonitoringInfoSpec.Merge(m, src)
-}
-func (m *MonitoringInfoSpec) XXX_Size() int {
-	return xxx_messageInfo_MonitoringInfoSpec.Size(m)
-}
-func (m *MonitoringInfoSpec) XXX_DiscardUnknown() {
-	xxx_messageInfo_MonitoringInfoSpec.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_MonitoringInfoSpec proto.InternalMessageInfo
-
-func (m *MonitoringInfoSpec) GetUrn() string {
-	if m != nil {
-		return m.Urn
+func (x *MonitoringInfoSpec) GetUrn() string {
+	if x != nil {
+		return x.Urn
 	}
 	return ""
 }
 
-func (m *MonitoringInfoSpec) GetType() string {
-	if m != nil {
-		return m.Type
+func (x *MonitoringInfoSpec) GetType() string {
+	if x != nil {
+		return x.Type
 	}
 	return ""
 }
 
-func (m *MonitoringInfoSpec) GetRequiredLabels() []string {
-	if m != nil {
-		return m.RequiredLabels
+func (x *MonitoringInfoSpec) GetRequiredLabels() []string {
+	if x != nil {
+		return x.RequiredLabels
 	}
 	return nil
 }
 
-func (m *MonitoringInfoSpec) GetAnnotations() []*Annotation {
-	if m != nil {
-		return m.Annotations
+func (x *MonitoringInfoSpec) GetAnnotations() []*Annotation {
+	if x != nil {
+		return x.Annotations
 	}
 	return nil
 }
 
 // The key name and value string of MonitoringInfo annotations.
 type Annotation struct {
-	Key                  string   `protobuf:"bytes,1,opt,name=key,proto3" json:"key,omitempty"`
-	Value                string   `protobuf:"bytes,2,opt,name=value,proto3" json:"value,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	Key   string `protobuf:"bytes,1,opt,name=key,proto3" json:"key,omitempty"`
+	Value string `protobuf:"bytes,2,opt,name=value,proto3" json:"value,omitempty"`
 }
 
-func (m *Annotation) Reset()         { *m = Annotation{} }
-func (m *Annotation) String() string { return proto.CompactTextString(m) }
-func (*Annotation) ProtoMessage()    {}
+func (x *Annotation) Reset() {
+	*x = Annotation{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_metrics_proto_msgTypes[1]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *Annotation) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*Annotation) ProtoMessage() {}
+
+func (x *Annotation) ProtoReflect() protoreflect.Message {
+	mi := &file_metrics_proto_msgTypes[1]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use Annotation.ProtoReflect.Descriptor instead.
 func (*Annotation) Descriptor() ([]byte, []int) {
-	return fileDescriptor_6039342a2ba47b72, []int{1}
+	return file_metrics_proto_rawDescGZIP(), []int{1}
 }
 
-func (m *Annotation) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_Annotation.Unmarshal(m, b)
-}
-func (m *Annotation) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_Annotation.Marshal(b, m, deterministic)
-}
-func (m *Annotation) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Annotation.Merge(m, src)
-}
-func (m *Annotation) XXX_Size() int {
-	return xxx_messageInfo_Annotation.Size(m)
-}
-func (m *Annotation) XXX_DiscardUnknown() {
-	xxx_messageInfo_Annotation.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_Annotation proto.InternalMessageInfo
-
-func (m *Annotation) GetKey() string {
-	if m != nil {
-		return m.Key
+func (x *Annotation) GetKey() string {
+	if x != nil {
+		return x.Key
 	}
 	return ""
 }
 
-func (m *Annotation) GetValue() string {
-	if m != nil {
-		return m.Value
+func (x *Annotation) GetValue() string {
+	if x != nil {
+		return x.Value
 	}
 	return ""
 }
 
 // A set of well known MonitoringInfo specifications.
 type MonitoringInfoSpecs struct {
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
 }
 
-func (m *MonitoringInfoSpecs) Reset()         { *m = MonitoringInfoSpecs{} }
-func (m *MonitoringInfoSpecs) String() string { return proto.CompactTextString(m) }
-func (*MonitoringInfoSpecs) ProtoMessage()    {}
+func (x *MonitoringInfoSpecs) Reset() {
+	*x = MonitoringInfoSpecs{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_metrics_proto_msgTypes[2]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *MonitoringInfoSpecs) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*MonitoringInfoSpecs) ProtoMessage() {}
+
+func (x *MonitoringInfoSpecs) ProtoReflect() protoreflect.Message {
+	mi := &file_metrics_proto_msgTypes[2]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use MonitoringInfoSpecs.ProtoReflect.Descriptor instead.
 func (*MonitoringInfoSpecs) Descriptor() ([]byte, []int) {
-	return fileDescriptor_6039342a2ba47b72, []int{2}
+	return file_metrics_proto_rawDescGZIP(), []int{2}
 }
 
-func (m *MonitoringInfoSpecs) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_MonitoringInfoSpecs.Unmarshal(m, b)
-}
-func (m *MonitoringInfoSpecs) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_MonitoringInfoSpecs.Marshal(b, m, deterministic)
-}
-func (m *MonitoringInfoSpecs) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_MonitoringInfoSpecs.Merge(m, src)
-}
-func (m *MonitoringInfoSpecs) XXX_Size() int {
-	return xxx_messageInfo_MonitoringInfoSpecs.Size(m)
-}
-func (m *MonitoringInfoSpecs) XXX_DiscardUnknown() {
-	xxx_messageInfo_MonitoringInfoSpecs.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_MonitoringInfoSpecs proto.InternalMessageInfo
-
 // A set of properties for the MonitoringInfoLabel, this is useful to obtain
 // the proper label string for the MonitoringInfoLabel.
 type MonitoringInfoLabelProps struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
 	// The label key to use in the MonitoringInfo labels map.
-	Name                 string   `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
+	Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"`
 }
 
-func (m *MonitoringInfoLabelProps) Reset()         { *m = MonitoringInfoLabelProps{} }
-func (m *MonitoringInfoLabelProps) String() string { return proto.CompactTextString(m) }
-func (*MonitoringInfoLabelProps) ProtoMessage()    {}
+func (x *MonitoringInfoLabelProps) Reset() {
+	*x = MonitoringInfoLabelProps{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_metrics_proto_msgTypes[3]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *MonitoringInfoLabelProps) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*MonitoringInfoLabelProps) ProtoMessage() {}
+
+func (x *MonitoringInfoLabelProps) ProtoReflect() protoreflect.Message {
+	mi := &file_metrics_proto_msgTypes[3]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use MonitoringInfoLabelProps.ProtoReflect.Descriptor instead.
 func (*MonitoringInfoLabelProps) Descriptor() ([]byte, []int) {
-	return fileDescriptor_6039342a2ba47b72, []int{3}
+	return file_metrics_proto_rawDescGZIP(), []int{3}
 }
 
-func (m *MonitoringInfoLabelProps) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_MonitoringInfoLabelProps.Unmarshal(m, b)
-}
-func (m *MonitoringInfoLabelProps) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_MonitoringInfoLabelProps.Marshal(b, m, deterministic)
-}
-func (m *MonitoringInfoLabelProps) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_MonitoringInfoLabelProps.Merge(m, src)
-}
-func (m *MonitoringInfoLabelProps) XXX_Size() int {
-	return xxx_messageInfo_MonitoringInfoLabelProps.Size(m)
-}
-func (m *MonitoringInfoLabelProps) XXX_DiscardUnknown() {
-	xxx_messageInfo_MonitoringInfoLabelProps.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_MonitoringInfoLabelProps proto.InternalMessageInfo
-
-func (m *MonitoringInfoLabelProps) GetName() string {
-	if m != nil {
-		return m.Name
+func (x *MonitoringInfoLabelProps) GetName() string {
+	if x != nil {
+		return x.Name
 	}
 	return ""
 }
 
 type MonitoringInfo struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
 	// (Required) Defines the semantic meaning of the metric or monitored state.
 	//
 	// See MonitoringInfoSpecs.Enum for the set of well known metrics/monitored
@@ -516,255 +633,656 @@
 	// together for all updates having the same URN and labels. Some systems such
 	// as Stackdriver will be able to aggregate the metrics using a subset of the
 	// provided labels
-	Labels               map[string]string `protobuf:"bytes,4,rep,name=labels,proto3" json:"labels,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"`
-	XXX_NoUnkeyedLiteral struct{}          `json:"-"`
-	XXX_unrecognized     []byte            `json:"-"`
-	XXX_sizecache        int32             `json:"-"`
+	Labels map[string]string `protobuf:"bytes,4,rep,name=labels,proto3" json:"labels,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"`
 }
 
-func (m *MonitoringInfo) Reset()         { *m = MonitoringInfo{} }
-func (m *MonitoringInfo) String() string { return proto.CompactTextString(m) }
-func (*MonitoringInfo) ProtoMessage()    {}
+func (x *MonitoringInfo) Reset() {
+	*x = MonitoringInfo{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_metrics_proto_msgTypes[4]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *MonitoringInfo) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*MonitoringInfo) ProtoMessage() {}
+
+func (x *MonitoringInfo) ProtoReflect() protoreflect.Message {
+	mi := &file_metrics_proto_msgTypes[4]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use MonitoringInfo.ProtoReflect.Descriptor instead.
 func (*MonitoringInfo) Descriptor() ([]byte, []int) {
-	return fileDescriptor_6039342a2ba47b72, []int{4}
+	return file_metrics_proto_rawDescGZIP(), []int{4}
 }
 
-func (m *MonitoringInfo) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_MonitoringInfo.Unmarshal(m, b)
-}
-func (m *MonitoringInfo) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_MonitoringInfo.Marshal(b, m, deterministic)
-}
-func (m *MonitoringInfo) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_MonitoringInfo.Merge(m, src)
-}
-func (m *MonitoringInfo) XXX_Size() int {
-	return xxx_messageInfo_MonitoringInfo.Size(m)
-}
-func (m *MonitoringInfo) XXX_DiscardUnknown() {
-	xxx_messageInfo_MonitoringInfo.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_MonitoringInfo proto.InternalMessageInfo
-
-func (m *MonitoringInfo) GetUrn() string {
-	if m != nil {
-		return m.Urn
+func (x *MonitoringInfo) GetUrn() string {
+	if x != nil {
+		return x.Urn
 	}
 	return ""
 }
 
-func (m *MonitoringInfo) GetType() string {
-	if m != nil {
-		return m.Type
+func (x *MonitoringInfo) GetType() string {
+	if x != nil {
+		return x.Type
 	}
 	return ""
 }
 
-func (m *MonitoringInfo) GetPayload() []byte {
-	if m != nil {
-		return m.Payload
+func (x *MonitoringInfo) GetPayload() []byte {
+	if x != nil {
+		return x.Payload
 	}
 	return nil
 }
 
-func (m *MonitoringInfo) GetLabels() map[string]string {
-	if m != nil {
-		return m.Labels
+func (x *MonitoringInfo) GetLabels() map[string]string {
+	if x != nil {
+		return x.Labels
 	}
 	return nil
 }
 
 // A set of well known URNs that specify the encoding and aggregation method.
 type MonitoringInfoTypeUrns struct {
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
 }
 
-func (m *MonitoringInfoTypeUrns) Reset()         { *m = MonitoringInfoTypeUrns{} }
-func (m *MonitoringInfoTypeUrns) String() string { return proto.CompactTextString(m) }
-func (*MonitoringInfoTypeUrns) ProtoMessage()    {}
+func (x *MonitoringInfoTypeUrns) Reset() {
+	*x = MonitoringInfoTypeUrns{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_metrics_proto_msgTypes[5]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *MonitoringInfoTypeUrns) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*MonitoringInfoTypeUrns) ProtoMessage() {}
+
+func (x *MonitoringInfoTypeUrns) ProtoReflect() protoreflect.Message {
+	mi := &file_metrics_proto_msgTypes[5]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use MonitoringInfoTypeUrns.ProtoReflect.Descriptor instead.
 func (*MonitoringInfoTypeUrns) Descriptor() ([]byte, []int) {
-	return fileDescriptor_6039342a2ba47b72, []int{5}
+	return file_metrics_proto_rawDescGZIP(), []int{5}
 }
 
-func (m *MonitoringInfoTypeUrns) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_MonitoringInfoTypeUrns.Unmarshal(m, b)
-}
-func (m *MonitoringInfoTypeUrns) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_MonitoringInfoTypeUrns.Marshal(b, m, deterministic)
-}
-func (m *MonitoringInfoTypeUrns) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_MonitoringInfoTypeUrns.Merge(m, src)
-}
-func (m *MonitoringInfoTypeUrns) XXX_Size() int {
-	return xxx_messageInfo_MonitoringInfoTypeUrns.Size(m)
-}
-func (m *MonitoringInfoTypeUrns) XXX_DiscardUnknown() {
-	xxx_messageInfo_MonitoringInfoTypeUrns.DiscardUnknown(m)
+var file_metrics_proto_extTypes = []protoimpl.ExtensionInfo{
+	{
+		ExtendedType:  (*descriptor.EnumValueOptions)(nil),
+		ExtensionType: (*MonitoringInfoLabelProps)(nil),
+		Field:         127337796,
+		Name:          "org.apache.beam.model.pipeline.v1.label_props",
+		Tag:           "bytes,127337796,opt,name=label_props",
+		Filename:      "metrics.proto",
+	},
+	{
+		ExtendedType:  (*descriptor.EnumValueOptions)(nil),
+		ExtensionType: (*MonitoringInfoSpec)(nil),
+		Field:         207174266,
+		Name:          "org.apache.beam.model.pipeline.v1.monitoring_info_spec",
+		Tag:           "bytes,207174266,opt,name=monitoring_info_spec",
+		Filename:      "metrics.proto",
+	},
 }
 
-var xxx_messageInfo_MonitoringInfoTypeUrns proto.InternalMessageInfo
+// Extension fields to descriptor.EnumValueOptions.
+var (
+	// optional org.apache.beam.model.pipeline.v1.MonitoringInfoLabelProps label_props = 127337796;
+	E_LabelProps = &file_metrics_proto_extTypes[0] // From: commit 0x7970544.
+	// Enum extension to store the MonitoringInfoSpecs.
+	//
+	// optional org.apache.beam.model.pipeline.v1.MonitoringInfoSpec monitoring_info_spec = 207174266;
+	E_MonitoringInfoSpec = &file_metrics_proto_extTypes[1]
+)
 
-var E_LabelProps = &proto.ExtensionDesc{
-	ExtendedType:  (*descriptor.EnumValueOptions)(nil),
-	ExtensionType: (*MonitoringInfoLabelProps)(nil),
-	Field:         127337796,
-	Name:          "org.apache.beam.model.pipeline.v1.label_props",
-	Tag:           "bytes,127337796,opt,name=label_props",
-	Filename:      "metrics.proto",
+var File_metrics_proto protoreflect.FileDescriptor
+
+var file_metrics_proto_rawDesc = []byte{
+	0x0a, 0x0d, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12,
+	0x21, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d,
+	0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e,
+	0x76, 0x31, 0x1a, 0x15, 0x62, 0x65, 0x61, 0x6d, 0x5f, 0x72, 0x75, 0x6e, 0x6e, 0x65, 0x72, 0x5f,
+	0x61, 0x70, 0x69, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x20, 0x67, 0x6f, 0x6f, 0x67, 0x6c,
+	0x65, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x64, 0x65, 0x73, 0x63, 0x72,
+	0x69, 0x70, 0x74, 0x6f, 0x72, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0xb4, 0x01, 0x0a, 0x12,
+	0x4d, 0x6f, 0x6e, 0x69, 0x74, 0x6f, 0x72, 0x69, 0x6e, 0x67, 0x49, 0x6e, 0x66, 0x6f, 0x53, 0x70,
+	0x65, 0x63, 0x12, 0x10, 0x0a, 0x03, 0x75, 0x72, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52,
+	0x03, 0x75, 0x72, 0x6e, 0x12, 0x12, 0x0a, 0x04, 0x74, 0x79, 0x70, 0x65, 0x18, 0x02, 0x20, 0x01,
+	0x28, 0x09, 0x52, 0x04, 0x74, 0x79, 0x70, 0x65, 0x12, 0x27, 0x0a, 0x0f, 0x72, 0x65, 0x71, 0x75,
+	0x69, 0x72, 0x65, 0x64, 0x5f, 0x6c, 0x61, 0x62, 0x65, 0x6c, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28,
+	0x09, 0x52, 0x0e, 0x72, 0x65, 0x71, 0x75, 0x69, 0x72, 0x65, 0x64, 0x4c, 0x61, 0x62, 0x65, 0x6c,
+	0x73, 0x12, 0x4f, 0x0a, 0x0b, 0x61, 0x6e, 0x6e, 0x6f, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x73,
+	0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x2d, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61,
+	0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70,
+	0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x6e, 0x6e, 0x6f, 0x74,
+	0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x0b, 0x61, 0x6e, 0x6e, 0x6f, 0x74, 0x61, 0x74, 0x69, 0x6f,
+	0x6e, 0x73, 0x22, 0x34, 0x0a, 0x0a, 0x41, 0x6e, 0x6e, 0x6f, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e,
+	0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x6b,
+	0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28,
+	0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x22, 0x8e, 0x1f, 0x0a, 0x13, 0x4d, 0x6f, 0x6e,
+	0x69, 0x74, 0x6f, 0x72, 0x69, 0x6e, 0x67, 0x49, 0x6e, 0x66, 0x6f, 0x53, 0x70, 0x65, 0x63, 0x73,
+	0x22, 0xf6, 0x1e, 0x0a, 0x04, 0x45, 0x6e, 0x75, 0x6d, 0x12, 0xa7, 0x01, 0x0a, 0x0e, 0x55, 0x53,
+	0x45, 0x52, 0x5f, 0x53, 0x55, 0x4d, 0x5f, 0x49, 0x4e, 0x54, 0x36, 0x34, 0x10, 0x00, 0x1a, 0x92,
+	0x01, 0xd2, 0xa7, 0xa7, 0x96, 0x06, 0x8b, 0x01, 0x0a, 0x1d, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d,
+	0x65, 0x74, 0x72, 0x69, 0x63, 0x3a, 0x75, 0x73, 0x65, 0x72, 0x3a, 0x73, 0x75, 0x6d, 0x5f, 0x69,
+	0x6e, 0x74, 0x36, 0x34, 0x3a, 0x76, 0x31, 0x12, 0x19, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65,
+	0x74, 0x72, 0x69, 0x63, 0x73, 0x3a, 0x73, 0x75, 0x6d, 0x5f, 0x69, 0x6e, 0x74, 0x36, 0x34, 0x3a,
+	0x76, 0x31, 0x1a, 0x0a, 0x50, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x46, 0x4f, 0x52, 0x4d, 0x1a, 0x09,
+	0x4e, 0x41, 0x4d, 0x45, 0x53, 0x50, 0x41, 0x43, 0x45, 0x1a, 0x04, 0x4e, 0x41, 0x4d, 0x45, 0x22,
+	0x32, 0x0a, 0x0b, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x23,
+	0x55, 0x52, 0x4e, 0x20, 0x75, 0x74, 0x69, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x20, 0x74, 0x6f, 0x20,
+	0x72, 0x65, 0x70, 0x6f, 0x72, 0x74, 0x20, 0x75, 0x73, 0x65, 0x72, 0x20, 0x6d, 0x65, 0x74, 0x72,
+	0x69, 0x63, 0x2e, 0x12, 0xaa, 0x01, 0x0a, 0x0f, 0x55, 0x53, 0x45, 0x52, 0x5f, 0x53, 0x55, 0x4d,
+	0x5f, 0x44, 0x4f, 0x55, 0x42, 0x4c, 0x45, 0x10, 0x01, 0x1a, 0x94, 0x01, 0xd2, 0xa7, 0xa7, 0x96,
+	0x06, 0x8d, 0x01, 0x0a, 0x1e, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63,
+	0x3a, 0x75, 0x73, 0x65, 0x72, 0x3a, 0x73, 0x75, 0x6d, 0x5f, 0x64, 0x6f, 0x75, 0x62, 0x6c, 0x65,
+	0x3a, 0x76, 0x31, 0x12, 0x1a, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63,
+	0x73, 0x3a, 0x73, 0x75, 0x6d, 0x5f, 0x64, 0x6f, 0x75, 0x62, 0x6c, 0x65, 0x3a, 0x76, 0x31, 0x1a,
+	0x0a, 0x50, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x46, 0x4f, 0x52, 0x4d, 0x1a, 0x09, 0x4e, 0x41, 0x4d,
+	0x45, 0x53, 0x50, 0x41, 0x43, 0x45, 0x1a, 0x04, 0x4e, 0x41, 0x4d, 0x45, 0x22, 0x32, 0x0a, 0x0b,
+	0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x23, 0x55, 0x52, 0x4e,
+	0x20, 0x75, 0x74, 0x69, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x20, 0x74, 0x6f, 0x20, 0x72, 0x65, 0x70,
+	0x6f, 0x72, 0x74, 0x20, 0x75, 0x73, 0x65, 0x72, 0x20, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x2e,
+	0x12, 0xc2, 0x01, 0x0a, 0x17, 0x55, 0x53, 0x45, 0x52, 0x5f, 0x44, 0x49, 0x53, 0x54, 0x52, 0x49,
+	0x42, 0x55, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x49, 0x4e, 0x54, 0x36, 0x34, 0x10, 0x02, 0x1a, 0xa4,
+	0x01, 0xd2, 0xa7, 0xa7, 0x96, 0x06, 0x9d, 0x01, 0x0a, 0x26, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d,
+	0x65, 0x74, 0x72, 0x69, 0x63, 0x3a, 0x75, 0x73, 0x65, 0x72, 0x3a, 0x64, 0x69, 0x73, 0x74, 0x72,
+	0x69, 0x62, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x6e, 0x74, 0x36, 0x34, 0x3a, 0x76, 0x31,
+	0x12, 0x22, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x3a, 0x64,
+	0x69, 0x73, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x6e, 0x74, 0x36,
+	0x34, 0x3a, 0x76, 0x31, 0x1a, 0x0a, 0x50, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x46, 0x4f, 0x52, 0x4d,
+	0x1a, 0x09, 0x4e, 0x41, 0x4d, 0x45, 0x53, 0x50, 0x41, 0x43, 0x45, 0x1a, 0x04, 0x4e, 0x41, 0x4d,
+	0x45, 0x22, 0x32, 0x0a, 0x0b, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e,
+	0x12, 0x23, 0x55, 0x52, 0x4e, 0x20, 0x75, 0x74, 0x69, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x20, 0x74,
+	0x6f, 0x20, 0x72, 0x65, 0x70, 0x6f, 0x72, 0x74, 0x20, 0x75, 0x73, 0x65, 0x72, 0x20, 0x6d, 0x65,
+	0x74, 0x72, 0x69, 0x63, 0x2e, 0x12, 0xc5, 0x01, 0x0a, 0x18, 0x55, 0x53, 0x45, 0x52, 0x5f, 0x44,
+	0x49, 0x53, 0x54, 0x52, 0x49, 0x42, 0x55, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x44, 0x4f, 0x55, 0x42,
+	0x4c, 0x45, 0x10, 0x03, 0x1a, 0xa6, 0x01, 0xd2, 0xa7, 0xa7, 0x96, 0x06, 0x9f, 0x01, 0x0a, 0x27,
+	0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x3a, 0x75, 0x73, 0x65, 0x72,
+	0x3a, 0x64, 0x69, 0x73, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x64, 0x6f,
+	0x75, 0x62, 0x6c, 0x65, 0x3a, 0x76, 0x31, 0x12, 0x23, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65,
+	0x74, 0x72, 0x69, 0x63, 0x73, 0x3a, 0x64, 0x69, 0x73, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x69,
+	0x6f, 0x6e, 0x5f, 0x64, 0x6f, 0x75, 0x62, 0x6c, 0x65, 0x3a, 0x76, 0x31, 0x1a, 0x0a, 0x50, 0x54,
+	0x52, 0x41, 0x4e, 0x53, 0x46, 0x4f, 0x52, 0x4d, 0x1a, 0x09, 0x4e, 0x41, 0x4d, 0x45, 0x53, 0x50,
+	0x41, 0x43, 0x45, 0x1a, 0x04, 0x4e, 0x41, 0x4d, 0x45, 0x22, 0x32, 0x0a, 0x0b, 0x64, 0x65, 0x73,
+	0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x23, 0x55, 0x52, 0x4e, 0x20, 0x75, 0x74,
+	0x69, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x20, 0x74, 0x6f, 0x20, 0x72, 0x65, 0x70, 0x6f, 0x72, 0x74,
+	0x20, 0x75, 0x73, 0x65, 0x72, 0x20, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x2e, 0x12, 0xb0, 0x01,
+	0x0a, 0x11, 0x55, 0x53, 0x45, 0x52, 0x5f, 0x4c, 0x41, 0x54, 0x45, 0x53, 0x54, 0x5f, 0x49, 0x4e,
+	0x54, 0x36, 0x34, 0x10, 0x04, 0x1a, 0x98, 0x01, 0xd2, 0xa7, 0xa7, 0x96, 0x06, 0x91, 0x01, 0x0a,
+	0x20, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x3a, 0x75, 0x73, 0x65,
+	0x72, 0x3a, 0x6c, 0x61, 0x74, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x6e, 0x74, 0x36, 0x34, 0x3a, 0x76,
+	0x31, 0x12, 0x1c, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x3a,
+	0x6c, 0x61, 0x74, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x6e, 0x74, 0x36, 0x34, 0x3a, 0x76, 0x31, 0x1a,
+	0x0a, 0x50, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x46, 0x4f, 0x52, 0x4d, 0x1a, 0x09, 0x4e, 0x41, 0x4d,
+	0x45, 0x53, 0x50, 0x41, 0x43, 0x45, 0x1a, 0x04, 0x4e, 0x41, 0x4d, 0x45, 0x22, 0x32, 0x0a, 0x0b,
+	0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x23, 0x55, 0x52, 0x4e,
+	0x20, 0x75, 0x74, 0x69, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x20, 0x74, 0x6f, 0x20, 0x72, 0x65, 0x70,
+	0x6f, 0x72, 0x74, 0x20, 0x75, 0x73, 0x65, 0x72, 0x20, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x2e,
+	0x12, 0xb3, 0x01, 0x0a, 0x12, 0x55, 0x53, 0x45, 0x52, 0x5f, 0x4c, 0x41, 0x54, 0x45, 0x53, 0x54,
+	0x5f, 0x44, 0x4f, 0x55, 0x42, 0x4c, 0x45, 0x10, 0x05, 0x1a, 0x9a, 0x01, 0xd2, 0xa7, 0xa7, 0x96,
+	0x06, 0x93, 0x01, 0x0a, 0x21, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63,
+	0x3a, 0x75, 0x73, 0x65, 0x72, 0x3a, 0x6c, 0x61, 0x74, 0x65, 0x73, 0x74, 0x5f, 0x64, 0x6f, 0x75,
+	0x62, 0x6c, 0x65, 0x3a, 0x76, 0x31, 0x12, 0x1d, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74,
+	0x72, 0x69, 0x63, 0x73, 0x3a, 0x6c, 0x61, 0x74, 0x65, 0x73, 0x74, 0x5f, 0x64, 0x6f, 0x75, 0x62,
+	0x6c, 0x65, 0x3a, 0x76, 0x31, 0x1a, 0x0a, 0x50, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x46, 0x4f, 0x52,
+	0x4d, 0x1a, 0x09, 0x4e, 0x41, 0x4d, 0x45, 0x53, 0x50, 0x41, 0x43, 0x45, 0x1a, 0x04, 0x4e, 0x41,
+	0x4d, 0x45, 0x22, 0x32, 0x0a, 0x0b, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f,
+	0x6e, 0x12, 0x23, 0x55, 0x52, 0x4e, 0x20, 0x75, 0x74, 0x69, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x20,
+	0x74, 0x6f, 0x20, 0x72, 0x65, 0x70, 0x6f, 0x72, 0x74, 0x20, 0x75, 0x73, 0x65, 0x72, 0x20, 0x6d,
+	0x65, 0x74, 0x72, 0x69, 0x63, 0x2e, 0x12, 0xad, 0x01, 0x0a, 0x10, 0x55, 0x53, 0x45, 0x52, 0x5f,
+	0x54, 0x4f, 0x50, 0x5f, 0x4e, 0x5f, 0x49, 0x4e, 0x54, 0x36, 0x34, 0x10, 0x06, 0x1a, 0x96, 0x01,
+	0xd2, 0xa7, 0xa7, 0x96, 0x06, 0x8f, 0x01, 0x0a, 0x1f, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65,
+	0x74, 0x72, 0x69, 0x63, 0x3a, 0x75, 0x73, 0x65, 0x72, 0x3a, 0x74, 0x6f, 0x70, 0x5f, 0x6e, 0x5f,
+	0x69, 0x6e, 0x74, 0x36, 0x34, 0x3a, 0x76, 0x31, 0x12, 0x1b, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d,
+	0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x3a, 0x74, 0x6f, 0x70, 0x5f, 0x6e, 0x5f, 0x69, 0x6e, 0x74,
+	0x36, 0x34, 0x3a, 0x76, 0x31, 0x1a, 0x0a, 0x50, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x46, 0x4f, 0x52,
+	0x4d, 0x1a, 0x09, 0x4e, 0x41, 0x4d, 0x45, 0x53, 0x50, 0x41, 0x43, 0x45, 0x1a, 0x04, 0x4e, 0x41,
+	0x4d, 0x45, 0x22, 0x32, 0x0a, 0x0b, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f,
+	0x6e, 0x12, 0x23, 0x55, 0x52, 0x4e, 0x20, 0x75, 0x74, 0x69, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x20,
+	0x74, 0x6f, 0x20, 0x72, 0x65, 0x70, 0x6f, 0x72, 0x74, 0x20, 0x75, 0x73, 0x65, 0x72, 0x20, 0x6d,
+	0x65, 0x74, 0x72, 0x69, 0x63, 0x2e, 0x12, 0xb0, 0x01, 0x0a, 0x11, 0x55, 0x53, 0x45, 0x52, 0x5f,
+	0x54, 0x4f, 0x50, 0x5f, 0x4e, 0x5f, 0x44, 0x4f, 0x55, 0x42, 0x4c, 0x45, 0x10, 0x07, 0x1a, 0x98,
+	0x01, 0xd2, 0xa7, 0xa7, 0x96, 0x06, 0x91, 0x01, 0x0a, 0x20, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d,
+	0x65, 0x74, 0x72, 0x69, 0x63, 0x3a, 0x75, 0x73, 0x65, 0x72, 0x3a, 0x74, 0x6f, 0x70, 0x5f, 0x6e,
+	0x5f, 0x64, 0x6f, 0x75, 0x62, 0x6c, 0x65, 0x3a, 0x76, 0x31, 0x12, 0x1c, 0x62, 0x65, 0x61, 0x6d,
+	0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x3a, 0x74, 0x6f, 0x70, 0x5f, 0x6e, 0x5f, 0x64,
+	0x6f, 0x75, 0x62, 0x6c, 0x65, 0x3a, 0x76, 0x31, 0x1a, 0x0a, 0x50, 0x54, 0x52, 0x41, 0x4e, 0x53,
+	0x46, 0x4f, 0x52, 0x4d, 0x1a, 0x09, 0x4e, 0x41, 0x4d, 0x45, 0x53, 0x50, 0x41, 0x43, 0x45, 0x1a,
+	0x04, 0x4e, 0x41, 0x4d, 0x45, 0x22, 0x32, 0x0a, 0x0b, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70,
+	0x74, 0x69, 0x6f, 0x6e, 0x12, 0x23, 0x55, 0x52, 0x4e, 0x20, 0x75, 0x74, 0x69, 0x6c, 0x69, 0x7a,
+	0x65, 0x64, 0x20, 0x74, 0x6f, 0x20, 0x72, 0x65, 0x70, 0x6f, 0x72, 0x74, 0x20, 0x75, 0x73, 0x65,
+	0x72, 0x20, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x2e, 0x12, 0xb6, 0x01, 0x0a, 0x13, 0x55, 0x53,
+	0x45, 0x52, 0x5f, 0x42, 0x4f, 0x54, 0x54, 0x4f, 0x4d, 0x5f, 0x4e, 0x5f, 0x49, 0x4e, 0x54, 0x36,
+	0x34, 0x10, 0x08, 0x1a, 0x9c, 0x01, 0xd2, 0xa7, 0xa7, 0x96, 0x06, 0x95, 0x01, 0x0a, 0x22, 0x62,
+	0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x3a, 0x75, 0x73, 0x65, 0x72, 0x3a,
+	0x62, 0x6f, 0x74, 0x74, 0x6f, 0x6d, 0x5f, 0x6e, 0x5f, 0x69, 0x6e, 0x74, 0x36, 0x34, 0x3a, 0x76,
+	0x31, 0x12, 0x1e, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x3a,
+	0x62, 0x6f, 0x74, 0x74, 0x6f, 0x6d, 0x5f, 0x6e, 0x5f, 0x69, 0x6e, 0x74, 0x36, 0x34, 0x3a, 0x76,
+	0x31, 0x1a, 0x0a, 0x50, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x46, 0x4f, 0x52, 0x4d, 0x1a, 0x09, 0x4e,
+	0x41, 0x4d, 0x45, 0x53, 0x50, 0x41, 0x43, 0x45, 0x1a, 0x04, 0x4e, 0x41, 0x4d, 0x45, 0x22, 0x32,
+	0x0a, 0x0b, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x23, 0x55,
+	0x52, 0x4e, 0x20, 0x75, 0x74, 0x69, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x20, 0x74, 0x6f, 0x20, 0x72,
+	0x65, 0x70, 0x6f, 0x72, 0x74, 0x20, 0x75, 0x73, 0x65, 0x72, 0x20, 0x6d, 0x65, 0x74, 0x72, 0x69,
+	0x63, 0x2e, 0x12, 0xb9, 0x01, 0x0a, 0x14, 0x55, 0x53, 0x45, 0x52, 0x5f, 0x42, 0x4f, 0x54, 0x54,
+	0x4f, 0x4d, 0x5f, 0x4e, 0x5f, 0x44, 0x4f, 0x55, 0x42, 0x4c, 0x45, 0x10, 0x09, 0x1a, 0x9e, 0x01,
+	0xd2, 0xa7, 0xa7, 0x96, 0x06, 0x97, 0x01, 0x0a, 0x23, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65,
+	0x74, 0x72, 0x69, 0x63, 0x3a, 0x75, 0x73, 0x65, 0x72, 0x3a, 0x62, 0x6f, 0x74, 0x74, 0x6f, 0x6d,
+	0x5f, 0x6e, 0x5f, 0x64, 0x6f, 0x75, 0x62, 0x6c, 0x65, 0x3a, 0x76, 0x31, 0x12, 0x1f, 0x62, 0x65,
+	0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x3a, 0x62, 0x6f, 0x74, 0x74, 0x6f,
+	0x6d, 0x5f, 0x6e, 0x5f, 0x64, 0x6f, 0x75, 0x62, 0x6c, 0x65, 0x3a, 0x76, 0x31, 0x1a, 0x0a, 0x50,
+	0x54, 0x52, 0x41, 0x4e, 0x53, 0x46, 0x4f, 0x52, 0x4d, 0x1a, 0x09, 0x4e, 0x41, 0x4d, 0x45, 0x53,
+	0x50, 0x41, 0x43, 0x45, 0x1a, 0x04, 0x4e, 0x41, 0x4d, 0x45, 0x22, 0x32, 0x0a, 0x0b, 0x64, 0x65,
+	0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x23, 0x55, 0x52, 0x4e, 0x20, 0x75,
+	0x74, 0x69, 0x6c, 0x69, 0x7a, 0x65, 0x64, 0x20, 0x74, 0x6f, 0x20, 0x72, 0x65, 0x70, 0x6f, 0x72,
+	0x74, 0x20, 0x75, 0x73, 0x65, 0x72, 0x20, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x2e, 0x12, 0xad,
+	0x01, 0x0a, 0x0d, 0x45, 0x4c, 0x45, 0x4d, 0x45, 0x4e, 0x54, 0x5f, 0x43, 0x4f, 0x55, 0x4e, 0x54,
+	0x10, 0x0a, 0x1a, 0x99, 0x01, 0xd2, 0xa7, 0xa7, 0x96, 0x06, 0x92, 0x01, 0x0a, 0x1c, 0x62, 0x65,
+	0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x3a, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e,
+	0x74, 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x3a, 0x76, 0x31, 0x12, 0x19, 0x62, 0x65, 0x61, 0x6d,
+	0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x3a, 0x73, 0x75, 0x6d, 0x5f, 0x69, 0x6e, 0x74,
+	0x36, 0x34, 0x3a, 0x76, 0x31, 0x1a, 0x0b, 0x50, 0x43, 0x4f, 0x4c, 0x4c, 0x45, 0x43, 0x54, 0x49,
+	0x4f, 0x4e, 0x22, 0x4a, 0x0a, 0x0b, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f,
+	0x6e, 0x12, 0x3b, 0x54, 0x68, 0x65, 0x20, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x20, 0x65, 0x6c, 0x65,
+	0x6d, 0x65, 0x6e, 0x74, 0x73, 0x20, 0x6f, 0x75, 0x74, 0x70, 0x75, 0x74, 0x20, 0x74, 0x6f, 0x20,
+	0x61, 0x20, 0x50, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x20, 0x62, 0x79,
+	0x20, 0x61, 0x20, 0x50, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x2e, 0x12, 0xcd,
+	0x02, 0x0a, 0x11, 0x53, 0x41, 0x4d, 0x50, 0x4c, 0x45, 0x44, 0x5f, 0x42, 0x59, 0x54, 0x45, 0x5f,
+	0x53, 0x49, 0x5a, 0x45, 0x10, 0x0b, 0x1a, 0xb5, 0x02, 0xd2, 0xa7, 0xa7, 0x96, 0x06, 0xae, 0x02,
+	0x0a, 0x20, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x3a, 0x73, 0x61,
+	0x6d, 0x70, 0x6c, 0x65, 0x64, 0x5f, 0x62, 0x79, 0x74, 0x65, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x3a,
+	0x76, 0x31, 0x12, 0x22, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73,
+	0x3a, 0x64, 0x69, 0x73, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x6e,
+	0x74, 0x36, 0x34, 0x3a, 0x76, 0x31, 0x1a, 0x0b, 0x50, 0x43, 0x4f, 0x4c, 0x4c, 0x45, 0x43, 0x54,
+	0x49, 0x4f, 0x4e, 0x22, 0xd8, 0x01, 0x0a, 0x0b, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74,
+	0x69, 0x6f, 0x6e, 0x12, 0xc8, 0x01, 0x54, 0x68, 0x65, 0x20, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x20,
+	0x62, 0x79, 0x74, 0x65, 0x20, 0x73, 0x69, 0x7a, 0x65, 0x20, 0x61, 0x6e, 0x64, 0x20, 0x63, 0x6f,
+	0x75, 0x6e, 0x74, 0x20, 0x6f, 0x66, 0x20, 0x61, 0x20, 0x73, 0x61, 0x6d, 0x70, 0x6c, 0x65, 0x64,
+	0x20, 0x20, 0x73, 0x65, 0x74, 0x20, 0x28, 0x6f, 0x72, 0x20, 0x61, 0x6c, 0x6c, 0x29, 0x20, 0x6f,
+	0x66, 0x20, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x20, 0x69, 0x6e, 0x20, 0x74, 0x68,
+	0x65, 0x20, 0x70, 0x63, 0x6f, 0x6c, 0x6c, 0x65, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x2e, 0x20, 0x53,
+	0x61, 0x6d, 0x70, 0x6c, 0x69, 0x6e, 0x67, 0x20, 0x69, 0x73, 0x20, 0x75, 0x73, 0x65, 0x64, 0x20,
+	0x20, 0x62, 0x65, 0x63, 0x61, 0x75, 0x73, 0x65, 0x20, 0x63, 0x61, 0x6c, 0x63, 0x75, 0x6c, 0x61,
+	0x74, 0x69, 0x6e, 0x67, 0x20, 0x74, 0x68, 0x65, 0x20, 0x62, 0x79, 0x74, 0x65, 0x20, 0x63, 0x6f,
+	0x75, 0x6e, 0x74, 0x20, 0x69, 0x6e, 0x76, 0x6f, 0x6c, 0x76, 0x65, 0x73, 0x20, 0x73, 0x65, 0x72,
+	0x69, 0x61, 0x6c, 0x69, 0x7a, 0x69, 0x6e, 0x67, 0x20, 0x74, 0x68, 0x65, 0x20, 0x20, 0x65, 0x6c,
+	0x65, 0x6d, 0x65, 0x6e, 0x74, 0x73, 0x20, 0x77, 0x68, 0x69, 0x63, 0x68, 0x20, 0x69, 0x73, 0x20,
+	0x43, 0x50, 0x55, 0x20, 0x69, 0x6e, 0x74, 0x65, 0x6e, 0x73, 0x69, 0x76, 0x65, 0x2e, 0x12, 0xd9,
+	0x01, 0x0a, 0x12, 0x53, 0x54, 0x41, 0x52, 0x54, 0x5f, 0x42, 0x55, 0x4e, 0x44, 0x4c, 0x45, 0x5f,
+	0x4d, 0x53, 0x45, 0x43, 0x53, 0x10, 0x0c, 0x1a, 0xc0, 0x01, 0xd2, 0xa7, 0xa7, 0x96, 0x06, 0xb9,
+	0x01, 0x0a, 0x36, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x3a, 0x70,
+	0x61, 0x72, 0x64, 0x6f, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x74,
+	0x69, 0x6d, 0x65, 0x3a, 0x73, 0x74, 0x61, 0x72, 0x74, 0x5f, 0x62, 0x75, 0x6e, 0x64, 0x6c, 0x65,
+	0x5f, 0x6d, 0x73, 0x65, 0x63, 0x73, 0x3a, 0x76, 0x31, 0x12, 0x19, 0x62, 0x65, 0x61, 0x6d, 0x3a,
+	0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x3a, 0x73, 0x75, 0x6d, 0x5f, 0x69, 0x6e, 0x74, 0x36,
+	0x34, 0x3a, 0x76, 0x31, 0x1a, 0x0a, 0x50, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x46, 0x4f, 0x52, 0x4d,
+	0x22, 0x58, 0x0a, 0x0b, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x12,
+	0x49, 0x54, 0x68, 0x65, 0x20, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x20, 0x65, 0x73, 0x74, 0x69, 0x6d,
+	0x61, 0x74, 0x65, 0x64, 0x20, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x20, 0x74,
+	0x69, 0x6d, 0x65, 0x20, 0x6f, 0x66, 0x20, 0x74, 0x68, 0x65, 0x20, 0x73, 0x74, 0x61, 0x72, 0x74,
+	0x20, 0x62, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x66, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x20,
+	0x69, 0x6e, 0x20, 0x61, 0x20, 0x70, 0x61, 0x72, 0x64, 0x6f, 0x12, 0xdf, 0x01, 0x0a, 0x14, 0x50,
+	0x52, 0x4f, 0x43, 0x45, 0x53, 0x53, 0x5f, 0x42, 0x55, 0x4e, 0x44, 0x4c, 0x45, 0x5f, 0x4d, 0x53,
+	0x45, 0x43, 0x53, 0x10, 0x0d, 0x1a, 0xc4, 0x01, 0xd2, 0xa7, 0xa7, 0x96, 0x06, 0xbd, 0x01, 0x0a,
+	0x38, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x3a, 0x70, 0x61, 0x72,
+	0x64, 0x6f, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x74, 0x69, 0x6d,
+	0x65, 0x3a, 0x70, 0x72, 0x6f, 0x63, 0x65, 0x73, 0x73, 0x5f, 0x62, 0x75, 0x6e, 0x64, 0x6c, 0x65,
+	0x5f, 0x6d, 0x73, 0x65, 0x63, 0x73, 0x3a, 0x76, 0x31, 0x12, 0x19, 0x62, 0x65, 0x61, 0x6d, 0x3a,
+	0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x3a, 0x73, 0x75, 0x6d, 0x5f, 0x69, 0x6e, 0x74, 0x36,
+	0x34, 0x3a, 0x76, 0x31, 0x1a, 0x0a, 0x50, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x46, 0x4f, 0x52, 0x4d,
+	0x22, 0x5a, 0x0a, 0x0b, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x12,
+	0x4b, 0x54, 0x68, 0x65, 0x20, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x20, 0x65, 0x73, 0x74, 0x69, 0x6d,
+	0x61, 0x74, 0x65, 0x64, 0x20, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x20, 0x74,
+	0x69, 0x6d, 0x65, 0x20, 0x6f, 0x66, 0x20, 0x74, 0x68, 0x65, 0x20, 0x70, 0x72, 0x6f, 0x63, 0x65,
+	0x73, 0x73, 0x20, 0x62, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x66, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f,
+	0x6e, 0x20, 0x69, 0x6e, 0x20, 0x61, 0x20, 0x70, 0x61, 0x72, 0x64, 0x6f, 0x12, 0xdd, 0x01, 0x0a,
+	0x13, 0x46, 0x49, 0x4e, 0x49, 0x53, 0x48, 0x5f, 0x42, 0x55, 0x4e, 0x44, 0x4c, 0x45, 0x5f, 0x4d,
+	0x53, 0x45, 0x43, 0x53, 0x10, 0x0e, 0x1a, 0xc3, 0x01, 0xd2, 0xa7, 0xa7, 0x96, 0x06, 0xbc, 0x01,
+	0x0a, 0x37, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x3a, 0x70, 0x61,
+	0x72, 0x64, 0x6f, 0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x74, 0x69,
+	0x6d, 0x65, 0x3a, 0x66, 0x69, 0x6e, 0x69, 0x73, 0x68, 0x5f, 0x62, 0x75, 0x6e, 0x64, 0x6c, 0x65,
+	0x5f, 0x6d, 0x73, 0x65, 0x63, 0x73, 0x3a, 0x76, 0x31, 0x12, 0x19, 0x62, 0x65, 0x61, 0x6d, 0x3a,
+	0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x3a, 0x73, 0x75, 0x6d, 0x5f, 0x69, 0x6e, 0x74, 0x36,
+	0x34, 0x3a, 0x76, 0x31, 0x1a, 0x0a, 0x50, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x46, 0x4f, 0x52, 0x4d,
+	0x22, 0x5a, 0x0a, 0x0b, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x12,
+	0x4b, 0x54, 0x68, 0x65, 0x20, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x20, 0x65, 0x73, 0x74, 0x69, 0x6d,
+	0x61, 0x74, 0x65, 0x64, 0x20, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x20, 0x74,
+	0x69, 0x6d, 0x65, 0x20, 0x6f, 0x66, 0x20, 0x74, 0x68, 0x65, 0x20, 0x66, 0x69, 0x6e, 0x69, 0x73,
+	0x68, 0x20, 0x62, 0x75, 0x6e, 0x64, 0x6c, 0x65, 0x20, 0x66, 0x75, 0x6e, 0x63, 0x74, 0x69, 0x6f,
+	0x6e, 0x20, 0x69, 0x6e, 0x20, 0x61, 0x20, 0x70, 0x61, 0x72, 0x64, 0x6f, 0x12, 0xbb, 0x01, 0x0a,
+	0x0b, 0x54, 0x4f, 0x54, 0x41, 0x4c, 0x5f, 0x4d, 0x53, 0x45, 0x43, 0x53, 0x10, 0x0f, 0x1a, 0xa9,
+	0x01, 0xd2, 0xa7, 0xa7, 0x96, 0x06, 0xa2, 0x01, 0x0a, 0x34, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d,
+	0x65, 0x74, 0x72, 0x69, 0x63, 0x3a, 0x70, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d,
+	0x5f, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x3a,
+	0x74, 0x6f, 0x74, 0x61, 0x6c, 0x5f, 0x6d, 0x73, 0x65, 0x63, 0x73, 0x3a, 0x76, 0x31, 0x12, 0x19,
+	0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x3a, 0x73, 0x75, 0x6d,
+	0x5f, 0x69, 0x6e, 0x74, 0x36, 0x34, 0x3a, 0x76, 0x31, 0x1a, 0x0a, 0x50, 0x54, 0x52, 0x41, 0x4e,
+	0x53, 0x46, 0x4f, 0x52, 0x4d, 0x22, 0x43, 0x0a, 0x0b, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70,
+	0x74, 0x69, 0x6f, 0x6e, 0x12, 0x34, 0x54, 0x68, 0x65, 0x20, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x20,
+	0x65, 0x73, 0x74, 0x69, 0x6d, 0x61, 0x74, 0x65, 0x64, 0x20, 0x65, 0x78, 0x65, 0x63, 0x75, 0x74,
+	0x69, 0x6f, 0x6e, 0x20, 0x74, 0x69, 0x6d, 0x65, 0x20, 0x6f, 0x66, 0x20, 0x74, 0x68, 0x65, 0x20,
+	0x70, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72, 0x6d, 0x12, 0x9f, 0x02, 0x0a, 0x0e, 0x57,
+	0x4f, 0x52, 0x4b, 0x5f, 0x52, 0x45, 0x4d, 0x41, 0x49, 0x4e, 0x49, 0x4e, 0x47, 0x10, 0x10, 0x1a,
+	0x8a, 0x02, 0xd2, 0xa7, 0xa7, 0x96, 0x06, 0x83, 0x02, 0x0a, 0x2c, 0x62, 0x65, 0x61, 0x6d, 0x3a,
+	0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x3a, 0x70, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66, 0x6f, 0x72,
+	0x6d, 0x5f, 0x70, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x3a, 0x72, 0x65, 0x6d, 0x61, 0x69,
+	0x6e, 0x69, 0x6e, 0x67, 0x3a, 0x76, 0x31, 0x12, 0x18, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65,
+	0x74, 0x72, 0x69, 0x63, 0x73, 0x3a, 0x70, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x3a, 0x76,
+	0x31, 0x1a, 0x0a, 0x50, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x46, 0x4f, 0x52, 0x4d, 0x22, 0xac, 0x01,
+	0x0a, 0x0b, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x9c, 0x01,
+	0x54, 0x68, 0x65, 0x20, 0x72, 0x65, 0x6d, 0x61, 0x69, 0x6e, 0x69, 0x6e, 0x67, 0x20, 0x61, 0x6d,
+	0x6f, 0x75, 0x6e, 0x74, 0x20, 0x6f, 0x66, 0x20, 0x77, 0x6f, 0x72, 0x6b, 0x20, 0x66, 0x6f, 0x72,
+	0x20, 0x65, 0x61, 0x63, 0x68, 0x20, 0x61, 0x63, 0x74, 0x69, 0x76, 0x65, 0x20, 0x65, 0x6c, 0x65,
+	0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x20, 0x45, 0x61, 0x63, 0x68, 0x20, 0x61, 0x63, 0x74, 0x69, 0x76,
+	0x65, 0x20, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x20, 0x72, 0x65, 0x70, 0x72, 0x65, 0x73,
+	0x65, 0x6e, 0x74, 0x73, 0x20, 0x61, 0x6e, 0x20, 0x69, 0x6e, 0x64, 0x65, 0x70, 0x65, 0x6e, 0x64,
+	0x65, 0x6e, 0x74, 0x20, 0x61, 0x6d, 0x6f, 0x75, 0x6e, 0x74, 0x20, 0x6f, 0x66, 0x20, 0x77, 0x6f,
+	0x72, 0x6b, 0x20, 0x6e, 0x6f, 0x74, 0x20, 0x73, 0x68, 0x61, 0x72, 0x65, 0x64, 0x20, 0x77, 0x69,
+	0x74, 0x68, 0x20, 0x61, 0x6e, 0x79, 0x20, 0x6f, 0x74, 0x68, 0x65, 0x72, 0x20, 0x61, 0x63, 0x74,
+	0x69, 0x76, 0x65, 0x20, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x12, 0x9f, 0x02, 0x0a,
+	0x0e, 0x57, 0x4f, 0x52, 0x4b, 0x5f, 0x43, 0x4f, 0x4d, 0x50, 0x4c, 0x45, 0x54, 0x45, 0x44, 0x10,
+	0x11, 0x1a, 0x8a, 0x02, 0xd2, 0xa7, 0xa7, 0x96, 0x06, 0x83, 0x02, 0x0a, 0x2c, 0x62, 0x65, 0x61,
+	0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x3a, 0x70, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x66,
+	0x6f, 0x72, 0x6d, 0x5f, 0x70, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x3a, 0x63, 0x6f, 0x6d,
+	0x70, 0x6c, 0x65, 0x74, 0x65, 0x64, 0x3a, 0x76, 0x31, 0x12, 0x18, 0x62, 0x65, 0x61, 0x6d, 0x3a,
+	0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x3a, 0x70, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73,
+	0x3a, 0x76, 0x31, 0x1a, 0x0a, 0x50, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x46, 0x4f, 0x52, 0x4d, 0x22,
+	0xac, 0x01, 0x0a, 0x0b, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x12,
+	0x9c, 0x01, 0x54, 0x68, 0x65, 0x20, 0x72, 0x65, 0x6d, 0x61, 0x69, 0x6e, 0x69, 0x6e, 0x67, 0x20,
+	0x61, 0x6d, 0x6f, 0x75, 0x6e, 0x74, 0x20, 0x6f, 0x66, 0x20, 0x77, 0x6f, 0x72, 0x6b, 0x20, 0x66,
+	0x6f, 0x72, 0x20, 0x65, 0x61, 0x63, 0x68, 0x20, 0x61, 0x63, 0x74, 0x69, 0x76, 0x65, 0x20, 0x65,
+	0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x20, 0x45, 0x61, 0x63, 0x68, 0x20, 0x61, 0x63, 0x74,
+	0x69, 0x76, 0x65, 0x20, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x20, 0x72, 0x65, 0x70, 0x72,
+	0x65, 0x73, 0x65, 0x6e, 0x74, 0x73, 0x20, 0x61, 0x6e, 0x20, 0x69, 0x6e, 0x64, 0x65, 0x70, 0x65,
+	0x6e, 0x64, 0x65, 0x6e, 0x74, 0x20, 0x61, 0x6d, 0x6f, 0x75, 0x6e, 0x74, 0x20, 0x6f, 0x66, 0x20,
+	0x77, 0x6f, 0x72, 0x6b, 0x20, 0x6e, 0x6f, 0x74, 0x20, 0x73, 0x68, 0x61, 0x72, 0x65, 0x64, 0x20,
+	0x77, 0x69, 0x74, 0x68, 0x20, 0x61, 0x6e, 0x79, 0x20, 0x6f, 0x74, 0x68, 0x65, 0x72, 0x20, 0x61,
+	0x63, 0x74, 0x69, 0x76, 0x65, 0x20, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x2e, 0x12, 0xa8,
+	0x01, 0x0a, 0x17, 0x44, 0x41, 0x54, 0x41, 0x5f, 0x43, 0x48, 0x41, 0x4e, 0x4e, 0x45, 0x4c, 0x5f,
+	0x52, 0x45, 0x41, 0x44, 0x5f, 0x49, 0x4e, 0x44, 0x45, 0x58, 0x10, 0x12, 0x1a, 0x8a, 0x01, 0xd2,
+	0xa7, 0xa7, 0x96, 0x06, 0x83, 0x01, 0x0a, 0x26, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74,
+	0x72, 0x69, 0x63, 0x3a, 0x64, 0x61, 0x74, 0x61, 0x5f, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c,
+	0x3a, 0x72, 0x65, 0x61, 0x64, 0x5f, 0x69, 0x6e, 0x64, 0x65, 0x78, 0x3a, 0x76, 0x31, 0x12, 0x19,
+	0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x3a, 0x73, 0x75, 0x6d,
+	0x5f, 0x69, 0x6e, 0x74, 0x36, 0x34, 0x3a, 0x76, 0x31, 0x1a, 0x0a, 0x50, 0x54, 0x52, 0x41, 0x4e,
+	0x53, 0x46, 0x4f, 0x52, 0x4d, 0x22, 0x32, 0x0a, 0x0b, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70,
+	0x74, 0x69, 0x6f, 0x6e, 0x12, 0x23, 0x54, 0x68, 0x65, 0x20, 0x72, 0x65, 0x61, 0x64, 0x20, 0x69,
+	0x6e, 0x64, 0x65, 0x78, 0x20, 0x6f, 0x66, 0x20, 0x74, 0x68, 0x65, 0x20, 0x64, 0x61, 0x74, 0x61,
+	0x20, 0x63, 0x68, 0x61, 0x6e, 0x6e, 0x65, 0x6c, 0x2e, 0x22, 0x2e, 0x0a, 0x18, 0x4d, 0x6f, 0x6e,
+	0x69, 0x74, 0x6f, 0x72, 0x69, 0x6e, 0x67, 0x49, 0x6e, 0x66, 0x6f, 0x4c, 0x61, 0x62, 0x65, 0x6c,
+	0x50, 0x72, 0x6f, 0x70, 0x73, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20,
+	0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x22, 0xf2, 0x03, 0x0a, 0x0e, 0x4d, 0x6f,
+	0x6e, 0x69, 0x74, 0x6f, 0x72, 0x69, 0x6e, 0x67, 0x49, 0x6e, 0x66, 0x6f, 0x12, 0x10, 0x0a, 0x03,
+	0x75, 0x72, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x75, 0x72, 0x6e, 0x12, 0x12,
+	0x0a, 0x04, 0x74, 0x79, 0x70, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x74, 0x79,
+	0x70, 0x65, 0x12, 0x18, 0x0a, 0x07, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x18, 0x03, 0x20,
+	0x01, 0x28, 0x0c, 0x52, 0x07, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x55, 0x0a, 0x06,
+	0x6c, 0x61, 0x62, 0x65, 0x6c, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x3d, 0x2e, 0x6f,
+	0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d,
+	0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31,
+	0x2e, 0x4d, 0x6f, 0x6e, 0x69, 0x74, 0x6f, 0x72, 0x69, 0x6e, 0x67, 0x49, 0x6e, 0x66, 0x6f, 0x2e,
+	0x4c, 0x61, 0x62, 0x65, 0x6c, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x06, 0x6c, 0x61, 0x62,
+	0x65, 0x6c, 0x73, 0x1a, 0x39, 0x0a, 0x0b, 0x4c, 0x61, 0x62, 0x65, 0x6c, 0x73, 0x45, 0x6e, 0x74,
+	0x72, 0x79, 0x12, 0x10, 0x0a, 0x03, 0x6b, 0x65, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52,
+	0x03, 0x6b, 0x65, 0x79, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20,
+	0x01, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x3a, 0x02, 0x38, 0x01, 0x22, 0x8d,
+	0x02, 0x0a, 0x14, 0x4d, 0x6f, 0x6e, 0x69, 0x74, 0x6f, 0x72, 0x69, 0x6e, 0x67, 0x49, 0x6e, 0x66,
+	0x6f, 0x4c, 0x61, 0x62, 0x65, 0x6c, 0x73, 0x12, 0x21, 0x0a, 0x09, 0x54, 0x52, 0x41, 0x4e, 0x53,
+	0x46, 0x4f, 0x52, 0x4d, 0x10, 0x00, 0x1a, 0x12, 0xa2, 0xd4, 0xe0, 0xe5, 0x03, 0x0c, 0x0a, 0x0a,
+	0x50, 0x54, 0x52, 0x41, 0x4e, 0x53, 0x46, 0x4f, 0x52, 0x4d, 0x12, 0x24, 0x0a, 0x0b, 0x50, 0x43,
+	0x4f, 0x4c, 0x4c, 0x45, 0x43, 0x54, 0x49, 0x4f, 0x4e, 0x10, 0x01, 0x1a, 0x13, 0xa2, 0xd4, 0xe0,
+	0xe5, 0x03, 0x0d, 0x0a, 0x0b, 0x50, 0x43, 0x4f, 0x4c, 0x4c, 0x45, 0x43, 0x54, 0x49, 0x4f, 0x4e,
+	0x12, 0x32, 0x0a, 0x12, 0x57, 0x49, 0x4e, 0x44, 0x4f, 0x57, 0x49, 0x4e, 0x47, 0x5f, 0x53, 0x54,
+	0x52, 0x41, 0x54, 0x45, 0x47, 0x59, 0x10, 0x02, 0x1a, 0x1a, 0xa2, 0xd4, 0xe0, 0xe5, 0x03, 0x14,
+	0x0a, 0x12, 0x57, 0x49, 0x4e, 0x44, 0x4f, 0x57, 0x49, 0x4e, 0x47, 0x5f, 0x53, 0x54, 0x52, 0x41,
+	0x54, 0x45, 0x47, 0x59, 0x12, 0x18, 0x0a, 0x05, 0x43, 0x4f, 0x44, 0x45, 0x52, 0x10, 0x03, 0x1a,
+	0x0d, 0xa2, 0xd4, 0xe0, 0xe5, 0x03, 0x07, 0x0a, 0x05, 0x43, 0x4f, 0x44, 0x45, 0x52, 0x12, 0x24,
+	0x0a, 0x0b, 0x45, 0x4e, 0x56, 0x49, 0x52, 0x4f, 0x4e, 0x4d, 0x45, 0x4e, 0x54, 0x10, 0x04, 0x1a,
+	0x13, 0xa2, 0xd4, 0xe0, 0xe5, 0x03, 0x0d, 0x0a, 0x0b, 0x45, 0x4e, 0x56, 0x49, 0x52, 0x4f, 0x4e,
+	0x4d, 0x45, 0x4e, 0x54, 0x12, 0x20, 0x0a, 0x09, 0x4e, 0x41, 0x4d, 0x45, 0x53, 0x50, 0x41, 0x43,
+	0x45, 0x10, 0x05, 0x1a, 0x11, 0xa2, 0xd4, 0xe0, 0xe5, 0x03, 0x0b, 0x0a, 0x09, 0x4e, 0x41, 0x4d,
+	0x45, 0x53, 0x50, 0x41, 0x43, 0x45, 0x12, 0x16, 0x0a, 0x04, 0x4e, 0x41, 0x4d, 0x45, 0x10, 0x06,
+	0x1a, 0x0c, 0xa2, 0xd4, 0xe0, 0xe5, 0x03, 0x06, 0x0a, 0x04, 0x4e, 0x41, 0x4d, 0x45, 0x22, 0xbc,
+	0x05, 0x0a, 0x16, 0x4d, 0x6f, 0x6e, 0x69, 0x74, 0x6f, 0x72, 0x69, 0x6e, 0x67, 0x49, 0x6e, 0x66,
+	0x6f, 0x54, 0x79, 0x70, 0x65, 0x55, 0x72, 0x6e, 0x73, 0x22, 0xa1, 0x05, 0x0a, 0x04, 0x45, 0x6e,
+	0x75, 0x6d, 0x12, 0x33, 0x0a, 0x0e, 0x53, 0x55, 0x4d, 0x5f, 0x49, 0x4e, 0x54, 0x36, 0x34, 0x5f,
+	0x54, 0x59, 0x50, 0x45, 0x10, 0x00, 0x1a, 0x1f, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x19, 0x62, 0x65,
+	0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x3a, 0x73, 0x75, 0x6d, 0x5f, 0x69,
+	0x6e, 0x74, 0x36, 0x34, 0x3a, 0x76, 0x31, 0x12, 0x35, 0x0a, 0x0f, 0x53, 0x55, 0x4d, 0x5f, 0x44,
+	0x4f, 0x55, 0x42, 0x4c, 0x45, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x10, 0x01, 0x1a, 0x20, 0xa2, 0xb4,
+	0xfa, 0xc2, 0x05, 0x1a, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73,
+	0x3a, 0x73, 0x75, 0x6d, 0x5f, 0x64, 0x6f, 0x75, 0x62, 0x6c, 0x65, 0x3a, 0x76, 0x31, 0x12, 0x45,
+	0x0a, 0x17, 0x44, 0x49, 0x53, 0x54, 0x52, 0x49, 0x42, 0x55, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x49,
+	0x4e, 0x54, 0x36, 0x34, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x10, 0x02, 0x1a, 0x28, 0xa2, 0xb4, 0xfa,
+	0xc2, 0x05, 0x22, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x3a,
+	0x64, 0x69, 0x73, 0x74, 0x72, 0x69, 0x62, 0x75, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x69, 0x6e, 0x74,
+	0x36, 0x34, 0x3a, 0x76, 0x31, 0x12, 0x47, 0x0a, 0x18, 0x44, 0x49, 0x53, 0x54, 0x52, 0x49, 0x42,
+	0x55, 0x54, 0x49, 0x4f, 0x4e, 0x5f, 0x44, 0x4f, 0x55, 0x42, 0x4c, 0x45, 0x5f, 0x54, 0x59, 0x50,
+	0x45, 0x10, 0x03, 0x1a, 0x29, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x23, 0x62, 0x65, 0x61, 0x6d, 0x3a,
+	0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x3a, 0x64, 0x69, 0x73, 0x74, 0x72, 0x69, 0x62, 0x75,
+	0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x64, 0x6f, 0x75, 0x62, 0x6c, 0x65, 0x3a, 0x76, 0x31, 0x12, 0x39,
+	0x0a, 0x11, 0x4c, 0x41, 0x54, 0x45, 0x53, 0x54, 0x5f, 0x49, 0x4e, 0x54, 0x36, 0x34, 0x5f, 0x54,
+	0x59, 0x50, 0x45, 0x10, 0x04, 0x1a, 0x22, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x1c, 0x62, 0x65, 0x61,
+	0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x3a, 0x6c, 0x61, 0x74, 0x65, 0x73, 0x74,
+	0x5f, 0x69, 0x6e, 0x74, 0x36, 0x34, 0x3a, 0x76, 0x31, 0x12, 0x3b, 0x0a, 0x12, 0x4c, 0x41, 0x54,
+	0x45, 0x53, 0x54, 0x5f, 0x44, 0x4f, 0x55, 0x42, 0x4c, 0x45, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x10,
+	0x05, 0x1a, 0x23, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x1d, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65,
+	0x74, 0x72, 0x69, 0x63, 0x73, 0x3a, 0x6c, 0x61, 0x74, 0x65, 0x73, 0x74, 0x5f, 0x64, 0x6f, 0x75,
+	0x62, 0x6c, 0x65, 0x3a, 0x76, 0x31, 0x12, 0x37, 0x0a, 0x10, 0x54, 0x4f, 0x50, 0x5f, 0x4e, 0x5f,
+	0x49, 0x4e, 0x54, 0x36, 0x34, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x10, 0x06, 0x1a, 0x21, 0xa2, 0xb4,
+	0xfa, 0xc2, 0x05, 0x1b, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73,
+	0x3a, 0x74, 0x6f, 0x70, 0x5f, 0x6e, 0x5f, 0x69, 0x6e, 0x74, 0x36, 0x34, 0x3a, 0x76, 0x31, 0x12,
+	0x39, 0x0a, 0x11, 0x54, 0x4f, 0x50, 0x5f, 0x4e, 0x5f, 0x44, 0x4f, 0x55, 0x42, 0x4c, 0x45, 0x5f,
+	0x54, 0x59, 0x50, 0x45, 0x10, 0x07, 0x1a, 0x22, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x1c, 0x62, 0x65,
+	0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x3a, 0x74, 0x6f, 0x70, 0x5f, 0x6e,
+	0x5f, 0x64, 0x6f, 0x75, 0x62, 0x6c, 0x65, 0x3a, 0x76, 0x31, 0x12, 0x3d, 0x0a, 0x13, 0x42, 0x4f,
+	0x54, 0x54, 0x4f, 0x4d, 0x5f, 0x4e, 0x5f, 0x49, 0x4e, 0x54, 0x36, 0x34, 0x5f, 0x54, 0x59, 0x50,
+	0x45, 0x10, 0x08, 0x1a, 0x24, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x1e, 0x62, 0x65, 0x61, 0x6d, 0x3a,
+	0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x3a, 0x62, 0x6f, 0x74, 0x74, 0x6f, 0x6d, 0x5f, 0x6e,
+	0x5f, 0x69, 0x6e, 0x74, 0x36, 0x34, 0x3a, 0x76, 0x31, 0x12, 0x3f, 0x0a, 0x14, 0x42, 0x4f, 0x54,
+	0x54, 0x4f, 0x4d, 0x5f, 0x4e, 0x5f, 0x44, 0x4f, 0x55, 0x42, 0x4c, 0x45, 0x5f, 0x54, 0x59, 0x50,
+	0x45, 0x10, 0x09, 0x1a, 0x25, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x1f, 0x62, 0x65, 0x61, 0x6d, 0x3a,
+	0x6d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73, 0x3a, 0x62, 0x6f, 0x74, 0x74, 0x6f, 0x6d, 0x5f, 0x6e,
+	0x5f, 0x64, 0x6f, 0x75, 0x62, 0x6c, 0x65, 0x3a, 0x76, 0x31, 0x12, 0x31, 0x0a, 0x0d, 0x50, 0x52,
+	0x4f, 0x47, 0x52, 0x45, 0x53, 0x53, 0x5f, 0x54, 0x59, 0x50, 0x45, 0x10, 0x0a, 0x1a, 0x1e, 0xa2,
+	0xb4, 0xfa, 0xc2, 0x05, 0x18, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x6d, 0x65, 0x74, 0x72, 0x69, 0x63,
+	0x73, 0x3a, 0x70, 0x72, 0x6f, 0x67, 0x72, 0x65, 0x73, 0x73, 0x3a, 0x76, 0x31, 0x3a, 0x82, 0x01,
+	0x0a, 0x0b, 0x6c, 0x61, 0x62, 0x65, 0x6c, 0x5f, 0x70, 0x72, 0x6f, 0x70, 0x73, 0x12, 0x21, 0x2e,
+	0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e,
+	0x45, 0x6e, 0x75, 0x6d, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73,
+	0x18, 0xc4, 0x8a, 0xdc, 0x3c, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x3b, 0x2e, 0x6f, 0x72, 0x67, 0x2e,
+	0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65,
+	0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x4d, 0x6f,
+	0x6e, 0x69, 0x74, 0x6f, 0x72, 0x69, 0x6e, 0x67, 0x49, 0x6e, 0x66, 0x6f, 0x4c, 0x61, 0x62, 0x65,
+	0x6c, 0x50, 0x72, 0x6f, 0x70, 0x73, 0x52, 0x0a, 0x6c, 0x61, 0x62, 0x65, 0x6c, 0x50, 0x72, 0x6f,
+	0x70, 0x73, 0x3a, 0x8d, 0x01, 0x0a, 0x14, 0x6d, 0x6f, 0x6e, 0x69, 0x74, 0x6f, 0x72, 0x69, 0x6e,
+	0x67, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x5f, 0x73, 0x70, 0x65, 0x63, 0x12, 0x21, 0x2e, 0x67, 0x6f,
+	0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6e,
+	0x75, 0x6d, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0xfa,
+	0xf4, 0xe4, 0x62, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x35, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70,
+	0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e,
+	0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x4d, 0x6f, 0x6e, 0x69,
+	0x74, 0x6f, 0x72, 0x69, 0x6e, 0x67, 0x49, 0x6e, 0x66, 0x6f, 0x53, 0x70, 0x65, 0x63, 0x52, 0x12,
+	0x6d, 0x6f, 0x6e, 0x69, 0x74, 0x6f, 0x72, 0x69, 0x6e, 0x67, 0x49, 0x6e, 0x66, 0x6f, 0x53, 0x70,
+	0x65, 0x63, 0x42, 0x76, 0x0a, 0x21, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65,
+	0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65,
+	0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x42, 0x0a, 0x4d, 0x65, 0x74, 0x72, 0x69, 0x63, 0x73,
+	0x41, 0x70, 0x69, 0x5a, 0x45, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f,
+	0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2f, 0x62, 0x65, 0x61, 0x6d, 0x2f, 0x73, 0x64, 0x6b, 0x73,
+	0x2f, 0x67, 0x6f, 0x2f, 0x70, 0x6b, 0x67, 0x2f, 0x62, 0x65, 0x61, 0x6d, 0x2f, 0x6d, 0x6f, 0x64,
+	0x65, 0x6c, 0x2f, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x5f, 0x76, 0x31, 0x3b, 0x70,
+	0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x5f, 0x76, 0x31, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74,
+	0x6f, 0x33,
 }
 
-var E_MonitoringInfoSpec = &proto.ExtensionDesc{
-	ExtendedType:  (*descriptor.EnumValueOptions)(nil),
-	ExtensionType: (*MonitoringInfoSpec)(nil),
-	Field:         207174266,
-	Name:          "org.apache.beam.model.pipeline.v1.monitoring_info_spec",
-	Tag:           "bytes,207174266,opt,name=monitoring_info_spec",
-	Filename:      "metrics.proto",
+var (
+	file_metrics_proto_rawDescOnce sync.Once
+	file_metrics_proto_rawDescData = file_metrics_proto_rawDesc
+)
+
+func file_metrics_proto_rawDescGZIP() []byte {
+	file_metrics_proto_rawDescOnce.Do(func() {
+		file_metrics_proto_rawDescData = protoimpl.X.CompressGZIP(file_metrics_proto_rawDescData)
+	})
+	return file_metrics_proto_rawDescData
 }
 
-func init() {
-	proto.RegisterEnum("org.apache.beam.model.pipeline.v1.MonitoringInfoSpecs_Enum", MonitoringInfoSpecs_Enum_name, MonitoringInfoSpecs_Enum_value)
-	proto.RegisterEnum("org.apache.beam.model.pipeline.v1.MonitoringInfo_MonitoringInfoLabels", MonitoringInfo_MonitoringInfoLabels_name, MonitoringInfo_MonitoringInfoLabels_value)
-	proto.RegisterEnum("org.apache.beam.model.pipeline.v1.MonitoringInfoTypeUrns_Enum", MonitoringInfoTypeUrns_Enum_name, MonitoringInfoTypeUrns_Enum_value)
-	proto.RegisterType((*MonitoringInfoSpec)(nil), "org.apache.beam.model.pipeline.v1.MonitoringInfoSpec")
-	proto.RegisterType((*Annotation)(nil), "org.apache.beam.model.pipeline.v1.Annotation")
-	proto.RegisterType((*MonitoringInfoSpecs)(nil), "org.apache.beam.model.pipeline.v1.MonitoringInfoSpecs")
-	proto.RegisterType((*MonitoringInfoLabelProps)(nil), "org.apache.beam.model.pipeline.v1.MonitoringInfoLabelProps")
-	proto.RegisterType((*MonitoringInfo)(nil), "org.apache.beam.model.pipeline.v1.MonitoringInfo")
-	proto.RegisterMapType((map[string]string)(nil), "org.apache.beam.model.pipeline.v1.MonitoringInfo.LabelsEntry")
-	proto.RegisterType((*MonitoringInfoTypeUrns)(nil), "org.apache.beam.model.pipeline.v1.MonitoringInfoTypeUrns")
-	proto.RegisterExtension(E_LabelProps)
-	proto.RegisterExtension(E_MonitoringInfoSpec)
+var file_metrics_proto_enumTypes = make([]protoimpl.EnumInfo, 3)
+var file_metrics_proto_msgTypes = make([]protoimpl.MessageInfo, 7)
+var file_metrics_proto_goTypes = []interface{}{
+	(MonitoringInfoSpecs_Enum)(0),            // 0: org.apache.beam.model.pipeline.v1.MonitoringInfoSpecs.Enum
+	(MonitoringInfo_MonitoringInfoLabels)(0), // 1: org.apache.beam.model.pipeline.v1.MonitoringInfo.MonitoringInfoLabels
+	(MonitoringInfoTypeUrns_Enum)(0),         // 2: org.apache.beam.model.pipeline.v1.MonitoringInfoTypeUrns.Enum
+	(*MonitoringInfoSpec)(nil),               // 3: org.apache.beam.model.pipeline.v1.MonitoringInfoSpec
+	(*Annotation)(nil),                       // 4: org.apache.beam.model.pipeline.v1.Annotation
+	(*MonitoringInfoSpecs)(nil),              // 5: org.apache.beam.model.pipeline.v1.MonitoringInfoSpecs
+	(*MonitoringInfoLabelProps)(nil),         // 6: org.apache.beam.model.pipeline.v1.MonitoringInfoLabelProps
+	(*MonitoringInfo)(nil),                   // 7: org.apache.beam.model.pipeline.v1.MonitoringInfo
+	(*MonitoringInfoTypeUrns)(nil),           // 8: org.apache.beam.model.pipeline.v1.MonitoringInfoTypeUrns
+	nil,                                      // 9: org.apache.beam.model.pipeline.v1.MonitoringInfo.LabelsEntry
+	(*descriptor.EnumValueOptions)(nil),      // 10: google.protobuf.EnumValueOptions
+}
+var file_metrics_proto_depIdxs = []int32{
+	4,  // 0: org.apache.beam.model.pipeline.v1.MonitoringInfoSpec.annotations:type_name -> org.apache.beam.model.pipeline.v1.Annotation
+	9,  // 1: org.apache.beam.model.pipeline.v1.MonitoringInfo.labels:type_name -> org.apache.beam.model.pipeline.v1.MonitoringInfo.LabelsEntry
+	10, // 2: org.apache.beam.model.pipeline.v1.label_props:extendee -> google.protobuf.EnumValueOptions
+	10, // 3: org.apache.beam.model.pipeline.v1.monitoring_info_spec:extendee -> google.protobuf.EnumValueOptions
+	6,  // 4: org.apache.beam.model.pipeline.v1.label_props:type_name -> org.apache.beam.model.pipeline.v1.MonitoringInfoLabelProps
+	3,  // 5: org.apache.beam.model.pipeline.v1.monitoring_info_spec:type_name -> org.apache.beam.model.pipeline.v1.MonitoringInfoSpec
+	6,  // [6:6] is the sub-list for method output_type
+	6,  // [6:6] is the sub-list for method input_type
+	4,  // [4:6] is the sub-list for extension type_name
+	2,  // [2:4] is the sub-list for extension extendee
+	0,  // [0:2] is the sub-list for field type_name
 }
 
-func init() { proto.RegisterFile("metrics.proto", fileDescriptor_6039342a2ba47b72) }
-
-var fileDescriptor_6039342a2ba47b72 = []byte{
-	// 1955 bytes of a gzipped FileDescriptorProto
-	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xdc, 0x98, 0xcf, 0x8f, 0x1c, 0x47,
-	0x15, 0xc7, 0x5d, 0x33, 0xbb, 0xeb, 0x6c, 0x8d, 0x77, 0xdd, 0x5b, 0xbb, 0xc4, 0x43, 0x91, 0x78,
-	0x6b, 0xc7, 0x81, 0x6c, 0x24, 0x98, 0xc8, 0xce, 0xc6, 0x49, 0xc6, 0x89, 0xd0, 0xec, 0x4c, 0x67,
-	0x33, 0x64, 0x7e, 0xa9, 0xa7, 0x27, 0x8e, 0x7d, 0x69, 0xd5, 0x74, 0xd7, 0xee, 0xb4, 0xdc, 0xd3,
-	0xdd, 0x74, 0xd7, 0xac, 0xbd, 0x3e, 0x92, 0x9b, 0x91, 0x41, 0x04, 0x08, 0x3f, 0x14, 0x88, 0xb0,
-	0x10, 0x06, 0x04, 0x08, 0x89, 0x80, 0x14, 0x11, 0x24, 0x14, 0x82, 0xc4, 0x85, 0x0b, 0xe2, 0x00,
-	0x12, 0x08, 0x0e, 0xf0, 0x0f, 0x20, 0xc4, 0x21, 0x27, 0x54, 0x5d, 0xf3, 0xab, 0x67, 0x66, 0x7f,
-	0x21, 0x2f, 0x87, 0xdc, 0xba, 0xab, 0xea, 0xd5, 0xfb, 0x7c, 0x5f, 0x55, 0x57, 0xd7, 0x7b, 0x70,
-	0xa1, 0xc3, 0x78, 0x60, 0x9b, 0x61, 0xd6, 0x0f, 0x3c, 0xee, 0xa1, 0x35, 0x2f, 0xd8, 0xc9, 0x52,
-	0x9f, 0x9a, 0x6d, 0x96, 0x6d, 0x31, 0xda, 0xc9, 0x76, 0x3c, 0x8b, 0x39, 0x59, 0xdf, 0xf6, 0x99,
-	0x63, 0xbb, 0x2c, 0xbb, 0x7b, 0x11, 0x7f, 0x44, 0xb4, 0x1b, 0x41, 0xd7, 0x75, 0x59, 0x60, 0x50,
-	0xdf, 0x96, 0x96, 0x98, 0xec, 0x78, 0xde, 0x8e, 0xc3, 0x9e, 0x8c, 0xde, 0x5a, 0xdd, 0xed, 0x27,
-	0x2d, 0x16, 0x9a, 0x81, 0xed, 0x73, 0x2f, 0xe8, 0x8d, 0x58, 0x1d, 0x1f, 0xc1, 0xed, 0x0e, 0x0b,
-	0x39, 0xed, 0xf8, 0x72, 0x40, 0xe6, 0x6d, 0x00, 0x51, 0xc5, 0x73, 0x6d, 0xee, 0x05, 0xb6, 0xbb,
-	0x53, 0x72, 0xb7, 0xbd, 0x86, 0xcf, 0x4c, 0xa4, 0xc0, 0x64, 0x37, 0x70, 0xd3, 0x80, 0x80, 0xf5,
-	0x79, 0x4d, 0x3c, 0x22, 0x04, 0x67, 0xf8, 0x9e, 0xcf, 0xd2, 0x89, 0xa8, 0x29, 0x7a, 0x46, 0x8f,
-	0xc3, 0xb3, 0x01, 0xfb, 0x6c, 0xd7, 0x0e, 0x98, 0x65, 0x38, 0xb4, 0xc5, 0x9c, 0x30, 0x9d, 0x24,
-	0xc9, 0xf5, 0x79, 0x6d, 0xb1, 0xdf, 0x5c, 0x8e, 0x5a, 0x51, 0x0d, 0xa6, 0xa8, 0xeb, 0x7a, 0x9c,
-	0x72, 0xdb, 0x73, 0xc3, 0xf4, 0x0c, 0x49, 0xae, 0xa7, 0x2e, 0x7d, 0x2a, 0x7b, 0xa8, 0xf0, 0x6c,
-	0x7e, 0x60, 0xa5, 0x8d, 0xce, 0x90, 0xd9, 0x80, 0x70, 0xd8, 0x25, 0x68, 0x6f, 0xb0, 0xbd, 0x3e,
-	0xed, 0x0d, 0xb6, 0x87, 0x56, 0xe0, 0xec, 0x2e, 0x75, 0xba, 0x7d, 0x5c, 0xf9, 0x92, 0xf9, 0xc2,
-	0x2a, 0x5c, 0x9e, 0x14, 0x1b, 0x66, 0xfe, 0x73, 0x1e, 0xce, 0xa8, 0x6e, 0xb7, 0x83, 0xee, 0x03,
-	0xb8, 0xd8, 0x6c, 0xa8, 0x9a, 0xd1, 0x68, 0x56, 0x8c, 0x52, 0x55, 0xbf, 0xbc, 0xa1, 0x9c, 0xc2,
-	0xaf, 0x83, 0x3f, 0xdc, 0xbf, 0xff, 0xc6, 0xdc, 0xe7, 0x01, 0x7c, 0x54, 0x50, 0xe6, 0xe4, 0xea,
-	0xe5, 0xba, 0x21, 0x0b, 0x72, 0x61, 0xb7, 0x63, 0xd8, 0x2e, 0xbf, 0xbc, 0x91, 0xdb, 0xbd, 0x88,
-	0x3e, 0x3a, 0xd2, 0x1d, 0xc6, 0xba, 0x30, 0xac, 0xeb, 0x5a, 0xbe, 0xda, 0x78, 0xb1, 0xa6, 0x55,
-	0xf0, 0x7c, 0x35, 0x5f, 0x51, 0x1b, 0xf5, 0x7c, 0x41, 0xc5, 0x33, 0xe2, 0x31, 0x73, 0x09, 0xa6,
-	0xfa, 0x0b, 0x27, 0xd4, 0x5c, 0x68, 0x6a, 0x55, 0xd2, 0xe5, 0xb6, 0x63, 0xdf, 0x66, 0x16, 0xe1,
-	0x1e, 0x09, 0x98, 0xef, 0x05, 0x9c, 0x08, 0x87, 0x44, 0xce, 0x9e, 0x45, 0x3f, 0x04, 0xf0, 0xec,
-	0x80, 0xb4, 0x58, 0x6b, 0x6e, 0x96, 0x55, 0x05, 0xe0, 0xaf, 0x48, 0xd4, 0xbb, 0x00, 0x9e, 0x9f,
-	0x8a, 0x6a, 0x79, 0xdd, 0x96, 0xc3, 0x04, 0x2b, 0x9e, 0x60, 0x1d, 0xf4, 0x3d, 0x78, 0xd8, 0xf7,
-	0x00, 0x3c, 0x17, 0xc1, 0x16, 0x4b, 0x0d, 0x5d, 0x2b, 0x6d, 0x36, 0xf5, 0x52, 0xad, 0xda, 0x8b,
-	0x6f, 0x02, 0x7f, 0x57, 0x42, 0x7f, 0x0b, 0xc0, 0x4f, 0x4c, 0x40, 0x5b, 0x76, 0xc8, 0x03, 0xbb,
-	0xd5, 0x15, 0xbe, 0x86, 0x81, 0xce, 0xc4, 0xe0, 0xa7, 0x8e, 0x79, 0xf0, 0x22, 0x7e, 0x0b, 0x60,
-	0x7a, 0x52, 0x44, 0x2f, 0xf4, 0x49, 0xfc, 0x3d, 0xa9, 0xe2, 0x2d, 0x00, 0x1f, 0x3f, 0x58, 0xc5,
-	0x70, 0x0d, 0x2e, 0xec, 0x2f, 0xe3, 0x04, 0x17, 0xe3, 0xa7, 0x00, 0x2e, 0x45, 0x3a, 0xca, 0x79,
-	0x5d, 0x6d, 0xe8, 0xbd, 0x65, 0x98, 0xc1, 0x5f, 0x97, 0x02, 0xbe, 0x04, 0x20, 0x99, 0x10, 0xe0,
-	0x50, 0xce, 0x42, 0x3e, 0x5c, 0x80, 0x47, 0x62, 0xe4, 0x63, 0xbd, 0x0f, 0x1e, 0xf9, 0x67, 0x00,
-	0xa2, 0x51, 0xe4, 0x5e, 0xd0, 0x67, 0xf1, 0x37, 0x25, 0xf3, 0x97, 0x01, 0x5c, 0xdb, 0x8f, 0x79,
-	0x18, 0xee, 0x47, 0xa7, 0x41, 0x9f, 0x60, 0xa0, 0x7f, 0x0c, 0xa0, 0x12, 0x51, 0xeb, 0xb5, 0xba,
-	0xd1, 0xdf, 0xee, 0x73, 0xf8, 0x0d, 0xc9, 0xfc, 0x45, 0x00, 0x57, 0x27, 0x98, 0xb9, 0xe7, 0x1b,
-	0x23, 0xfb, 0xfc, 0x63, 0x31, 0xe2, 0x78, 0xe7, 0x09, 0x6e, 0x0c, 0xc9, 0xdb, 0x0b, 0xf2, 0xe9,
-	0x03, 0x37, 0x86, 0x64, 0x1a, 0xc6, 0xf8, 0x91, 0x29, 0xc4, 0x27, 0x18, 0xe2, 0x5f, 0x00, 0xb8,
-	0x1c, 0x21, 0x6f, 0xd6, 0x74, 0xbd, 0x56, 0x19, 0x44, 0xf9, 0x21, 0xfc, 0xa6, 0x84, 0xfe, 0x2a,
-	0x80, 0x99, 0x09, 0xe8, 0x96, 0xc7, 0xb9, 0xd7, 0x19, 0x0d, 0xf4, 0xf9, 0x18, 0xf6, 0x44, 0xff,
-	0x83, 0x07, 0x7f, 0x07, 0xc0, 0x95, 0x38, 0x78, 0x2f, 0xdc, 0xf3, 0xf8, 0xdb, 0x92, 0xfc, 0x6b,
-	0x00, 0x5e, 0xd8, 0x9f, 0x7c, 0x18, 0xf1, 0xd5, 0xe9, 0xe8, 0x27, 0xbb, 0xaf, 0x17, 0xd4, 0xb2,
-	0x5a, 0x51, 0xab, 0xba, 0x51, 0xa8, 0x35, 0xab, 0xba, 0x02, 0xf1, 0x37, 0x24, 0xf4, 0xeb, 0x00,
-	0x8e, 0xee, 0x80, 0x1c, 0x73, 0x58, 0x87, 0xb9, 0xdc, 0x30, 0xbd, 0xae, 0xcb, 0x0f, 0xf9, 0x45,
-	0xa6, 0xea, 0x85, 0x5a, 0xb9, 0xac, 0x16, 0xc4, 0xb1, 0x9a, 0xf9, 0x4c, 0x9c, 0xeb, 0x8a, 0xde,
-	0x66, 0x84, 0x7b, 0x9c, 0x3a, 0xa4, 0x37, 0x65, 0x48, 0xbc, 0x2e, 0xf7, 0xbb, 0x5c, 0x40, 0x52,
-	0x52, 0x37, 0x3d, 0xc7, 0x61, 0xa6, 0x18, 0x4b, 0x5a, 0x7b, 0xa2, 0x41, 0x0f, 0xa8, 0x1b, 0x6e,
-	0x7b, 0x41, 0x27, 0x8b, 0x7e, 0x9f, 0x80, 0x4b, 0x8d, 0x7c, 0xa5, 0x5e, 0x56, 0x8b, 0xc6, 0xe6,
-	0x35, 0x5d, 0x35, 0x1a, 0xa5, 0xeb, 0xaa, 0x92, 0xc2, 0x3f, 0x4f, 0x44, 0xcc, 0x3f, 0x49, 0xc4,
-	0xf7, 0x75, 0x48, 0x3b, 0xbe, 0xc3, 0x2c, 0xa3, 0xb5, 0xc7, 0x99, 0x11, 0xda, 0xb7, 0xd9, 0x51,
-	0xff, 0x38, 0x31, 0x01, 0x7f, 0x02, 0x71, 0x05, 0xbf, 0x03, 0x43, 0x09, 0x62, 0x66, 0x22, 0x66,
-	0x26, 0xd4, 0xb5, 0x48, 0x14, 0x1b, 0xe2, 0x6d, 0x13, 0x4a, 0x7a, 0xae, 0x09, 0x09, 0x19, 0x27,
-	0xeb, 0x5e, 0x40, 0xa8, 0xe3, 0x3c, 0x21, 0xba, 0x06, 0xa2, 0x6d, 0x97, 0xf0, 0x36, 0x23, 0xfe,
-	0x50, 0x6f, 0x96, 0x34, 0x84, 0x95, 0xed, 0xee, 0x10, 0x3b, 0x14, 0x0b, 0x65, 0x11, 0xd2, 0x62,
-	0x26, 0xed, 0x86, 0x8c, 0x98, 0xd4, 0x31, 0xbb, 0x0e, 0xe5, 0xa2, 0x53, 0x98, 0x45, 0x8e, 0xa5,
-	0x3f, 0xdb, 0xdd, 0xf5, 0x9c, 0x5d, 0x16, 0x92, 0x90, 0x05, 0x36, 0x75, 0xec, 0xdb, 0xfd, 0x31,
-	0x43, 0x5f, 0x37, 0xdb, 0xb6, 0xd9, 0x16, 0x93, 0x16, 0xea, 0x4d, 0x62, 0xbb, 0x9c, 0xb9, 0xa1,
-	0xbd, 0xcb, 0xb2, 0xe8, 0xcf, 0x00, 0xa2, 0x86, 0x9e, 0xd7, 0x74, 0x63, 0xb3, 0x59, 0x2d, 0x96,
-	0x55, 0xa3, 0xd2, 0x50, 0x0b, 0x0d, 0xe5, 0x0c, 0xfe, 0xb5, 0xdc, 0x04, 0xef, 0x00, 0x78, 0x79,
-	0x34, 0xa0, 0x3e, 0x0d, 0x2c, 0xcf, 0x60, 0xb7, 0x98, 0x29, 0x03, 0x26, 0xae, 0xa0, 0xb9, 0x90,
-	0xd3, 0x80, 0x1b, 0xad, 0xae, 0x6b, 0x39, 0xcc, 0xe8, 0x84, 0xcc, 0x0c, 0x8f, 0x7e, 0x83, 0xca,
-	0xbc, 0x1a, 0x8f, 0x6d, 0x69, 0x64, 0x77, 0x84, 0xdc, 0xee, 0x50, 0xce, 0x2c, 0x32, 0xf0, 0x47,
-	0x84, 0x3f, 0x11, 0x44, 0xa1, 0x2e, 0x72, 0x4b, 0xa4, 0xdb, 0xed, 0xae, 0x2b, 0x77, 0x8c, 0xed,
-	0x12, 0x4a, 0x22, 0x4a, 0xf4, 0x37, 0x00, 0x57, 0xea, 0x5a, 0xad, 0xa0, 0x36, 0x1a, 0x71, 0x75,
-	0x0b, 0xf8, 0x7d, 0xa9, 0xee, 0x57, 0x00, 0x3e, 0x7b, 0xa8, 0x3a, 0x3f, 0xf0, 0x4c, 0x16, 0x86,
-	0xff, 0xab, 0xbe, 0xeb, 0x71, 0x7d, 0x2f, 0x1f, 0x5d, 0x5f, 0xcf, 0xf1, 0x01, 0x0a, 0xff, 0x0a,
-	0xe0, 0xf2, 0x8b, 0xa5, 0x6a, 0xa9, 0xf1, 0x52, 0x5c, 0xe0, 0x22, 0xfe, 0x8d, 0x14, 0xf8, 0x2e,
-	0x80, 0xcf, 0x1c, 0x2a, 0x70, 0xdb, 0x76, 0xed, 0xb0, 0xfd, 0xff, 0xd7, 0x27, 0xfd, 0xf6, 0xe4,
-	0x91, 0x69, 0xfa, 0x7e, 0x09, 0x60, 0x4a, 0xaf, 0xe9, 0xf9, 0x72, 0x4f, 0xd7, 0x59, 0xfc, 0x03,
-	0xa9, 0xeb, 0x1e, 0x80, 0x1b, 0x31, 0x5d, 0xbc, 0x7f, 0x44, 0x8c, 0x8b, 0x8b, 0x50, 0x8e, 0x2d,
-	0xaa, 0x10, 0x17, 0xb5, 0x71, 0x8c, 0x45, 0x1b, 0x90, 0xa0, 0xb7, 0x12, 0x70, 0xf1, 0x6a, 0x4d,
-	0x7b, 0xd9, 0xd0, 0xd4, 0x4a, 0xbe, 0x54, 0x2d, 0x55, 0xb7, 0x14, 0x05, 0xdf, 0x91, 0x07, 0xd5,
-	0x6b, 0x09, 0xf8, 0xc9, 0x7d, 0x04, 0xf8, 0x81, 0xb7, 0x13, 0xb0, 0x30, 0xcc, 0x05, 0xac, 0x43,
-	0x6d, 0xd7, 0x76, 0x77, 0x04, 0x78, 0x3a, 0x06, 0x3e, 0x18, 0x33, 0xc6, 0xfd, 0xa3, 0xb1, 0x93,
-	0xea, 0xcd, 0xe8, 0xa4, 0x1a, 0x4c, 0x44, 0x68, 0xa7, 0x7f, 0x40, 0xdd, 0xf4, 0x82, 0x1b, 0x64,
-	0xdb, 0x0b, 0x08, 0xa3, 0x66, 0x9b, 0x50, 0x93, 0xdb, 0xbb, 0xac, 0x7f, 0x5c, 0x64, 0x89, 0x3a,
-	0xd9, 0x28, 0x7e, 0x1f, 0x01, 0x0b, 0xa3, 0xd3, 0x84, 0x8a, 0x55, 0xb2, 0x98, 0xcf, 0x5c, 0x4b,
-	0xf4, 0x8c, 0xcd, 0xea, 0x7a, 0x9c, 0x84, 0x6d, 0x1a, 0x30, 0x8b, 0xdc, 0xb4, 0x79, 0x9b, 0x50,
-	0x77, 0x8f, 0x78, 0xbc, 0xcd, 0x82, 0x71, 0x37, 0xc3, 0x08, 0x15, 0x6a, 0xe2, 0x48, 0xd7, 0xd5,
-	0xa2, 0xb2, 0x74, 0x9c, 0x08, 0x99, 0x9e, 0x38, 0x63, 0x39, 0xb3, 0x3e, 0xb4, 0x11, 0xfa, 0x3e,
-	0x80, 0xe7, 0x8a, 0x79, 0x3d, 0x6f, 0x14, 0x5e, 0xca, 0x57, 0xab, 0x6a, 0xd9, 0xd0, 0xd4, 0x7c,
-	0xd1, 0x28, 0x55, 0x8b, 0xea, 0xab, 0x0a, 0xc2, 0x77, 0xe4, 0xd7, 0xf0, 0xda, 0x58, 0xb6, 0x65,
-	0x51, 0x4e, 0x0d, 0xb3, 0x4d, 0x5d, 0x97, 0x39, 0xb9, 0x80, 0x51, 0xcb, 0x10, 0x2c, 0xb7, 0x8e,
-	0xb1, 0xff, 0xc7, 0xaf, 0x12, 0x32, 0x46, 0xd4, 0x8a, 0x44, 0xdd, 0xea, 0xef, 0x75, 0xe1, 0x87,
-	0xf4, 0xfc, 0x64, 0x33, 0x59, 0x98, 0x8e, 0xe7, 0xe3, 0x51, 0xbd, 0xa0, 0x1e, 0x78, 0x7e, 0x88,
-	0x10, 0x9c, 0x71, 0x69, 0x87, 0xf5, 0xb2, 0xfa, 0xe8, 0x39, 0xf3, 0xaf, 0x24, 0x5c, 0x8c, 0x1b,
-	0x1c, 0xb1, 0x52, 0x91, 0x86, 0xa7, 0x7d, 0xba, 0xe7, 0x78, 0xd4, 0x4a, 0x27, 0x09, 0x58, 0x3f,
-	0xa3, 0xf5, 0x5f, 0x51, 0x13, 0xce, 0xf5, 0x4a, 0x17, 0xb2, 0x2a, 0xf1, 0xc2, 0x11, 0xaa, 0x12,
-	0x71, 0x84, 0xac, 0x2c, 0x72, 0xa8, 0x2e, 0x0f, 0xf6, 0xb4, 0xde, 0x64, 0xf8, 0x39, 0x98, 0x1a,
-	0x69, 0x3e, 0x6a, 0x85, 0x22, 0x97, 0x78, 0x16, 0x64, 0xee, 0x26, 0xe0, 0xca, 0x94, 0xa8, 0x84,
-	0x68, 0x0d, 0xce, 0x0f, 0xc2, 0xad, 0x9c, 0xc2, 0xe8, 0xde, 0x1f, 0xff, 0xfe, 0xcf, 0xe4, 0x19,
-	0x38, 0xb2, 0x08, 0xe8, 0x31, 0x38, 0x7a, 0x0b, 0x51, 0x00, 0x5e, 0x8e, 0x06, 0x2d, 0xc4, 0x9a,
-	0xd1, 0x25, 0x88, 0xae, 0x96, 0xaa, 0xc5, 0xda, 0xd5, 0x52, 0x75, 0xcb, 0x68, 0xe8, 0x5a, 0x5e,
-	0x57, 0xb7, 0xae, 0x29, 0x09, 0x8c, 0xa3, 0xc1, 0x2b, 0xd3, 0x7a, 0x51, 0x1a, 0xce, 0x16, 0x6a,
-	0x45, 0x55, 0x53, 0x92, 0x78, 0x21, 0x1a, 0x76, 0xba, 0xd7, 0x20, 0x7c, 0xaa, 0xd5, 0x57, 0x4a,
-	0x5a, 0xad, 0x2a, 0xae, 0x84, 0xca, 0xcc, 0xd0, 0xe7, 0x48, 0x33, 0x22, 0x70, 0x78, 0xf9, 0x54,
-	0x66, 0xf1, 0x52, 0x34, 0x26, 0x35, 0xd2, 0x88, 0x1e, 0x86, 0xd1, 0x9d, 0x54, 0x99, 0xc3, 0x67,
-	0xa2, 0xce, 0x39, 0xf9, 0x9e, 0x79, 0x77, 0x16, 0x3e, 0x1c, 0x8f, 0x87, 0xbe, 0xe7, 0xb3, 0x66,
-	0xe0, 0x86, 0x99, 0xef, 0xcc, 0xf6, 0x0a, 0x37, 0x4f, 0xc1, 0xc5, 0x41, 0xc9, 0xc6, 0xd0, 0xaf,
-	0xd5, 0x55, 0xe5, 0x14, 0x5e, 0xbd, 0xf7, 0xf6, 0x07, 0xef, 0xcd, 0xee, 0xbf, 0x7b, 0xd1, 0xd3,
-	0xf0, 0xec, 0xb0, 0x7a, 0x22, 0xad, 0x00, 0x26, 0x91, 0xd5, 0x01, 0xe5, 0x11, 0xa4, 0xc2, 0x73,
-	0x93, 0x75, 0x0c, 0x69, 0x9e, 0xc0, 0xeb, 0x91, 0xf9, 0x11, 0xae, 0x8b, 0x68, 0x0b, 0xa6, 0xa7,
-	0x54, 0x12, 0xe4, 0x3c, 0x49, 0xfc, 0x44, 0x34, 0xcf, 0x51, 0x2a, 0x04, 0xe8, 0x39, 0xb8, 0x34,
-	0x9a, 0xca, 0xcb, 0x19, 0x66, 0x70, 0x26, 0x9a, 0xe1, 0xc0, 0x4c, 0x1d, 0x5d, 0x81, 0x28, 0x96,
-	0x52, 0x4b, 0xdb, 0x59, 0x7c, 0x21, 0xb2, 0x3d, 0x38, 0x61, 0x46, 0xcf, 0x40, 0x65, 0x24, 0xb3,
-	0x95, 0xa6, 0x73, 0x78, 0x2d, 0x32, 0x3d, 0x28, 0x73, 0x15, 0xc0, 0xa3, 0x29, 0xa6, 0xb4, 0x3c,
-	0x3d, 0x15, 0x78, 0x3c, 0xbf, 0x7c, 0x01, 0x2e, 0xc7, 0x53, 0x3d, 0x69, 0xfc, 0x10, 0x7e, 0x2c,
-	0x32, 0x3e, 0x24, 0x8f, 0x43, 0x9f, 0x86, 0x2b, 0x63, 0x09, 0x97, 0xb4, 0x9f, 0xc7, 0x1f, 0x8f,
-	0xec, 0x0f, 0x4b, 0xa6, 0xd0, 0x45, 0xb8, 0x50, 0xd7, 0x6a, 0x5b, 0x9a, 0xb8, 0x1f, 0x46, 0x96,
-	0x10, 0x9f, 0x8f, 0x2c, 0xf7, 0xfd, 0x93, 0xe4, 0x3e, 0x07, 0x60, 0x2a, 0x3a, 0x14, 0xc4, 0x3f,
-	0xc8, 0x0f, 0xd1, 0x5a, 0x56, 0xd6, 0x64, 0xb3, 0xfd, 0x9a, 0x6c, 0x56, 0xec, 0xe0, 0x57, 0xc4,
-	0xe7, 0x5f, 0xf3, 0x65, 0x7d, 0xf4, 0xfd, 0x3b, 0x7f, 0x79, 0x9e, 0x80, 0xf5, 0xd4, 0xa5, 0x2b,
-	0xc7, 0x3e, 0x8c, 0x86, 0x07, 0xa8, 0x06, 0x9d, 0xc1, 0x73, 0xee, 0x2e, 0x80, 0x2b, 0x9d, 0xc1,
-	0x40, 0xc3, 0x76, 0xb7, 0x3d, 0x23, 0xf4, 0x99, 0x79, 0x14, 0x9a, 0x0f, 0xfe, 0xfd, 0x8f, 0x56,
-	0x44, 0xf3, 0xf4, 0xb1, 0x69, 0x1a, 0x3e, 0x33, 0x35, 0xd4, 0x99, 0x68, 0xdb, 0x7c, 0x1e, 0x1e,
-	0x5e, 0xf5, 0xde, 0x84, 0x15, 0x19, 0xce, 0xbc, 0x6f, 0x5f, 0x4f, 0xf5, 0x3b, 0x8c, 0xdd, 0x8b,
-	0xad, 0xb9, 0x08, 0xf6, 0xa9, 0xff, 0x06, 0x00, 0x00, 0xff, 0xff, 0x1f, 0x47, 0x01, 0x2c, 0x49,
-	0x17, 0x00, 0x00,
+func init() { file_metrics_proto_init() }
+func file_metrics_proto_init() {
+	if File_metrics_proto != nil {
+		return
+	}
+	file_beam_runner_api_proto_init()
+	if !protoimpl.UnsafeEnabled {
+		file_metrics_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*MonitoringInfoSpec); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_metrics_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*Annotation); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_metrics_proto_msgTypes[2].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*MonitoringInfoSpecs); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_metrics_proto_msgTypes[3].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*MonitoringInfoLabelProps); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_metrics_proto_msgTypes[4].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*MonitoringInfo); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_metrics_proto_msgTypes[5].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*MonitoringInfoTypeUrns); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+	}
+	type x struct{}
+	out := protoimpl.TypeBuilder{
+		File: protoimpl.DescBuilder{
+			GoPackagePath: reflect.TypeOf(x{}).PkgPath(),
+			RawDescriptor: file_metrics_proto_rawDesc,
+			NumEnums:      3,
+			NumMessages:   7,
+			NumExtensions: 2,
+			NumServices:   0,
+		},
+		GoTypes:           file_metrics_proto_goTypes,
+		DependencyIndexes: file_metrics_proto_depIdxs,
+		EnumInfos:         file_metrics_proto_enumTypes,
+		MessageInfos:      file_metrics_proto_msgTypes,
+		ExtensionInfos:    file_metrics_proto_extTypes,
+	}.Build()
+	File_metrics_proto = out.File
+	file_metrics_proto_rawDesc = nil
+	file_metrics_proto_goTypes = nil
+	file_metrics_proto_depIdxs = nil
 }
diff --git a/sdks/go/pkg/beam/model/pipeline_v1/schema.pb.go b/sdks/go/pkg/beam/model/pipeline_v1/schema.pb.go
index 7e166ca..720b19e 100644
--- a/sdks/go/pkg/beam/model/pipeline_v1/schema.pb.go
+++ b/sdks/go/pkg/beam/model/pipeline_v1/schema.pb.go
@@ -1,24 +1,48 @@
+//
+// 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.
+
+// ** Experimental **
+// Protocol Buffers describing Beam Schemas, a portable representation for
+// complex types.
+//
+// The primary application of Schema is as the payload for the standard coder
+// "beam:coder:row:v1", defined in beam_runner_api.proto
+
 // Code generated by protoc-gen-go. DO NOT EDIT.
+// versions:
+// 	protoc-gen-go v1.25.0-devel
+// 	protoc        v3.13.0
 // source: schema.proto
 
 package pipeline_v1
 
 import (
-	fmt "fmt"
-	proto "github.com/golang/protobuf/proto"
-	math "math"
+	protoreflect "google.golang.org/protobuf/reflect/protoreflect"
+	protoimpl "google.golang.org/protobuf/runtime/protoimpl"
+	reflect "reflect"
+	sync "sync"
 )
 
-// Reference imports to suppress errors if they are not otherwise used.
-var _ = proto.Marshal
-var _ = fmt.Errorf
-var _ = math.Inf
-
-// This is a compile-time assertion to ensure that this generated file
-// is compatible with the proto package it is being compiled against.
-// A compilation error at this line likely means your copy of the
-// proto package needs to be updated.
-const _ = proto.ProtoPackageIsVersion3 // please upgrade the proto package
+const (
+	// Verify that this generated code is sufficiently up-to-date.
+	_ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion)
+	// Verify that runtime/protoimpl is sufficiently up-to-date.
+	_ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20)
+)
 
 type AtomicType int32
 
@@ -35,221 +59,324 @@
 	AtomicType_BYTES       AtomicType = 9
 )
 
-var AtomicType_name = map[int32]string{
-	0: "UNSPECIFIED",
-	1: "BYTE",
-	2: "INT16",
-	3: "INT32",
-	4: "INT64",
-	5: "FLOAT",
-	6: "DOUBLE",
-	7: "STRING",
-	8: "BOOLEAN",
-	9: "BYTES",
-}
+// Enum value maps for AtomicType.
+var (
+	AtomicType_name = map[int32]string{
+		0: "UNSPECIFIED",
+		1: "BYTE",
+		2: "INT16",
+		3: "INT32",
+		4: "INT64",
+		5: "FLOAT",
+		6: "DOUBLE",
+		7: "STRING",
+		8: "BOOLEAN",
+		9: "BYTES",
+	}
+	AtomicType_value = map[string]int32{
+		"UNSPECIFIED": 0,
+		"BYTE":        1,
+		"INT16":       2,
+		"INT32":       3,
+		"INT64":       4,
+		"FLOAT":       5,
+		"DOUBLE":      6,
+		"STRING":      7,
+		"BOOLEAN":     8,
+		"BYTES":       9,
+	}
+)
 
-var AtomicType_value = map[string]int32{
-	"UNSPECIFIED": 0,
-	"BYTE":        1,
-	"INT16":       2,
-	"INT32":       3,
-	"INT64":       4,
-	"FLOAT":       5,
-	"DOUBLE":      6,
-	"STRING":      7,
-	"BOOLEAN":     8,
-	"BYTES":       9,
+func (x AtomicType) Enum() *AtomicType {
+	p := new(AtomicType)
+	*p = x
+	return p
 }
 
 func (x AtomicType) String() string {
-	return proto.EnumName(AtomicType_name, int32(x))
+	return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x))
 }
 
+func (AtomicType) Descriptor() protoreflect.EnumDescriptor {
+	return file_schema_proto_enumTypes[0].Descriptor()
+}
+
+func (AtomicType) Type() protoreflect.EnumType {
+	return &file_schema_proto_enumTypes[0]
+}
+
+func (x AtomicType) Number() protoreflect.EnumNumber {
+	return protoreflect.EnumNumber(x)
+}
+
+// Deprecated: Use AtomicType.Descriptor instead.
 func (AtomicType) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_1c5fb4d8cc22d66a, []int{0}
+	return file_schema_proto_rawDescGZIP(), []int{0}
 }
 
 type Schema struct {
-	Fields               []*Field  `protobuf:"bytes,1,rep,name=fields,proto3" json:"fields,omitempty"`
-	Id                   string    `protobuf:"bytes,2,opt,name=id,proto3" json:"id,omitempty"`
-	Options              []*Option `protobuf:"bytes,3,rep,name=options,proto3" json:"options,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}  `json:"-"`
-	XXX_unrecognized     []byte    `json:"-"`
-	XXX_sizecache        int32     `json:"-"`
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	// List of fields for this schema. Two fields may not share a name.
+	Fields []*Field `protobuf:"bytes,1,rep,name=fields,proto3" json:"fields,omitempty"`
+	// REQUIRED. An RFC 4122 UUID.
+	Id      string    `protobuf:"bytes,2,opt,name=id,proto3" json:"id,omitempty"`
+	Options []*Option `protobuf:"bytes,3,rep,name=options,proto3" json:"options,omitempty"`
 }
 
-func (m *Schema) Reset()         { *m = Schema{} }
-func (m *Schema) String() string { return proto.CompactTextString(m) }
-func (*Schema) ProtoMessage()    {}
+func (x *Schema) Reset() {
+	*x = Schema{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_schema_proto_msgTypes[0]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *Schema) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*Schema) ProtoMessage() {}
+
+func (x *Schema) ProtoReflect() protoreflect.Message {
+	mi := &file_schema_proto_msgTypes[0]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use Schema.ProtoReflect.Descriptor instead.
 func (*Schema) Descriptor() ([]byte, []int) {
-	return fileDescriptor_1c5fb4d8cc22d66a, []int{0}
+	return file_schema_proto_rawDescGZIP(), []int{0}
 }
 
-func (m *Schema) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_Schema.Unmarshal(m, b)
-}
-func (m *Schema) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_Schema.Marshal(b, m, deterministic)
-}
-func (m *Schema) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Schema.Merge(m, src)
-}
-func (m *Schema) XXX_Size() int {
-	return xxx_messageInfo_Schema.Size(m)
-}
-func (m *Schema) XXX_DiscardUnknown() {
-	xxx_messageInfo_Schema.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_Schema proto.InternalMessageInfo
-
-func (m *Schema) GetFields() []*Field {
-	if m != nil {
-		return m.Fields
+func (x *Schema) GetFields() []*Field {
+	if x != nil {
+		return x.Fields
 	}
 	return nil
 }
 
-func (m *Schema) GetId() string {
-	if m != nil {
-		return m.Id
+func (x *Schema) GetId() string {
+	if x != nil {
+		return x.Id
 	}
 	return ""
 }
 
-func (m *Schema) GetOptions() []*Option {
-	if m != nil {
-		return m.Options
+func (x *Schema) GetOptions() []*Option {
+	if x != nil {
+		return x.Options
 	}
 	return nil
 }
 
 type Field struct {
-	Name                 string     `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"`
-	Description          string     `protobuf:"bytes,2,opt,name=description,proto3" json:"description,omitempty"`
-	Type                 *FieldType `protobuf:"bytes,3,opt,name=type,proto3" json:"type,omitempty"`
-	Id                   int32      `protobuf:"varint,4,opt,name=id,proto3" json:"id,omitempty"`
-	EncodingPosition     int32      `protobuf:"varint,5,opt,name=encoding_position,json=encodingPosition,proto3" json:"encoding_position,omitempty"`
-	Options              []*Option  `protobuf:"bytes,6,rep,name=options,proto3" json:"options,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}   `json:"-"`
-	XXX_unrecognized     []byte     `json:"-"`
-	XXX_sizecache        int32      `json:"-"`
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	// REQUIRED. Name of this field within the schema.
+	Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"`
+	// OPTIONAL. Human readable description of this field, such as the query that generated it.
+	Description string     `protobuf:"bytes,2,opt,name=description,proto3" json:"description,omitempty"`
+	Type        *FieldType `protobuf:"bytes,3,opt,name=type,proto3" json:"type,omitempty"`
+	Id          int32      `protobuf:"varint,4,opt,name=id,proto3" json:"id,omitempty"`
+	// OPTIONAL. The position of this field's data when encoded, e.g. with beam:coder:row:v1.
+	// Either no fields in a given row are have encoding position populated,
+	// or all of them are. Used to support backwards compatibility with schema
+	// changes.
+	// If no fields have encoding position populated the order of encoding is the same as the order in the Schema.
+	EncodingPosition int32     `protobuf:"varint,5,opt,name=encoding_position,json=encodingPosition,proto3" json:"encoding_position,omitempty"`
+	Options          []*Option `protobuf:"bytes,6,rep,name=options,proto3" json:"options,omitempty"`
 }
 
-func (m *Field) Reset()         { *m = Field{} }
-func (m *Field) String() string { return proto.CompactTextString(m) }
-func (*Field) ProtoMessage()    {}
+func (x *Field) Reset() {
+	*x = Field{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_schema_proto_msgTypes[1]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *Field) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*Field) ProtoMessage() {}
+
+func (x *Field) ProtoReflect() protoreflect.Message {
+	mi := &file_schema_proto_msgTypes[1]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use Field.ProtoReflect.Descriptor instead.
 func (*Field) Descriptor() ([]byte, []int) {
-	return fileDescriptor_1c5fb4d8cc22d66a, []int{1}
+	return file_schema_proto_rawDescGZIP(), []int{1}
 }
 
-func (m *Field) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_Field.Unmarshal(m, b)
-}
-func (m *Field) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_Field.Marshal(b, m, deterministic)
-}
-func (m *Field) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Field.Merge(m, src)
-}
-func (m *Field) XXX_Size() int {
-	return xxx_messageInfo_Field.Size(m)
-}
-func (m *Field) XXX_DiscardUnknown() {
-	xxx_messageInfo_Field.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_Field proto.InternalMessageInfo
-
-func (m *Field) GetName() string {
-	if m != nil {
-		return m.Name
+func (x *Field) GetName() string {
+	if x != nil {
+		return x.Name
 	}
 	return ""
 }
 
-func (m *Field) GetDescription() string {
-	if m != nil {
-		return m.Description
+func (x *Field) GetDescription() string {
+	if x != nil {
+		return x.Description
 	}
 	return ""
 }
 
-func (m *Field) GetType() *FieldType {
-	if m != nil {
-		return m.Type
+func (x *Field) GetType() *FieldType {
+	if x != nil {
+		return x.Type
 	}
 	return nil
 }
 
-func (m *Field) GetId() int32 {
-	if m != nil {
-		return m.Id
+func (x *Field) GetId() int32 {
+	if x != nil {
+		return x.Id
 	}
 	return 0
 }
 
-func (m *Field) GetEncodingPosition() int32 {
-	if m != nil {
-		return m.EncodingPosition
+func (x *Field) GetEncodingPosition() int32 {
+	if x != nil {
+		return x.EncodingPosition
 	}
 	return 0
 }
 
-func (m *Field) GetOptions() []*Option {
-	if m != nil {
-		return m.Options
+func (x *Field) GetOptions() []*Option {
+	if x != nil {
+		return x.Options
 	}
 	return nil
 }
 
 type FieldType struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
 	Nullable bool `protobuf:"varint,1,opt,name=nullable,proto3" json:"nullable,omitempty"`
-	// Types that are valid to be assigned to TypeInfo:
+	// Types that are assignable to TypeInfo:
 	//	*FieldType_AtomicType
 	//	*FieldType_ArrayType
 	//	*FieldType_IterableType
 	//	*FieldType_MapType
 	//	*FieldType_RowType
 	//	*FieldType_LogicalType
-	TypeInfo             isFieldType_TypeInfo `protobuf_oneof:"type_info"`
-	XXX_NoUnkeyedLiteral struct{}             `json:"-"`
-	XXX_unrecognized     []byte               `json:"-"`
-	XXX_sizecache        int32                `json:"-"`
+	TypeInfo isFieldType_TypeInfo `protobuf_oneof:"type_info"`
 }
 
-func (m *FieldType) Reset()         { *m = FieldType{} }
-func (m *FieldType) String() string { return proto.CompactTextString(m) }
-func (*FieldType) ProtoMessage()    {}
+func (x *FieldType) Reset() {
+	*x = FieldType{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_schema_proto_msgTypes[2]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *FieldType) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*FieldType) ProtoMessage() {}
+
+func (x *FieldType) ProtoReflect() protoreflect.Message {
+	mi := &file_schema_proto_msgTypes[2]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use FieldType.ProtoReflect.Descriptor instead.
 func (*FieldType) Descriptor() ([]byte, []int) {
-	return fileDescriptor_1c5fb4d8cc22d66a, []int{2}
+	return file_schema_proto_rawDescGZIP(), []int{2}
 }
 
-func (m *FieldType) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_FieldType.Unmarshal(m, b)
-}
-func (m *FieldType) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_FieldType.Marshal(b, m, deterministic)
-}
-func (m *FieldType) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_FieldType.Merge(m, src)
-}
-func (m *FieldType) XXX_Size() int {
-	return xxx_messageInfo_FieldType.Size(m)
-}
-func (m *FieldType) XXX_DiscardUnknown() {
-	xxx_messageInfo_FieldType.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_FieldType proto.InternalMessageInfo
-
-func (m *FieldType) GetNullable() bool {
-	if m != nil {
-		return m.Nullable
+func (x *FieldType) GetNullable() bool {
+	if x != nil {
+		return x.Nullable
 	}
 	return false
 }
 
+func (m *FieldType) GetTypeInfo() isFieldType_TypeInfo {
+	if m != nil {
+		return m.TypeInfo
+	}
+	return nil
+}
+
+func (x *FieldType) GetAtomicType() AtomicType {
+	if x, ok := x.GetTypeInfo().(*FieldType_AtomicType); ok {
+		return x.AtomicType
+	}
+	return AtomicType_UNSPECIFIED
+}
+
+func (x *FieldType) GetArrayType() *ArrayType {
+	if x, ok := x.GetTypeInfo().(*FieldType_ArrayType); ok {
+		return x.ArrayType
+	}
+	return nil
+}
+
+func (x *FieldType) GetIterableType() *IterableType {
+	if x, ok := x.GetTypeInfo().(*FieldType_IterableType); ok {
+		return x.IterableType
+	}
+	return nil
+}
+
+func (x *FieldType) GetMapType() *MapType {
+	if x, ok := x.GetTypeInfo().(*FieldType_MapType); ok {
+		return x.MapType
+	}
+	return nil
+}
+
+func (x *FieldType) GetRowType() *RowType {
+	if x, ok := x.GetTypeInfo().(*FieldType_RowType); ok {
+		return x.RowType
+	}
+	return nil
+}
+
+func (x *FieldType) GetLogicalType() *LogicalType {
+	if x, ok := x.GetTypeInfo().(*FieldType_LogicalType); ok {
+		return x.LogicalType
+	}
+	return nil
+}
+
 type isFieldType_TypeInfo interface {
 	isFieldType_TypeInfo()
 }
@@ -290,434 +417,490 @@
 
 func (*FieldType_LogicalType) isFieldType_TypeInfo() {}
 
-func (m *FieldType) GetTypeInfo() isFieldType_TypeInfo {
-	if m != nil {
-		return m.TypeInfo
-	}
-	return nil
-}
-
-func (m *FieldType) GetAtomicType() AtomicType {
-	if x, ok := m.GetTypeInfo().(*FieldType_AtomicType); ok {
-		return x.AtomicType
-	}
-	return AtomicType_UNSPECIFIED
-}
-
-func (m *FieldType) GetArrayType() *ArrayType {
-	if x, ok := m.GetTypeInfo().(*FieldType_ArrayType); ok {
-		return x.ArrayType
-	}
-	return nil
-}
-
-func (m *FieldType) GetIterableType() *IterableType {
-	if x, ok := m.GetTypeInfo().(*FieldType_IterableType); ok {
-		return x.IterableType
-	}
-	return nil
-}
-
-func (m *FieldType) GetMapType() *MapType {
-	if x, ok := m.GetTypeInfo().(*FieldType_MapType); ok {
-		return x.MapType
-	}
-	return nil
-}
-
-func (m *FieldType) GetRowType() *RowType {
-	if x, ok := m.GetTypeInfo().(*FieldType_RowType); ok {
-		return x.RowType
-	}
-	return nil
-}
-
-func (m *FieldType) GetLogicalType() *LogicalType {
-	if x, ok := m.GetTypeInfo().(*FieldType_LogicalType); ok {
-		return x.LogicalType
-	}
-	return nil
-}
-
-// XXX_OneofWrappers is for the internal use of the proto package.
-func (*FieldType) XXX_OneofWrappers() []interface{} {
-	return []interface{}{
-		(*FieldType_AtomicType)(nil),
-		(*FieldType_ArrayType)(nil),
-		(*FieldType_IterableType)(nil),
-		(*FieldType_MapType)(nil),
-		(*FieldType_RowType)(nil),
-		(*FieldType_LogicalType)(nil),
-	}
-}
-
 type ArrayType struct {
-	ElementType          *FieldType `protobuf:"bytes,1,opt,name=element_type,json=elementType,proto3" json:"element_type,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}   `json:"-"`
-	XXX_unrecognized     []byte     `json:"-"`
-	XXX_sizecache        int32      `json:"-"`
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	ElementType *FieldType `protobuf:"bytes,1,opt,name=element_type,json=elementType,proto3" json:"element_type,omitempty"`
 }
 
-func (m *ArrayType) Reset()         { *m = ArrayType{} }
-func (m *ArrayType) String() string { return proto.CompactTextString(m) }
-func (*ArrayType) ProtoMessage()    {}
+func (x *ArrayType) Reset() {
+	*x = ArrayType{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_schema_proto_msgTypes[3]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *ArrayType) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*ArrayType) ProtoMessage() {}
+
+func (x *ArrayType) ProtoReflect() protoreflect.Message {
+	mi := &file_schema_proto_msgTypes[3]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use ArrayType.ProtoReflect.Descriptor instead.
 func (*ArrayType) Descriptor() ([]byte, []int) {
-	return fileDescriptor_1c5fb4d8cc22d66a, []int{3}
+	return file_schema_proto_rawDescGZIP(), []int{3}
 }
 
-func (m *ArrayType) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_ArrayType.Unmarshal(m, b)
-}
-func (m *ArrayType) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_ArrayType.Marshal(b, m, deterministic)
-}
-func (m *ArrayType) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ArrayType.Merge(m, src)
-}
-func (m *ArrayType) XXX_Size() int {
-	return xxx_messageInfo_ArrayType.Size(m)
-}
-func (m *ArrayType) XXX_DiscardUnknown() {
-	xxx_messageInfo_ArrayType.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_ArrayType proto.InternalMessageInfo
-
-func (m *ArrayType) GetElementType() *FieldType {
-	if m != nil {
-		return m.ElementType
+func (x *ArrayType) GetElementType() *FieldType {
+	if x != nil {
+		return x.ElementType
 	}
 	return nil
 }
 
 type IterableType struct {
-	ElementType          *FieldType `protobuf:"bytes,1,opt,name=element_type,json=elementType,proto3" json:"element_type,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}   `json:"-"`
-	XXX_unrecognized     []byte     `json:"-"`
-	XXX_sizecache        int32      `json:"-"`
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	ElementType *FieldType `protobuf:"bytes,1,opt,name=element_type,json=elementType,proto3" json:"element_type,omitempty"`
 }
 
-func (m *IterableType) Reset()         { *m = IterableType{} }
-func (m *IterableType) String() string { return proto.CompactTextString(m) }
-func (*IterableType) ProtoMessage()    {}
+func (x *IterableType) Reset() {
+	*x = IterableType{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_schema_proto_msgTypes[4]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *IterableType) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*IterableType) ProtoMessage() {}
+
+func (x *IterableType) ProtoReflect() protoreflect.Message {
+	mi := &file_schema_proto_msgTypes[4]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use IterableType.ProtoReflect.Descriptor instead.
 func (*IterableType) Descriptor() ([]byte, []int) {
-	return fileDescriptor_1c5fb4d8cc22d66a, []int{4}
+	return file_schema_proto_rawDescGZIP(), []int{4}
 }
 
-func (m *IterableType) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_IterableType.Unmarshal(m, b)
-}
-func (m *IterableType) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_IterableType.Marshal(b, m, deterministic)
-}
-func (m *IterableType) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_IterableType.Merge(m, src)
-}
-func (m *IterableType) XXX_Size() int {
-	return xxx_messageInfo_IterableType.Size(m)
-}
-func (m *IterableType) XXX_DiscardUnknown() {
-	xxx_messageInfo_IterableType.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_IterableType proto.InternalMessageInfo
-
-func (m *IterableType) GetElementType() *FieldType {
-	if m != nil {
-		return m.ElementType
+func (x *IterableType) GetElementType() *FieldType {
+	if x != nil {
+		return x.ElementType
 	}
 	return nil
 }
 
 type MapType struct {
-	KeyType              *FieldType `protobuf:"bytes,1,opt,name=key_type,json=keyType,proto3" json:"key_type,omitempty"`
-	ValueType            *FieldType `protobuf:"bytes,2,opt,name=value_type,json=valueType,proto3" json:"value_type,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}   `json:"-"`
-	XXX_unrecognized     []byte     `json:"-"`
-	XXX_sizecache        int32      `json:"-"`
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	KeyType   *FieldType `protobuf:"bytes,1,opt,name=key_type,json=keyType,proto3" json:"key_type,omitempty"`
+	ValueType *FieldType `protobuf:"bytes,2,opt,name=value_type,json=valueType,proto3" json:"value_type,omitempty"`
 }
 
-func (m *MapType) Reset()         { *m = MapType{} }
-func (m *MapType) String() string { return proto.CompactTextString(m) }
-func (*MapType) ProtoMessage()    {}
+func (x *MapType) Reset() {
+	*x = MapType{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_schema_proto_msgTypes[5]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *MapType) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*MapType) ProtoMessage() {}
+
+func (x *MapType) ProtoReflect() protoreflect.Message {
+	mi := &file_schema_proto_msgTypes[5]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use MapType.ProtoReflect.Descriptor instead.
 func (*MapType) Descriptor() ([]byte, []int) {
-	return fileDescriptor_1c5fb4d8cc22d66a, []int{5}
+	return file_schema_proto_rawDescGZIP(), []int{5}
 }
 
-func (m *MapType) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_MapType.Unmarshal(m, b)
-}
-func (m *MapType) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_MapType.Marshal(b, m, deterministic)
-}
-func (m *MapType) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_MapType.Merge(m, src)
-}
-func (m *MapType) XXX_Size() int {
-	return xxx_messageInfo_MapType.Size(m)
-}
-func (m *MapType) XXX_DiscardUnknown() {
-	xxx_messageInfo_MapType.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_MapType proto.InternalMessageInfo
-
-func (m *MapType) GetKeyType() *FieldType {
-	if m != nil {
-		return m.KeyType
+func (x *MapType) GetKeyType() *FieldType {
+	if x != nil {
+		return x.KeyType
 	}
 	return nil
 }
 
-func (m *MapType) GetValueType() *FieldType {
-	if m != nil {
-		return m.ValueType
+func (x *MapType) GetValueType() *FieldType {
+	if x != nil {
+		return x.ValueType
 	}
 	return nil
 }
 
 type RowType struct {
-	Schema               *Schema  `protobuf:"bytes,1,opt,name=schema,proto3" json:"schema,omitempty"`
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	Schema *Schema `protobuf:"bytes,1,opt,name=schema,proto3" json:"schema,omitempty"`
 }
 
-func (m *RowType) Reset()         { *m = RowType{} }
-func (m *RowType) String() string { return proto.CompactTextString(m) }
-func (*RowType) ProtoMessage()    {}
+func (x *RowType) Reset() {
+	*x = RowType{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_schema_proto_msgTypes[6]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *RowType) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*RowType) ProtoMessage() {}
+
+func (x *RowType) ProtoReflect() protoreflect.Message {
+	mi := &file_schema_proto_msgTypes[6]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use RowType.ProtoReflect.Descriptor instead.
 func (*RowType) Descriptor() ([]byte, []int) {
-	return fileDescriptor_1c5fb4d8cc22d66a, []int{6}
+	return file_schema_proto_rawDescGZIP(), []int{6}
 }
 
-func (m *RowType) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_RowType.Unmarshal(m, b)
-}
-func (m *RowType) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_RowType.Marshal(b, m, deterministic)
-}
-func (m *RowType) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_RowType.Merge(m, src)
-}
-func (m *RowType) XXX_Size() int {
-	return xxx_messageInfo_RowType.Size(m)
-}
-func (m *RowType) XXX_DiscardUnknown() {
-	xxx_messageInfo_RowType.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_RowType proto.InternalMessageInfo
-
-func (m *RowType) GetSchema() *Schema {
-	if m != nil {
-		return m.Schema
+func (x *RowType) GetSchema() *Schema {
+	if x != nil {
+		return x.Schema
 	}
 	return nil
 }
 
 type LogicalType struct {
-	Urn                  string      `protobuf:"bytes,1,opt,name=urn,proto3" json:"urn,omitempty"`
-	Payload              []byte      `protobuf:"bytes,2,opt,name=payload,proto3" json:"payload,omitempty"`
-	Representation       *FieldType  `protobuf:"bytes,3,opt,name=representation,proto3" json:"representation,omitempty"`
-	ArgumentType         *FieldType  `protobuf:"bytes,4,opt,name=argument_type,json=argumentType,proto3" json:"argument_type,omitempty"`
-	Argument             *FieldValue `protobuf:"bytes,5,opt,name=argument,proto3" json:"argument,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}    `json:"-"`
-	XXX_unrecognized     []byte      `json:"-"`
-	XXX_sizecache        int32       `json:"-"`
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	Urn            string      `protobuf:"bytes,1,opt,name=urn,proto3" json:"urn,omitempty"`
+	Payload        []byte      `protobuf:"bytes,2,opt,name=payload,proto3" json:"payload,omitempty"`
+	Representation *FieldType  `protobuf:"bytes,3,opt,name=representation,proto3" json:"representation,omitempty"`
+	ArgumentType   *FieldType  `protobuf:"bytes,4,opt,name=argument_type,json=argumentType,proto3" json:"argument_type,omitempty"`
+	Argument       *FieldValue `protobuf:"bytes,5,opt,name=argument,proto3" json:"argument,omitempty"`
 }
 
-func (m *LogicalType) Reset()         { *m = LogicalType{} }
-func (m *LogicalType) String() string { return proto.CompactTextString(m) }
-func (*LogicalType) ProtoMessage()    {}
+func (x *LogicalType) Reset() {
+	*x = LogicalType{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_schema_proto_msgTypes[7]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *LogicalType) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*LogicalType) ProtoMessage() {}
+
+func (x *LogicalType) ProtoReflect() protoreflect.Message {
+	mi := &file_schema_proto_msgTypes[7]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use LogicalType.ProtoReflect.Descriptor instead.
 func (*LogicalType) Descriptor() ([]byte, []int) {
-	return fileDescriptor_1c5fb4d8cc22d66a, []int{7}
+	return file_schema_proto_rawDescGZIP(), []int{7}
 }
 
-func (m *LogicalType) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_LogicalType.Unmarshal(m, b)
-}
-func (m *LogicalType) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_LogicalType.Marshal(b, m, deterministic)
-}
-func (m *LogicalType) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_LogicalType.Merge(m, src)
-}
-func (m *LogicalType) XXX_Size() int {
-	return xxx_messageInfo_LogicalType.Size(m)
-}
-func (m *LogicalType) XXX_DiscardUnknown() {
-	xxx_messageInfo_LogicalType.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_LogicalType proto.InternalMessageInfo
-
-func (m *LogicalType) GetUrn() string {
-	if m != nil {
-		return m.Urn
+func (x *LogicalType) GetUrn() string {
+	if x != nil {
+		return x.Urn
 	}
 	return ""
 }
 
-func (m *LogicalType) GetPayload() []byte {
-	if m != nil {
-		return m.Payload
+func (x *LogicalType) GetPayload() []byte {
+	if x != nil {
+		return x.Payload
 	}
 	return nil
 }
 
-func (m *LogicalType) GetRepresentation() *FieldType {
-	if m != nil {
-		return m.Representation
+func (x *LogicalType) GetRepresentation() *FieldType {
+	if x != nil {
+		return x.Representation
 	}
 	return nil
 }
 
-func (m *LogicalType) GetArgumentType() *FieldType {
-	if m != nil {
-		return m.ArgumentType
+func (x *LogicalType) GetArgumentType() *FieldType {
+	if x != nil {
+		return x.ArgumentType
 	}
 	return nil
 }
 
-func (m *LogicalType) GetArgument() *FieldValue {
-	if m != nil {
-		return m.Argument
+func (x *LogicalType) GetArgument() *FieldValue {
+	if x != nil {
+		return x.Argument
 	}
 	return nil
 }
 
 type Option struct {
-	Name                 string      `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"`
-	Type                 *FieldType  `protobuf:"bytes,2,opt,name=type,proto3" json:"type,omitempty"`
-	Value                *FieldValue `protobuf:"bytes,3,opt,name=value,proto3" json:"value,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}    `json:"-"`
-	XXX_unrecognized     []byte      `json:"-"`
-	XXX_sizecache        int32       `json:"-"`
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	// REQUIRED. Identifier for the option.
+	Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"`
+	// OPTIONAL. Type specifer for the structure of value.
+	// If not present, assumes no additional configuration is needed
+	// for this option and value is ignored.
+	Type  *FieldType  `protobuf:"bytes,2,opt,name=type,proto3" json:"type,omitempty"`
+	Value *FieldValue `protobuf:"bytes,3,opt,name=value,proto3" json:"value,omitempty"`
 }
 
-func (m *Option) Reset()         { *m = Option{} }
-func (m *Option) String() string { return proto.CompactTextString(m) }
-func (*Option) ProtoMessage()    {}
+func (x *Option) Reset() {
+	*x = Option{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_schema_proto_msgTypes[8]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *Option) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*Option) ProtoMessage() {}
+
+func (x *Option) ProtoReflect() protoreflect.Message {
+	mi := &file_schema_proto_msgTypes[8]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use Option.ProtoReflect.Descriptor instead.
 func (*Option) Descriptor() ([]byte, []int) {
-	return fileDescriptor_1c5fb4d8cc22d66a, []int{8}
+	return file_schema_proto_rawDescGZIP(), []int{8}
 }
 
-func (m *Option) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_Option.Unmarshal(m, b)
-}
-func (m *Option) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_Option.Marshal(b, m, deterministic)
-}
-func (m *Option) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Option.Merge(m, src)
-}
-func (m *Option) XXX_Size() int {
-	return xxx_messageInfo_Option.Size(m)
-}
-func (m *Option) XXX_DiscardUnknown() {
-	xxx_messageInfo_Option.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_Option proto.InternalMessageInfo
-
-func (m *Option) GetName() string {
-	if m != nil {
-		return m.Name
+func (x *Option) GetName() string {
+	if x != nil {
+		return x.Name
 	}
 	return ""
 }
 
-func (m *Option) GetType() *FieldType {
-	if m != nil {
-		return m.Type
+func (x *Option) GetType() *FieldType {
+	if x != nil {
+		return x.Type
 	}
 	return nil
 }
 
-func (m *Option) GetValue() *FieldValue {
-	if m != nil {
-		return m.Value
+func (x *Option) GetValue() *FieldValue {
+	if x != nil {
+		return x.Value
 	}
 	return nil
 }
 
 type Row struct {
-	Values               []*FieldValue `protobuf:"bytes,1,rep,name=values,proto3" json:"values,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}      `json:"-"`
-	XXX_unrecognized     []byte        `json:"-"`
-	XXX_sizecache        int32         `json:"-"`
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	Values []*FieldValue `protobuf:"bytes,1,rep,name=values,proto3" json:"values,omitempty"`
 }
 
-func (m *Row) Reset()         { *m = Row{} }
-func (m *Row) String() string { return proto.CompactTextString(m) }
-func (*Row) ProtoMessage()    {}
+func (x *Row) Reset() {
+	*x = Row{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_schema_proto_msgTypes[9]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *Row) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*Row) ProtoMessage() {}
+
+func (x *Row) ProtoReflect() protoreflect.Message {
+	mi := &file_schema_proto_msgTypes[9]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use Row.ProtoReflect.Descriptor instead.
 func (*Row) Descriptor() ([]byte, []int) {
-	return fileDescriptor_1c5fb4d8cc22d66a, []int{9}
+	return file_schema_proto_rawDescGZIP(), []int{9}
 }
 
-func (m *Row) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_Row.Unmarshal(m, b)
-}
-func (m *Row) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_Row.Marshal(b, m, deterministic)
-}
-func (m *Row) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_Row.Merge(m, src)
-}
-func (m *Row) XXX_Size() int {
-	return xxx_messageInfo_Row.Size(m)
-}
-func (m *Row) XXX_DiscardUnknown() {
-	xxx_messageInfo_Row.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_Row proto.InternalMessageInfo
-
-func (m *Row) GetValues() []*FieldValue {
-	if m != nil {
-		return m.Values
+func (x *Row) GetValues() []*FieldValue {
+	if x != nil {
+		return x.Values
 	}
 	return nil
 }
 
 type FieldValue struct {
-	// Types that are valid to be assigned to FieldValue:
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	// Types that are assignable to FieldValue:
 	//	*FieldValue_AtomicValue
 	//	*FieldValue_ArrayValue
 	//	*FieldValue_IterableValue
 	//	*FieldValue_MapValue
 	//	*FieldValue_RowValue
 	//	*FieldValue_LogicalTypeValue
-	FieldValue           isFieldValue_FieldValue `protobuf_oneof:"field_value"`
-	XXX_NoUnkeyedLiteral struct{}                `json:"-"`
-	XXX_unrecognized     []byte                  `json:"-"`
-	XXX_sizecache        int32                   `json:"-"`
+	FieldValue isFieldValue_FieldValue `protobuf_oneof:"field_value"`
 }
 
-func (m *FieldValue) Reset()         { *m = FieldValue{} }
-func (m *FieldValue) String() string { return proto.CompactTextString(m) }
-func (*FieldValue) ProtoMessage()    {}
+func (x *FieldValue) Reset() {
+	*x = FieldValue{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_schema_proto_msgTypes[10]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *FieldValue) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*FieldValue) ProtoMessage() {}
+
+func (x *FieldValue) ProtoReflect() protoreflect.Message {
+	mi := &file_schema_proto_msgTypes[10]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use FieldValue.ProtoReflect.Descriptor instead.
 func (*FieldValue) Descriptor() ([]byte, []int) {
-	return fileDescriptor_1c5fb4d8cc22d66a, []int{10}
+	return file_schema_proto_rawDescGZIP(), []int{10}
 }
 
-func (m *FieldValue) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_FieldValue.Unmarshal(m, b)
-}
-func (m *FieldValue) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_FieldValue.Marshal(b, m, deterministic)
-}
-func (m *FieldValue) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_FieldValue.Merge(m, src)
-}
-func (m *FieldValue) XXX_Size() int {
-	return xxx_messageInfo_FieldValue.Size(m)
-}
-func (m *FieldValue) XXX_DiscardUnknown() {
-	xxx_messageInfo_FieldValue.DiscardUnknown(m)
+func (m *FieldValue) GetFieldValue() isFieldValue_FieldValue {
+	if m != nil {
+		return m.FieldValue
+	}
+	return nil
 }
 
-var xxx_messageInfo_FieldValue proto.InternalMessageInfo
+func (x *FieldValue) GetAtomicValue() *AtomicTypeValue {
+	if x, ok := x.GetFieldValue().(*FieldValue_AtomicValue); ok {
+		return x.AtomicValue
+	}
+	return nil
+}
+
+func (x *FieldValue) GetArrayValue() *ArrayTypeValue {
+	if x, ok := x.GetFieldValue().(*FieldValue_ArrayValue); ok {
+		return x.ArrayValue
+	}
+	return nil
+}
+
+func (x *FieldValue) GetIterableValue() *IterableTypeValue {
+	if x, ok := x.GetFieldValue().(*FieldValue_IterableValue); ok {
+		return x.IterableValue
+	}
+	return nil
+}
+
+func (x *FieldValue) GetMapValue() *MapTypeValue {
+	if x, ok := x.GetFieldValue().(*FieldValue_MapValue); ok {
+		return x.MapValue
+	}
+	return nil
+}
+
+func (x *FieldValue) GetRowValue() *Row {
+	if x, ok := x.GetFieldValue().(*FieldValue_RowValue); ok {
+		return x.RowValue
+	}
+	return nil
+}
+
+func (x *FieldValue) GetLogicalTypeValue() *LogicalTypeValue {
+	if x, ok := x.GetFieldValue().(*FieldValue_LogicalTypeValue); ok {
+		return x.LogicalTypeValue
+	}
+	return nil
+}
 
 type isFieldValue_FieldValue interface {
 	isFieldValue_FieldValue()
@@ -759,69 +942,12 @@
 
 func (*FieldValue_LogicalTypeValue) isFieldValue_FieldValue() {}
 
-func (m *FieldValue) GetFieldValue() isFieldValue_FieldValue {
-	if m != nil {
-		return m.FieldValue
-	}
-	return nil
-}
-
-func (m *FieldValue) GetAtomicValue() *AtomicTypeValue {
-	if x, ok := m.GetFieldValue().(*FieldValue_AtomicValue); ok {
-		return x.AtomicValue
-	}
-	return nil
-}
-
-func (m *FieldValue) GetArrayValue() *ArrayTypeValue {
-	if x, ok := m.GetFieldValue().(*FieldValue_ArrayValue); ok {
-		return x.ArrayValue
-	}
-	return nil
-}
-
-func (m *FieldValue) GetIterableValue() *IterableTypeValue {
-	if x, ok := m.GetFieldValue().(*FieldValue_IterableValue); ok {
-		return x.IterableValue
-	}
-	return nil
-}
-
-func (m *FieldValue) GetMapValue() *MapTypeValue {
-	if x, ok := m.GetFieldValue().(*FieldValue_MapValue); ok {
-		return x.MapValue
-	}
-	return nil
-}
-
-func (m *FieldValue) GetRowValue() *Row {
-	if x, ok := m.GetFieldValue().(*FieldValue_RowValue); ok {
-		return x.RowValue
-	}
-	return nil
-}
-
-func (m *FieldValue) GetLogicalTypeValue() *LogicalTypeValue {
-	if x, ok := m.GetFieldValue().(*FieldValue_LogicalTypeValue); ok {
-		return x.LogicalTypeValue
-	}
-	return nil
-}
-
-// XXX_OneofWrappers is for the internal use of the proto package.
-func (*FieldValue) XXX_OneofWrappers() []interface{} {
-	return []interface{}{
-		(*FieldValue_AtomicValue)(nil),
-		(*FieldValue_ArrayValue)(nil),
-		(*FieldValue_IterableValue)(nil),
-		(*FieldValue_MapValue)(nil),
-		(*FieldValue_RowValue)(nil),
-		(*FieldValue_LogicalTypeValue)(nil),
-	}
-}
-
 type AtomicTypeValue struct {
-	// Types that are valid to be assigned to Value:
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	// Types that are assignable to Value:
 	//	*AtomicTypeValue_Byte
 	//	*AtomicTypeValue_Int16
 	//	*AtomicTypeValue_Int32
@@ -831,36 +957,110 @@
 	//	*AtomicTypeValue_String_
 	//	*AtomicTypeValue_Boolean
 	//	*AtomicTypeValue_Bytes
-	Value                isAtomicTypeValue_Value `protobuf_oneof:"value"`
-	XXX_NoUnkeyedLiteral struct{}                `json:"-"`
-	XXX_unrecognized     []byte                  `json:"-"`
-	XXX_sizecache        int32                   `json:"-"`
+	Value isAtomicTypeValue_Value `protobuf_oneof:"value"`
 }
 
-func (m *AtomicTypeValue) Reset()         { *m = AtomicTypeValue{} }
-func (m *AtomicTypeValue) String() string { return proto.CompactTextString(m) }
-func (*AtomicTypeValue) ProtoMessage()    {}
+func (x *AtomicTypeValue) Reset() {
+	*x = AtomicTypeValue{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_schema_proto_msgTypes[11]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *AtomicTypeValue) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*AtomicTypeValue) ProtoMessage() {}
+
+func (x *AtomicTypeValue) ProtoReflect() protoreflect.Message {
+	mi := &file_schema_proto_msgTypes[11]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use AtomicTypeValue.ProtoReflect.Descriptor instead.
 func (*AtomicTypeValue) Descriptor() ([]byte, []int) {
-	return fileDescriptor_1c5fb4d8cc22d66a, []int{11}
+	return file_schema_proto_rawDescGZIP(), []int{11}
 }
 
-func (m *AtomicTypeValue) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_AtomicTypeValue.Unmarshal(m, b)
-}
-func (m *AtomicTypeValue) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_AtomicTypeValue.Marshal(b, m, deterministic)
-}
-func (m *AtomicTypeValue) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_AtomicTypeValue.Merge(m, src)
-}
-func (m *AtomicTypeValue) XXX_Size() int {
-	return xxx_messageInfo_AtomicTypeValue.Size(m)
-}
-func (m *AtomicTypeValue) XXX_DiscardUnknown() {
-	xxx_messageInfo_AtomicTypeValue.DiscardUnknown(m)
+func (m *AtomicTypeValue) GetValue() isAtomicTypeValue_Value {
+	if m != nil {
+		return m.Value
+	}
+	return nil
 }
 
-var xxx_messageInfo_AtomicTypeValue proto.InternalMessageInfo
+func (x *AtomicTypeValue) GetByte() int32 {
+	if x, ok := x.GetValue().(*AtomicTypeValue_Byte); ok {
+		return x.Byte
+	}
+	return 0
+}
+
+func (x *AtomicTypeValue) GetInt16() int32 {
+	if x, ok := x.GetValue().(*AtomicTypeValue_Int16); ok {
+		return x.Int16
+	}
+	return 0
+}
+
+func (x *AtomicTypeValue) GetInt32() int32 {
+	if x, ok := x.GetValue().(*AtomicTypeValue_Int32); ok {
+		return x.Int32
+	}
+	return 0
+}
+
+func (x *AtomicTypeValue) GetInt64() int64 {
+	if x, ok := x.GetValue().(*AtomicTypeValue_Int64); ok {
+		return x.Int64
+	}
+	return 0
+}
+
+func (x *AtomicTypeValue) GetFloat() float32 {
+	if x, ok := x.GetValue().(*AtomicTypeValue_Float); ok {
+		return x.Float
+	}
+	return 0
+}
+
+func (x *AtomicTypeValue) GetDouble() float64 {
+	if x, ok := x.GetValue().(*AtomicTypeValue_Double); ok {
+		return x.Double
+	}
+	return 0
+}
+
+func (x *AtomicTypeValue) GetString_() string {
+	if x, ok := x.GetValue().(*AtomicTypeValue_String_); ok {
+		return x.String_
+	}
+	return ""
+}
+
+func (x *AtomicTypeValue) GetBoolean() bool {
+	if x, ok := x.GetValue().(*AtomicTypeValue_Boolean); ok {
+		return x.Boolean
+	}
+	return false
+}
+
+func (x *AtomicTypeValue) GetBytes() []byte {
+	if x, ok := x.GetValue().(*AtomicTypeValue_Bytes); ok {
+		return x.Bytes
+	}
+	return nil
+}
 
 type isAtomicTypeValue_Value interface {
 	isAtomicTypeValue_Value()
@@ -920,79 +1120,780 @@
 
 func (*AtomicTypeValue_Bytes) isAtomicTypeValue_Value() {}
 
-func (m *AtomicTypeValue) GetValue() isAtomicTypeValue_Value {
-	if m != nil {
-		return m.Value
+type ArrayTypeValue struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	Element []*FieldValue `protobuf:"bytes,1,rep,name=element,proto3" json:"element,omitempty"`
+}
+
+func (x *ArrayTypeValue) Reset() {
+	*x = ArrayTypeValue{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_schema_proto_msgTypes[12]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *ArrayTypeValue) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*ArrayTypeValue) ProtoMessage() {}
+
+func (x *ArrayTypeValue) ProtoReflect() protoreflect.Message {
+	mi := &file_schema_proto_msgTypes[12]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use ArrayTypeValue.ProtoReflect.Descriptor instead.
+func (*ArrayTypeValue) Descriptor() ([]byte, []int) {
+	return file_schema_proto_rawDescGZIP(), []int{12}
+}
+
+func (x *ArrayTypeValue) GetElement() []*FieldValue {
+	if x != nil {
+		return x.Element
 	}
 	return nil
 }
 
-func (m *AtomicTypeValue) GetByte() int32 {
-	if x, ok := m.GetValue().(*AtomicTypeValue_Byte); ok {
-		return x.Byte
-	}
-	return 0
+type IterableTypeValue struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	Element []*FieldValue `protobuf:"bytes,1,rep,name=element,proto3" json:"element,omitempty"`
 }
 
-func (m *AtomicTypeValue) GetInt16() int32 {
-	if x, ok := m.GetValue().(*AtomicTypeValue_Int16); ok {
-		return x.Int16
+func (x *IterableTypeValue) Reset() {
+	*x = IterableTypeValue{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_schema_proto_msgTypes[13]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
 	}
-	return 0
 }
 
-func (m *AtomicTypeValue) GetInt32() int32 {
-	if x, ok := m.GetValue().(*AtomicTypeValue_Int32); ok {
-		return x.Int32
-	}
-	return 0
+func (x *IterableTypeValue) String() string {
+	return protoimpl.X.MessageStringOf(x)
 }
 
-func (m *AtomicTypeValue) GetInt64() int64 {
-	if x, ok := m.GetValue().(*AtomicTypeValue_Int64); ok {
-		return x.Int64
+func (*IterableTypeValue) ProtoMessage() {}
+
+func (x *IterableTypeValue) ProtoReflect() protoreflect.Message {
+	mi := &file_schema_proto_msgTypes[13]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
 	}
-	return 0
+	return mi.MessageOf(x)
 }
 
-func (m *AtomicTypeValue) GetFloat() float32 {
-	if x, ok := m.GetValue().(*AtomicTypeValue_Float); ok {
-		return x.Float
-	}
-	return 0
+// Deprecated: Use IterableTypeValue.ProtoReflect.Descriptor instead.
+func (*IterableTypeValue) Descriptor() ([]byte, []int) {
+	return file_schema_proto_rawDescGZIP(), []int{13}
 }
 
-func (m *AtomicTypeValue) GetDouble() float64 {
-	if x, ok := m.GetValue().(*AtomicTypeValue_Double); ok {
-		return x.Double
-	}
-	return 0
-}
-
-func (m *AtomicTypeValue) GetString_() string {
-	if x, ok := m.GetValue().(*AtomicTypeValue_String_); ok {
-		return x.String_
-	}
-	return ""
-}
-
-func (m *AtomicTypeValue) GetBoolean() bool {
-	if x, ok := m.GetValue().(*AtomicTypeValue_Boolean); ok {
-		return x.Boolean
-	}
-	return false
-}
-
-func (m *AtomicTypeValue) GetBytes() []byte {
-	if x, ok := m.GetValue().(*AtomicTypeValue_Bytes); ok {
-		return x.Bytes
+func (x *IterableTypeValue) GetElement() []*FieldValue {
+	if x != nil {
+		return x.Element
 	}
 	return nil
 }
 
-// XXX_OneofWrappers is for the internal use of the proto package.
-func (*AtomicTypeValue) XXX_OneofWrappers() []interface{} {
-	return []interface{}{
+type MapTypeValue struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	Entries []*MapTypeEntry `protobuf:"bytes,1,rep,name=entries,proto3" json:"entries,omitempty"`
+}
+
+func (x *MapTypeValue) Reset() {
+	*x = MapTypeValue{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_schema_proto_msgTypes[14]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *MapTypeValue) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*MapTypeValue) ProtoMessage() {}
+
+func (x *MapTypeValue) ProtoReflect() protoreflect.Message {
+	mi := &file_schema_proto_msgTypes[14]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use MapTypeValue.ProtoReflect.Descriptor instead.
+func (*MapTypeValue) Descriptor() ([]byte, []int) {
+	return file_schema_proto_rawDescGZIP(), []int{14}
+}
+
+func (x *MapTypeValue) GetEntries() []*MapTypeEntry {
+	if x != nil {
+		return x.Entries
+	}
+	return nil
+}
+
+type MapTypeEntry struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	Key   *FieldValue `protobuf:"bytes,1,opt,name=key,proto3" json:"key,omitempty"`
+	Value *FieldValue `protobuf:"bytes,2,opt,name=value,proto3" json:"value,omitempty"`
+}
+
+func (x *MapTypeEntry) Reset() {
+	*x = MapTypeEntry{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_schema_proto_msgTypes[15]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *MapTypeEntry) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*MapTypeEntry) ProtoMessage() {}
+
+func (x *MapTypeEntry) ProtoReflect() protoreflect.Message {
+	mi := &file_schema_proto_msgTypes[15]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use MapTypeEntry.ProtoReflect.Descriptor instead.
+func (*MapTypeEntry) Descriptor() ([]byte, []int) {
+	return file_schema_proto_rawDescGZIP(), []int{15}
+}
+
+func (x *MapTypeEntry) GetKey() *FieldValue {
+	if x != nil {
+		return x.Key
+	}
+	return nil
+}
+
+func (x *MapTypeEntry) GetValue() *FieldValue {
+	if x != nil {
+		return x.Value
+	}
+	return nil
+}
+
+type LogicalTypeValue struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	Value *FieldValue `protobuf:"bytes,1,opt,name=value,proto3" json:"value,omitempty"`
+}
+
+func (x *LogicalTypeValue) Reset() {
+	*x = LogicalTypeValue{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_schema_proto_msgTypes[16]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *LogicalTypeValue) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*LogicalTypeValue) ProtoMessage() {}
+
+func (x *LogicalTypeValue) ProtoReflect() protoreflect.Message {
+	mi := &file_schema_proto_msgTypes[16]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use LogicalTypeValue.ProtoReflect.Descriptor instead.
+func (*LogicalTypeValue) Descriptor() ([]byte, []int) {
+	return file_schema_proto_rawDescGZIP(), []int{16}
+}
+
+func (x *LogicalTypeValue) GetValue() *FieldValue {
+	if x != nil {
+		return x.Value
+	}
+	return nil
+}
+
+var File_schema_proto protoreflect.FileDescriptor
+
+var file_schema_proto_rawDesc = []byte{
+	0x0a, 0x0c, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x21,
+	0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e,
+	0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76,
+	0x31, 0x22, 0x9f, 0x01, 0x0a, 0x06, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x12, 0x40, 0x0a, 0x06,
+	0x66, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x28, 0x2e, 0x6f,
+	0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d,
+	0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31,
+	0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x52, 0x06, 0x66, 0x69, 0x65, 0x6c, 0x64, 0x73, 0x12, 0x0e,
+	0x0a, 0x02, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, 0x12, 0x43,
+	0x0a, 0x07, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32,
+	0x29, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61,
+	0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65,
+	0x2e, 0x76, 0x31, 0x2e, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x07, 0x6f, 0x70, 0x74, 0x69,
+	0x6f, 0x6e, 0x73, 0x22, 0x81, 0x02, 0x0a, 0x05, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x12, 0x12, 0x0a,
+	0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d,
+	0x65, 0x12, 0x20, 0x0a, 0x0b, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74, 0x69, 0x6f, 0x6e,
+	0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0b, 0x64, 0x65, 0x73, 0x63, 0x72, 0x69, 0x70, 0x74,
+	0x69, 0x6f, 0x6e, 0x12, 0x40, 0x0a, 0x04, 0x74, 0x79, 0x70, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28,
+	0x0b, 0x32, 0x2c, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62,
+	0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69,
+	0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x54, 0x79, 0x70, 0x65, 0x52,
+	0x04, 0x74, 0x79, 0x70, 0x65, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x04, 0x20, 0x01, 0x28,
+	0x05, 0x52, 0x02, 0x69, 0x64, 0x12, 0x2b, 0x0a, 0x11, 0x65, 0x6e, 0x63, 0x6f, 0x64, 0x69, 0x6e,
+	0x67, 0x5f, 0x70, 0x6f, 0x73, 0x69, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x05, 0x20, 0x01, 0x28, 0x05,
+	0x52, 0x10, 0x65, 0x6e, 0x63, 0x6f, 0x64, 0x69, 0x6e, 0x67, 0x50, 0x6f, 0x73, 0x69, 0x74, 0x69,
+	0x6f, 0x6e, 0x12, 0x43, 0x0a, 0x07, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x06, 0x20,
+	0x03, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65,
+	0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65,
+	0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x07,
+	0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x22, 0x94, 0x04, 0x0a, 0x09, 0x46, 0x69, 0x65, 0x6c,
+	0x64, 0x54, 0x79, 0x70, 0x65, 0x12, 0x1a, 0x0a, 0x08, 0x6e, 0x75, 0x6c, 0x6c, 0x61, 0x62, 0x6c,
+	0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x08, 0x52, 0x08, 0x6e, 0x75, 0x6c, 0x6c, 0x61, 0x62, 0x6c,
+	0x65, 0x12, 0x50, 0x0a, 0x0b, 0x61, 0x74, 0x6f, 0x6d, 0x69, 0x63, 0x5f, 0x74, 0x79, 0x70, 0x65,
+	0x18, 0x02, 0x20, 0x01, 0x28, 0x0e, 0x32, 0x2d, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61,
+	0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70,
+	0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x74, 0x6f, 0x6d, 0x69,
+	0x63, 0x54, 0x79, 0x70, 0x65, 0x48, 0x00, 0x52, 0x0a, 0x61, 0x74, 0x6f, 0x6d, 0x69, 0x63, 0x54,
+	0x79, 0x70, 0x65, 0x12, 0x4d, 0x0a, 0x0a, 0x61, 0x72, 0x72, 0x61, 0x79, 0x5f, 0x74, 0x79, 0x70,
+	0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2c, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70,
+	0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e,
+	0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x72, 0x72, 0x61,
+	0x79, 0x54, 0x79, 0x70, 0x65, 0x48, 0x00, 0x52, 0x09, 0x61, 0x72, 0x72, 0x61, 0x79, 0x54, 0x79,
+	0x70, 0x65, 0x12, 0x56, 0x0a, 0x0d, 0x69, 0x74, 0x65, 0x72, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x74,
+	0x79, 0x70, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2f, 0x2e, 0x6f, 0x72, 0x67, 0x2e,
+	0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65,
+	0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x49, 0x74,
+	0x65, 0x72, 0x61, 0x62, 0x6c, 0x65, 0x54, 0x79, 0x70, 0x65, 0x48, 0x00, 0x52, 0x0c, 0x69, 0x74,
+	0x65, 0x72, 0x61, 0x62, 0x6c, 0x65, 0x54, 0x79, 0x70, 0x65, 0x12, 0x47, 0x0a, 0x08, 0x6d, 0x61,
+	0x70, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2a, 0x2e, 0x6f,
+	0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d,
+	0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31,
+	0x2e, 0x4d, 0x61, 0x70, 0x54, 0x79, 0x70, 0x65, 0x48, 0x00, 0x52, 0x07, 0x6d, 0x61, 0x70, 0x54,
+	0x79, 0x70, 0x65, 0x12, 0x47, 0x0a, 0x08, 0x72, 0x6f, 0x77, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18,
+	0x06, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2a, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63,
+	0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69,
+	0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x6f, 0x77, 0x54, 0x79, 0x70,
+	0x65, 0x48, 0x00, 0x52, 0x07, 0x72, 0x6f, 0x77, 0x54, 0x79, 0x70, 0x65, 0x12, 0x53, 0x0a, 0x0c,
+	0x6c, 0x6f, 0x67, 0x69, 0x63, 0x61, 0x6c, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x07, 0x20, 0x01,
+	0x28, 0x0b, 0x32, 0x2e, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e,
+	0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c,
+	0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x4c, 0x6f, 0x67, 0x69, 0x63, 0x61, 0x6c, 0x54, 0x79,
+	0x70, 0x65, 0x48, 0x00, 0x52, 0x0b, 0x6c, 0x6f, 0x67, 0x69, 0x63, 0x61, 0x6c, 0x54, 0x79, 0x70,
+	0x65, 0x42, 0x0b, 0x0a, 0x09, 0x74, 0x79, 0x70, 0x65, 0x5f, 0x69, 0x6e, 0x66, 0x6f, 0x22, 0x5c,
+	0x0a, 0x09, 0x41, 0x72, 0x72, 0x61, 0x79, 0x54, 0x79, 0x70, 0x65, 0x12, 0x4f, 0x0a, 0x0c, 0x65,
+	0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28,
+	0x0b, 0x32, 0x2c, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62,
+	0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69,
+	0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x54, 0x79, 0x70, 0x65, 0x52,
+	0x0b, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x54, 0x79, 0x70, 0x65, 0x22, 0x5f, 0x0a, 0x0c,
+	0x49, 0x74, 0x65, 0x72, 0x61, 0x62, 0x6c, 0x65, 0x54, 0x79, 0x70, 0x65, 0x12, 0x4f, 0x0a, 0x0c,
+	0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x01, 0x20, 0x01,
+	0x28, 0x0b, 0x32, 0x2c, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e,
+	0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c,
+	0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x54, 0x79, 0x70, 0x65,
+	0x52, 0x0b, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x54, 0x79, 0x70, 0x65, 0x22, 0x9f, 0x01,
+	0x0a, 0x07, 0x4d, 0x61, 0x70, 0x54, 0x79, 0x70, 0x65, 0x12, 0x47, 0x0a, 0x08, 0x6b, 0x65, 0x79,
+	0x5f, 0x74, 0x79, 0x70, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2c, 0x2e, 0x6f, 0x72,
+	0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f,
+	0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e,
+	0x46, 0x69, 0x65, 0x6c, 0x64, 0x54, 0x79, 0x70, 0x65, 0x52, 0x07, 0x6b, 0x65, 0x79, 0x54, 0x79,
+	0x70, 0x65, 0x12, 0x4b, 0x0a, 0x0a, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x5f, 0x74, 0x79, 0x70, 0x65,
+	0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2c, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61,
+	0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70,
+	0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64,
+	0x54, 0x79, 0x70, 0x65, 0x52, 0x09, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x54, 0x79, 0x70, 0x65, 0x22,
+	0x4c, 0x0a, 0x07, 0x52, 0x6f, 0x77, 0x54, 0x79, 0x70, 0x65, 0x12, 0x41, 0x0a, 0x06, 0x73, 0x63,
+	0x68, 0x65, 0x6d, 0x61, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x29, 0x2e, 0x6f, 0x72, 0x67,
+	0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64,
+	0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x53,
+	0x63, 0x68, 0x65, 0x6d, 0x61, 0x52, 0x06, 0x73, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x22, 0xad, 0x02,
+	0x0a, 0x0b, 0x4c, 0x6f, 0x67, 0x69, 0x63, 0x61, 0x6c, 0x54, 0x79, 0x70, 0x65, 0x12, 0x10, 0x0a,
+	0x03, 0x75, 0x72, 0x6e, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x03, 0x75, 0x72, 0x6e, 0x12,
+	0x18, 0x0a, 0x07, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c,
+	0x52, 0x07, 0x70, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x54, 0x0a, 0x0e, 0x72, 0x65, 0x70,
+	0x72, 0x65, 0x73, 0x65, 0x6e, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x03, 0x20, 0x01, 0x28,
+	0x0b, 0x32, 0x2c, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62,
+	0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69,
+	0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x54, 0x79, 0x70, 0x65, 0x52,
+	0x0e, 0x72, 0x65, 0x70, 0x72, 0x65, 0x73, 0x65, 0x6e, 0x74, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12,
+	0x51, 0x0a, 0x0d, 0x61, 0x72, 0x67, 0x75, 0x6d, 0x65, 0x6e, 0x74, 0x5f, 0x74, 0x79, 0x70, 0x65,
+	0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2c, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61,
+	0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70,
+	0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64,
+	0x54, 0x79, 0x70, 0x65, 0x52, 0x0c, 0x61, 0x72, 0x67, 0x75, 0x6d, 0x65, 0x6e, 0x74, 0x54, 0x79,
+	0x70, 0x65, 0x12, 0x49, 0x0a, 0x08, 0x61, 0x72, 0x67, 0x75, 0x6d, 0x65, 0x6e, 0x74, 0x18, 0x05,
+	0x20, 0x01, 0x28, 0x0b, 0x32, 0x2d, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68,
+	0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70,
+	0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x56, 0x61,
+	0x6c, 0x75, 0x65, 0x52, 0x08, 0x61, 0x72, 0x67, 0x75, 0x6d, 0x65, 0x6e, 0x74, 0x22, 0xa3, 0x01,
+	0x0a, 0x06, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65,
+	0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x12, 0x40, 0x0a, 0x04,
+	0x74, 0x79, 0x70, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2c, 0x2e, 0x6f, 0x72, 0x67,
+	0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64,
+	0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x46,
+	0x69, 0x65, 0x6c, 0x64, 0x54, 0x79, 0x70, 0x65, 0x52, 0x04, 0x74, 0x79, 0x70, 0x65, 0x12, 0x43,
+	0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2d, 0x2e,
+	0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e,
+	0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76,
+	0x31, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x52, 0x05, 0x76, 0x61,
+	0x6c, 0x75, 0x65, 0x22, 0x4c, 0x0a, 0x03, 0x52, 0x6f, 0x77, 0x12, 0x45, 0x0a, 0x06, 0x76, 0x61,
+	0x6c, 0x75, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x2d, 0x2e, 0x6f, 0x72, 0x67,
+	0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64,
+	0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x46,
+	0x69, 0x65, 0x6c, 0x64, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x52, 0x06, 0x76, 0x61, 0x6c, 0x75, 0x65,
+	0x73, 0x22, 0xa5, 0x04, 0x0a, 0x0a, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x56, 0x61, 0x6c, 0x75, 0x65,
+	0x12, 0x57, 0x0a, 0x0c, 0x61, 0x74, 0x6f, 0x6d, 0x69, 0x63, 0x5f, 0x76, 0x61, 0x6c, 0x75, 0x65,
+	0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x32, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61,
+	0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70,
+	0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x41, 0x74, 0x6f, 0x6d, 0x69,
+	0x63, 0x54, 0x79, 0x70, 0x65, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x48, 0x00, 0x52, 0x0b, 0x61, 0x74,
+	0x6f, 0x6d, 0x69, 0x63, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x54, 0x0a, 0x0b, 0x61, 0x72, 0x72,
+	0x61, 0x79, 0x5f, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x31,
+	0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d,
+	0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e,
+	0x76, 0x31, 0x2e, 0x41, 0x72, 0x72, 0x61, 0x79, 0x54, 0x79, 0x70, 0x65, 0x56, 0x61, 0x6c, 0x75,
+	0x65, 0x48, 0x00, 0x52, 0x0a, 0x61, 0x72, 0x72, 0x61, 0x79, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x12,
+	0x5d, 0x0a, 0x0e, 0x69, 0x74, 0x65, 0x72, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x76, 0x61, 0x6c, 0x75,
+	0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x34, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70,
+	0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e,
+	0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x49, 0x74, 0x65, 0x72,
+	0x61, 0x62, 0x6c, 0x65, 0x54, 0x79, 0x70, 0x65, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x48, 0x00, 0x52,
+	0x0d, 0x69, 0x74, 0x65, 0x72, 0x61, 0x62, 0x6c, 0x65, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x4e,
+	0x0a, 0x09, 0x6d, 0x61, 0x70, 0x5f, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28,
+	0x0b, 0x32, 0x2f, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62,
+	0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69,
+	0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x4d, 0x61, 0x70, 0x54, 0x79, 0x70, 0x65, 0x56, 0x61, 0x6c,
+	0x75, 0x65, 0x48, 0x00, 0x52, 0x08, 0x6d, 0x61, 0x70, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x45,
+	0x0a, 0x09, 0x72, 0x6f, 0x77, 0x5f, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x05, 0x20, 0x01, 0x28,
+	0x0b, 0x32, 0x26, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62,
+	0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69,
+	0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x52, 0x6f, 0x77, 0x48, 0x00, 0x52, 0x08, 0x72, 0x6f, 0x77,
+	0x56, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x63, 0x0a, 0x12, 0x6c, 0x6f, 0x67, 0x69, 0x63, 0x61, 0x6c,
+	0x5f, 0x74, 0x79, 0x70, 0x65, 0x5f, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28,
+	0x0b, 0x32, 0x33, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62,
+	0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69,
+	0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x4c, 0x6f, 0x67, 0x69, 0x63, 0x61, 0x6c, 0x54, 0x79, 0x70,
+	0x65, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x48, 0x00, 0x52, 0x10, 0x6c, 0x6f, 0x67, 0x69, 0x63, 0x61,
+	0x6c, 0x54, 0x79, 0x70, 0x65, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x42, 0x0d, 0x0a, 0x0b, 0x66, 0x69,
+	0x65, 0x6c, 0x64, 0x5f, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x22, 0xf8, 0x01, 0x0a, 0x0f, 0x41, 0x74,
+	0x6f, 0x6d, 0x69, 0x63, 0x54, 0x79, 0x70, 0x65, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x14, 0x0a,
+	0x04, 0x62, 0x79, 0x74, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x05, 0x48, 0x00, 0x52, 0x04, 0x62,
+	0x79, 0x74, 0x65, 0x12, 0x16, 0x0a, 0x05, 0x69, 0x6e, 0x74, 0x31, 0x36, 0x18, 0x02, 0x20, 0x01,
+	0x28, 0x05, 0x48, 0x00, 0x52, 0x05, 0x69, 0x6e, 0x74, 0x31, 0x36, 0x12, 0x16, 0x0a, 0x05, 0x69,
+	0x6e, 0x74, 0x33, 0x32, 0x18, 0x03, 0x20, 0x01, 0x28, 0x05, 0x48, 0x00, 0x52, 0x05, 0x69, 0x6e,
+	0x74, 0x33, 0x32, 0x12, 0x16, 0x0a, 0x05, 0x69, 0x6e, 0x74, 0x36, 0x34, 0x18, 0x04, 0x20, 0x01,
+	0x28, 0x03, 0x48, 0x00, 0x52, 0x05, 0x69, 0x6e, 0x74, 0x36, 0x34, 0x12, 0x16, 0x0a, 0x05, 0x66,
+	0x6c, 0x6f, 0x61, 0x74, 0x18, 0x05, 0x20, 0x01, 0x28, 0x02, 0x48, 0x00, 0x52, 0x05, 0x66, 0x6c,
+	0x6f, 0x61, 0x74, 0x12, 0x18, 0x0a, 0x06, 0x64, 0x6f, 0x75, 0x62, 0x6c, 0x65, 0x18, 0x06, 0x20,
+	0x01, 0x28, 0x01, 0x48, 0x00, 0x52, 0x06, 0x64, 0x6f, 0x75, 0x62, 0x6c, 0x65, 0x12, 0x18, 0x0a,
+	0x06, 0x73, 0x74, 0x72, 0x69, 0x6e, 0x67, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x48, 0x00, 0x52,
+	0x06, 0x73, 0x74, 0x72, 0x69, 0x6e, 0x67, 0x12, 0x1a, 0x0a, 0x07, 0x62, 0x6f, 0x6f, 0x6c, 0x65,
+	0x61, 0x6e, 0x18, 0x08, 0x20, 0x01, 0x28, 0x08, 0x48, 0x00, 0x52, 0x07, 0x62, 0x6f, 0x6f, 0x6c,
+	0x65, 0x61, 0x6e, 0x12, 0x16, 0x0a, 0x05, 0x62, 0x79, 0x74, 0x65, 0x73, 0x18, 0x09, 0x20, 0x01,
+	0x28, 0x0c, 0x48, 0x00, 0x52, 0x05, 0x62, 0x79, 0x74, 0x65, 0x73, 0x42, 0x07, 0x0a, 0x05, 0x76,
+	0x61, 0x6c, 0x75, 0x65, 0x22, 0x59, 0x0a, 0x0e, 0x41, 0x72, 0x72, 0x61, 0x79, 0x54, 0x79, 0x70,
+	0x65, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x47, 0x0a, 0x07, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e,
+	0x74, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x2d, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70,
+	0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e,
+	0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x46, 0x69, 0x65, 0x6c,
+	0x64, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x52, 0x07, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x22,
+	0x5c, 0x0a, 0x11, 0x49, 0x74, 0x65, 0x72, 0x61, 0x62, 0x6c, 0x65, 0x54, 0x79, 0x70, 0x65, 0x56,
+	0x61, 0x6c, 0x75, 0x65, 0x12, 0x47, 0x0a, 0x07, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x18,
+	0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x2d, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63,
+	0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69,
+	0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x56,
+	0x61, 0x6c, 0x75, 0x65, 0x52, 0x07, 0x65, 0x6c, 0x65, 0x6d, 0x65, 0x6e, 0x74, 0x22, 0x59, 0x0a,
+	0x0c, 0x4d, 0x61, 0x70, 0x54, 0x79, 0x70, 0x65, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x49, 0x0a,
+	0x07, 0x65, 0x6e, 0x74, 0x72, 0x69, 0x65, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x2f,
+	0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d,
+	0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e,
+	0x76, 0x31, 0x2e, 0x4d, 0x61, 0x70, 0x54, 0x79, 0x70, 0x65, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x52,
+	0x07, 0x65, 0x6e, 0x74, 0x72, 0x69, 0x65, 0x73, 0x22, 0x94, 0x01, 0x0a, 0x0c, 0x4d, 0x61, 0x70,
+	0x54, 0x79, 0x70, 0x65, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x3f, 0x0a, 0x03, 0x6b, 0x65, 0x79,
+	0x18, 0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2d, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61,
+	0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70,
+	0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64,
+	0x56, 0x61, 0x6c, 0x75, 0x65, 0x52, 0x03, 0x6b, 0x65, 0x79, 0x12, 0x43, 0x0a, 0x05, 0x76, 0x61,
+	0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x2d, 0x2e, 0x6f, 0x72, 0x67, 0x2e,
+	0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65,
+	0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x46, 0x69,
+	0x65, 0x6c, 0x64, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x22,
+	0x57, 0x0a, 0x10, 0x4c, 0x6f, 0x67, 0x69, 0x63, 0x61, 0x6c, 0x54, 0x79, 0x70, 0x65, 0x56, 0x61,
+	0x6c, 0x75, 0x65, 0x12, 0x43, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x01, 0x20, 0x01,
+	0x28, 0x0b, 0x32, 0x2d, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e,
+	0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c,
+	0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x46, 0x69, 0x65, 0x6c, 0x64, 0x56, 0x61, 0x6c, 0x75,
+	0x65, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x2a, 0x83, 0x01, 0x0a, 0x0a, 0x41, 0x74, 0x6f,
+	0x6d, 0x69, 0x63, 0x54, 0x79, 0x70, 0x65, 0x12, 0x0f, 0x0a, 0x0b, 0x55, 0x4e, 0x53, 0x50, 0x45,
+	0x43, 0x49, 0x46, 0x49, 0x45, 0x44, 0x10, 0x00, 0x12, 0x08, 0x0a, 0x04, 0x42, 0x59, 0x54, 0x45,
+	0x10, 0x01, 0x12, 0x09, 0x0a, 0x05, 0x49, 0x4e, 0x54, 0x31, 0x36, 0x10, 0x02, 0x12, 0x09, 0x0a,
+	0x05, 0x49, 0x4e, 0x54, 0x33, 0x32, 0x10, 0x03, 0x12, 0x09, 0x0a, 0x05, 0x49, 0x4e, 0x54, 0x36,
+	0x34, 0x10, 0x04, 0x12, 0x09, 0x0a, 0x05, 0x46, 0x4c, 0x4f, 0x41, 0x54, 0x10, 0x05, 0x12, 0x0a,
+	0x0a, 0x06, 0x44, 0x4f, 0x55, 0x42, 0x4c, 0x45, 0x10, 0x06, 0x12, 0x0a, 0x0a, 0x06, 0x53, 0x54,
+	0x52, 0x49, 0x4e, 0x47, 0x10, 0x07, 0x12, 0x0b, 0x0a, 0x07, 0x42, 0x4f, 0x4f, 0x4c, 0x45, 0x41,
+	0x4e, 0x10, 0x08, 0x12, 0x09, 0x0a, 0x05, 0x42, 0x59, 0x54, 0x45, 0x53, 0x10, 0x09, 0x42, 0x75,
+	0x0a, 0x21, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61,
+	0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65,
+	0x2e, 0x76, 0x31, 0x42, 0x09, 0x53, 0x63, 0x68, 0x65, 0x6d, 0x61, 0x41, 0x70, 0x69, 0x5a, 0x45,
+	0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x61, 0x70, 0x61, 0x63, 0x68,
+	0x65, 0x2f, 0x62, 0x65, 0x61, 0x6d, 0x2f, 0x73, 0x64, 0x6b, 0x73, 0x2f, 0x67, 0x6f, 0x2f, 0x70,
+	0x6b, 0x67, 0x2f, 0x62, 0x65, 0x61, 0x6d, 0x2f, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2f, 0x70, 0x69,
+	0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x5f, 0x76, 0x31, 0x3b, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69,
+	0x6e, 0x65, 0x5f, 0x76, 0x31, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
+}
+
+var (
+	file_schema_proto_rawDescOnce sync.Once
+	file_schema_proto_rawDescData = file_schema_proto_rawDesc
+)
+
+func file_schema_proto_rawDescGZIP() []byte {
+	file_schema_proto_rawDescOnce.Do(func() {
+		file_schema_proto_rawDescData = protoimpl.X.CompressGZIP(file_schema_proto_rawDescData)
+	})
+	return file_schema_proto_rawDescData
+}
+
+var file_schema_proto_enumTypes = make([]protoimpl.EnumInfo, 1)
+var file_schema_proto_msgTypes = make([]protoimpl.MessageInfo, 17)
+var file_schema_proto_goTypes = []interface{}{
+	(AtomicType)(0),           // 0: org.apache.beam.model.pipeline.v1.AtomicType
+	(*Schema)(nil),            // 1: org.apache.beam.model.pipeline.v1.Schema
+	(*Field)(nil),             // 2: org.apache.beam.model.pipeline.v1.Field
+	(*FieldType)(nil),         // 3: org.apache.beam.model.pipeline.v1.FieldType
+	(*ArrayType)(nil),         // 4: org.apache.beam.model.pipeline.v1.ArrayType
+	(*IterableType)(nil),      // 5: org.apache.beam.model.pipeline.v1.IterableType
+	(*MapType)(nil),           // 6: org.apache.beam.model.pipeline.v1.MapType
+	(*RowType)(nil),           // 7: org.apache.beam.model.pipeline.v1.RowType
+	(*LogicalType)(nil),       // 8: org.apache.beam.model.pipeline.v1.LogicalType
+	(*Option)(nil),            // 9: org.apache.beam.model.pipeline.v1.Option
+	(*Row)(nil),               // 10: org.apache.beam.model.pipeline.v1.Row
+	(*FieldValue)(nil),        // 11: org.apache.beam.model.pipeline.v1.FieldValue
+	(*AtomicTypeValue)(nil),   // 12: org.apache.beam.model.pipeline.v1.AtomicTypeValue
+	(*ArrayTypeValue)(nil),    // 13: org.apache.beam.model.pipeline.v1.ArrayTypeValue
+	(*IterableTypeValue)(nil), // 14: org.apache.beam.model.pipeline.v1.IterableTypeValue
+	(*MapTypeValue)(nil),      // 15: org.apache.beam.model.pipeline.v1.MapTypeValue
+	(*MapTypeEntry)(nil),      // 16: org.apache.beam.model.pipeline.v1.MapTypeEntry
+	(*LogicalTypeValue)(nil),  // 17: org.apache.beam.model.pipeline.v1.LogicalTypeValue
+}
+var file_schema_proto_depIdxs = []int32{
+	2,  // 0: org.apache.beam.model.pipeline.v1.Schema.fields:type_name -> org.apache.beam.model.pipeline.v1.Field
+	9,  // 1: org.apache.beam.model.pipeline.v1.Schema.options:type_name -> org.apache.beam.model.pipeline.v1.Option
+	3,  // 2: org.apache.beam.model.pipeline.v1.Field.type:type_name -> org.apache.beam.model.pipeline.v1.FieldType
+	9,  // 3: org.apache.beam.model.pipeline.v1.Field.options:type_name -> org.apache.beam.model.pipeline.v1.Option
+	0,  // 4: org.apache.beam.model.pipeline.v1.FieldType.atomic_type:type_name -> org.apache.beam.model.pipeline.v1.AtomicType
+	4,  // 5: org.apache.beam.model.pipeline.v1.FieldType.array_type:type_name -> org.apache.beam.model.pipeline.v1.ArrayType
+	5,  // 6: org.apache.beam.model.pipeline.v1.FieldType.iterable_type:type_name -> org.apache.beam.model.pipeline.v1.IterableType
+	6,  // 7: org.apache.beam.model.pipeline.v1.FieldType.map_type:type_name -> org.apache.beam.model.pipeline.v1.MapType
+	7,  // 8: org.apache.beam.model.pipeline.v1.FieldType.row_type:type_name -> org.apache.beam.model.pipeline.v1.RowType
+	8,  // 9: org.apache.beam.model.pipeline.v1.FieldType.logical_type:type_name -> org.apache.beam.model.pipeline.v1.LogicalType
+	3,  // 10: org.apache.beam.model.pipeline.v1.ArrayType.element_type:type_name -> org.apache.beam.model.pipeline.v1.FieldType
+	3,  // 11: org.apache.beam.model.pipeline.v1.IterableType.element_type:type_name -> org.apache.beam.model.pipeline.v1.FieldType
+	3,  // 12: org.apache.beam.model.pipeline.v1.MapType.key_type:type_name -> org.apache.beam.model.pipeline.v1.FieldType
+	3,  // 13: org.apache.beam.model.pipeline.v1.MapType.value_type:type_name -> org.apache.beam.model.pipeline.v1.FieldType
+	1,  // 14: org.apache.beam.model.pipeline.v1.RowType.schema:type_name -> org.apache.beam.model.pipeline.v1.Schema
+	3,  // 15: org.apache.beam.model.pipeline.v1.LogicalType.representation:type_name -> org.apache.beam.model.pipeline.v1.FieldType
+	3,  // 16: org.apache.beam.model.pipeline.v1.LogicalType.argument_type:type_name -> org.apache.beam.model.pipeline.v1.FieldType
+	11, // 17: org.apache.beam.model.pipeline.v1.LogicalType.argument:type_name -> org.apache.beam.model.pipeline.v1.FieldValue
+	3,  // 18: org.apache.beam.model.pipeline.v1.Option.type:type_name -> org.apache.beam.model.pipeline.v1.FieldType
+	11, // 19: org.apache.beam.model.pipeline.v1.Option.value:type_name -> org.apache.beam.model.pipeline.v1.FieldValue
+	11, // 20: org.apache.beam.model.pipeline.v1.Row.values:type_name -> org.apache.beam.model.pipeline.v1.FieldValue
+	12, // 21: org.apache.beam.model.pipeline.v1.FieldValue.atomic_value:type_name -> org.apache.beam.model.pipeline.v1.AtomicTypeValue
+	13, // 22: org.apache.beam.model.pipeline.v1.FieldValue.array_value:type_name -> org.apache.beam.model.pipeline.v1.ArrayTypeValue
+	14, // 23: org.apache.beam.model.pipeline.v1.FieldValue.iterable_value:type_name -> org.apache.beam.model.pipeline.v1.IterableTypeValue
+	15, // 24: org.apache.beam.model.pipeline.v1.FieldValue.map_value:type_name -> org.apache.beam.model.pipeline.v1.MapTypeValue
+	10, // 25: org.apache.beam.model.pipeline.v1.FieldValue.row_value:type_name -> org.apache.beam.model.pipeline.v1.Row
+	17, // 26: org.apache.beam.model.pipeline.v1.FieldValue.logical_type_value:type_name -> org.apache.beam.model.pipeline.v1.LogicalTypeValue
+	11, // 27: org.apache.beam.model.pipeline.v1.ArrayTypeValue.element:type_name -> org.apache.beam.model.pipeline.v1.FieldValue
+	11, // 28: org.apache.beam.model.pipeline.v1.IterableTypeValue.element:type_name -> org.apache.beam.model.pipeline.v1.FieldValue
+	16, // 29: org.apache.beam.model.pipeline.v1.MapTypeValue.entries:type_name -> org.apache.beam.model.pipeline.v1.MapTypeEntry
+	11, // 30: org.apache.beam.model.pipeline.v1.MapTypeEntry.key:type_name -> org.apache.beam.model.pipeline.v1.FieldValue
+	11, // 31: org.apache.beam.model.pipeline.v1.MapTypeEntry.value:type_name -> org.apache.beam.model.pipeline.v1.FieldValue
+	11, // 32: org.apache.beam.model.pipeline.v1.LogicalTypeValue.value:type_name -> org.apache.beam.model.pipeline.v1.FieldValue
+	33, // [33:33] is the sub-list for method output_type
+	33, // [33:33] is the sub-list for method input_type
+	33, // [33:33] is the sub-list for extension type_name
+	33, // [33:33] is the sub-list for extension extendee
+	0,  // [0:33] is the sub-list for field type_name
+}
+
+func init() { file_schema_proto_init() }
+func file_schema_proto_init() {
+	if File_schema_proto != nil {
+		return
+	}
+	if !protoimpl.UnsafeEnabled {
+		file_schema_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*Schema); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_schema_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*Field); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_schema_proto_msgTypes[2].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*FieldType); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_schema_proto_msgTypes[3].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*ArrayType); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_schema_proto_msgTypes[4].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*IterableType); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_schema_proto_msgTypes[5].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*MapType); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_schema_proto_msgTypes[6].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*RowType); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_schema_proto_msgTypes[7].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*LogicalType); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_schema_proto_msgTypes[8].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*Option); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_schema_proto_msgTypes[9].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*Row); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_schema_proto_msgTypes[10].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*FieldValue); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_schema_proto_msgTypes[11].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*AtomicTypeValue); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_schema_proto_msgTypes[12].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*ArrayTypeValue); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_schema_proto_msgTypes[13].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*IterableTypeValue); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_schema_proto_msgTypes[14].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*MapTypeValue); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_schema_proto_msgTypes[15].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*MapTypeEntry); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_schema_proto_msgTypes[16].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*LogicalTypeValue); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+	}
+	file_schema_proto_msgTypes[2].OneofWrappers = []interface{}{
+		(*FieldType_AtomicType)(nil),
+		(*FieldType_ArrayType)(nil),
+		(*FieldType_IterableType)(nil),
+		(*FieldType_MapType)(nil),
+		(*FieldType_RowType)(nil),
+		(*FieldType_LogicalType)(nil),
+	}
+	file_schema_proto_msgTypes[10].OneofWrappers = []interface{}{
+		(*FieldValue_AtomicValue)(nil),
+		(*FieldValue_ArrayValue)(nil),
+		(*FieldValue_IterableValue)(nil),
+		(*FieldValue_MapValue)(nil),
+		(*FieldValue_RowValue)(nil),
+		(*FieldValue_LogicalTypeValue)(nil),
+	}
+	file_schema_proto_msgTypes[11].OneofWrappers = []interface{}{
 		(*AtomicTypeValue_Byte)(nil),
 		(*AtomicTypeValue_Int16)(nil),
 		(*AtomicTypeValue_Int32)(nil),
@@ -1003,301 +1904,23 @@
 		(*AtomicTypeValue_Boolean)(nil),
 		(*AtomicTypeValue_Bytes)(nil),
 	}
-}
-
-type ArrayTypeValue struct {
-	Element              []*FieldValue `protobuf:"bytes,1,rep,name=element,proto3" json:"element,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}      `json:"-"`
-	XXX_unrecognized     []byte        `json:"-"`
-	XXX_sizecache        int32         `json:"-"`
-}
-
-func (m *ArrayTypeValue) Reset()         { *m = ArrayTypeValue{} }
-func (m *ArrayTypeValue) String() string { return proto.CompactTextString(m) }
-func (*ArrayTypeValue) ProtoMessage()    {}
-func (*ArrayTypeValue) Descriptor() ([]byte, []int) {
-	return fileDescriptor_1c5fb4d8cc22d66a, []int{12}
-}
-
-func (m *ArrayTypeValue) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_ArrayTypeValue.Unmarshal(m, b)
-}
-func (m *ArrayTypeValue) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_ArrayTypeValue.Marshal(b, m, deterministic)
-}
-func (m *ArrayTypeValue) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_ArrayTypeValue.Merge(m, src)
-}
-func (m *ArrayTypeValue) XXX_Size() int {
-	return xxx_messageInfo_ArrayTypeValue.Size(m)
-}
-func (m *ArrayTypeValue) XXX_DiscardUnknown() {
-	xxx_messageInfo_ArrayTypeValue.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_ArrayTypeValue proto.InternalMessageInfo
-
-func (m *ArrayTypeValue) GetElement() []*FieldValue {
-	if m != nil {
-		return m.Element
-	}
-	return nil
-}
-
-type IterableTypeValue struct {
-	Element              []*FieldValue `protobuf:"bytes,1,rep,name=element,proto3" json:"element,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}      `json:"-"`
-	XXX_unrecognized     []byte        `json:"-"`
-	XXX_sizecache        int32         `json:"-"`
-}
-
-func (m *IterableTypeValue) Reset()         { *m = IterableTypeValue{} }
-func (m *IterableTypeValue) String() string { return proto.CompactTextString(m) }
-func (*IterableTypeValue) ProtoMessage()    {}
-func (*IterableTypeValue) Descriptor() ([]byte, []int) {
-	return fileDescriptor_1c5fb4d8cc22d66a, []int{13}
-}
-
-func (m *IterableTypeValue) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_IterableTypeValue.Unmarshal(m, b)
-}
-func (m *IterableTypeValue) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_IterableTypeValue.Marshal(b, m, deterministic)
-}
-func (m *IterableTypeValue) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_IterableTypeValue.Merge(m, src)
-}
-func (m *IterableTypeValue) XXX_Size() int {
-	return xxx_messageInfo_IterableTypeValue.Size(m)
-}
-func (m *IterableTypeValue) XXX_DiscardUnknown() {
-	xxx_messageInfo_IterableTypeValue.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_IterableTypeValue proto.InternalMessageInfo
-
-func (m *IterableTypeValue) GetElement() []*FieldValue {
-	if m != nil {
-		return m.Element
-	}
-	return nil
-}
-
-type MapTypeValue struct {
-	Entries              []*MapTypeEntry `protobuf:"bytes,1,rep,name=entries,proto3" json:"entries,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}        `json:"-"`
-	XXX_unrecognized     []byte          `json:"-"`
-	XXX_sizecache        int32           `json:"-"`
-}
-
-func (m *MapTypeValue) Reset()         { *m = MapTypeValue{} }
-func (m *MapTypeValue) String() string { return proto.CompactTextString(m) }
-func (*MapTypeValue) ProtoMessage()    {}
-func (*MapTypeValue) Descriptor() ([]byte, []int) {
-	return fileDescriptor_1c5fb4d8cc22d66a, []int{14}
-}
-
-func (m *MapTypeValue) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_MapTypeValue.Unmarshal(m, b)
-}
-func (m *MapTypeValue) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_MapTypeValue.Marshal(b, m, deterministic)
-}
-func (m *MapTypeValue) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_MapTypeValue.Merge(m, src)
-}
-func (m *MapTypeValue) XXX_Size() int {
-	return xxx_messageInfo_MapTypeValue.Size(m)
-}
-func (m *MapTypeValue) XXX_DiscardUnknown() {
-	xxx_messageInfo_MapTypeValue.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_MapTypeValue proto.InternalMessageInfo
-
-func (m *MapTypeValue) GetEntries() []*MapTypeEntry {
-	if m != nil {
-		return m.Entries
-	}
-	return nil
-}
-
-type MapTypeEntry struct {
-	Key                  *FieldValue `protobuf:"bytes,1,opt,name=key,proto3" json:"key,omitempty"`
-	Value                *FieldValue `protobuf:"bytes,2,opt,name=value,proto3" json:"value,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}    `json:"-"`
-	XXX_unrecognized     []byte      `json:"-"`
-	XXX_sizecache        int32       `json:"-"`
-}
-
-func (m *MapTypeEntry) Reset()         { *m = MapTypeEntry{} }
-func (m *MapTypeEntry) String() string { return proto.CompactTextString(m) }
-func (*MapTypeEntry) ProtoMessage()    {}
-func (*MapTypeEntry) Descriptor() ([]byte, []int) {
-	return fileDescriptor_1c5fb4d8cc22d66a, []int{15}
-}
-
-func (m *MapTypeEntry) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_MapTypeEntry.Unmarshal(m, b)
-}
-func (m *MapTypeEntry) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_MapTypeEntry.Marshal(b, m, deterministic)
-}
-func (m *MapTypeEntry) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_MapTypeEntry.Merge(m, src)
-}
-func (m *MapTypeEntry) XXX_Size() int {
-	return xxx_messageInfo_MapTypeEntry.Size(m)
-}
-func (m *MapTypeEntry) XXX_DiscardUnknown() {
-	xxx_messageInfo_MapTypeEntry.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_MapTypeEntry proto.InternalMessageInfo
-
-func (m *MapTypeEntry) GetKey() *FieldValue {
-	if m != nil {
-		return m.Key
-	}
-	return nil
-}
-
-func (m *MapTypeEntry) GetValue() *FieldValue {
-	if m != nil {
-		return m.Value
-	}
-	return nil
-}
-
-type LogicalTypeValue struct {
-	Value                *FieldValue `protobuf:"bytes,1,opt,name=value,proto3" json:"value,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}    `json:"-"`
-	XXX_unrecognized     []byte      `json:"-"`
-	XXX_sizecache        int32       `json:"-"`
-}
-
-func (m *LogicalTypeValue) Reset()         { *m = LogicalTypeValue{} }
-func (m *LogicalTypeValue) String() string { return proto.CompactTextString(m) }
-func (*LogicalTypeValue) ProtoMessage()    {}
-func (*LogicalTypeValue) Descriptor() ([]byte, []int) {
-	return fileDescriptor_1c5fb4d8cc22d66a, []int{16}
-}
-
-func (m *LogicalTypeValue) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_LogicalTypeValue.Unmarshal(m, b)
-}
-func (m *LogicalTypeValue) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_LogicalTypeValue.Marshal(b, m, deterministic)
-}
-func (m *LogicalTypeValue) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_LogicalTypeValue.Merge(m, src)
-}
-func (m *LogicalTypeValue) XXX_Size() int {
-	return xxx_messageInfo_LogicalTypeValue.Size(m)
-}
-func (m *LogicalTypeValue) XXX_DiscardUnknown() {
-	xxx_messageInfo_LogicalTypeValue.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_LogicalTypeValue proto.InternalMessageInfo
-
-func (m *LogicalTypeValue) GetValue() *FieldValue {
-	if m != nil {
-		return m.Value
-	}
-	return nil
-}
-
-func init() {
-	proto.RegisterEnum("org.apache.beam.model.pipeline.v1.AtomicType", AtomicType_name, AtomicType_value)
-	proto.RegisterType((*Schema)(nil), "org.apache.beam.model.pipeline.v1.Schema")
-	proto.RegisterType((*Field)(nil), "org.apache.beam.model.pipeline.v1.Field")
-	proto.RegisterType((*FieldType)(nil), "org.apache.beam.model.pipeline.v1.FieldType")
-	proto.RegisterType((*ArrayType)(nil), "org.apache.beam.model.pipeline.v1.ArrayType")
-	proto.RegisterType((*IterableType)(nil), "org.apache.beam.model.pipeline.v1.IterableType")
-	proto.RegisterType((*MapType)(nil), "org.apache.beam.model.pipeline.v1.MapType")
-	proto.RegisterType((*RowType)(nil), "org.apache.beam.model.pipeline.v1.RowType")
-	proto.RegisterType((*LogicalType)(nil), "org.apache.beam.model.pipeline.v1.LogicalType")
-	proto.RegisterType((*Option)(nil), "org.apache.beam.model.pipeline.v1.Option")
-	proto.RegisterType((*Row)(nil), "org.apache.beam.model.pipeline.v1.Row")
-	proto.RegisterType((*FieldValue)(nil), "org.apache.beam.model.pipeline.v1.FieldValue")
-	proto.RegisterType((*AtomicTypeValue)(nil), "org.apache.beam.model.pipeline.v1.AtomicTypeValue")
-	proto.RegisterType((*ArrayTypeValue)(nil), "org.apache.beam.model.pipeline.v1.ArrayTypeValue")
-	proto.RegisterType((*IterableTypeValue)(nil), "org.apache.beam.model.pipeline.v1.IterableTypeValue")
-	proto.RegisterType((*MapTypeValue)(nil), "org.apache.beam.model.pipeline.v1.MapTypeValue")
-	proto.RegisterType((*MapTypeEntry)(nil), "org.apache.beam.model.pipeline.v1.MapTypeEntry")
-	proto.RegisterType((*LogicalTypeValue)(nil), "org.apache.beam.model.pipeline.v1.LogicalTypeValue")
-}
-
-func init() { proto.RegisterFile("schema.proto", fileDescriptor_1c5fb4d8cc22d66a) }
-
-var fileDescriptor_1c5fb4d8cc22d66a = []byte{
-	// 1067 bytes of a gzipped FileDescriptorProto
-	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xac, 0x97, 0xcf, 0x8f, 0xda, 0x46,
-	0x14, 0xc7, 0xd7, 0x60, 0x6c, 0xfc, 0xcc, 0x12, 0x67, 0x54, 0x55, 0x28, 0x27, 0xea, 0x43, 0x45,
-	0xd3, 0x86, 0x0a, 0x58, 0x71, 0xe9, 0xa1, 0x81, 0x0d, 0xbb, 0x8b, 0xca, 0xc2, 0xd6, 0x90, 0x44,
-	0x5b, 0xa5, 0x42, 0x03, 0xcc, 0x12, 0x6b, 0x8d, 0xc7, 0x32, 0x66, 0x57, 0x1c, 0xab, 0xfe, 0x0b,
-	0xb9, 0xe7, 0x50, 0xf5, 0xd8, 0xff, 0xaf, 0xc7, 0x6a, 0x7e, 0xd8, 0xd0, 0x6d, 0xa5, 0xd8, 0x64,
-	0x6f, 0xf3, 0x9e, 0xf9, 0x7e, 0x66, 0xe6, 0xcd, 0xf3, 0x77, 0x0c, 0x94, 0xd6, 0xf3, 0xf7, 0x64,
-	0x85, 0xeb, 0x41, 0x48, 0x23, 0x8a, 0xbe, 0xa2, 0xe1, 0xb2, 0x8e, 0x03, 0x3c, 0x7f, 0x4f, 0xea,
-	0x33, 0x82, 0x57, 0xf5, 0x15, 0x5d, 0x10, 0xaf, 0x1e, 0xb8, 0x01, 0xf1, 0x5c, 0x9f, 0xd4, 0xef,
-	0x1a, 0xf6, 0x47, 0x05, 0xb4, 0x31, 0xd7, 0xa0, 0x97, 0xa0, 0xdd, 0xb8, 0xc4, 0x5b, 0xac, 0x2b,
-	0x4a, 0x35, 0x5f, 0x33, 0x9b, 0xb5, 0xfa, 0x27, 0xe5, 0xf5, 0x33, 0x26, 0x70, 0xa4, 0x0e, 0x95,
-	0x21, 0xe7, 0x2e, 0x2a, 0xb9, 0xaa, 0x52, 0x33, 0x9c, 0x9c, 0xbb, 0x40, 0xa7, 0xa0, 0xd3, 0x20,
-	0x72, 0xa9, 0xbf, 0xae, 0xe4, 0x39, 0xf2, 0x9b, 0x14, 0xc8, 0x11, 0x57, 0x38, 0xb1, 0xd2, 0xfe,
-	0x2d, 0x07, 0x05, 0x3e, 0x0d, 0x42, 0xa0, 0xfa, 0x78, 0x45, 0x2a, 0x0a, 0x9f, 0x80, 0x8f, 0x51,
-	0x15, 0xcc, 0x05, 0x59, 0xcf, 0x43, 0x97, 0xff, 0x5a, 0xce, 0xbd, 0x9f, 0x42, 0x2f, 0x41, 0x8d,
-	0xb6, 0x01, 0xa9, 0xe4, 0xab, 0x4a, 0xcd, 0x6c, 0x7e, 0x97, 0x76, 0x53, 0x93, 0x6d, 0x40, 0x1c,
-	0xae, 0x94, 0xdb, 0x52, 0xab, 0x4a, 0xad, 0xc0, 0xb7, 0xf5, 0x2d, 0x3c, 0x25, 0xfe, 0x9c, 0x2e,
-	0x5c, 0x7f, 0x39, 0x0d, 0xe8, 0xda, 0xe5, 0x33, 0x17, 0xf8, 0x63, 0x2b, 0x7e, 0x70, 0x25, 0xf3,
-	0xfb, 0x35, 0xd0, 0x0e, 0xae, 0xc1, 0x07, 0x15, 0x8c, 0x64, 0x55, 0xe8, 0x19, 0x14, 0xfd, 0x8d,
-	0xe7, 0xe1, 0x99, 0x27, 0x6a, 0x51, 0x74, 0x92, 0x18, 0x5d, 0x81, 0x89, 0x23, 0xba, 0x72, 0xe7,
-	0x53, 0xbe, 0x69, 0x56, 0x8f, 0x72, 0xf3, 0x45, 0x8a, 0x29, 0x3b, 0x5c, 0xc5, 0xf8, 0x17, 0x47,
-	0x0e, 0xe0, 0x24, 0x42, 0x97, 0x00, 0x38, 0x0c, 0xf1, 0x76, 0x9a, 0xb1, 0x8a, 0x1d, 0x26, 0x92,
-	0x3c, 0x03, 0xc7, 0x01, 0x7a, 0x03, 0xc7, 0x6e, 0x44, 0x42, 0xb6, 0x58, 0x41, 0x54, 0x39, 0xf1,
-	0xfb, 0x14, 0xc4, 0xbe, 0xd4, 0x49, 0x68, 0xc9, 0xdd, 0x8b, 0xd1, 0x39, 0x14, 0x57, 0x38, 0x10,
-	0xc8, 0x02, 0x47, 0x3e, 0x4f, 0x81, 0xbc, 0xc4, 0x81, 0xa4, 0xe9, 0x2b, 0x31, 0x64, 0xa0, 0x90,
-	0xde, 0x0b, 0x90, 0x96, 0x1a, 0xe4, 0xd0, 0xfb, 0x18, 0x14, 0x8a, 0x21, 0x1a, 0x43, 0xc9, 0xa3,
-	0x4b, 0x77, 0x8e, 0x3d, 0x01, 0xd3, 0x39, 0xac, 0x9e, 0x02, 0x36, 0x10, 0x32, 0x09, 0x34, 0xbd,
-	0x5d, 0xd8, 0x35, 0xc1, 0x60, 0xb0, 0xa9, 0xeb, 0xdf, 0x50, 0xfb, 0x1d, 0x18, 0x49, 0x95, 0xd1,
-	0x08, 0x4a, 0xc4, 0x23, 0x2b, 0xe2, 0x47, 0x62, 0x3a, 0xe5, 0x80, 0x7e, 0x37, 0x25, 0x81, 0x05,
-	0xf6, 0x14, 0x4a, 0xfb, 0x15, 0x7f, 0xfc, 0x09, 0x3e, 0x2a, 0xa0, 0x5f, 0xee, 0xaa, 0x7e, 0x4b,
-	0xb6, 0x87, 0x83, 0xf5, 0x5b, 0x22, 0xca, 0xf0, 0x13, 0xc0, 0x1d, 0xf6, 0x36, 0x64, 0xd7, 0xff,
-	0x59, 0x51, 0x06, 0xd7, 0xf3, 0x15, 0x0e, 0x40, 0x97, 0x07, 0x8b, 0x3a, 0xa0, 0x09, 0x6f, 0x95,
-	0xcb, 0x4b, 0xf3, 0x1a, 0x0b, 0x63, 0x75, 0xa4, 0xd0, 0xfe, 0x2b, 0x07, 0xe6, 0xde, 0xd1, 0x22,
-	0x0b, 0xf2, 0x9b, 0xd0, 0x97, 0x76, 0xc6, 0x86, 0xa8, 0x02, 0x7a, 0x80, 0xb7, 0x1e, 0xc5, 0xc2,
-	0x45, 0x4b, 0x4e, 0x1c, 0xa2, 0x09, 0x94, 0x43, 0x12, 0x84, 0x64, 0x4d, 0xfc, 0x08, 0x73, 0xc3,
-	0x39, 0xc4, 0xcf, 0x1e, 0x30, 0xd0, 0xcf, 0x70, 0x8c, 0xc3, 0xe5, 0x66, 0x77, 0xa6, 0xea, 0x01,
-	0xd0, 0x52, 0x8c, 0xe0, 0x9b, 0xea, 0x43, 0x31, 0x8e, 0xe5, 0x7b, 0xf8, 0x22, 0x2d, 0xed, 0x0d,
-	0xab, 0xbb, 0x93, 0xc8, 0xed, 0x3f, 0x14, 0xd0, 0x84, 0x13, 0xfe, 0xaf, 0xf5, 0xc7, 0xc6, 0x9e,
-	0x3b, 0xd8, 0xd8, 0x4f, 0xa1, 0xc0, 0xcf, 0x5a, 0xd6, 0x32, 0xe3, 0x42, 0x85, 0xd6, 0x1e, 0x40,
-	0xde, 0xa1, 0xf7, 0xa8, 0x07, 0x1a, 0x8f, 0xe3, 0xdb, 0x33, 0x23, 0x4c, 0x8a, 0xed, 0x3f, 0x55,
-	0x80, 0x5d, 0x1a, 0xbd, 0x85, 0x92, 0xb4, 0x73, 0xb1, 0x50, 0xd1, 0x7b, 0xcd, 0x4c, 0x7e, 0xce,
-	0x49, 0xcc, 0x47, 0x04, 0x49, 0x80, 0x27, 0x60, 0x0a, 0x57, 0x17, 0x5c, 0x51, 0xc3, 0x46, 0x16,
-	0x5b, 0x8f, 0xb1, 0xe2, 0x76, 0x10, 0xd4, 0x5f, 0xa1, 0x9c, 0x98, 0xfb, 0x7e, 0x65, 0x4f, 0x32,
-	0xba, 0x7b, 0xcc, 0x4e, 0xae, 0x0a, 0x81, 0x1f, 0x82, 0xc1, 0x3c, 0x5e, 0x90, 0xd3, 0xdf, 0x1b,
-	0xd2, 0x63, 0x62, 0x28, 0xbb, 0x27, 0x04, 0xaf, 0x07, 0x06, 0xb3, 0x7a, 0xc1, 0x13, 0xcd, 0xfa,
-	0x75, 0x3a, 0xaf, 0x67, 0x98, 0x90, 0xde, 0x0b, 0xcc, 0x1c, 0xd0, 0xbe, 0xd1, 0x4b, 0x9e, 0xb8,
-	0x3b, 0x5a, 0xd9, 0xec, 0x3e, 0x5e, 0xa3, 0xe5, 0x3d, 0xc8, 0x75, 0x8f, 0xc1, 0xe4, 0x5f, 0x59,
-	0x82, 0x6e, 0xff, 0xad, 0xc0, 0x93, 0x07, 0x47, 0x8c, 0xbe, 0x00, 0x75, 0xb6, 0x8d, 0x44, 0x93,
-	0x14, 0x2e, 0x8e, 0x1c, 0x1e, 0xa1, 0x2f, 0xa1, 0xe0, 0xfa, 0x51, 0xa3, 0xcd, 0xcf, 0x98, 0xa5,
-	0x45, 0x28, 0xf3, 0xad, 0x26, 0x3f, 0xa2, 0x38, 0xdf, 0x6a, 0xca, 0x7c, 0xfb, 0x84, 0x17, 0x38,
-	0x2f, 0xf3, 0xed, 0x13, 0x96, 0xbf, 0xf1, 0x28, 0x16, 0x6f, 0x75, 0x8e, 0xe5, 0x79, 0x88, 0x2a,
-	0xa0, 0x2d, 0xe8, 0x86, 0x7d, 0x8b, 0xb0, 0x1d, 0x2b, 0x17, 0x47, 0x8e, 0x8c, 0xd9, 0x93, 0x75,
-	0x14, 0xba, 0xfe, 0x92, 0x5f, 0x7d, 0x06, 0x7b, 0x22, 0x62, 0xf4, 0x0c, 0xf4, 0x19, 0xa5, 0x1e,
-	0xc1, 0x7e, 0xa5, 0xc8, 0x3e, 0x60, 0xd8, 0xb5, 0x29, 0x13, 0x6c, 0x1e, 0xb6, 0xee, 0x75, 0xc5,
-	0x60, 0x0e, 0xc8, 0xe6, 0xe1, 0x61, 0x57, 0x97, 0x2f, 0xab, 0x7d, 0x0d, 0xe5, 0x7f, 0x37, 0x21,
-	0x3a, 0x07, 0x5d, 0xde, 0x2b, 0x87, 0xbd, 0x7c, 0xb1, 0xda, 0x7e, 0x07, 0x4f, 0xff, 0xd3, 0x86,
-	0x8f, 0x47, 0xbf, 0x86, 0xd2, 0x7e, 0x2b, 0xa2, 0x3e, 0xe8, 0xc4, 0x8f, 0x42, 0x37, 0xf1, 0x8c,
-	0x0c, 0xcd, 0xdc, 0xf3, 0xa3, 0x70, 0xeb, 0xc4, 0x7a, 0xfb, 0x83, 0x92, 0xb0, 0xf9, 0x13, 0xf4,
-	0x23, 0xe4, 0x6f, 0xc9, 0x56, 0xfa, 0x45, 0xc6, 0x05, 0x33, 0xe5, 0xce, 0x1b, 0x73, 0x9f, 0xe1,
-	0x8d, 0x6f, 0xc1, 0x7a, 0xd8, 0xdc, 0x3b, 0xb0, 0x72, 0x38, 0xf8, 0xf9, 0xef, 0x0a, 0xc0, 0xae,
-	0xfd, 0xd1, 0x13, 0x30, 0x5f, 0x0f, 0xc7, 0x57, 0xbd, 0xd3, 0xfe, 0x59, 0xbf, 0xf7, 0xca, 0x3a,
-	0x42, 0x45, 0x50, 0xbb, 0xd7, 0x93, 0x9e, 0xa5, 0x20, 0x03, 0x0a, 0xfd, 0xe1, 0xa4, 0xd1, 0xb6,
-	0x72, 0x72, 0xd8, 0x6a, 0x5a, 0x79, 0x39, 0x6c, 0x9f, 0x58, 0x2a, 0x1b, 0x9e, 0x0d, 0x46, 0x9d,
-	0x89, 0x55, 0x40, 0x00, 0xda, 0xab, 0xd1, 0xeb, 0xee, 0xa0, 0x67, 0x69, 0x6c, 0x3c, 0x9e, 0x38,
-	0xfd, 0xe1, 0xb9, 0xa5, 0x23, 0x13, 0xf4, 0xee, 0x68, 0x34, 0xe8, 0x75, 0x86, 0x56, 0x91, 0xfd,
-	0x9e, 0xa1, 0xc7, 0x96, 0xd1, 0xfd, 0x01, 0x3e, 0xfd, 0x0f, 0xab, 0x6b, 0x88, 0xaf, 0x80, 0x4e,
-	0xe0, 0xfe, 0x62, 0xc6, 0xf9, 0xe9, 0x5d, 0x63, 0xa6, 0xf1, 0xff, 0x68, 0xad, 0x7f, 0x02, 0x00,
-	0x00, 0xff, 0xff, 0x93, 0x02, 0x87, 0xb0, 0xb3, 0x0d, 0x00, 0x00,
+	type x struct{}
+	out := protoimpl.TypeBuilder{
+		File: protoimpl.DescBuilder{
+			GoPackagePath: reflect.TypeOf(x{}).PkgPath(),
+			RawDescriptor: file_schema_proto_rawDesc,
+			NumEnums:      1,
+			NumMessages:   17,
+			NumExtensions: 0,
+			NumServices:   0,
+		},
+		GoTypes:           file_schema_proto_goTypes,
+		DependencyIndexes: file_schema_proto_depIdxs,
+		EnumInfos:         file_schema_proto_enumTypes,
+		MessageInfos:      file_schema_proto_msgTypes,
+	}.Build()
+	File_schema_proto = out.File
+	file_schema_proto_rawDesc = nil
+	file_schema_proto_goTypes = nil
+	file_schema_proto_depIdxs = nil
 }
diff --git a/sdks/go/pkg/beam/model/pipeline_v1/standard_window_fns.pb.go b/sdks/go/pkg/beam/model/pipeline_v1/standard_window_fns.pb.go
index 58a2a8b..30c9204 100644
--- a/sdks/go/pkg/beam/model/pipeline_v1/standard_window_fns.pb.go
+++ b/sdks/go/pkg/beam/model/pipeline_v1/standard_window_fns.pb.go
@@ -1,26 +1,47 @@
+//
+// 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.
+
+//
+// Protocol Buffers describing the Runner API, which is the runner-independent,
+// SDK-independent definition of the Beam model.
+
 // Code generated by protoc-gen-go. DO NOT EDIT.
+// versions:
+// 	protoc-gen-go v1.25.0-devel
+// 	protoc        v3.13.0
 // source: standard_window_fns.proto
 
 package pipeline_v1
 
 import (
-	fmt "fmt"
-	proto "github.com/golang/protobuf/proto"
 	duration "github.com/golang/protobuf/ptypes/duration"
 	timestamp "github.com/golang/protobuf/ptypes/timestamp"
-	math "math"
+	protoreflect "google.golang.org/protobuf/reflect/protoreflect"
+	protoimpl "google.golang.org/protobuf/runtime/protoimpl"
+	reflect "reflect"
+	sync "sync"
 )
 
-// Reference imports to suppress errors if they are not otherwise used.
-var _ = proto.Marshal
-var _ = fmt.Errorf
-var _ = math.Inf
-
-// This is a compile-time assertion to ensure that this generated file
-// is compatible with the proto package it is being compiled against.
-// A compilation error at this line likely means your copy of the
-// proto package needs to be updated.
-const _ = proto.ProtoPackageIsVersion3 // please upgrade the proto package
+const (
+	// Verify that this generated code is sufficiently up-to-date.
+	_ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion)
+	// Verify that runtime/protoimpl is sufficiently up-to-date.
+	_ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20)
+)
 
 type GlobalWindowsPayload_Enum int32
 
@@ -28,20 +49,41 @@
 	GlobalWindowsPayload_PROPERTIES GlobalWindowsPayload_Enum = 0
 )
 
-var GlobalWindowsPayload_Enum_name = map[int32]string{
-	0: "PROPERTIES",
-}
+// Enum value maps for GlobalWindowsPayload_Enum.
+var (
+	GlobalWindowsPayload_Enum_name = map[int32]string{
+		0: "PROPERTIES",
+	}
+	GlobalWindowsPayload_Enum_value = map[string]int32{
+		"PROPERTIES": 0,
+	}
+)
 
-var GlobalWindowsPayload_Enum_value = map[string]int32{
-	"PROPERTIES": 0,
+func (x GlobalWindowsPayload_Enum) Enum() *GlobalWindowsPayload_Enum {
+	p := new(GlobalWindowsPayload_Enum)
+	*p = x
+	return p
 }
 
 func (x GlobalWindowsPayload_Enum) String() string {
-	return proto.EnumName(GlobalWindowsPayload_Enum_name, int32(x))
+	return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x))
 }
 
+func (GlobalWindowsPayload_Enum) Descriptor() protoreflect.EnumDescriptor {
+	return file_standard_window_fns_proto_enumTypes[0].Descriptor()
+}
+
+func (GlobalWindowsPayload_Enum) Type() protoreflect.EnumType {
+	return &file_standard_window_fns_proto_enumTypes[0]
+}
+
+func (x GlobalWindowsPayload_Enum) Number() protoreflect.EnumNumber {
+	return protoreflect.EnumNumber(x)
+}
+
+// Deprecated: Use GlobalWindowsPayload_Enum.Descriptor instead.
 func (GlobalWindowsPayload_Enum) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_fab9dd76b0d0d680, []int{0, 0}
+	return file_standard_window_fns_proto_rawDescGZIP(), []int{0, 0}
 }
 
 type FixedWindowsPayload_Enum int32
@@ -50,20 +92,41 @@
 	FixedWindowsPayload_PROPERTIES FixedWindowsPayload_Enum = 0
 )
 
-var FixedWindowsPayload_Enum_name = map[int32]string{
-	0: "PROPERTIES",
-}
+// Enum value maps for FixedWindowsPayload_Enum.
+var (
+	FixedWindowsPayload_Enum_name = map[int32]string{
+		0: "PROPERTIES",
+	}
+	FixedWindowsPayload_Enum_value = map[string]int32{
+		"PROPERTIES": 0,
+	}
+)
 
-var FixedWindowsPayload_Enum_value = map[string]int32{
-	"PROPERTIES": 0,
+func (x FixedWindowsPayload_Enum) Enum() *FixedWindowsPayload_Enum {
+	p := new(FixedWindowsPayload_Enum)
+	*p = x
+	return p
 }
 
 func (x FixedWindowsPayload_Enum) String() string {
-	return proto.EnumName(FixedWindowsPayload_Enum_name, int32(x))
+	return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x))
 }
 
+func (FixedWindowsPayload_Enum) Descriptor() protoreflect.EnumDescriptor {
+	return file_standard_window_fns_proto_enumTypes[1].Descriptor()
+}
+
+func (FixedWindowsPayload_Enum) Type() protoreflect.EnumType {
+	return &file_standard_window_fns_proto_enumTypes[1]
+}
+
+func (x FixedWindowsPayload_Enum) Number() protoreflect.EnumNumber {
+	return protoreflect.EnumNumber(x)
+}
+
+// Deprecated: Use FixedWindowsPayload_Enum.Descriptor instead.
 func (FixedWindowsPayload_Enum) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_fab9dd76b0d0d680, []int{1, 0}
+	return file_standard_window_fns_proto_rawDescGZIP(), []int{1, 0}
 }
 
 type SlidingWindowsPayload_Enum int32
@@ -72,20 +135,41 @@
 	SlidingWindowsPayload_PROPERTIES SlidingWindowsPayload_Enum = 0
 )
 
-var SlidingWindowsPayload_Enum_name = map[int32]string{
-	0: "PROPERTIES",
-}
+// Enum value maps for SlidingWindowsPayload_Enum.
+var (
+	SlidingWindowsPayload_Enum_name = map[int32]string{
+		0: "PROPERTIES",
+	}
+	SlidingWindowsPayload_Enum_value = map[string]int32{
+		"PROPERTIES": 0,
+	}
+)
 
-var SlidingWindowsPayload_Enum_value = map[string]int32{
-	"PROPERTIES": 0,
+func (x SlidingWindowsPayload_Enum) Enum() *SlidingWindowsPayload_Enum {
+	p := new(SlidingWindowsPayload_Enum)
+	*p = x
+	return p
 }
 
 func (x SlidingWindowsPayload_Enum) String() string {
-	return proto.EnumName(SlidingWindowsPayload_Enum_name, int32(x))
+	return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x))
 }
 
+func (SlidingWindowsPayload_Enum) Descriptor() protoreflect.EnumDescriptor {
+	return file_standard_window_fns_proto_enumTypes[2].Descriptor()
+}
+
+func (SlidingWindowsPayload_Enum) Type() protoreflect.EnumType {
+	return &file_standard_window_fns_proto_enumTypes[2]
+}
+
+func (x SlidingWindowsPayload_Enum) Number() protoreflect.EnumNumber {
+	return protoreflect.EnumNumber(x)
+}
+
+// Deprecated: Use SlidingWindowsPayload_Enum.Descriptor instead.
 func (SlidingWindowsPayload_Enum) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_fab9dd76b0d0d680, []int{2, 0}
+	return file_standard_window_fns_proto_rawDescGZIP(), []int{2, 0}
 }
 
 type SessionWindowsPayload_Enum int32
@@ -94,20 +178,41 @@
 	SessionWindowsPayload_PROPERTIES SessionWindowsPayload_Enum = 0
 )
 
-var SessionWindowsPayload_Enum_name = map[int32]string{
-	0: "PROPERTIES",
-}
+// Enum value maps for SessionWindowsPayload_Enum.
+var (
+	SessionWindowsPayload_Enum_name = map[int32]string{
+		0: "PROPERTIES",
+	}
+	SessionWindowsPayload_Enum_value = map[string]int32{
+		"PROPERTIES": 0,
+	}
+)
 
-var SessionWindowsPayload_Enum_value = map[string]int32{
-	"PROPERTIES": 0,
+func (x SessionWindowsPayload_Enum) Enum() *SessionWindowsPayload_Enum {
+	p := new(SessionWindowsPayload_Enum)
+	*p = x
+	return p
 }
 
 func (x SessionWindowsPayload_Enum) String() string {
-	return proto.EnumName(SessionWindowsPayload_Enum_name, int32(x))
+	return protoimpl.X.EnumStringOf(x.Descriptor(), protoreflect.EnumNumber(x))
 }
 
+func (SessionWindowsPayload_Enum) Descriptor() protoreflect.EnumDescriptor {
+	return file_standard_window_fns_proto_enumTypes[3].Descriptor()
+}
+
+func (SessionWindowsPayload_Enum) Type() protoreflect.EnumType {
+	return &file_standard_window_fns_proto_enumTypes[3]
+}
+
+func (x SessionWindowsPayload_Enum) Number() protoreflect.EnumNumber {
+	return protoreflect.EnumNumber(x)
+}
+
+// Deprecated: Use SessionWindowsPayload_Enum.Descriptor instead.
 func (SessionWindowsPayload_Enum) EnumDescriptor() ([]byte, []int) {
-	return fileDescriptor_fab9dd76b0d0d680, []int{3, 0}
+	return file_standard_window_fns_proto_rawDescGZIP(), []int{3, 0}
 }
 
 // By default, all data in a PCollection is assigned to the single global
@@ -116,87 +221,102 @@
 // See https://beam.apache.org/documentation/programming-guide/#single-global-window
 // for additional details.
 type GlobalWindowsPayload struct {
-	XXX_NoUnkeyedLiteral struct{} `json:"-"`
-	XXX_unrecognized     []byte   `json:"-"`
-	XXX_sizecache        int32    `json:"-"`
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
 }
 
-func (m *GlobalWindowsPayload) Reset()         { *m = GlobalWindowsPayload{} }
-func (m *GlobalWindowsPayload) String() string { return proto.CompactTextString(m) }
-func (*GlobalWindowsPayload) ProtoMessage()    {}
+func (x *GlobalWindowsPayload) Reset() {
+	*x = GlobalWindowsPayload{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_standard_window_fns_proto_msgTypes[0]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *GlobalWindowsPayload) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*GlobalWindowsPayload) ProtoMessage() {}
+
+func (x *GlobalWindowsPayload) ProtoReflect() protoreflect.Message {
+	mi := &file_standard_window_fns_proto_msgTypes[0]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use GlobalWindowsPayload.ProtoReflect.Descriptor instead.
 func (*GlobalWindowsPayload) Descriptor() ([]byte, []int) {
-	return fileDescriptor_fab9dd76b0d0d680, []int{0}
+	return file_standard_window_fns_proto_rawDescGZIP(), []int{0}
 }
 
-func (m *GlobalWindowsPayload) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_GlobalWindowsPayload.Unmarshal(m, b)
-}
-func (m *GlobalWindowsPayload) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_GlobalWindowsPayload.Marshal(b, m, deterministic)
-}
-func (m *GlobalWindowsPayload) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_GlobalWindowsPayload.Merge(m, src)
-}
-func (m *GlobalWindowsPayload) XXX_Size() int {
-	return xxx_messageInfo_GlobalWindowsPayload.Size(m)
-}
-func (m *GlobalWindowsPayload) XXX_DiscardUnknown() {
-	xxx_messageInfo_GlobalWindowsPayload.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_GlobalWindowsPayload proto.InternalMessageInfo
-
 // A fixed time window represents a consistent duration size, non overlapping
 // time interval in the data stream.
 //
 // See https://beam.apache.org/documentation/programming-guide/#fixed-time-windows
 // for additional details.
 type FixedWindowsPayload struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
 	// (Required) Represents the size of the window.
 	Size *duration.Duration `protobuf:"bytes,1,opt,name=size,proto3" json:"size,omitempty"`
 	// (Required) Represents the timestamp of when the first window begins.
 	// Window N will start at offset + N * size.
-	Offset               *timestamp.Timestamp `protobuf:"bytes,2,opt,name=offset,proto3" json:"offset,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}             `json:"-"`
-	XXX_unrecognized     []byte               `json:"-"`
-	XXX_sizecache        int32                `json:"-"`
+	Offset *timestamp.Timestamp `protobuf:"bytes,2,opt,name=offset,proto3" json:"offset,omitempty"`
 }
 
-func (m *FixedWindowsPayload) Reset()         { *m = FixedWindowsPayload{} }
-func (m *FixedWindowsPayload) String() string { return proto.CompactTextString(m) }
-func (*FixedWindowsPayload) ProtoMessage()    {}
+func (x *FixedWindowsPayload) Reset() {
+	*x = FixedWindowsPayload{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_standard_window_fns_proto_msgTypes[1]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *FixedWindowsPayload) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*FixedWindowsPayload) ProtoMessage() {}
+
+func (x *FixedWindowsPayload) ProtoReflect() protoreflect.Message {
+	mi := &file_standard_window_fns_proto_msgTypes[1]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use FixedWindowsPayload.ProtoReflect.Descriptor instead.
 func (*FixedWindowsPayload) Descriptor() ([]byte, []int) {
-	return fileDescriptor_fab9dd76b0d0d680, []int{1}
+	return file_standard_window_fns_proto_rawDescGZIP(), []int{1}
 }
 
-func (m *FixedWindowsPayload) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_FixedWindowsPayload.Unmarshal(m, b)
-}
-func (m *FixedWindowsPayload) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_FixedWindowsPayload.Marshal(b, m, deterministic)
-}
-func (m *FixedWindowsPayload) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_FixedWindowsPayload.Merge(m, src)
-}
-func (m *FixedWindowsPayload) XXX_Size() int {
-	return xxx_messageInfo_FixedWindowsPayload.Size(m)
-}
-func (m *FixedWindowsPayload) XXX_DiscardUnknown() {
-	xxx_messageInfo_FixedWindowsPayload.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_FixedWindowsPayload proto.InternalMessageInfo
-
-func (m *FixedWindowsPayload) GetSize() *duration.Duration {
-	if m != nil {
-		return m.Size
+func (x *FixedWindowsPayload) GetSize() *duration.Duration {
+	if x != nil {
+		return x.Size
 	}
 	return nil
 }
 
-func (m *FixedWindowsPayload) GetOffset() *timestamp.Timestamp {
-	if m != nil {
-		return m.Offset
+func (x *FixedWindowsPayload) GetOffset() *timestamp.Timestamp {
+	if x != nil {
+		return x.Offset
 	}
 	return nil
 }
@@ -216,60 +336,68 @@
 // See https://beam.apache.org/documentation/programming-guide/#sliding-time-windows
 // for additional details.
 type SlidingWindowsPayload struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
 	// (Required) Represents the size of the window.
 	Size *duration.Duration `protobuf:"bytes,1,opt,name=size,proto3" json:"size,omitempty"`
 	// (Required) Represents the timestamp of when the first window begins.
 	// Window N will start at offset + N * period.
 	Offset *timestamp.Timestamp `protobuf:"bytes,2,opt,name=offset,proto3" json:"offset,omitempty"`
 	// (Required) Represents the amount of time between each start of a window.
-	Period               *duration.Duration `protobuf:"bytes,3,opt,name=period,proto3" json:"period,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}           `json:"-"`
-	XXX_unrecognized     []byte             `json:"-"`
-	XXX_sizecache        int32              `json:"-"`
+	Period *duration.Duration `protobuf:"bytes,3,opt,name=period,proto3" json:"period,omitempty"`
 }
 
-func (m *SlidingWindowsPayload) Reset()         { *m = SlidingWindowsPayload{} }
-func (m *SlidingWindowsPayload) String() string { return proto.CompactTextString(m) }
-func (*SlidingWindowsPayload) ProtoMessage()    {}
+func (x *SlidingWindowsPayload) Reset() {
+	*x = SlidingWindowsPayload{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_standard_window_fns_proto_msgTypes[2]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *SlidingWindowsPayload) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*SlidingWindowsPayload) ProtoMessage() {}
+
+func (x *SlidingWindowsPayload) ProtoReflect() protoreflect.Message {
+	mi := &file_standard_window_fns_proto_msgTypes[2]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use SlidingWindowsPayload.ProtoReflect.Descriptor instead.
 func (*SlidingWindowsPayload) Descriptor() ([]byte, []int) {
-	return fileDescriptor_fab9dd76b0d0d680, []int{2}
+	return file_standard_window_fns_proto_rawDescGZIP(), []int{2}
 }
 
-func (m *SlidingWindowsPayload) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_SlidingWindowsPayload.Unmarshal(m, b)
-}
-func (m *SlidingWindowsPayload) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_SlidingWindowsPayload.Marshal(b, m, deterministic)
-}
-func (m *SlidingWindowsPayload) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_SlidingWindowsPayload.Merge(m, src)
-}
-func (m *SlidingWindowsPayload) XXX_Size() int {
-	return xxx_messageInfo_SlidingWindowsPayload.Size(m)
-}
-func (m *SlidingWindowsPayload) XXX_DiscardUnknown() {
-	xxx_messageInfo_SlidingWindowsPayload.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_SlidingWindowsPayload proto.InternalMessageInfo
-
-func (m *SlidingWindowsPayload) GetSize() *duration.Duration {
-	if m != nil {
-		return m.Size
+func (x *SlidingWindowsPayload) GetSize() *duration.Duration {
+	if x != nil {
+		return x.Size
 	}
 	return nil
 }
 
-func (m *SlidingWindowsPayload) GetOffset() *timestamp.Timestamp {
-	if m != nil {
-		return m.Offset
+func (x *SlidingWindowsPayload) GetOffset() *timestamp.Timestamp {
+	if x != nil {
+		return x.Offset
 	}
 	return nil
 }
 
-func (m *SlidingWindowsPayload) GetPeriod() *duration.Duration {
-	if m != nil {
-		return m.Period
+func (x *SlidingWindowsPayload) GetPeriod() *duration.Duration {
+	if x != nil {
+		return x.Period
 	}
 	return nil
 }
@@ -285,84 +413,231 @@
 // See https://beam.apache.org/documentation/programming-guide/#session-windows
 // for additional details.
 type SessionWindowsPayload struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
 	// (Required) Minimum duration of gaps between sessions.
-	GapSize              *duration.Duration `protobuf:"bytes,1,opt,name=gap_size,json=gapSize,proto3" json:"gap_size,omitempty"`
-	XXX_NoUnkeyedLiteral struct{}           `json:"-"`
-	XXX_unrecognized     []byte             `json:"-"`
-	XXX_sizecache        int32              `json:"-"`
+	GapSize *duration.Duration `protobuf:"bytes,1,opt,name=gap_size,json=gapSize,proto3" json:"gap_size,omitempty"`
 }
 
-func (m *SessionWindowsPayload) Reset()         { *m = SessionWindowsPayload{} }
-func (m *SessionWindowsPayload) String() string { return proto.CompactTextString(m) }
-func (*SessionWindowsPayload) ProtoMessage()    {}
+func (x *SessionWindowsPayload) Reset() {
+	*x = SessionWindowsPayload{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_standard_window_fns_proto_msgTypes[3]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *SessionWindowsPayload) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*SessionWindowsPayload) ProtoMessage() {}
+
+func (x *SessionWindowsPayload) ProtoReflect() protoreflect.Message {
+	mi := &file_standard_window_fns_proto_msgTypes[3]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use SessionWindowsPayload.ProtoReflect.Descriptor instead.
 func (*SessionWindowsPayload) Descriptor() ([]byte, []int) {
-	return fileDescriptor_fab9dd76b0d0d680, []int{3}
+	return file_standard_window_fns_proto_rawDescGZIP(), []int{3}
 }
 
-func (m *SessionWindowsPayload) XXX_Unmarshal(b []byte) error {
-	return xxx_messageInfo_SessionWindowsPayload.Unmarshal(m, b)
-}
-func (m *SessionWindowsPayload) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) {
-	return xxx_messageInfo_SessionWindowsPayload.Marshal(b, m, deterministic)
-}
-func (m *SessionWindowsPayload) XXX_Merge(src proto.Message) {
-	xxx_messageInfo_SessionWindowsPayload.Merge(m, src)
-}
-func (m *SessionWindowsPayload) XXX_Size() int {
-	return xxx_messageInfo_SessionWindowsPayload.Size(m)
-}
-func (m *SessionWindowsPayload) XXX_DiscardUnknown() {
-	xxx_messageInfo_SessionWindowsPayload.DiscardUnknown(m)
-}
-
-var xxx_messageInfo_SessionWindowsPayload proto.InternalMessageInfo
-
-func (m *SessionWindowsPayload) GetGapSize() *duration.Duration {
-	if m != nil {
-		return m.GapSize
+func (x *SessionWindowsPayload) GetGapSize() *duration.Duration {
+	if x != nil {
+		return x.GapSize
 	}
 	return nil
 }
 
-func init() {
-	proto.RegisterEnum("org.apache.beam.model.pipeline.v1.GlobalWindowsPayload_Enum", GlobalWindowsPayload_Enum_name, GlobalWindowsPayload_Enum_value)
-	proto.RegisterEnum("org.apache.beam.model.pipeline.v1.FixedWindowsPayload_Enum", FixedWindowsPayload_Enum_name, FixedWindowsPayload_Enum_value)
-	proto.RegisterEnum("org.apache.beam.model.pipeline.v1.SlidingWindowsPayload_Enum", SlidingWindowsPayload_Enum_name, SlidingWindowsPayload_Enum_value)
-	proto.RegisterEnum("org.apache.beam.model.pipeline.v1.SessionWindowsPayload_Enum", SessionWindowsPayload_Enum_name, SessionWindowsPayload_Enum_value)
-	proto.RegisterType((*GlobalWindowsPayload)(nil), "org.apache.beam.model.pipeline.v1.GlobalWindowsPayload")
-	proto.RegisterType((*FixedWindowsPayload)(nil), "org.apache.beam.model.pipeline.v1.FixedWindowsPayload")
-	proto.RegisterType((*SlidingWindowsPayload)(nil), "org.apache.beam.model.pipeline.v1.SlidingWindowsPayload")
-	proto.RegisterType((*SessionWindowsPayload)(nil), "org.apache.beam.model.pipeline.v1.SessionWindowsPayload")
+var File_standard_window_fns_proto protoreflect.FileDescriptor
+
+var file_standard_window_fns_proto_rawDesc = []byte{
+	0x0a, 0x19, 0x73, 0x74, 0x61, 0x6e, 0x64, 0x61, 0x72, 0x64, 0x5f, 0x77, 0x69, 0x6e, 0x64, 0x6f,
+	0x77, 0x5f, 0x66, 0x6e, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x21, 0x6f, 0x72, 0x67,
+	0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64,
+	0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x1a, 0x15,
+	0x62, 0x65, 0x61, 0x6d, 0x5f, 0x72, 0x75, 0x6e, 0x6e, 0x65, 0x72, 0x5f, 0x61, 0x70, 0x69, 0x2e,
+	0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x1e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2f, 0x70, 0x72,
+	0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x64, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x2e,
+	0x70, 0x72, 0x6f, 0x74, 0x6f, 0x1a, 0x1f, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2f, 0x70, 0x72,
+	0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70,
+	0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0x56, 0x0a, 0x14, 0x47, 0x6c, 0x6f, 0x62, 0x61, 0x6c,
+	0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x73, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x22, 0x3e,
+	0x0a, 0x04, 0x45, 0x6e, 0x75, 0x6d, 0x12, 0x36, 0x0a, 0x0a, 0x50, 0x52, 0x4f, 0x50, 0x45, 0x52,
+	0x54, 0x49, 0x45, 0x53, 0x10, 0x00, 0x1a, 0x26, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x20, 0x62, 0x65,
+	0x61, 0x6d, 0x3a, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x5f, 0x66, 0x6e, 0x3a, 0x67, 0x6c, 0x6f,
+	0x62, 0x61, 0x6c, 0x5f, 0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x73, 0x3a, 0x76, 0x31, 0x22, 0xb7,
+	0x01, 0x0a, 0x13, 0x46, 0x69, 0x78, 0x65, 0x64, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x73, 0x50,
+	0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x2d, 0x0a, 0x04, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x01,
+	0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72,
+	0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52,
+	0x04, 0x73, 0x69, 0x7a, 0x65, 0x12, 0x32, 0x0a, 0x06, 0x6f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x18,
+	0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70,
+	0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d,
+	0x70, 0x52, 0x06, 0x6f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x22, 0x3d, 0x0a, 0x04, 0x45, 0x6e, 0x75,
+	0x6d, 0x12, 0x35, 0x0a, 0x0a, 0x50, 0x52, 0x4f, 0x50, 0x45, 0x52, 0x54, 0x49, 0x45, 0x53, 0x10,
+	0x00, 0x1a, 0x25, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x1f, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x77, 0x69,
+	0x6e, 0x64, 0x6f, 0x77, 0x5f, 0x66, 0x6e, 0x3a, 0x66, 0x69, 0x78, 0x65, 0x64, 0x5f, 0x77, 0x69,
+	0x6e, 0x64, 0x6f, 0x77, 0x73, 0x3a, 0x76, 0x31, 0x22, 0xee, 0x01, 0x0a, 0x15, 0x53, 0x6c, 0x69,
+	0x64, 0x69, 0x6e, 0x67, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x73, 0x50, 0x61, 0x79, 0x6c, 0x6f,
+	0x61, 0x64, 0x12, 0x2d, 0x0a, 0x04, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0b,
+	0x32, 0x19, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62,
+	0x75, 0x66, 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x04, 0x73, 0x69, 0x7a,
+	0x65, 0x12, 0x32, 0x0a, 0x06, 0x6f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28,
+	0x0b, 0x32, 0x1a, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f,
+	0x62, 0x75, 0x66, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x52, 0x06, 0x6f,
+	0x66, 0x66, 0x73, 0x65, 0x74, 0x12, 0x31, 0x0a, 0x06, 0x70, 0x65, 0x72, 0x69, 0x6f, 0x64, 0x18,
+	0x03, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70,
+	0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e,
+	0x52, 0x06, 0x70, 0x65, 0x72, 0x69, 0x6f, 0x64, 0x22, 0x3f, 0x0a, 0x04, 0x45, 0x6e, 0x75, 0x6d,
+	0x12, 0x37, 0x0a, 0x0a, 0x50, 0x52, 0x4f, 0x50, 0x45, 0x52, 0x54, 0x49, 0x45, 0x53, 0x10, 0x00,
+	0x1a, 0x27, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x21, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x77, 0x69, 0x6e,
+	0x64, 0x6f, 0x77, 0x5f, 0x66, 0x6e, 0x3a, 0x73, 0x6c, 0x69, 0x64, 0x69, 0x6e, 0x67, 0x5f, 0x77,
+	0x69, 0x6e, 0x64, 0x6f, 0x77, 0x73, 0x3a, 0x76, 0x31, 0x22, 0x8e, 0x01, 0x0a, 0x15, 0x53, 0x65,
+	0x73, 0x73, 0x69, 0x6f, 0x6e, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x73, 0x50, 0x61, 0x79, 0x6c,
+	0x6f, 0x61, 0x64, 0x12, 0x34, 0x0a, 0x08, 0x67, 0x61, 0x70, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18,
+	0x01, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x19, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70,
+	0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x44, 0x75, 0x72, 0x61, 0x74, 0x69, 0x6f, 0x6e,
+	0x52, 0x07, 0x67, 0x61, 0x70, 0x53, 0x69, 0x7a, 0x65, 0x22, 0x3f, 0x0a, 0x04, 0x45, 0x6e, 0x75,
+	0x6d, 0x12, 0x37, 0x0a, 0x0a, 0x50, 0x52, 0x4f, 0x50, 0x45, 0x52, 0x54, 0x49, 0x45, 0x53, 0x10,
+	0x00, 0x1a, 0x27, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x21, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x77, 0x69,
+	0x6e, 0x64, 0x6f, 0x77, 0x5f, 0x66, 0x6e, 0x3a, 0x73, 0x65, 0x73, 0x73, 0x69, 0x6f, 0x6e, 0x5f,
+	0x77, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x73, 0x3a, 0x76, 0x31, 0x42, 0x7d, 0x0a, 0x21, 0x6f, 0x72,
+	0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f,
+	0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x42,
+	0x11, 0x53, 0x74, 0x61, 0x6e, 0x64, 0x61, 0x72, 0x64, 0x57, 0x69, 0x6e, 0x64, 0x6f, 0x77, 0x46,
+	0x6e, 0x73, 0x5a, 0x45, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x61,
+	0x70, 0x61, 0x63, 0x68, 0x65, 0x2f, 0x62, 0x65, 0x61, 0x6d, 0x2f, 0x73, 0x64, 0x6b, 0x73, 0x2f,
+	0x67, 0x6f, 0x2f, 0x70, 0x6b, 0x67, 0x2f, 0x62, 0x65, 0x61, 0x6d, 0x2f, 0x6d, 0x6f, 0x64, 0x65,
+	0x6c, 0x2f, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x5f, 0x76, 0x31, 0x3b, 0x70, 0x69,
+	0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x5f, 0x76, 0x31, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f,
+	0x33,
 }
 
-func init() { proto.RegisterFile("standard_window_fns.proto", fileDescriptor_fab9dd76b0d0d680) }
+var (
+	file_standard_window_fns_proto_rawDescOnce sync.Once
+	file_standard_window_fns_proto_rawDescData = file_standard_window_fns_proto_rawDesc
+)
 
-var fileDescriptor_fab9dd76b0d0d680 = []byte{
-	// 404 bytes of a gzipped FileDescriptorProto
-	0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xbc, 0x93, 0x4d, 0x4e, 0xdb, 0x40,
-	0x14, 0xc7, 0xeb, 0x36, 0x4d, 0xab, 0xc9, 0xa6, 0x75, 0x1b, 0x29, 0xf1, 0xa2, 0x49, 0x2c, 0xf5,
-	0x63, 0xd3, 0x89, 0x9c, 0x7e, 0x49, 0x91, 0xda, 0x4a, 0x81, 0x04, 0xb1, 0x22, 0xb2, 0x23, 0x90,
-	0xd8, 0x58, 0x63, 0x66, 0x6c, 0x46, 0xb2, 0x67, 0x46, 0x1e, 0x3b, 0x81, 0x1c, 0x82, 0x43, 0x70,
-	0x06, 0x24, 0xf6, 0x1c, 0x88, 0x03, 0xb0, 0x42, 0x1e, 0x4f, 0x50, 0x30, 0x8b, 0xc0, 0x86, 0xe5,
-	0xcc, 0xfb, 0xbf, 0xf7, 0x7e, 0xbf, 0xc5, 0x03, 0x6d, 0x99, 0x21, 0x86, 0x51, 0x8a, 0xfd, 0x05,
-	0x65, 0x98, 0x2f, 0xfc, 0x90, 0x49, 0x28, 0x52, 0x9e, 0x71, 0xb3, 0xc7, 0xd3, 0x08, 0x22, 0x81,
-	0x8e, 0x8e, 0x09, 0x0c, 0x08, 0x4a, 0x60, 0xc2, 0x31, 0x89, 0xa1, 0xa0, 0x82, 0xc4, 0x94, 0x11,
-	0x38, 0x77, 0xac, 0x66, 0xf1, 0xef, 0xa7, 0x39, 0x63, 0x24, 0xf5, 0x91, 0xa0, 0x65, 0xa7, 0xf5,
-	0x29, 0xe2, 0x3c, 0x8a, 0x49, 0x5f, 0xbd, 0x82, 0x3c, 0xec, 0xe3, 0x3c, 0x45, 0x19, 0xe5, 0x4c,
-	0xd7, 0x3b, 0xd5, 0x7a, 0x46, 0x13, 0x22, 0x33, 0x94, 0x88, 0x32, 0x60, 0xef, 0x83, 0x8f, 0x3b,
-	0x31, 0x0f, 0x50, 0x7c, 0xa0, 0xa0, 0xe4, 0x14, 0x9d, 0xc6, 0x1c, 0x61, 0xfb, 0x1f, 0xa8, 0x8d,
-	0x59, 0x9e, 0x98, 0xbf, 0x01, 0x98, 0xba, 0x7b, 0xd3, 0xb1, 0x3b, 0xdb, 0x1d, 0x7b, 0xef, 0x5e,
-	0x58, 0x5f, 0xce, 0x2f, 0x6e, 0xae, 0x5e, 0x77, 0x0b, 0x9a, 0xe1, 0x9d, 0xc7, 0x30, 0x52, 0x23,
-	0xb4, 0x98, 0x1c, 0xce, 0x1d, 0xfb, 0xd2, 0x00, 0x1f, 0x26, 0xf4, 0x84, 0xe0, 0xfb, 0x73, 0xcd,
-	0xef, 0xa0, 0x26, 0xe9, 0x92, 0xb4, 0x8c, 0xae, 0xf1, 0xad, 0x31, 0x68, 0xc3, 0x92, 0x0f, 0xae,
-	0xf8, 0xe0, 0xb6, 0xe6, 0x77, 0x55, 0xcc, 0x1c, 0x80, 0x3a, 0x0f, 0x43, 0x49, 0xb2, 0xd6, 0x4b,
-	0xd5, 0x60, 0x3d, 0x68, 0x98, 0xad, 0x84, 0x5c, 0x9d, 0xb4, 0xff, 0x6a, 0xf4, 0x5f, 0x15, 0xf4,
-	0xcf, 0x0a, 0xbd, 0x53, 0x41, 0x0f, 0x0b, 0xc8, 0x75, 0xf2, 0x6b, 0x03, 0x34, 0xbd, 0x98, 0x62,
-	0xca, 0xa2, 0x67, 0x67, 0x37, 0x1d, 0x50, 0x17, 0x24, 0xa5, 0x1c, 0xb7, 0x5e, 0x6d, 0x5a, 0xa2,
-	0x83, 0xf6, 0x7f, 0xad, 0xfb, 0xa7, 0xa2, 0xfb, 0x55, 0xe9, 0xf6, 0x2a, 0xba, 0xb2, 0x34, 0x5b,
-	0x17, 0x3e, 0x2b, 0x84, 0x89, 0x94, 0x94, 0xb3, 0x8a, 0xf0, 0x4f, 0xf0, 0x36, 0x42, 0xc2, 0x7f,
-	0x9c, 0xf4, 0x9b, 0x08, 0x09, 0x8f, 0x2e, 0xc9, 0x53, 0x81, 0xca, 0xcd, 0x6b, 0x40, 0xa3, 0x2d,
-	0xb0, 0xf9, 0x20, 0x46, 0xef, 0x3d, 0x7d, 0x4e, 0x25, 0xf3, 0x84, 0xc9, 0xc3, 0xc6, 0xaa, 0xee,
-	0xcf, 0x9d, 0xa0, 0xae, 0x08, 0x7f, 0xdc, 0x06, 0x00, 0x00, 0xff, 0xff, 0xb9, 0x67, 0xf3, 0xc1,
-	0x77, 0x03, 0x00, 0x00,
+func file_standard_window_fns_proto_rawDescGZIP() []byte {
+	file_standard_window_fns_proto_rawDescOnce.Do(func() {
+		file_standard_window_fns_proto_rawDescData = protoimpl.X.CompressGZIP(file_standard_window_fns_proto_rawDescData)
+	})
+	return file_standard_window_fns_proto_rawDescData
+}
+
+var file_standard_window_fns_proto_enumTypes = make([]protoimpl.EnumInfo, 4)
+var file_standard_window_fns_proto_msgTypes = make([]protoimpl.MessageInfo, 4)
+var file_standard_window_fns_proto_goTypes = []interface{}{
+	(GlobalWindowsPayload_Enum)(0),  // 0: org.apache.beam.model.pipeline.v1.GlobalWindowsPayload.Enum
+	(FixedWindowsPayload_Enum)(0),   // 1: org.apache.beam.model.pipeline.v1.FixedWindowsPayload.Enum
+	(SlidingWindowsPayload_Enum)(0), // 2: org.apache.beam.model.pipeline.v1.SlidingWindowsPayload.Enum
+	(SessionWindowsPayload_Enum)(0), // 3: org.apache.beam.model.pipeline.v1.SessionWindowsPayload.Enum
+	(*GlobalWindowsPayload)(nil),    // 4: org.apache.beam.model.pipeline.v1.GlobalWindowsPayload
+	(*FixedWindowsPayload)(nil),     // 5: org.apache.beam.model.pipeline.v1.FixedWindowsPayload
+	(*SlidingWindowsPayload)(nil),   // 6: org.apache.beam.model.pipeline.v1.SlidingWindowsPayload
+	(*SessionWindowsPayload)(nil),   // 7: org.apache.beam.model.pipeline.v1.SessionWindowsPayload
+	(*duration.Duration)(nil),       // 8: google.protobuf.Duration
+	(*timestamp.Timestamp)(nil),     // 9: google.protobuf.Timestamp
+}
+var file_standard_window_fns_proto_depIdxs = []int32{
+	8, // 0: org.apache.beam.model.pipeline.v1.FixedWindowsPayload.size:type_name -> google.protobuf.Duration
+	9, // 1: org.apache.beam.model.pipeline.v1.FixedWindowsPayload.offset:type_name -> google.protobuf.Timestamp
+	8, // 2: org.apache.beam.model.pipeline.v1.SlidingWindowsPayload.size:type_name -> google.protobuf.Duration
+	9, // 3: org.apache.beam.model.pipeline.v1.SlidingWindowsPayload.offset:type_name -> google.protobuf.Timestamp
+	8, // 4: org.apache.beam.model.pipeline.v1.SlidingWindowsPayload.period:type_name -> google.protobuf.Duration
+	8, // 5: org.apache.beam.model.pipeline.v1.SessionWindowsPayload.gap_size:type_name -> google.protobuf.Duration
+	6, // [6:6] is the sub-list for method output_type
+	6, // [6:6] is the sub-list for method input_type
+	6, // [6:6] is the sub-list for extension type_name
+	6, // [6:6] is the sub-list for extension extendee
+	0, // [0:6] is the sub-list for field type_name
+}
+
+func init() { file_standard_window_fns_proto_init() }
+func file_standard_window_fns_proto_init() {
+	if File_standard_window_fns_proto != nil {
+		return
+	}
+	file_beam_runner_api_proto_init()
+	if !protoimpl.UnsafeEnabled {
+		file_standard_window_fns_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*GlobalWindowsPayload); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_standard_window_fns_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*FixedWindowsPayload); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_standard_window_fns_proto_msgTypes[2].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*SlidingWindowsPayload); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_standard_window_fns_proto_msgTypes[3].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*SessionWindowsPayload); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+	}
+	type x struct{}
+	out := protoimpl.TypeBuilder{
+		File: protoimpl.DescBuilder{
+			GoPackagePath: reflect.TypeOf(x{}).PkgPath(),
+			RawDescriptor: file_standard_window_fns_proto_rawDesc,
+			NumEnums:      4,
+			NumMessages:   4,
+			NumExtensions: 0,
+			NumServices:   0,
+		},
+		GoTypes:           file_standard_window_fns_proto_goTypes,
+		DependencyIndexes: file_standard_window_fns_proto_depIdxs,
+		EnumInfos:         file_standard_window_fns_proto_enumTypes,
+		MessageInfos:      file_standard_window_fns_proto_msgTypes,
+	}.Build()
+	File_standard_window_fns_proto = out.File
+	file_standard_window_fns_proto_rawDesc = nil
+	file_standard_window_fns_proto_goTypes = nil
+	file_standard_window_fns_proto_depIdxs = nil
 }
diff --git a/sdks/go/test/regression/coders/fromyaml/fromyaml.go b/sdks/go/test/regression/coders/fromyaml/fromyaml.go
index c7d67d3..3544a5f 100644
--- a/sdks/go/test/regression/coders/fromyaml/fromyaml.go
+++ b/sdks/go/test/regression/coders/fromyaml/fromyaml.go
@@ -44,6 +44,7 @@
 var unimplementedCoders = map[string]bool{
 	"beam:coder:param_windowed_value:v1": true,
 	"beam:coder:timer:v1":                true,
+	"beam:coder:sharded_key:v1":          true,
 }
 
 // Coder is a representation a serialized beam coder.
diff --git a/sdks/java/bom/build.gradle b/sdks/java/bom/build.gradle
index f462240..b8a71db 100644
--- a/sdks/java/bom/build.gradle
+++ b/sdks/java/bom/build.gradle
@@ -66,7 +66,15 @@
 // the pom.xml generation and have the publish tasks depend on `copyPom` instead.
 tasks.whenTaskAdded { task ->
   if (task.name == 'generatePomFileForMavenJavaPublication') {
-    task.enabled = false
+    // Ensures the pom file is signed later if we are performing a release (see BEAM-11068)
+    task.doLast {
+      copy {
+        from 'pom.xml.template'
+        into mavenJavaDir
+        rename 'pom.xml.template', 'pom-default.xml'
+        expand(version: project.version, modules: bomModuleNames)
+      }
+    }
   } else if (task.name.startsWith('publishMavenJavaPublication')) {
     task.dependsOn copyPom
   }
diff --git a/sdks/java/container/build.gradle b/sdks/java/container/build.gradle
index e0c46b8..042dcf6 100644
--- a/sdks/java/container/build.gradle
+++ b/sdks/java/container/build.gradle
@@ -24,6 +24,9 @@
 applyGoNature()
 applyDockerNature()
 
+evaluationDependsOn(":sdks:java:harness")
+evaluationDependsOn(":sdks:java:io:kafka")
+
 description = "Apache Beam :: SDKs :: Java :: Container"
 
 // Figure out why the golang plugin does not add a build dependency between projects.
@@ -74,7 +77,6 @@
 // command: ./gradlew generateLicenseReport
 import com.github.jk1.license.render.*
 licenseReport {
-  outputDir = 'java_third_party_licenses'
   projects = [project,
               project.rootProject.findProject(':sdks:java:harness'),
               project.rootProject.findProject(':sdks:java:io:kafka')]
@@ -102,30 +104,31 @@
   ])
 }
 
+task pullLicenses(type: Exec) {
+  commandLine './license_scripts/license_script.sh'
+  outputs.dir('build/target/third_party_licenses')
+  outputs.dir('build/target/java_third_party_licenses')
+}
+pullLicenses.dependsOn generateLicenseReport
+generateLicenseReport.outputs.cacheIf { false }
+
+task copyGolangLicenses(type: Copy) {
+  from "${project(':release:go-licenses:java').buildDir}/output"
+  into "build/target/go-licenses"
+  dependsOn ':release:go-licenses:java:createLicenses'
+}
+
+task skipPullLicenses(type: Exec) {
+  executable "sh"
+  args "-c", "mkdir -p build/target/third_party_licenses && touch build/target/third_party_licenses/skip"
+}
+
 if (project.rootProject.hasProperty(["docker-pull-licenses"]) ||
         project.rootProject.hasProperty(["isRelease"])) {
-  task pullLicenses(type: Exec) {
-    workingDir project.rootProject.projectDir
-    commandLine './sdks/java/container/license_scripts/license_script.sh'
-    doLast {
-      file('third_party_licenses').deleteDir()
-      file('java_third_party_licenses').deleteDir()
-    }
-  }
   dockerPrepare.dependsOn pullLicenses
-
-  task copyGolangLicenses(type: Copy) {
-    from "${project(':release:go-licenses:java').buildDir}/output"
-    into "build/target/go-licenses"
-    dependsOn ':release:go-licenses:java:createLicenses'
-  }
-  dockerPrepare.dependsOn 'copyGolangLicenses'
+  dockerPrepare.dependsOn copyGolangLicenses
 } else {
-  task createFile(type: Exec) {
-      executable "sh"
-      args "-c", "mkdir -p build/target/third_party_licenses && touch build/target/third_party_licenses/skip"
-  }
-  dockerPrepare.dependsOn createFile
+  dockerPrepare.dependsOn skipPullLicenses
 }
 
 // Ensure that we build the required resources and copy and file dependencies from related projects
diff --git a/sdks/java/container/license_scripts/license_script.sh b/sdks/java/container/license_scripts/license_script.sh
index 0088c5b..48025ce 100755
--- a/sdks/java/container/license_scripts/license_script.sh
+++ b/sdks/java/container/license_scripts/license_script.sh
@@ -16,13 +16,27 @@
 
 set -e
 
-ROOT=$(pwd)
-SCRIPT_DIR="${ROOT}/sdks/java/container/license_scripts"
-ENV_DIR="${ROOT}/sdks/java/container/build/virtualenv"
-LICENSE_DIR="${ROOT}/sdks/java/container/build/target/third_party_licenses"
+SCRIPT_DIR="${PWD}/license_scripts"
+ENV_DIR="${PWD}/build/virtualenv"
 
-# reports are generated at ~/beam/java_third_party_licenses
-./gradlew generateLicenseReport --rerun-tasks
+# This file must already exist before this helper script is run.
+# It is created by :sdks:java:container:generateLicenseReport
+INDEX_FILE="${PWD}/build/reports/dependency-license/index.json"
+
+# The licenses already pulled by generateDependencyReport are alongside index.json.
+# The script first copies those over.
+EXISTING_LICENSE_DIR="${PWD}/build/reports/dependency-license"
+
+# The python will download Java licenses here
+DOWNLOAD_DIR="${PWD}/build/target/java_third_party_licenses"
+
+# All licenses will be put here by this script
+DEST_DIR="${PWD}/build/target/third_party_licenses"
+
+echo "Copying already-fetched licenses from ${EXISTING_LICENSE_DIR} to ${DOWNLOAD_DIR}"
+if [ -d "$DOWNLOAD_DIR" ]; then rm -rf "$DOWNLOAD_DIR" ; fi
+mkdir -p "$DOWNLOAD_DIR"
+cp -r "${EXISTING_LICENSE_DIR}"/*.jar "${DOWNLOAD_DIR}"
 
 # activate virtualenv
 virtualenv --python=python3 ${ENV_DIR} && . ${ENV_DIR}/bin/activate
@@ -31,18 +45,20 @@
 ${ENV_DIR}/bin/pip install -r ${SCRIPT_DIR}/requirement.txt
 
 # pull licenses, notices and source code
-FLAGS="--license_dir=${ROOT}/java_third_party_licenses \
+FLAGS="--license_index=${INDEX_FILE} \
+       --output_dir=${DOWNLOAD_DIR} \
        --dep_url_yaml=${SCRIPT_DIR}/dep_urls_java.yaml "
 
 echo "Executing ${ENV_DIR}/bin/python ${SCRIPT_DIR}/pull_licenses_java.py $FLAGS"
-${ENV_DIR}/bin/python ${SCRIPT_DIR}/pull_licenses_java.py $FLAGS
+"${ENV_DIR}/bin/python" "${SCRIPT_DIR}/pull_licenses_java.py" $FLAGS
 
-if [ -d "$LICENSE_DIR" ]; then rm -rf $LICENSE_DIR; fi
-mkdir -p ${LICENSE_DIR}
-echo "Copy licenses to ${LICENSE_DIR}."
-cp -r ${ROOT}/java_third_party_licenses/*.jar ${LICENSE_DIR}/
-cp -r ${ROOT}/java_third_party_licenses/*.csv ${LICENSE_DIR}/
-gzip -r ${LICENSE_DIR}/*
+# If this script is running, it is assumed that outputs are out of date and should be cleared and rewritten
+if [ -d "$DEST_DIR" ]; then rm -rf "$DEST_DIR"; fi
+mkdir -p "$DEST_DIR"
 
-rm -rf ${ROOT}/java_third_party_licenses
+echo "Copying licenses from ${DOWNLOAD_DIR} to ${DEST_DIR}."
+cp -r "$DOWNLOAD_DIR"/*.jar "$DEST_DIR"/
+cp -r "$DOWNLOAD_DIR"/*.csv "$DEST_DIR"/
+gzip -r "$DEST_DIR"/*
+
 echo "Finished license_scripts.sh"
\ No newline at end of file
diff --git a/sdks/java/container/license_scripts/pull_licenses_java.py b/sdks/java/container/license_scripts/pull_licenses_java.py
index a587e30..3ca24bd 100644
--- a/sdks/java/container/license_scripts/pull_licenses_java.py
+++ b/sdks/java/container/license_scripts/pull_licenses_java.py
@@ -38,7 +38,7 @@
 from tenacity import wait_fixed
 from urllib.request import urlopen, URLError, HTTPError
 
-SOURCE_CODE_REQUIRED_LICENSES = ['lgpl', 'glp', 'cddl', 'mpl']
+SOURCE_CODE_REQUIRED_LICENSES = ['lgpl', 'glp', 'cddl', 'mpl', 'gnu', 'mozilla public license']
 RETRY_NUM = 9
 THREADS = 16
 
@@ -101,13 +101,22 @@
 
 def pull_source_code(base_url, dir_name, dep):
     # base_url example: https://repo1.maven.org/maven2/org/mortbay/jetty/jsp-2.1/6.1.14/
-    soup = BeautifulSoup(urlopen(base_url).read(), "html.parser")
+    try:
+      soup = BeautifulSoup(urlopen(base_url).read(), "html.parser")
+    except:
+      logging.error('Error reading source base from {base_url}'.format(base_url=base_url))
+      raise
+    source_count = 0
     for href in (a["href"] for a in soup.select("a[href]")):
         if href.endswith(
-                '.jar') and not 'javadoc' in href:  # download jar file only
+                '.jar') and 'sources.jar' in href:  # download sources jar file only
             file_name = dir_name + '/' + href
             url = base_url + '/' + href
+            logging.debug('Pulling source from {url}'.format(url=url))
             pull_from_url(file_name, url, dep, incorrect_source_url)
+            source_count = source_count + 1
+    if source_count == 0:
+      raise RuntimeError('No source found at {base_url}'.format(base_url=base_url))
 
 
 @retry(reraise=True, stop=stop_after_attempt(3))
@@ -115,8 +124,8 @@
     csv_columns = [
         'dependency_name', 'url_to_license', 'license_type', 'source_included'
     ]
-    csv_file = "{license_dir}/beam_java_dependency_list.csv".format(
-        license_dir=license_dir)
+    csv_file = "{output_dir}/beam_java_dependency_list.csv".format(
+        output_dir=output_dir)
     try:
         with open(csv_file, 'w') as csvfile:
             writer = csv.DictWriter(csvfile, fieldnames=csv_columns)
@@ -140,17 +149,22 @@
     }
     '''
 
-    name = dep['moduleName'].split(':')[1].lower()
+    name = dep['moduleName'].split(':')[1]
     version = dep['moduleVersion']
     name_version = name + '-' + version
-    dir_name = '{license_dir}/{name_version}.jar'.format(
-        license_dir=license_dir, name_version=name_version)
+    # javac is not a runtime dependency
+    if name == 'javac':
+      logging.debug('Skipping', name_version)
+      return
+    # skip self dependencies
+    if dep['moduleName'].lower().startswith('beam'):
+      logging.debug('Skipping', name_version)
+      return
+    dir_name = '{output_dir}/{name_version}.jar'.format(
+        output_dir=output_dir, name_version=name_version)
 
-    # if auto pulled, directory is existing at {license_dir}
+    # if auto pulled, directory is existing at {output_dir}
     if not os.path.isdir(dir_name):
-        # skip self dependencies
-        if dep['moduleName'].startswith('beam'):
-            logging.debug('Skippig', name_version)
         os.mkdir(dir_name)
         # pull license
         try:
@@ -217,17 +231,19 @@
 if __name__ == "__main__":
     start = datetime.now()
     parser = argparse.ArgumentParser()
-    parser.add_argument('--license_dir', required=True)
+    parser.add_argument('--license_index', required=True)
+    parser.add_argument('--output_dir', required=True)
     parser.add_argument('--dep_url_yaml', required=True)
 
     args = parser.parse_args()
-    license_dir = args.license_dir
+    license_index = args.license_index
+    output_dir = args.output_dir
     dep_url_yaml = args.dep_url_yaml
 
     logging.getLogger().setLevel(logging.INFO)
 
     # index.json is generated by Gradle plugin.
-    with open('{license_dir}/index.json'.format(license_dir=license_dir)) as f:
+    with open(license_index) as f:
         dependencies = json.load(f)
 
     with open(dep_url_yaml) as file:
@@ -261,6 +277,7 @@
                  'and "notice" (if available) field to {yaml_file} for each ' \
                  'missing license. Dependency List: [{dep_list}]'.format(
             dep_list=','.join(sorted(no_licenses)), yaml_file=dep_url_yaml)
+        logging.error(how_to)
         error_msg.append(how_to)
         run_status = 'failed'
 
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/Pipeline.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/Pipeline.java
index 55a16ef..0348311 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/Pipeline.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/Pipeline.java
@@ -51,6 +51,7 @@
 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.TupleTag;
 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.Function;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Joiner;
@@ -59,6 +60,7 @@
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ArrayListMultimap;
 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.HashMultimap;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableSet;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Multimap;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.SetMultimap;
 import org.checkerframework.checker.nullness.qual.Nullable;
@@ -118,6 +120,7 @@
  *
  * }</pre>
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class Pipeline {
   private static final Logger LOG = LoggerFactory.getLogger(Pipeline.class);
   /**
@@ -562,16 +565,26 @@
       return;
     }
     InputT originalInput = replacement.getInput();
+    Map<TupleTag<?>, PCollection<?>> originalOutputs = original.getOutputs();
 
     LOG.debug("Replacing {} with {}", original, replacement);
     transforms.replaceNode(original, originalInput, replacement.getTransform());
     try {
       OutputT newOutput = replacement.getTransform().expand(originalInput);
-      Map<PValue, ReplacementOutput> originalToReplacement =
+      Map<PCollection<?>, ReplacementOutput> originalToReplacement =
           replacementFactory.mapOutputs(original.getOutputs(), newOutput);
       // Ensure the internal TransformHierarchy data structures are consistent.
       transforms.setOutput(newOutput);
       transforms.replaceOutputs(originalToReplacement);
+      checkState(
+          ImmutableSet.copyOf(originalOutputs.values())
+              .equals(ImmutableSet.copyOf(transforms.getCurrent().getOutputs().values())),
+          "After replacing %s with %s, outputs were not rewired correctly:"
+              + " Original outputs %s became %s.",
+          original,
+          transforms.getCurrent(),
+          originalOutputs,
+          transforms.getCurrent().getOutputs());
     } finally {
       transforms.popNode();
     }
@@ -594,8 +607,8 @@
         case ERROR: // be very verbose here since it will just fail the execution
           throw new IllegalStateException(
               String.format(
-                      "Pipeline update will not be possible"
-                          + " because the following transforms do not have stable unique names: %s.",
+                      "Pipeline update will not be possible because the following transforms do"
+                          + " not have stable unique names: %s.",
                       Joiner.on(", ").join(transform(errors, new KeysExtractor())))
                   + "\n\n"
                   + "Conflicting instances:\n"
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/AtomicCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/AtomicCoder.java
index 335cf28..fd4d414 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/AtomicCoder.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/AtomicCoder.java
@@ -30,6 +30,7 @@
  *
  * @param <T> the type of the values being transcoded
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public abstract class AtomicCoder<T> extends StructuredCoder<T> {
   /**
    * {@inheritDoc}.
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/AvroCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/AvroCoder.java
index 8b77446..26e6dd6 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/AvroCoder.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/AvroCoder.java
@@ -103,6 +103,7 @@
  *
  * @param <T> the type of elements handled by this coder
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class AvroCoder<T> extends CustomCoder<T> {
 
   /**
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderProviders.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderProviders.java
index 2def8c4..2a09c9d 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderProviders.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderProviders.java
@@ -28,6 +28,7 @@
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.MoreObjects;
 
 /** Static utility methods for creating and working with {@link CoderProvider}s. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public final class CoderProviders {
   private CoderProviders() {} // Static utility class
 
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderRegistry.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderRegistry.java
index 12e1cb3..7d7cae5 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderRegistry.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderRegistry.java
@@ -89,6 +89,7 @@
  * <p>Note that if multiple {@link CoderProvider coder providers} can provide a {@link Coder} for a
  * given type, the precedence order above defines which {@link CoderProvider} is chosen.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class CoderRegistry {
 
   private static final Logger LOG = LoggerFactory.getLogger(CoderRegistry.class);
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DefaultCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DefaultCoder.java
index 5974150..9fbc2c5 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DefaultCoder.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DefaultCoder.java
@@ -49,7 +49,10 @@
 @Documented
 @Retention(RetentionPolicy.RUNTIME)
 @Target(ElementType.TYPE)
-@SuppressWarnings("rawtypes")
+@SuppressWarnings({
+  "rawtypes",
+  "nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
+})
 public @interface DefaultCoder {
   @CheckForNull
   Class<? extends Coder> value();
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/NullableCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/NullableCoder.java
index 2c8b186..954a746 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/NullableCoder.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/NullableCoder.java
@@ -35,7 +35,8 @@
  *
  * @param <T> the type of the values being transcoded
  */
-public class NullableCoder<T> extends StructuredCoder<T> {
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
+public class NullableCoder<T> extends StructuredCoder<@Nullable T> {
   public static <T> NullableCoder<T> of(Coder<T> valueCoder) {
     if (valueCoder instanceof NullableCoder) {
       return (NullableCoder<T>) valueCoder;
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/RowCoderGenerator.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/RowCoderGenerator.java
index 6929fc4..bc2c966 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/RowCoderGenerator.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/RowCoderGenerator.java
@@ -95,6 +95,7 @@
  * </code></pre>
  */
 @Experimental(Kind.SCHEMAS)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public abstract class RowCoderGenerator {
   private static final ByteBuddy BYTE_BUDDY = new ByteBuddy();
   private static final BitSetCoder NULL_LIST_CODER = BitSetCoder.of();
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/StringDelegateCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/StringDelegateCoder.java
index 5ba07b7..996e58f 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/StringDelegateCoder.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/StringDelegateCoder.java
@@ -41,6 +41,7 @@
  *
  * @param <T> The type of objects coded.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public final class StringDelegateCoder<T> extends CustomCoder<T> {
   public static <T> StringDelegateCoder<T> of(Class<T> clazz) {
     return StringDelegateCoder.of(clazz, TypeDescriptor.of(clazz));
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroIO.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroIO.java
index 0e69218..948073e 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroIO.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroIO.java
@@ -326,6 +326,7 @@
  *     .to(new UserDynamicAvroDestinations(userToSchemaMap)));
  * }</pre>
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class AvroIO {
   /**
    * Reads records of the given type from an Avro file (or multiple Avro files matching a pattern).
@@ -1848,9 +1849,8 @@
   @AutoValue
   public abstract static class Sink<ElementT> implements FileIO.Sink<ElementT> {
     /** @deprecated RecordFormatter will be removed in future versions. */
-    @Nullable
     @Deprecated
-    abstract RecordFormatter<ElementT> getRecordFormatter();
+    abstract @Nullable RecordFormatter<ElementT> getRecordFormatter();
 
     abstract @Nullable String getJsonSchema();
 
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroSchemaIOProvider.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroSchemaIOProvider.java
index ad3984b..e26751b 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroSchemaIOProvider.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroSchemaIOProvider.java
@@ -39,6 +39,7 @@
  */
 @Internal
 @AutoService(SchemaIOProvider.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class AvroSchemaIOProvider implements SchemaIOProvider {
   /** Returns an id that uniquely represents this IO. */
   @Override
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroSink.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroSink.java
index fe198b1..205e32b 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroSink.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroSink.java
@@ -33,6 +33,7 @@
 import org.checkerframework.checker.nullness.qual.Nullable;
 
 /** A {@link FileBasedSink} for Avro files. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class AvroSink<UserT, DestinationT, OutputT>
     extends FileBasedSink<UserT, DestinationT, OutputT> {
   private final boolean genericRecords;
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroSource.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroSource.java
index f016681..b255820 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroSource.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroSource.java
@@ -131,6 +131,7 @@
  */
 // CHECKSTYLE.ON: JavadocStyle
 @Experimental(Kind.SOURCE_SINK)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class AvroSource<T> extends BlockBasedSource<T> {
   // Default minimum bundle size (chosen as two default-size Avro blocks to attempt to
   // ensure that every source has at least one block of records).
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BlockBasedSource.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BlockBasedSource.java
index 7c2bd3a..cc04768 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BlockBasedSource.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BlockBasedSource.java
@@ -61,6 +61,7 @@
  * @param <T> The type of records to be read from the source.
  */
 @Experimental(Kind.SOURCE_SINK)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public abstract class BlockBasedSource<T> extends FileBasedSource<T> {
   /**
    * Creates a {@code BlockBasedSource} based on a file name or pattern. Subclasses must call this
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BoundedReadFromUnboundedSource.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BoundedReadFromUnboundedSource.java
index a846bc9..c14f86e 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BoundedReadFromUnboundedSource.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BoundedReadFromUnboundedSource.java
@@ -51,6 +51,7 @@
  * {@link PTransform} that reads a bounded amount of data from an {@link UnboundedSource}, specified
  * as one or both of a maximum number of elements or a maximum period of time to read.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BoundedReadFromUnboundedSource<T> extends PTransform<PBegin, PCollection<T>> {
   private final UnboundedSource<T, ?> source;
   private final long maxNumRecords;
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/ClassLoaderFileSystem.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/ClassLoaderFileSystem.java
index 07f940e..859b4cb 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/ClassLoaderFileSystem.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/ClassLoaderFileSystem.java
@@ -37,6 +37,7 @@
 import org.checkerframework.checker.nullness.qual.Nullable;
 
 /** A read-only {@link FileSystem} implementation looking up resources using a ClassLoader. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ClassLoaderFileSystem extends FileSystem<ClassLoaderFileSystem.ClassLoaderResourceId> {
 
   public static final String SCHEMA = "classpath";
@@ -137,9 +138,8 @@
       return SCHEMA;
     }
 
-    @Nullable
     @Override
-    public String getFilename() {
+    public @Nullable String getFilename() {
       return path;
     }
 
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CompressedSource.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CompressedSource.java
index c7b1bda..d6a34b1 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CompressedSource.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CompressedSource.java
@@ -62,6 +62,7 @@
  * @param <T> The type to read from the compressed file.
  */
 @Experimental(Kind.SOURCE_SINK)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class CompressedSource<T> extends FileBasedSource<T> {
   /**
    * Factory interface for creating channels that decompress the content of an underlying channel.
@@ -334,9 +335,9 @@
     @GuardedBy("progressLock")
     private long numRecordsRead;
 
-    @Nullable // Initialized in startReading
+    // Initialized in startReading
     @GuardedBy("progressLock")
-    private CountingChannel channel;
+    private @Nullable CountingChannel channel;
 
     private DecompressingChannelFactory channelFactory;
 
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/ConstantAvroDestination.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/ConstantAvroDestination.java
index cf76cdd..1754fef 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/ConstantAvroDestination.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/ConstantAvroDestination.java
@@ -32,6 +32,7 @@
 import org.checkerframework.checker.nullness.qual.Nullable;
 
 /** Always returns a constant {@link FilenamePolicy}, {@link Schema}, metadata, and codec. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class ConstantAvroDestination<UserT, OutputT>
     extends DynamicAvroDestinations<UserT, Void, OutputT> {
   private static class SchemaFunction implements Serializable, Function<String, Schema> {
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CountingSource.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CountingSource.java
index a36530e..b28b059 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CountingSource.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CountingSource.java
@@ -54,6 +54,7 @@
  * <p>To produce a bounded source, use {@link #createSourceForSubrange(long, long)}. To produce an
  * unbounded source, use {@link #createUnboundedFrom(long)}.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class CountingSource {
   /**
    * Creates a {@link BoundedSource} that will produce the specified number of elements, from {@code
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/DefaultFilenamePolicy.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/DefaultFilenamePolicy.java
index 74d556a..5e2298c 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/DefaultFilenamePolicy.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/DefaultFilenamePolicy.java
@@ -58,6 +58,7 @@
  * different files for each window and other sharding controls, see the {@code
  * WriteOneFilePerWindow} example pipeline.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public final class DefaultFilenamePolicy extends FilenamePolicy {
   /** The default sharding name template. */
   public static final String DEFAULT_UNWINDOWED_SHARD_TEMPLATE = ShardNameTemplate.INDEX_OF_MAX;
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/DynamicFileDestinations.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/DynamicFileDestinations.java
index b16ef8f..9d336a7 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/DynamicFileDestinations.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/DynamicFileDestinations.java
@@ -30,6 +30,7 @@
 import org.checkerframework.checker.nullness.qual.Nullable;
 
 /** Some helper classes that derive from {@link FileBasedSink.DynamicDestinations}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DynamicFileDestinations {
   /** Always returns a constant {@link FilenamePolicy}. */
   private static class ConstantFilenamePolicy<UserT, OutputT>
@@ -109,9 +110,8 @@
       return emptyDestination;
     }
 
-    @Nullable
     @Override
-    public Coder<Params> getDestinationCoder() {
+    public @Nullable Coder<Params> getDestinationCoder() {
       return ParamsCoder.of();
     }
 
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileBasedSink.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileBasedSink.java
index 37d83c7..4827ae6 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileBasedSink.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileBasedSink.java
@@ -120,6 +120,7 @@
  * @param <OutputT> the type of values written to the sink.
  */
 @Experimental(Kind.FILESYSTEM)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public abstract class FileBasedSink<UserT, DestinationT, OutputT>
     implements Serializable, HasDisplayData {
   private static final Logger LOG = LoggerFactory.getLogger(FileBasedSink.class);
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileBasedSource.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileBasedSource.java
index 1682c9c..18b3aaa 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileBasedSource.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileBasedSource.java
@@ -62,6 +62,7 @@
  *
  * @param <T> Type of records represented by the source.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public abstract class FileBasedSource<T> extends OffsetBasedSource<T> {
   private static final Logger LOG = LoggerFactory.getLogger(FileBasedSource.class);
 
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileIO.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileIO.java
index d321b13..36e3d25 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileIO.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileIO.java
@@ -308,6 +308,7 @@
  *     .withNaming(type -> defaultNaming(type + "-transactions", ".csv"));
  * }</pre>
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class FileIO {
   private static final Logger LOG = LoggerFactory.getLogger(FileIO.class);
 
@@ -1459,9 +1460,8 @@
                   false /* isDirectory */);
             }
 
-            @Nullable
             @Override
-            public ResourceId unwindowedFilename(
+            public @Nullable ResourceId unwindowedFilename(
                 int shardNumber, int numShards, OutputFileHints outputFileHints) {
               return FileSystems.matchNewResource(
                   namingFn.getFilename(
@@ -1480,9 +1480,8 @@
           return Lists.newArrayList(spec.getAllSideInputs());
         }
 
-        @Nullable
         @Override
-        public Coder<DestinationT> getDestinationCoder() {
+        public @Nullable Coder<DestinationT> getDestinationCoder() {
           return spec.getDestinationCoder();
         }
       }
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileSystems.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileSystems.java
index 4f26711..d8364f2 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileSystems.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/FileSystems.java
@@ -66,6 +66,7 @@
 
 /** Clients facing {@link FileSystem} utility. */
 @Experimental(Kind.FILESYSTEM)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class FileSystems {
 
   public static final String DEFAULT_SCHEME = "file";
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/GenerateSequence.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/GenerateSequence.java
index 7c5c483..a2a2387 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/GenerateSequence.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/GenerateSequence.java
@@ -70,6 +70,7 @@
  * will be present in the resulting {@link PCollection}.
  */
 @AutoValue
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public abstract class GenerateSequence extends PTransform<PBegin, PCollection<Long>> {
   abstract long getFrom();
 
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/LocalFileSystem.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/LocalFileSystem.java
index 5a201e8..12cd1de 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/LocalFileSystem.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/LocalFileSystem.java
@@ -80,6 +80,7 @@
  *   <li>file:///C:/Users/beam/Documents/pom.xml
  * </ul>
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class LocalFileSystem extends FileSystem<LocalResourceId> {
 
   private static final Logger LOG = LoggerFactory.getLogger(LocalFileSystem.class);
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/LocalResourceId.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/LocalResourceId.java
index 02400af..e6394f7 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/LocalResourceId.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/LocalResourceId.java
@@ -37,7 +37,7 @@
 
   private final String pathString;
 
-  private transient @Nullable volatile Path cachedPath;
+  private transient volatile @Nullable Path cachedPath;
 
   static LocalResourceId fromPath(Path path, boolean isDirectory) {
     checkNotNull(path, "path");
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/OffsetBasedSource.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/OffsetBasedSource.java
index 7b10174..7f6f0d8 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/OffsetBasedSource.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/OffsetBasedSource.java
@@ -54,6 +54,7 @@
  * @see FileBasedSource
  * @see RangeTracker
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public abstract class OffsetBasedSource<T> extends BoundedSource<T> {
   private final long startOffset;
   private final long endOffset;
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Read.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Read.java
index bc5b9dd..49d040e 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Read.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Read.java
@@ -76,6 +76,7 @@
  * p.apply(Read.from(new MySource().withFoo("foo").withBar("bar")));
  * </pre>
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class Read {
 
   /**
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/SerializableAvroCodecFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/SerializableAvroCodecFactory.java
index 7242dd1..ea19b52 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/SerializableAvroCodecFactory.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/SerializableAvroCodecFactory.java
@@ -40,6 +40,7 @@
  * A wrapper that allows {@link org.apache.avro.file.CodecFactory}s to be serialized using Java's
  * standard serialization mechanisms.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class SerializableAvroCodecFactory implements Externalizable {
   private static final long serialVersionUID = 7445324844109564303L;
   private static final List<String> noOptAvroCodecs =
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TFRecordIO.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TFRecordIO.java
index 89cfe34..b24a7fb 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TFRecordIO.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TFRecordIO.java
@@ -58,6 +58,7 @@
  * to write windowed data or writing to multiple destinations) use {@link #sink} in combination with
  * {@link FileIO#write} or {@link FileIO#writeDynamic}.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class TFRecordIO {
   /** The default coder, which returns each record of the input file as a byte array. */
   public static final Coder<byte[]> DEFAULT_BYTE_ARRAY_CODER = ByteArrayCoder.of();
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java
index dc08ffa..a3c8a74 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextIO.java
@@ -179,6 +179,7 @@
  * <p>For backwards compatibility, {@link TextIO} also supports the legacy {@link
  * DynamicDestinations} interface for advanced features via {@link Write#to(DynamicDestinations)}.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class TextIO {
   private static final long DEFAULT_BUNDLE_SIZE_BYTES = 64 * 1024 * 1024L;
 
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextRowCountEstimator.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextRowCountEstimator.java
index 9cccec3..5a78d68 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextRowCountEstimator.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextRowCountEstimator.java
@@ -29,6 +29,7 @@
 
 /** This returns a row count estimation for files associated with a file pattern. */
 @AutoValue
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public abstract class TextRowCountEstimator {
   private static final long DEFAULT_NUM_BYTES_PER_FILE = 64 * 1024L;
   private static final Compression DEFAULT_COMPRESSION = Compression.AUTO;
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextSink.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextSink.java
index b85ce90..21826b8 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextSink.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextSink.java
@@ -34,6 +34,7 @@
  * '\n'} represented in {@code UTF-8} format as the record separator. Each record (including the
  * last) is terminated.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class TextSink<UserT, DestinationT> extends FileBasedSink<UserT, DestinationT, String> {
   private final @Nullable String header;
   private final @Nullable String footer;
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextSource.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextSource.java
index 2807a7f..d392d7a 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextSource.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextSource.java
@@ -48,6 +48,7 @@
  * representing the beginning of the first record to be decoded.
  */
 @VisibleForTesting
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class TextSource extends FileBasedSource<String> {
   byte[] delimiter;
 
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/WriteFiles.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/WriteFiles.java
index b40f1d5..09ceddc 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/WriteFiles.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/WriteFiles.java
@@ -109,6 +109,7 @@
  */
 @Experimental(Kind.SOURCE_SINK)
 @AutoValue
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public abstract class WriteFiles<UserT, DestinationT, OutputT>
     extends PTransform<PCollection<UserT>, WriteFilesResult<DestinationT>> {
   private static final Logger LOG = LoggerFactory.getLogger(WriteFiles.class);
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/fs/ResourceIdTester.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/fs/ResourceIdTester.java
index 79eba30..193d722 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/fs/ResourceIdTester.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/fs/ResourceIdTester.java
@@ -37,6 +37,7 @@
 
 /** A utility to test {@link ResourceId} implementations. */
 @Experimental(Kind.FILESYSTEM)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public final class ResourceIdTester {
   /**
    * Enforces that the {@link ResourceId} implementation of {@code baseDirectory} meets the {@link
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/range/ByteKeyRangeTracker.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/range/ByteKeyRangeTracker.java
index 58e5c20..8a500ba 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/range/ByteKeyRangeTracker.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/range/ByteKeyRangeTracker.java
@@ -31,6 +31,7 @@
  * @see ByteKey
  * @see ByteKeyRange
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public final class ByteKeyRangeTracker implements RangeTracker<ByteKey> {
   private static final Logger LOG = LoggerFactory.getLogger(ByteKeyRangeTracker.class);
 
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/range/OffsetRangeTracker.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/range/OffsetRangeTracker.java
index b76ad3c..9d2281b 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/range/OffsetRangeTracker.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/range/OffsetRangeTracker.java
@@ -30,6 +30,7 @@
  * <p>Not to be confused with {@link
  * org.apache.beam.sdk.transforms.splittabledofn.OffsetRangeTracker}.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class OffsetRangeTracker implements RangeTracker<Long> {
   private static final Logger LOG = LoggerFactory.getLogger(OffsetRangeTracker.class);
 
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/MetricResult.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/MetricResult.java
index 75150cc..879ad38 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/MetricResult.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/MetricResult.java
@@ -33,6 +33,7 @@
 @Experimental(Kind.METRICS)
 @JsonFilter("committedMetrics")
 @AutoValue
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public abstract class MetricResult<T> {
   /** Return the name of the metric. */
   public MetricName getName() {
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ExperimentalOptions.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ExperimentalOptions.java
index 9ab4fc5..2846c83 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ExperimentalOptions.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ExperimentalOptions.java
@@ -29,6 +29,7 @@
  */
 @Experimental
 @Hidden
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public interface ExperimentalOptions extends PipelineOptions {
 
   String STATE_CACHE_SIZE = "state_cache_size";
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionsFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionsFactory.java
index f91263b..a1701f0 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionsFactory.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionsFactory.java
@@ -114,6 +114,7 @@
  * href="http://www.oracle.com/technetwork/java/javase/documentation/spec-136004.html">JavaBeans
  * specification</a> for more details as to what constitutes a property.
  */
+@SuppressWarnings({"nullness", "keyfor"}) // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class PipelineOptionsFactory {
   /**
    * Creates and returns an object that implements {@link PipelineOptions}. This sets the {@link
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionsValidator.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionsValidator.java
index affb28c..ab3238b 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionsValidator.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionsValidator.java
@@ -31,6 +31,7 @@
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.TreeMultimap;
 
 /** Validates that the {@link PipelineOptions} conforms to all the {@link Validation} criteria. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class PipelineOptionsValidator {
   /**
    * Validates that the passed {@link PipelineOptions} conforms to all the validation criteria from
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ProxyInvocationHandler.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ProxyInvocationHandler.java
index ff8e1c6..250169f 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ProxyInvocationHandler.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ProxyInvocationHandler.java
@@ -89,6 +89,7 @@
  * PipelineOptions#as(Class)}.
  */
 @ThreadSafe
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class ProxyInvocationHandler implements InvocationHandler, Serializable {
   /**
    * No two instances of this class are considered equivalent hence we generate a random hash code.
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ValueProvider.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ValueProvider.java
index a65de0c..48b6adc 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ValueProvider.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ValueProvider.java
@@ -61,6 +61,7 @@
  */
 @JsonSerialize(using = ValueProvider.Serializer.class)
 @JsonDeserialize(using = ValueProvider.Deserializer.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public interface ValueProvider<T> extends Serializable {
   /**
    * Returns the runtime value wrapped by this {@link ValueProvider} in case it is {@link
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/AppliedPTransform.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/AppliedPTransform.java
index da7c720..34d3d2c 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/AppliedPTransform.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/AppliedPTransform.java
@@ -23,9 +23,10 @@
 import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.annotations.Internal;
 import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PCollection;
 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.PValues;
 import org.apache.beam.sdk.values.TupleTag;
 
 /**
@@ -55,8 +56,8 @@
           TransformT extends PTransform<? super InputT, OutputT>>
       AppliedPTransform<InputT, OutputT, TransformT> of(
           String fullName,
-          Map<TupleTag<?>, PValue> input,
-          Map<TupleTag<?>, PValue> output,
+          Map<TupleTag<?>, PCollection<?>> input,
+          Map<TupleTag<?>, PCollection<?>> output,
           TransformT transform,
           Pipeline p) {
     return new AutoValue_AppliedPTransform<>(fullName, input, output, transform, p);
@@ -64,17 +65,18 @@
 
   public abstract String getFullName();
 
-  public abstract Map<TupleTag<?>, PValue> getInputs();
+  public abstract Map<TupleTag<?>, PCollection<?>> getInputs();
 
-  public abstract Map<TupleTag<?>, PValue> getOutputs();
+  public abstract Map<TupleTag<?>, PCollection<?>> getOutputs();
 
   public abstract TransformT getTransform();
 
   public abstract Pipeline getPipeline();
 
   /** @return map of {@link TupleTag TupleTags} which are not side inputs. */
-  public Map<TupleTag<?>, PValue> getMainInputs() {
-    Map<TupleTag<?>, PValue> sideInputs = getTransform().getAdditionalInputs();
+  public Map<TupleTag<?>, PCollection<?>> getMainInputs() {
+    Map<TupleTag<?>, PCollection<?>> sideInputs =
+        PValues.fullyExpand(getTransform().getAdditionalInputs());
     return getInputs().entrySet().stream()
         .filter(e -> !sideInputs.containsKey(e.getKey()))
         .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/PTransformOverrideFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/PTransformOverrideFactory.java
index 28cfc0a..b7dfdc0 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/PTransformOverrideFactory.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/PTransformOverrideFactory.java
@@ -24,9 +24,9 @@
 import org.apache.beam.sdk.annotations.Experimental.Kind;
 import org.apache.beam.sdk.annotations.Internal;
 import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.values.PCollection;
 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.TaggedPValue;
 import org.apache.beam.sdk.values.TupleTag;
 
@@ -53,7 +53,8 @@
    * Returns a {@link Map} from the expanded values in {@code newOutput} to the values produced by
    * the original transform.
    */
-  Map<PValue, ReplacementOutput> mapOutputs(Map<TupleTag<?>, PValue> outputs, OutputT newOutput);
+  Map<PCollection<?>, ReplacementOutput> mapOutputs(
+      Map<TupleTag<?>, PCollection<?>> outputs, OutputT newOutput);
 
   /**
    * A {@link PTransform} that replaces an {@link AppliedPTransform}, and the input required to do
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformHierarchy.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformHierarchy.java
index 4d51152..b283196 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformHierarchy.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/TransformHierarchy.java
@@ -25,7 +25,6 @@
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
-import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
@@ -40,11 +39,11 @@
 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.PValues;
 import org.apache.beam.sdk.values.TupleTag;
 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.MoreObjects;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
-import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables;
 import org.checkerframework.checker.nullness.qual.Nullable;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -53,17 +52,20 @@
  * <b><i>For internal use only; no backwards-compatibility guarantees.</i></b>
  *
  * <p>Captures information about a collection of transformations and their associated {@link
- * PValue}s.
+ * PCollection PCollections}.
  */
 @Internal
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class TransformHierarchy {
   private static final Logger LOG = LoggerFactory.getLogger(TransformHierarchy.class);
 
   private final Node root;
   private final Map<Node, PInput> unexpandedInputs;
-  private final Map<POutput, Node> producers;
-  // A map of PValue to the PInput the producing PTransform is applied to
-  private final Map<PValue, PInput> producerInput;
+  private final Map<PCollection<?>, Node> producers;
+
+  // A map of PCollection<?> to the PInput the producing PTransform is applied to
+  private final Map<PCollection<?>, PInput> producerInput;
+
   // Maintain a stack based on the enclosing nodes
   private Node current;
 
@@ -99,48 +101,6 @@
     return current;
   }
 
-  @Internal
-  public Node pushFinalizedNode(
-      String name,
-      Map<TupleTag<?>, PValue> inputs,
-      PTransform<?, ?> transform,
-      Map<TupleTag<?>, PValue> outputs) {
-    checkNotNull(
-        transform, "A %s must be provided for all Nodes", PTransform.class.getSimpleName());
-    checkNotNull(
-        name, "A name must be provided for all %s Nodes", PTransform.class.getSimpleName());
-    checkNotNull(
-        inputs, "An input must be provided for all %s Nodes", PTransform.class.getSimpleName());
-    Node node = new Node(current, transform, name, inputs, outputs);
-    node.finishedSpecifying = true;
-    current.addComposite(node);
-    current = node;
-    return current;
-  }
-
-  @Internal
-  public Node addFinalizedPrimitiveNode(
-      String name,
-      Map<TupleTag<?>, PValue> inputs,
-      PTransform<?, ?> transform,
-      Map<TupleTag<?>, PValue> outputs) {
-    checkNotNull(
-        transform, "A %s must be provided for all Nodes", PTransform.class.getSimpleName());
-    checkNotNull(
-        name, "A name must be provided for all %s Nodes", PTransform.class.getSimpleName());
-    checkNotNull(
-        inputs, "Inputs must be provided for all %s Nodes", PTransform.class.getSimpleName());
-    checkNotNull(
-        outputs, "Outputs must be provided for all %s Nodes", PTransform.class.getSimpleName());
-    Node node = new Node(current, transform, name, inputs, outputs);
-    node.finishedSpecifying = true;
-    for (PValue output : outputs.values()) {
-      producers.put(output, node);
-    }
-    current.addComposite(node);
-    return node;
-  }
-
   public Node replaceNode(Node existing, PInput input, PTransform<?, ?> transform) {
     checkNotNull(existing);
     checkNotNull(input);
@@ -150,7 +110,7 @@
         "Replacing a node when the graph has an unexpanded input. This is an SDK bug.");
     Node replacement =
         new Node(existing.getEnclosingNode(), transform, existing.getFullName(), input);
-    for (PValue output : existing.getOutputs().values()) {
+    for (PCollection<?> output : existing.getOutputs().values()) {
       Node producer = producers.get(output);
       boolean producedInExisting = false;
       do {
@@ -184,7 +144,7 @@
    */
   public void finishSpecifyingInput() {
     // Inputs must be completely specified before they are consumed by a transform.
-    for (PValue inputValue : current.getInputs().values()) {
+    for (PCollection<?> inputValue : current.getInputs().values()) {
       PInput input = producerInput.remove(inputValue);
       Node producerNode = maybeGetProducer(inputValue);
       if (producerNode != null) {
@@ -204,7 +164,7 @@
    * nodes.
    */
   public void setOutput(POutput output) {
-    for (PCollection<?> value : fullyExpand(output).values()) {
+    for (PCollection<?> value : PValues.fullyExpand(output.expand()).values()) {
       if (!producers.containsKey(value)) {
         producers.put(value, current);
         value.finishSpecifyingOutput(
@@ -222,7 +182,7 @@
    * node it is replacing. No value that is a key in {@code originalToReplacement} may be present
    * within the {@link TransformHierarchy} after this method completes.
    */
-  public void replaceOutputs(Map<PValue, ReplacementOutput> originalToReplacement) {
+  public void replaceOutputs(Map<PCollection<?>, ReplacementOutput> originalToReplacement) {
     current.replaceOutputs(originalToReplacement);
   }
 
@@ -237,11 +197,11 @@
     checkState(current != null, "Can't pop the root node of a TransformHierarchy");
   }
 
-  Node maybeGetProducer(PValue produced) {
+  Node maybeGetProducer(PCollection<?> produced) {
     return producers.get(produced);
   }
 
-  Node getProducer(PValue produced) {
+  Node getProducer(PCollection<?> produced) {
     return checkNotNull(maybeGetProducer(produced), "No producer found for %s", produced);
   }
 
@@ -258,9 +218,10 @@
    * as input. These values must still be finished specifying.
    */
   private void finishSpecifying() {
-    for (Entry<PValue, PInput> producerInputEntry : producerInput.entrySet()) {
-      PValue value = producerInputEntry.getKey();
-      value.finishSpecifying(producerInputEntry.getValue(), getProducer(value).getTransform());
+    for (Entry<PCollection<?>, PInput> producerInputEntry : producerInput.entrySet()) {
+      PCollection<?> pCollection = producerInputEntry.getKey();
+      pCollection.finishSpecifying(
+          producerInputEntry.getValue(), getProducer(pCollection).getTransform());
     }
     producerInput.clear();
   }
@@ -269,47 +230,6 @@
     return current;
   }
 
-  private Map<TupleTag<?>, PCollection<?>> fullyExpand(POutput output) {
-    Map<TupleTag<?>, PCollection<?>> result = new LinkedHashMap<>();
-    for (Map.Entry<TupleTag<?>, PValue> value : output.expand().entrySet()) {
-      if (value.getValue() instanceof PCollection) {
-        PCollection<?> previous = result.put(value.getKey(), (PCollection<?>) value.getValue());
-        checkArgument(
-            previous == null,
-            "Found conflicting %ss in flattened expansion of %s: %s maps to %s and %s",
-            output,
-            TupleTag.class.getSimpleName(),
-            value.getKey(),
-            previous,
-            value.getValue());
-      } else {
-        if (value.getValue().expand().size() == 1
-            && Iterables.getOnlyElement(value.getValue().expand().values())
-                .equals(value.getValue())) {
-          throw new IllegalStateException(
-              String.format(
-                  "Non %s %s that expands into itself %s",
-                  PCollection.class.getSimpleName(),
-                  PValue.class.getSimpleName(),
-                  value.getValue()));
-        }
-        for (Map.Entry<TupleTag<?>, PCollection<?>> valueComponent :
-            fullyExpand(value.getValue()).entrySet()) {
-          PCollection<?> previous = result.put(valueComponent.getKey(), valueComponent.getValue());
-          checkArgument(
-              previous == null,
-              "Found conflicting %ss in flattened expansion of %s: %s maps to %s and %s",
-              output,
-              TupleTag.class.getSimpleName(),
-              valueComponent.getKey(),
-              previous,
-              valueComponent.getValue());
-        }
-      }
-    }
-    return result;
-  }
-
   /**
    * Provides internal tracking of transform relationships with helper methods for initialization
    * and ordered visitation.
@@ -329,11 +249,11 @@
     private final List<Node> parts = new ArrayList<>();
 
     // Input to the transform, in expanded form.
-    private final Map<TupleTag<?>, PValue> inputs;
+    private final Map<TupleTag<?>, PCollection<?>> inputs;
 
     // TODO: track which outputs need to be exported to parent.
     // Output of the transform, in expanded form. Null if not yet set.
-    private @Nullable Map<TupleTag<?>, PValue> outputs;
+    private @Nullable Map<TupleTag<?>, PCollection<?>> outputs;
 
     @VisibleForTesting boolean finishedSpecifying = false;
 
@@ -361,9 +281,9 @@
       this.enclosingNode = enclosingNode;
       this.transform = transform;
       this.fullName = fullName;
-      ImmutableMap.Builder<TupleTag<?>, PValue> inputs = ImmutableMap.builder();
-      inputs.putAll(input.expand());
-      inputs.putAll(transform.getAdditionalInputs());
+      ImmutableMap.Builder<TupleTag<?>, PCollection<?>> inputs = ImmutableMap.builder();
+      inputs.putAll(PValues.expandInput(input));
+      inputs.putAll(PValues.fullyExpand(transform.getAdditionalInputs()));
       this.inputs = inputs.build();
     }
 
@@ -384,8 +304,8 @@
         @Nullable Node enclosingNode,
         @Nullable PTransform<?, ?> transform,
         String fullName,
-        @Nullable Map<TupleTag<?>, PValue> inputs,
-        @Nullable Map<TupleTag<?>, PValue> outputs) {
+        @Nullable Map<TupleTag<?>, PCollection<?>> inputs,
+        @Nullable Map<TupleTag<?>, PCollection<?>> outputs) {
       this.enclosingNode = enclosingNode;
       this.transform = transform;
       this.fullName = fullName;
@@ -455,7 +375,7 @@
     private boolean returnsOthersOutput() {
       PTransform<?, ?> transform = getTransform();
       if (outputs != null) {
-        for (PValue outputValue : outputs.values()) {
+        for (PCollection<?> outputValue : outputs.values()) {
           if (!getProducer(outputValue).getTransform().equals(transform)) {
             return true;
           }
@@ -473,7 +393,7 @@
     }
 
     /** Returns the transform input, in fully expanded form. */
-    public Map<TupleTag<?>, PValue> getInputs() {
+    public Map<TupleTag<?>, PCollection<?>> getInputs() {
       return inputs;
     }
 
@@ -483,12 +403,12 @@
       checkState(
           this.outputs == null, "Tried to specify more than one output for %s", getFullName());
       checkNotNull(output, "Tried to set the output of %s to null", getFullName());
-      this.outputs = output.expand();
+      this.outputs = PValues.fullyExpand(output.expand());
 
       // Validate that a primitive transform produces only primitive output, and a composite
       // transform does not produce primitive output.
       Set<Node> outputProducers = new HashSet<>();
-      for (PValue outputValue : output.expand().values()) {
+      for (PCollection<?> outputValue : PValues.fullyExpand(output.expand()).values()) {
         outputProducers.add(getProducer(outputValue));
       }
       if (outputProducers.contains(this) && (!parts.isEmpty() || outputProducers.size() > 1)) {
@@ -500,13 +420,14 @@
         }
         throw new IllegalArgumentException(
             String.format(
-                "Output of composite transform [%s] contains a primitive %s produced by it. "
-                    + "Only primitive transforms are permitted to produce primitive outputs."
+                "Output of composite transform [%s] contains a %s produced by it. "
+                    + "Only primitive transforms are permitted to produce %ss."
                     + "%n    Outputs: %s"
                     + "%n    Other Producers: %s"
                     + "%n    Components: %s",
                 getFullName(),
-                POutput.class.getSimpleName(),
+                PCollection.class.getSimpleName(),
+                PCollection.class.getSimpleName(),
                 output.expand(),
                 otherProducerNames,
                 parts));
@@ -524,14 +445,14 @@
      * @param originalToReplacement A map from the outputs of the replacement {@link Node} to the
      *     original output.
      */
-    void replaceOutputs(Map<PValue, ReplacementOutput> originalToReplacement) {
+    void replaceOutputs(Map<PCollection<?>, ReplacementOutput> originalToReplacement) {
       checkNotNull(this.outputs, "Outputs haven't been specified for node %s yet", getFullName());
       for (Node component : this.parts) {
         // Replace the outputs of the component nodes
         component.replaceOutputs(originalToReplacement);
       }
-      ImmutableMap.Builder<TupleTag<?>, PValue> newOutputsBuilder = ImmutableMap.builder();
-      for (Map.Entry<TupleTag<?>, PValue> output : outputs.entrySet()) {
+      ImmutableMap.Builder<TupleTag<?>, PCollection<?>> newOutputsBuilder = ImmutableMap.builder();
+      for (Map.Entry<TupleTag<?>, PCollection<?>> output : outputs.entrySet()) {
         ReplacementOutput mapping = originalToReplacement.get(output.getValue());
         if (mapping != null) {
           if (this.equals(producers.get(mapping.getReplacement().getValue()))) {
@@ -552,7 +473,7 @@
           newOutputsBuilder.put(output);
         }
       }
-      ImmutableMap<TupleTag<?>, PValue> newOutputs = newOutputsBuilder.build();
+      ImmutableMap<TupleTag<?>, PCollection<?>> newOutputs = newOutputsBuilder.build();
       checkState(
           outputs.size() == newOutputs.size(),
           "Number of outputs must be stable across replacement");
@@ -560,7 +481,7 @@
     }
 
     /** Returns the transform output, in expanded form. */
-    public Map<TupleTag<?>, PValue> getOutputs() {
+    public Map<TupleTag<?>, PCollection<?>> getOutputs() {
       return outputs == null ? Collections.emptyMap() : outputs;
     }
 
@@ -633,7 +554,7 @@
 
       if (!isRootNode()) {
         // Visit inputs.
-        for (PValue inputValue : inputs.values()) {
+        for (PCollection<?> inputValue : inputs.values()) {
           Node valueProducer = maybeGetProducer(inputValue);
           if (valueProducer != null) {
             if (!visitedNodes.contains(valueProducer)) {
@@ -704,7 +625,10 @@
       if (isRootNode()) {
         return "RootNode";
       }
-      return MoreObjects.toStringHelper(getClass()).add("fullName", fullName).toString();
+      return MoreObjects.toStringHelper(getClass())
+          .add("fullName", fullName)
+          .add("transform", transform)
+          .toString();
     }
   }
 }
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/AutoValueSchema.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/AutoValueSchema.java
index bf7d6f6..71ee62d 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/AutoValueSchema.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/AutoValueSchema.java
@@ -36,6 +36,7 @@
 
 /** A {@link SchemaProvider} for AutoValue classes. */
 @Experimental(Kind.SCHEMAS)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class AutoValueSchema extends GetterBasedSchemaProvider {
   /** {@link FieldValueTypeSupplier} that's based on AutoValue getters. */
   @VisibleForTesting
@@ -111,9 +112,8 @@
     return creatorFactory;
   }
 
-  @Nullable
   @Override
-  public <T> Schema schemaFor(TypeDescriptor<T> typeDescriptor) {
+  public <T> @Nullable Schema schemaFor(TypeDescriptor<T> typeDescriptor) {
     return JavaBeanUtils.schemaFromJavaBeanClass(
         typeDescriptor.getRawType(), AbstractGetterTypeSupplier.INSTANCE);
   }
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/CachingFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/CachingFactory.java
index c846225..09ba0d6 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/CachingFactory.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/CachingFactory.java
@@ -31,6 +31,7 @@
  * significant for larger schemas) on each lookup. This wrapper caches the value returned by the
  * inner factory, so the schema comparison only need happen on the first lookup.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class CachingFactory<CreatedT> implements Factory<CreatedT> {
   private transient @Nullable ConcurrentHashMap<Class, CreatedT> cache = null;
 
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/FieldAccessDescriptor.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/FieldAccessDescriptor.java
index 0924845..c2db82f 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/FieldAccessDescriptor.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/FieldAccessDescriptor.java
@@ -60,6 +60,7 @@
  */
 @Experimental(Kind.SCHEMAS)
 @AutoValue
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public abstract class FieldAccessDescriptor implements Serializable {
   /** Description of a single field. */
   @AutoValue
@@ -76,6 +77,7 @@
 
     /** OneOf union for a collection selector. */
     @AutoOneOf(Qualifier.Kind.class)
+    @SuppressWarnings("nullness")
     public abstract static class Qualifier implements Serializable {
       /** The kind of qualifier. */
       public enum Kind {
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/FieldTypeDescriptors.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/FieldTypeDescriptors.java
index 1399ef0..7f5634b 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/FieldTypeDescriptors.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/FieldTypeDescriptors.java
@@ -38,6 +38,7 @@
  * define Java objects which can represent these field types.
  */
 @Experimental(Kind.SCHEMAS)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class FieldTypeDescriptors {
   private static final BiMap<TypeName, TypeDescriptor> PRIMITIVE_MAPPING =
       ImmutableBiMap.<TypeName, TypeDescriptor>builder()
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 0dc4a28..bd26e37 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
@@ -32,6 +32,7 @@
 
 /** Represents type information for a Java type that will be used to infer a Schema type. */
 @AutoValue
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public abstract class FieldValueTypeInformation implements Serializable {
   /** Returns the field name. */
   public abstract String getName();
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 d404ce4..f310dad 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
@@ -43,6 +43,7 @@
 
 /** Function to convert a {@link Row} to a user type using a creator factory. */
 @Experimental(Kind.SCHEMAS)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class FromRowUsingCreator<T> implements SerializableFunction<Row, T> {
   private final Class<T> clazz;
   private final GetterBasedSchemaProvider schemaProvider;
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/GetterBasedSchemaProvider.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/GetterBasedSchemaProvider.java
index 9379d7d..1b01eb2 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/GetterBasedSchemaProvider.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/GetterBasedSchemaProvider.java
@@ -31,6 +31,7 @@
  * FieldValueGetter}s.
  */
 @Experimental(Kind.SCHEMAS)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public abstract class GetterBasedSchemaProvider implements SchemaProvider {
   /** Implementing class should override to return FieldValueGetters. */
   public abstract List<FieldValueGetter> fieldValueGetters(Class<?> targetClass, Schema schema);
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/JavaBeanSchema.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/JavaBeanSchema.java
index 254c503..a5c3bf2 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/JavaBeanSchema.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/JavaBeanSchema.java
@@ -47,6 +47,7 @@
  * on the schema.
  */
 @Experimental(Kind.SCHEMAS)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class JavaBeanSchema extends GetterBasedSchemaProvider {
   /** {@link FieldValueTypeSupplier} that's based on getter methods. */
   @VisibleForTesting
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/JavaFieldSchema.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/JavaFieldSchema.java
index a8a6e39..5b45f4d 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/JavaFieldSchema.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/JavaFieldSchema.java
@@ -49,6 +49,7 @@
  * on the schema.
  */
 @Experimental(Kind.SCHEMAS)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class JavaFieldSchema extends GetterBasedSchemaProvider {
   /** {@link FieldValueTypeSupplier} that's based on public fields. */
   @VisibleForTesting
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 5aed674..cba56ea 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
@@ -51,6 +51,7 @@
 
 /** {@link Schema} describes the fields in {@link Row}. */
 @Experimental(Kind.SCHEMAS)
+@SuppressWarnings({"nullness", "keyfor"}) // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class Schema implements Serializable {
   // This is the metadata field used to store the logical type identifier.
   private static final String LOGICAL_TYPE_IDENTIFIER = "SchemaLogicalTypeId";
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/SchemaCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/SchemaCoder.java
index 77a45c4..a28ba29 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/SchemaCoder.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/SchemaCoder.java
@@ -41,6 +41,7 @@
 
 /** {@link SchemaCoder} is used as the coder for types that have schemas registered. */
 @Experimental(Kind.SCHEMAS)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SchemaCoder<T> extends CustomCoder<T> {
   protected final Schema schema;
   private final TypeDescriptor<T> typeDescriptor;
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/SchemaCoderHelpers.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/SchemaCoderHelpers.java
index f243830..ae77ace 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/SchemaCoderHelpers.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/SchemaCoderHelpers.java
@@ -48,6 +48,7 @@
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
 import org.joda.time.ReadableInstant;
 
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class SchemaCoderHelpers {
   // This contains a map of primitive types to their coders.
   private static final Map<TypeName, Coder> CODER_MAP =
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/SchemaRegistry.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/SchemaRegistry.java
index d20b179..82b638c 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/SchemaRegistry.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/SchemaRegistry.java
@@ -49,6 +49,7 @@
  * optional modules to register schemas as well.
  */
 @Experimental(Kind.SCHEMAS)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SchemaRegistry {
   private static final List<SchemaProvider> REGISTERED_SCHEMA_PROVIDERS;
 
@@ -75,9 +76,8 @@
       providers.put(typeDescriptor, schemaProvider);
     }
 
-    @Nullable
     @Override
-    public <T> Schema schemaFor(TypeDescriptor<T> typeDescriptor) {
+    public <T> @Nullable Schema schemaFor(TypeDescriptor<T> typeDescriptor) {
       TypeDescriptor<?> type = typeDescriptor;
       do {
         SchemaProvider schemaProvider = providers.get(type);
@@ -92,9 +92,9 @@
       } while (true);
     }
 
-    @Nullable
     @Override
-    public <T> SerializableFunction<T, Row> toRowFunction(TypeDescriptor<T> typeDescriptor) {
+    public <T> @Nullable SerializableFunction<T, Row> toRowFunction(
+        TypeDescriptor<T> typeDescriptor) {
       TypeDescriptor<?> type = typeDescriptor;
       do {
         SchemaProvider schemaProvider = providers.get(type);
@@ -109,9 +109,9 @@
       } while (true);
     }
 
-    @Nullable
     @Override
-    public <T> SerializableFunction<Row, T> fromRowFunction(TypeDescriptor<T> typeDescriptor) {
+    public <T> @Nullable SerializableFunction<Row, T> fromRowFunction(
+        TypeDescriptor<T> typeDescriptor) {
       TypeDescriptor<?> type = typeDescriptor;
       do {
         SchemaProvider schemaProvider = providers.get(type);
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/SchemaTranslation.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/SchemaTranslation.java
index 274975e..0982134 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/SchemaTranslation.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/SchemaTranslation.java
@@ -47,6 +47,7 @@
 
 /** Utility methods for translating schemas. */
 @Experimental(Kind.SCHEMAS)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SchemaTranslation {
 
   private static final String URN_BEAM_LOGICAL_DATETIME = "beam:logical_type:datetime:v1";
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/SchemaUtils.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/SchemaUtils.java
index e2e749a..c2d7c07 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/SchemaUtils.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/SchemaUtils.java
@@ -20,6 +20,7 @@
 import org.apache.beam.sdk.schemas.Schema.FieldType;
 
 /** A set of utility functions for schemas. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SchemaUtils {
   /**
    * Given two schema that have matching types, return a nullable-widened schema.
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/annotations/DefaultSchema.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/annotations/DefaultSchema.java
index e47509f..ae9277d 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/annotations/DefaultSchema.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/annotations/DefaultSchema.java
@@ -59,7 +59,10 @@
 @Documented
 @Retention(RetentionPolicy.RUNTIME)
 @Target(ElementType.TYPE)
-@SuppressWarnings("rawtypes")
+@SuppressWarnings({
+  "rawtypes",
+  "nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
+})
 @Experimental(Kind.SCHEMAS)
 public @interface DefaultSchema {
 
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/logicaltypes/Date.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/logicaltypes/Date.java
index d942340..83eeea5 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/logicaltypes/Date.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/logicaltypes/Date.java
@@ -29,6 +29,7 @@
  * <p>Its input type is a {@link LocalDate}, and base type is a {@link Long} that represents a
  * incrementing count of days where day 0 is 1970-01-01 (ISO).
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class Date implements Schema.LogicalType<LocalDate, Long> {
 
   @Override
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/logicaltypes/DateTime.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/logicaltypes/DateTime.java
index 148c726..d06fb77 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/logicaltypes/DateTime.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/logicaltypes/DateTime.java
@@ -35,6 +35,7 @@
  * same as the base type of {@link Time}, which is a Long that represents a count of time in
  * nanoseconds.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DateTime implements Schema.LogicalType<LocalDateTime, Row> {
   public static final String DATE_FIELD_NAME = "Date";
   public static final String TIME_FIELD_NAME = "Time";
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
index 6716422..bd5dca6 100644
--- 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
@@ -37,6 +37,7 @@
 
 /** This {@link LogicalType} represent an enumeration over a fixed set of values. */
 @Experimental(Kind.SCHEMAS)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class EnumerationType implements LogicalType<Value, Integer> {
   public static final String IDENTIFIER = "Enum";
   final BiMap<String, Integer> enumValues = HashBiMap.create();
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/logicaltypes/MicrosInstant.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/logicaltypes/MicrosInstant.java
index e1fc8b9..d030bf9 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/logicaltypes/MicrosInstant.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/logicaltypes/MicrosInstant.java
@@ -32,6 +32,7 @@
  * <p>For a more faithful logical type to use with {@code java.time.Instant}, see {@link
  * NanosInstant}.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class MicrosInstant implements Schema.LogicalType<Instant, Row> {
   public static final String IDENTIFIER = "beam:logical_type:micros_instant:v1";
   // TODO(BEAM-10878): This should be a constant
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/logicaltypes/NanosDuration.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/logicaltypes/NanosDuration.java
index 76a83e1..125390e 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/logicaltypes/NanosDuration.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/logicaltypes/NanosDuration.java
@@ -21,6 +21,7 @@
 import org.apache.beam.sdk.values.Row;
 
 /** A duration represented in nanoseconds. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class NanosDuration extends NanosType<Duration> {
   public static final String IDENTIFIER = "beam:logical_type:nanos_duration:v1";
 
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/logicaltypes/NanosInstant.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/logicaltypes/NanosInstant.java
index 9d78959..4835715 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/logicaltypes/NanosInstant.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/logicaltypes/NanosInstant.java
@@ -21,6 +21,7 @@
 import org.apache.beam.sdk.values.Row;
 
 /** A timestamp represented as nanoseconds since the epoch. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class NanosInstant extends NanosType<Instant> {
   public static final String IDENTIFIER = "beam:logical_type:nanos_instant:v1";
 
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
index 60b0b83..6f9fb18 100644
--- 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
@@ -42,6 +42,7 @@
  * EnumerationType} logical type field that indicates which field is set.
  */
 @Experimental(Kind.SCHEMAS)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class OneOfType implements LogicalType<OneOfType.Value, Row> {
   public static final String IDENTIFIER = "OneOf";
 
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
index a42182f..b2f9001 100644
--- 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
@@ -24,6 +24,7 @@
 
 /** A base class for LogicalTypes that use the same Java type as the underlying base type. */
 @Experimental(Kind.SCHEMAS)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public abstract class PassThroughLogicalType<T> implements LogicalType<T, T> {
   private final String identifier;
   private final FieldType argumentType;
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/logicaltypes/Time.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/logicaltypes/Time.java
index c193d84..d4c7e64 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/logicaltypes/Time.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/logicaltypes/Time.java
@@ -29,6 +29,7 @@
  * <p>Its input type is a {@link LocalTime}, and base type is a {@link Long} that represents a count
  * of time in nanoseconds.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class Time implements Schema.LogicalType<LocalTime, Long> {
 
   @Override
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/parser/FieldAccessDescriptorParser.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/parser/FieldAccessDescriptorParser.java
index bd2c519..1534aa4 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/parser/FieldAccessDescriptorParser.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/parser/FieldAccessDescriptorParser.java
@@ -47,6 +47,7 @@
 
 /** Parser for textual field-access selector. */
 @Internal
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class FieldAccessDescriptorParser {
   public static FieldAccessDescriptor parse(String expr) {
     CharStream charStream = CharStreams.fromString(expr);
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/AddFields.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/AddFields.java
index bda292e..8f83223 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/AddFields.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/AddFields.java
@@ -64,6 +64,7 @@
  * }</pre>
  */
 @Experimental(Kind.SCHEMAS)
+@SuppressWarnings({"nullness", "keyfor"}) // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class AddFields {
   public static <T> Inner<T> create() {
     return new Inner<>();
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/Cast.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/Cast.java
index 037a4fe..7b5f162 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/Cast.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/Cast.java
@@ -47,6 +47,7 @@
 /** Set of utilities for casting rows between schemas. */
 @Experimental(Kind.SCHEMAS)
 @AutoValue
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public abstract class Cast<T> extends PTransform<PCollection<T>, PCollection<Row>> {
 
   public abstract Schema outputSchema();
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/CoGroup.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/CoGroup.java
index de065dc..ef937d1 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/CoGroup.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/CoGroup.java
@@ -206,6 +206,7 @@
  * <p>Do note that cross-product joins while simpler and easier to program, can cause performance problems.
  */
 @Experimental(Kind.SCHEMAS)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class CoGroup {
   private static final List NULL_LIST;
 
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/Convert.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/Convert.java
index 3b4d560..ae267c0 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/Convert.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/Convert.java
@@ -35,6 +35,7 @@
 
 /** A set of utilities for converting between different objects supporting schemas. */
 @Experimental(Kind.SCHEMAS)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class Convert {
   /**
    * Convert a {@link PCollection}{@literal <InputT>} into a {@link PCollection}{@literal <Row>}.
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/DropFields.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/DropFields.java
index badf138..96139c4 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/DropFields.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/DropFields.java
@@ -62,6 +62,7 @@
  * }</pre>
  */
 @Experimental(Kind.SCHEMAS)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DropFields {
   public static <T> Inner<T> fields(String... fields) {
     return fields(FieldAccessDescriptor.withFieldNames(fields));
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/Filter.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/Filter.java
index 52ebc75..2ff66ab 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/Filter.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/Filter.java
@@ -81,6 +81,7 @@
  * }</pre>
  */
 @Experimental(Kind.SCHEMAS)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class Filter {
   public static <T> Inner<T> create() {
     return new Inner<T>();
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/Group.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/Group.java
index 1d084b4..d92fedd 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/Group.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/Group.java
@@ -92,6 +92,7 @@
  * ApproximateQuantilesCombineFn in the above example.
  */
 @Experimental(Kind.SCHEMAS)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class Group {
   /**
    * Returns a transform that groups all elements in the input {@link PCollection}. The returned
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/Join.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/Join.java
index e0e607f..90d7fb1 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/Join.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/Join.java
@@ -53,6 +53,7 @@
  * <p>Full outer joins, left outer joins, and right outer joins are also supported.
  */
 @Experimental(Kind.SCHEMAS)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class Join {
   public static final String LHS_TAG = "lhs";
   public static final String RHS_TAG = "rhs";
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/RenameFields.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/RenameFields.java
index e0405bb..c7eba38 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/RenameFields.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/RenameFields.java
@@ -55,6 +55,7 @@
  * }</pre>
  */
 @Experimental(Kind.SCHEMAS)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class RenameFields {
   /** Create an instance of this transform. */
   public static <T> Inner<T> create() {
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/SchemaAggregateFn.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/SchemaAggregateFn.java
index d8ad5b1..61706cb 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/SchemaAggregateFn.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/SchemaAggregateFn.java
@@ -48,6 +48,7 @@
 
 /** This is the builder used by {@link Group} to build up a composed {@link CombineFn}. */
 @Experimental(Kind.SCHEMAS)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class SchemaAggregateFn {
   static Inner create() {
     return new AutoValue_SchemaAggregateFn_Inner.Builder()
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/Select.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/Select.java
index 400df59..b56db38 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/Select.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/transforms/Select.java
@@ -86,6 +86,7 @@
  * }</pre>
  */
 @Experimental(Kind.SCHEMAS)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class Select {
   public static <T> Fields<T> create() {
     return fieldAccess(FieldAccessDescriptor.create());
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/AutoValueUtils.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/AutoValueUtils.java
index 4cdbfa5..f797245 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/AutoValueUtils.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/AutoValueUtils.java
@@ -69,6 +69,7 @@
 
 /** Utilities for managing AutoValue schemas. */
 @Experimental(Kind.SCHEMAS)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class AutoValueUtils {
   public static Class getBaseAutoValueClass(Class<?> clazz) {
     int lastDot = clazz.getName().lastIndexOf('.');
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/AvroByteBuddyUtils.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/AvroByteBuddyUtils.java
index 240d7a7..2eb5a4d 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/AvroByteBuddyUtils.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/AvroByteBuddyUtils.java
@@ -49,6 +49,7 @@
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Maps;
 
 @Experimental(Kind.SCHEMAS)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class AvroByteBuddyUtils {
   private static final ByteBuddy BYTE_BUDDY = new ByteBuddy();
 
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 50b34df..60a176b 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
@@ -97,6 +97,7 @@
 
 /** Utils to convert AVRO records to Beam rows. */
 @Experimental(Kind.SCHEMAS)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class AvroUtils {
   static {
     // This works around a bug in the Avro library (AVRO-1891) around SpecificRecord's handling
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 523776a..ee29d49 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
@@ -93,6 +93,7 @@
 import org.joda.time.base.BaseLocal;
 
 @Internal
+@SuppressWarnings({"nullness", "keyfor"}) // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ByteBuddyUtils {
   private static final ForLoadedType ARRAYS_TYPE = new ForLoadedType(Arrays.class);
   private static final ForLoadedType ARRAY_UTILS_TYPE = new ForLoadedType(ArrayUtils.class);
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/ConvertHelpers.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/ConvertHelpers.java
index a378f1b..9018567 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/ConvertHelpers.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/ConvertHelpers.java
@@ -52,6 +52,7 @@
 
 /** Helper functions for converting between equivalent schema types. */
 @Experimental(Kind.SCHEMAS)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ConvertHelpers {
   /** Return value after converting a schema. */
   public static class ConvertedSchemaInformation<T> implements Serializable {
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/JavaBeanUtils.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/JavaBeanUtils.java
index c0b1042..c7e5b4f 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/JavaBeanUtils.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/JavaBeanUtils.java
@@ -58,6 +58,7 @@
 
 /** A set of utilities to generate getter and setter classes for JavaBean objects. */
 @Experimental(Kind.SCHEMAS)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class JavaBeanUtils {
   /** Create a {@link Schema} for a Java Bean class. */
   public static Schema schemaFromJavaBeanClass(
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/POJOUtils.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/POJOUtils.java
index 65e1ac6..f41f8d6 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/POJOUtils.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/POJOUtils.java
@@ -68,6 +68,7 @@
 
 /** A set of utilities to generate getter and setter classes for POJOs. */
 @Experimental(Kind.SCHEMAS)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class POJOUtils {
   public static Schema schemaFromPojoClass(
       Class<?> clazz, FieldValueTypeSupplier fieldValueTypeSupplier) {
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 81be207..e3b80c2 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
@@ -48,6 +48,7 @@
 
 /** A set of reflection helper methods. */
 @Internal
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ReflectUtils {
   /** Represents a class and a schema. */
   @AutoValue
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/SchemaZipFold.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/SchemaZipFold.java
index 983e0bd..d235bfe 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/SchemaZipFold.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/SchemaZipFold.java
@@ -37,6 +37,7 @@
  * <p>Values returned by `accept` are accumulated.
  */
 @Experimental(Kind.SCHEMAS)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public abstract class SchemaZipFold<T> implements Serializable {
 
   public final T apply(Schema left, Schema right) {
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/SelectByteBuddyHelpers.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/SelectByteBuddyHelpers.java
index c5f29a5..f22bd25 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/SelectByteBuddyHelpers.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/SelectByteBuddyHelpers.java
@@ -71,6 +71,7 @@
 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;
 
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class SelectByteBuddyHelpers {
   private static final ByteBuddy BYTE_BUDDY = new ByteBuddy();
   private static final String SELECT_SCHEMA_FIELD_NAME = "OUTPUTSCHEMA";
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/SelectHelpers.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/SelectHelpers.java
index d08c8ac..9777dcf 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/SelectHelpers.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/schemas/utils/SelectHelpers.java
@@ -43,6 +43,7 @@
 
 /** Helper methods to select subrows out of rows. */
 @Experimental(Kind.SCHEMAS)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SelectHelpers {
 
   private static Schema union(Iterable<Schema> schemas) {
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 c6cb0cb..efe8799 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
@@ -40,6 +40,7 @@
 
 /** A set of utilities for inferring a Beam {@link Schema} from static Java types. */
 @Experimental(Kind.SCHEMAS)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class StaticSchemaInference {
   public static List<FieldValueTypeInformation> sortBySchema(
       List<FieldValueTypeInformation> types, Schema schema) {
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/state/StateSpecs.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/state/StateSpecs.java
index fbe3a00..314ee58 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/state/StateSpecs.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/state/StateSpecs.java
@@ -36,6 +36,7 @@
 
 /** Static methods for working with {@link StateSpec StateSpecs}. */
 @Experimental(Kind.STATE)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class StateSpecs {
 
   private static final CoderRegistry STANDARD_REGISTRY = CoderRegistry.createDefault();
@@ -591,7 +592,7 @@
 
   private static class OrderedListStateSpec<T> implements StateSpec<OrderedListState<T>> {
 
-    @Nullable private Coder<T> elemCoder;
+    private @Nullable Coder<T> elemCoder;
 
     private OrderedListStateSpec(@Nullable Coder<T> elemCoder) {
       this.elemCoder = elemCoder;
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/Annotations.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/Annotations.java
index 22d9952..e766b74 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/Annotations.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/Annotations.java
@@ -24,6 +24,7 @@
 import org.junit.experimental.categories.Category;
 
 /** A utility class for querying annotations. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class Annotations {
 
   /** Annotation predicates. */
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/CoderProperties.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/CoderProperties.java
index e89cacd..bddfcab 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/CoderProperties.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/CoderProperties.java
@@ -54,6 +54,7 @@
  * coder is preserved. This causes tests written such that coders that lose information during
  * serialization or change state during encoding/decoding will fail.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class CoderProperties {
 
   /** All the contexts, for use in test cases. */
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/CombineFnTester.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/CombineFnTester.java
index 383c1f8..6ad3771 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/CombineFnTester.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/CombineFnTester.java
@@ -31,6 +31,7 @@
  * Utilities for testing {@link CombineFn CombineFns}. Ensures that the {@link CombineFn} gives
  * correct results across various permutations and shardings of the input.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class CombineFnTester {
   /**
    * Tests that the the {@link CombineFn}, when applied to the provided input, produces the provided
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/FileChecksumMatcher.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/FileChecksumMatcher.java
index 497d69b..532a571 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/FileChecksumMatcher.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/FileChecksumMatcher.java
@@ -53,6 +53,7 @@
  * <p>Checksum of outputs is generated based on SHA-1 algorithm. If output file is empty, SHA-1 hash
  * of empty string (da39a3ee5e6b4b0d3255bfef95601890afd80709) is used as expected.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class FileChecksumMatcher extends TypeSafeMatcher<ShardedFile>
     implements SerializableMatcher<ShardedFile> {
 
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/JsonMatcher.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/JsonMatcher.java
index 1f66dad..45c2429 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/JsonMatcher.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/JsonMatcher.java
@@ -35,6 +35,7 @@
  *              jsonStringLike("{\"height\": 80, \"name\": \"person\"}"));
  * </pre>
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public abstract class JsonMatcher<T> extends TypeSafeMatcher<T> {
   private Matcher<Map<String, Object>> mapMatcher;
   private static final ObjectMapper MAPPER = new ObjectMapper();
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/PAssert.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/PAssert.java
index 05b1914..689d7dd 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/PAssert.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/PAssert.java
@@ -108,6 +108,7 @@
  *
  * <p>JUnit and Hamcrest must be linked in by any code that uses PAssert.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class PAssert {
   public static final String SUCCESS_COUNTER = "PAssertSuccess";
   public static final String FAILURE_COUNTER = "PAssertFailure";
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/PaneExtractors.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/PaneExtractors.java
index c1213aa..b66489b 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/PaneExtractors.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/PaneExtractors.java
@@ -36,6 +36,7 @@
  * coder of any output {@link PCollection}, as appropriate {@link TypeDescriptor TypeDescriptors}
  * cannot be obtained when the extractor is created.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 final class PaneExtractors {
   private PaneExtractors() {}
 
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/SerializableMatchers.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/SerializableMatchers.java
index 70d35b8..dc2e741 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/SerializableMatchers.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/SerializableMatchers.java
@@ -58,6 +58,7 @@
  * iterable is undefined, use a matcher like {@code kv(equalTo("some key"), containsInAnyOrder(1, 2,
  * 3))}.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SerializableMatchers implements Serializable {
 
   // Serializable only because of capture by anonymous inner classes
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/SourceTestUtils.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/SourceTestUtils.java
index e748946..5717b57 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/SourceTestUtils.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/SourceTestUtils.java
@@ -77,6 +77,7 @@
  *
  * <p>Like {@link PAssert}, requires JUnit and Hamcrest to be present in the classpath.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SourceTestUtils {
   private static final Logger LOG = LoggerFactory.getLogger(SourceTestUtils.class);
 
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/SuccessOrFailure.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/SuccessOrFailure.java
index 820e1a9..73467b9 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/SuccessOrFailure.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/SuccessOrFailure.java
@@ -27,6 +27,7 @@
 
 /** Output of {@link PAssert}. Passed to a conclude function to act upon. */
 @DefaultCoder(SerializableCoder.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public final class SuccessOrFailure implements Serializable {
 
   private final boolean isSuccess;
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipeline.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipeline.java
index 1128980..89737d8 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipeline.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipeline.java
@@ -100,6 +100,7 @@
  * <p>See also the <a href="https://beam.apache.org/contribute/testing/">Testing</a> documentation
  * section.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class TestPipeline extends Pipeline implements TestRule {
 
   private final PipelineOptions options;
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/WindowSupplier.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/WindowSupplier.java
index 0458817..967f619 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/WindowSupplier.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/WindowSupplier.java
@@ -36,6 +36,7 @@
  */
 // Spotbugs thinks synchronization is for the fields, but it is for the act of decoding
 @SuppressFBWarnings("IS2_INCONSISTENT_SYNC")
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 final class WindowSupplier implements Supplier<Collection<BoundedWindow>>, Serializable {
   private final Coder<? extends BoundedWindow> coder;
   private final Collection<byte[]> encodedWindows;
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ApproximateQuantiles.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ApproximateQuantiles.java
index 3f6aa5e..a8f0705 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ApproximateQuantiles.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ApproximateQuantiles.java
@@ -55,6 +55,7 @@
  * {@code PTransform}s for getting an idea of a {@code PCollection}'s data distribution using
  * approximate {@code N}-tiles (e.g. quartiles, percentiles, etc.), either globally or per-key.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ApproximateQuantiles {
   private ApproximateQuantiles() {
     // do not instantiate
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java
index a4be721..1699a7c 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java
@@ -81,6 +81,7 @@
  * href="https://beam.apache.org/documentation/programming-guide/#transforms-combine">documentation</a>
  * for how to use the operations in this class.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class Combine {
   private Combine() {
     // do not instantiate
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/CombineFns.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/CombineFns.java
index b2e142d..3d62ca1 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/CombineFns.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/CombineFns.java
@@ -50,6 +50,10 @@
 import org.checkerframework.checker.nullness.qual.Nullable;
 
 /** Static utility methods that create combine function instances. */
+@SuppressWarnings({
+  "initialization",
+  "nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
+})
 public class CombineFns {
 
   /**
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Create.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Create.java
index bab85d1..20d123a 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Create.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Create.java
@@ -101,7 +101,10 @@
  *
  * @param <T> the type of the elements of the resulting {@code PCollection}
  */
-@SuppressWarnings("OptionalUsedAsFieldOrParameterType")
+@SuppressWarnings({
+  "OptionalUsedAsFieldOrParameterType",
+  "nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
+})
 public class Create<T> {
   /**
    * Returns a new {@code Create.Values} transform that produces a {@link PCollection} containing
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Distinct.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Distinct.java
index 2bae714..cc1705e 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Distinct.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Distinct.java
@@ -61,6 +61,7 @@
  *
  * @param <T> the type of the elements of the input and output {@code PCollection}s
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class Distinct<T> extends PTransform<PCollection<T>, PCollection<T>> {
 
   /**
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnOutputReceivers.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnOutputReceivers.java
index fd7f61d..84e20c1 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnOutputReceivers.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnOutputReceivers.java
@@ -33,6 +33,7 @@
 
 /** Common {@link OutputReceiver} and {@link MultiOutputReceiver} classes. */
 @Internal
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DoFnOutputReceivers {
   private static class RowOutputReceiver<T> implements OutputReceiver<Row> {
     WindowedContextOutputReceiver<T> outputReceiver;
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnSchemaInformation.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnSchemaInformation.java
index 1c54e73..8e066e2 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnSchemaInformation.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnSchemaInformation.java
@@ -40,6 +40,7 @@
 @Experimental(Kind.SCHEMAS)
 @AutoValue
 @Internal
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public abstract class DoFnSchemaInformation implements Serializable {
   /**
    * The schema of the @Element parameter. If the Java type does not match the input PCollection but
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/FlatMapElements.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/FlatMapElements.java
index 0e15b58..e15e86c 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/FlatMapElements.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/FlatMapElements.java
@@ -38,6 +38,7 @@
  * {@code PTransform}s for mapping a simple function that returns iterables over the elements of a
  * {@link PCollection} and merging the results.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class FlatMapElements<InputT, OutputT>
     extends PTransform<PCollection<? extends InputT>, PCollection<OutputT>> {
   private final transient @Nullable TypeDescriptor<InputT> inputType;
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/GroupIntoBatches.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/GroupIntoBatches.java
index 16bf08c..ae3b0f2 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/GroupIntoBatches.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/GroupIntoBatches.java
@@ -83,6 +83,7 @@
  *         .withMaxBufferingDuration(maxBufferingDuration));
  * }</pre>
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class GroupIntoBatches<K, InputT>
     extends PTransform<PCollection<KV<K, InputT>>, PCollection<KV<K, Iterable<InputT>>>> {
 
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/InferableFunction.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/InferableFunction.java
index 464868c..0737a5c 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/InferableFunction.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/InferableFunction.java
@@ -31,6 +31,7 @@
  * <p>See {@link SimpleFunction} for providing robust type information where a {@link
  * SerializableFunction} is required.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public abstract class InferableFunction<InputT, OutputT>
     implements ProcessFunction<InputT, OutputT>, HasDisplayData {
 
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Latest.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Latest.java
index 906e7e0..db5f268 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Latest.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Latest.java
@@ -50,6 +50,7 @@
  *
  * <p>For elements with the same timestamp, the element chosen for output is arbitrary.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class Latest {
   // Do not instantiate
   private Latest() {}
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/MapElements.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/MapElements.java
index 7628613..b10602d 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/MapElements.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/MapElements.java
@@ -36,6 +36,7 @@
 import org.checkerframework.checker.nullness.qual.Nullable;
 
 /** {@code PTransform}s for mapping a simple function over the elements of a {@link PCollection}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class MapElements<InputT, OutputT>
     extends PTransform<PCollection<? extends InputT>, PCollection<OutputT>> {
   private final transient @Nullable TypeDescriptor<InputT> inputType;
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Max.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Max.java
index 37966b7..79c5415 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Max.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Max.java
@@ -43,6 +43,7 @@
  *     .apply(Max.<String>integersPerKey());
  * }</pre>
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class Max {
 
   private Max() {
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Min.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Min.java
index 6690377..70ad23a 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Min.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Min.java
@@ -43,6 +43,7 @@
  *     .apply(Min.<String>integersPerKey());
  * }</pre>
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class Min {
 
   private Min() {
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ParDo.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ParDo.java
index abb64bc..104a519 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ParDo.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ParDo.java
@@ -386,6 +386,7 @@
  * @see <a href= "https://beam.apache.org/documentation/programming-guide/#pardo"> the web
  *     documentation for ParDo</a>
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ParDo {
 
   /**
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/PeriodicSequence.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/PeriodicSequence.java
index 5119e3a..f3bd475 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/PeriodicSequence.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/PeriodicSequence.java
@@ -43,6 +43,7 @@
  * output elements at any time after target time.
  */
 @Experimental(Experimental.Kind.SPLITTABLE_DO_FN)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class PeriodicSequence
     extends PTransform<PCollection<PeriodicSequence.SequenceDefinition>, PCollection<Instant>> {
 
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Regex.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Regex.java
index bd24637..66221bc 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Regex.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Regex.java
@@ -37,6 +37,7 @@
  *
  * <p>Lines that do not match the Regex will not be output.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class Regex {
   private Regex() {
     // do not instantiate
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Reify.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Reify.java
index 540c08d..e6f5029 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Reify.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Reify.java
@@ -36,6 +36,7 @@
  * {@link PTransform PTransforms} for converting between explicit and implicit form of various Beam
  * values.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class Reify {
   private static class ReifyView<K, V> extends PTransform<PCollection<K>, PCollection<KV<K, V>>> {
     private final PCollectionView<V> view;
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Sample.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Sample.java
index 4b93596..6a307f4 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Sample.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Sample.java
@@ -43,6 +43,7 @@
  * <p>{@link #combineFn} can also be used manually, in combination with state and with the {@link
  * Combine} transform.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class Sample {
 
   /** Returns a {@link CombineFn} that computes a fixed-sized uniform sample of its inputs. */
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/SerializableFunctions.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/SerializableFunctions.java
index 78fc89d..4fdfb59 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/SerializableFunctions.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/SerializableFunctions.java
@@ -22,6 +22,7 @@
 import org.checkerframework.checker.nullness.qual.Nullable;
 
 /** Useful {@link SerializableFunction} overrides. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SerializableFunctions {
   public static <InT, OutT extends Serializable> SerializableFunction<InT, OutT> clonesOf(
       final OutT base) {
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/SimpleFunction.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/SimpleFunction.java
index 83c8c60..c77a230 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/SimpleFunction.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/SimpleFunction.java
@@ -26,6 +26,7 @@
  * allow us to infer type information, which in turn aids {@link org.apache.beam.sdk.coders.Coder
  * Coder} inference.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public abstract class SimpleFunction<InputT, OutputT> extends InferableFunction<InputT, OutputT>
     implements SerializableFunction<InputT, OutputT> {
 
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ToString.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ToString.java
index 015e517..94c8ecd 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ToString.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ToString.java
@@ -29,6 +29,7 @@
  * <p><b>Note</b>: For any custom string conversion and formatting, we recommend applying your own
  * {@link ProcessFunction} using {@link MapElements#via(ProcessFunction)}
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public final class ToString {
   private ToString() {
     // do not instantiate
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Top.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Top.java
index 7c39814..6ff8fe5 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Top.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Top.java
@@ -52,6 +52,7 @@
  * PCollection}, or the largest (or smallest) set of values associated with each key in a {@code
  * PCollection} of {@code KV}s.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class Top {
 
   private Top() {
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/View.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/View.java
index f02305f..1bcb9a2 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/View.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/View.java
@@ -133,6 +133,7 @@
  * <p>See {@link ParDo.SingleOutput#withSideInputs} for details on how to access this variable
  * inside a {@link ParDo} over another {@link PCollection}.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class View {
 
   // Do not instantiate
@@ -608,7 +609,7 @@
    * various SDK requested views.
    */
   @Internal
-  static class VoidKeyToMultimapMaterialization<T>
+  public static class VoidKeyToMultimapMaterialization<T>
       extends PTransform<PCollection<T>, PCollection<KV<Void, T>>> {
 
     private static class VoidKeyToMultimapMaterializationDoFn<T> extends DoFn<T, KV<Void, T>> {
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Wait.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Wait.java
index 5788ef6..553f468 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Wait.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Wait.java
@@ -75,6 +75,7 @@
  * </ul>
  */
 @Experimental
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class Wait {
   /** Waits on the given signal collections. */
   public static <T> OnSignal<T> on(PCollection<?>... signals) {
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 258468a..a2fd7be 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
@@ -126,6 +126,7 @@
  * href="https://beam.apache.org/documentation/runners/capability-matrix/">capability matrix</a>.
  */
 @Experimental(Kind.SPLITTABLE_DO_FN)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class Watch {
   private static final Logger LOG = LoggerFactory.getLogger(Watch.class);
 
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/WithFailures.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/WithFailures.java
index 0c03300..15bdf2f 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/WithFailures.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/WithFailures.java
@@ -69,6 +69,7 @@
  * }</pre>
  */
 @Experimental(Kind.WITH_EXCEPTIONS)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class WithFailures {
 
   /**
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/WithKeys.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/WithKeys.java
index fc14928..4cc4639 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/WithKeys.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/WithKeys.java
@@ -53,6 +53,7 @@
  * @param <V> the type of the elements in the input {@code PCollection} and the values in the output
  *     {@code PCollection}
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class WithKeys<K, V> extends PTransform<PCollection<V>, PCollection<KV<K, V>>> {
   /**
    * Returns a {@code PTransform} that takes a {@code PCollection<V>} and returns a {@code
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/display/DisplayData.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/display/DisplayData.java
index fc93a1c..67805ee 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/display/DisplayData.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/display/DisplayData.java
@@ -51,6 +51,7 @@
  *
  * <p>Components specify their display data by implementing the {@link HasDisplayData} interface.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DisplayData implements Serializable {
   private static final DisplayData EMPTY = new DisplayData(Maps.newHashMap());
   private static final DateTimeFormatter TIMESTAMP_FORMATTER = ISODateTimeFormat.dateTime();
@@ -208,16 +209,16 @@
   public abstract static class Item implements Serializable {
 
     /** The path for the display item within a component hierarchy. */
-    @Nullable
     @JsonIgnore
+    @Nullable
     public abstract Path getPath();
 
     /**
      * The namespace for the display item. The namespace defaults to the component which the display
      * item belongs to.
      */
-    @Nullable
     @JsonGetter("namespace")
+    @Nullable
     public abstract Class<?> getNamespace();
 
     /**
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/CoGbkResult.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/CoGbkResult.java
index d559d40..0174b45 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/CoGbkResult.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/CoGbkResult.java
@@ -44,6 +44,7 @@
  * A row result of a {@link CoGroupByKey}. This is a tuple of {@link Iterable}s produced for a given
  * key, and these can be accessed in different ways.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class CoGbkResult {
   /**
    * A map of integer union tags to a list of union objects. Note: the key and the embedded union
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/UnionCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/UnionCoder.java
index 2bdb2ae..a72679a 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/UnionCoder.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/UnionCoder.java
@@ -29,6 +29,7 @@
 import org.apache.beam.sdk.util.common.ElementByteSizeObserver;
 
 /** A UnionCoder encodes RawUnionValues. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class UnionCoder extends StructuredCoder<RawUnionValue> {
   // TODO: Think about how to integrate this with a schema object (i.e.
   // a tuple of tuple tags).
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/ByteBuddyDoFnInvokerFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/ByteBuddyDoFnInvokerFactory.java
index 3cca6fc..504392c 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/ByteBuddyDoFnInvokerFactory.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/ByteBuddyDoFnInvokerFactory.java
@@ -109,6 +109,7 @@
 import org.joda.time.Instant;
 
 /** Dynamically generates a {@link DoFnInvoker} instances for invoking a {@link DoFn}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class ByteBuddyDoFnInvokerFactory implements DoFnInvokerFactory {
 
   public static final String START_BUNDLE_CONTEXT_PARAMETER_METHOD = "startBundleContext";
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/ByteBuddyOnTimerInvokerFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/ByteBuddyOnTimerInvokerFactory.java
index 5df43d2..8bd7017 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/ByteBuddyOnTimerInvokerFactory.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/ByteBuddyOnTimerInvokerFactory.java
@@ -53,6 +53,7 @@
  * Dynamically generates {@link OnTimerInvoker} instances for invoking a particular {@link TimerId}
  * on a particular {@link DoFn}.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class ByteBuddyOnTimerInvokerFactory implements OnTimerInvokerFactory {
 
   @Override
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignature.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignature.java
index 479f399..1c6e579 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignature.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignature.java
@@ -62,6 +62,7 @@
  */
 @AutoValue
 @Internal
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public abstract class DoFnSignature {
   /** Class of the original {@link DoFn} from which this signature was produced. */
   public abstract Class<? extends DoFn<?, ?>> fnClass();
@@ -973,9 +974,8 @@
     public abstract @Nullable TypeDescriptor<?> watermarkEstimatorT();
 
     /** The window type used by this method, if any. */
-    @Nullable
     @Override
-    public abstract TypeDescriptor<? extends BoundedWindow> windowT();
+    public abstract @Nullable TypeDescriptor<? extends BoundedWindow> windowT();
 
     /** Whether this {@link DoFn} returns a {@link ProcessContinuation} or void. */
     public abstract boolean hasReturnValue();
@@ -1051,9 +1051,8 @@
     public abstract boolean requiresStableInput();
 
     /** The window type used by this method, if any. */
-    @Nullable
     @Override
-    public abstract TypeDescriptor<? extends BoundedWindow> windowT();
+    public abstract @Nullable TypeDescriptor<? extends BoundedWindow> windowT();
 
     /** Types of optional parameters of the annotated method, in the order they appear. */
     @Override
@@ -1093,9 +1092,8 @@
     public abstract boolean requiresStableInput();
 
     /** The window type used by this method, if any. */
-    @Nullable
     @Override
-    public abstract TypeDescriptor<? extends BoundedWindow> windowT();
+    public abstract @Nullable TypeDescriptor<? extends BoundedWindow> windowT();
 
     /** Types of optional parameters of the annotated method, in the order they appear. */
     @Override
@@ -1133,9 +1131,8 @@
     public abstract boolean requiresStableInput();
 
     /** The window type used by this method, if any. */
-    @Nullable
     @Override
-    public abstract TypeDescriptor<? extends BoundedWindow> windowT();
+    public abstract @Nullable TypeDescriptor<? extends BoundedWindow> windowT();
 
     /** Types of optional parameters of the annotated method, in the order they appear. */
     @Override
@@ -1270,9 +1267,8 @@
     public abstract TypeDescriptor<?> restrictionT();
 
     /** The window type used by this method, if any. */
-    @Nullable
     @Override
-    public abstract TypeDescriptor<? extends BoundedWindow> windowT();
+    public abstract @Nullable TypeDescriptor<? extends BoundedWindow> windowT();
 
     /** Types of optional parameters of the annotated method, in the order they appear. */
     @Override
@@ -1296,9 +1292,8 @@
     public abstract Method targetMethod();
 
     /** The window type used by this method, if any. */
-    @Nullable
     @Override
-    public abstract TypeDescriptor<? extends BoundedWindow> windowT();
+    public abstract @Nullable TypeDescriptor<? extends BoundedWindow> windowT();
 
     /** Types of parameters of the annotated method, in the order they appear. */
     @Override
@@ -1321,9 +1316,8 @@
     public abstract Method targetMethod();
 
     /** The window type used by this method, if any. */
-    @Nullable
     @Override
-    public abstract TypeDescriptor<? extends BoundedWindow> windowT();
+    public abstract @Nullable TypeDescriptor<? extends BoundedWindow> windowT();
 
     /** Types of parameters of the annotated method, in the order they appear. */
     @Override
@@ -1349,9 +1343,8 @@
     public abstract TypeDescriptor<?> trackerT();
 
     /** The window type used by this method, if any. */
-    @Nullable
     @Override
-    public abstract TypeDescriptor<? extends BoundedWindow> windowT();
+    public abstract @Nullable TypeDescriptor<? extends BoundedWindow> windowT();
 
     /** Types of optional parameters of the annotated method, in the order they appear. */
     @Override
@@ -1375,9 +1368,8 @@
     public abstract Method targetMethod();
 
     /** The window type used by this method, if any. */
-    @Nullable
     @Override
-    public abstract TypeDescriptor<? extends BoundedWindow> windowT();
+    public abstract @Nullable TypeDescriptor<? extends BoundedWindow> windowT();
 
     /** Types of optional parameters of the annotated method, in the order they appear. */
     @Override
@@ -1418,9 +1410,8 @@
     public abstract TypeDescriptor<?> watermarkEstimatorStateT();
 
     /** The window type used by this method, if any. */
-    @Nullable
     @Override
-    public abstract TypeDescriptor<? extends BoundedWindow> windowT();
+    public abstract @Nullable TypeDescriptor<? extends BoundedWindow> windowT();
 
     /** Types of optional parameters of the annotated method, in the order they appear. */
     @Override
@@ -1447,9 +1438,8 @@
     public abstract TypeDescriptor<?> watermarkEstimatorT();
 
     /** The window type used by this method, if any. */
-    @Nullable
     @Override
-    public abstract TypeDescriptor<? extends BoundedWindow> windowT();
+    public abstract @Nullable TypeDescriptor<? extends BoundedWindow> windowT();
 
     /** Types of optional parameters of the annotated method, in the order they appear. */
     @Override
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignatures.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignatures.java
index b54847b..89872d2 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignatures.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignatures.java
@@ -104,6 +104,7 @@
 
 /** Utilities for working with {@link DoFnSignature}. See {@link #getSignature}. */
 @Internal
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DoFnSignatures {
 
   private DoFnSignatures() {}
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 042f3c1..aec3d1b 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
@@ -40,6 +40,7 @@
  * to process.
  */
 @Experimental(Kind.SPLITTABLE_DO_FN)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ByteKeyRangeTracker extends RestrictionTracker<ByteKeyRange, ByteKey>
     implements HasProgress {
   /* An empty range which contains no keys. */
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/splittabledofn/GrowableOffsetRangeTracker.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/splittabledofn/GrowableOffsetRangeTracker.java
index 690911c..f9472bf 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/splittabledofn/GrowableOffsetRangeTracker.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/splittabledofn/GrowableOffsetRangeTracker.java
@@ -36,6 +36,7 @@
  * <p>The growable range is marked as done by claiming {@code Long.MAX_VALUE}.
  */
 @Experimental(Kind.SPLITTABLE_DO_FN)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class GrowableOffsetRangeTracker extends OffsetRangeTracker {
   /**
    * Provides the estimated end offset of the range.
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 35475c0..373fdb1 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
@@ -38,6 +38,7 @@
  * - 1}.
  */
 @Experimental(Kind.SPLITTABLE_DO_FN)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class OffsetRangeTracker extends RestrictionTracker<OffsetRange, Long>
     implements HasProgress {
   protected OffsetRange range;
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/splittabledofn/WatermarkEstimators.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/splittabledofn/WatermarkEstimators.java
index 7dcd970..764df58 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/splittabledofn/WatermarkEstimators.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/splittabledofn/WatermarkEstimators.java
@@ -30,6 +30,7 @@
  * associated {@link DoFn splittable DoFn}s.
  */
 @Experimental(Kind.SPLITTABLE_DO_FN)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class WatermarkEstimators {
   /** Concrete implementation of a {@link ManualWatermarkEstimator}. */
   public static class Manual implements ManualWatermarkEstimator<Instant> {
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterPane.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterPane.java
index 682fc1f..8647e0d 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterPane.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterPane.java
@@ -30,6 +30,7 @@
  * arrived.
  */
 @Experimental(Kind.TRIGGER)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class AfterPane extends OnceTrigger {
 
   private final int countElems;
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterProcessingTime.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterProcessingTime.java
index 6393e49..8ea676f 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterProcessingTime.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterProcessingTime.java
@@ -36,6 +36,7 @@
  * input first arrives.
  */
 @Experimental(Kind.TRIGGER)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class AfterProcessingTime extends OnceTrigger {
 
   private static final PeriodFormatter DURATION_FORMATTER = PeriodFormat.wordBased(Locale.ENGLISH);
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterSynchronizedProcessingTime.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterSynchronizedProcessingTime.java
index 59f2013..63d7c41 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterSynchronizedProcessingTime.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterSynchronizedProcessingTime.java
@@ -31,6 +31,7 @@
  * triggers. In that use, this trigger is ready as soon as all upstream workers processing time
  * clocks have caught up to the moment that input arrived.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class AfterSynchronizedProcessingTime extends OnceTrigger {
 
   public static AfterSynchronizedProcessingTime ofFirstElement() {
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterWatermark.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterWatermark.java
index 509269f..24baca7 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterWatermark.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterWatermark.java
@@ -57,6 +57,7 @@
  * AfterWatermark.pastEndOfWindow.withLateFirings(OnceTrigger)}.
  */
 @Experimental(Kind.TRIGGER)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class AfterWatermark {
 
   private static final String TO_STRING = "AfterWatermark.pastEndOfWindow()";
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/GlobalWindows.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/GlobalWindows.java
index e6cf8d8..b7c6538 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/GlobalWindows.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/GlobalWindows.java
@@ -30,6 +30,7 @@
  * <p>This is the {@link WindowFn} used for data coming from a source, before a {@link Window}
  * transform has been applied.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class GlobalWindows extends NonMergingWindowFn<Object, GlobalWindow> {
 
   private static final Collection<GlobalWindow> GLOBAL_WINDOWS =
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/MergeOverlappingIntervalWindows.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/MergeOverlappingIntervalWindows.java
index b9674de..b5b35f9 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/MergeOverlappingIntervalWindows.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/MergeOverlappingIntervalWindows.java
@@ -30,6 +30,7 @@
  * <p>A utility function for merging overlapping {@link IntervalWindow IntervalWindows}.
  */
 @Internal
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class MergeOverlappingIntervalWindows {
 
   /** Merge overlapping {@link IntervalWindow}s. */
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Never.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Never.java
index e5a90fd..195260e 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Never.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Never.java
@@ -28,6 +28,7 @@
  * <p>Using this trigger will only produce output when the watermark passes the end of the {@link
  * BoundedWindow window} plus the {@link Window#withAllowedLateness allowed lateness}.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public final class Never {
   /**
    * Returns a trigger which never fires. Output will be produced from the using {@link GroupByKey}
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/TimestampCombiner.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/TimestampCombiner.java
index a1eb5f9..0dadf3a 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/TimestampCombiner.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/TimestampCombiner.java
@@ -36,6 +36,7 @@
  * GroupByKey} or {@link Combine}.
  */
 @Experimental(Kind.OUTPUT_TIME)
+@SuppressWarnings({"nullness", "keyfor"}) // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public enum TimestampCombiner {
   /**
    * The policy of taking at the earliest of a set of timestamps.
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Trigger.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Trigger.java
index 2648d0d..021dccf 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Trigger.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Trigger.java
@@ -71,6 +71,7 @@
  * </ul>
  */
 @Experimental(Kind.TRIGGER)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public abstract class Trigger implements Serializable {
 
   protected final List<Trigger> subTriggers;
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Window.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Window.java
index eaaeda3..2005e02 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Window.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Window.java
@@ -134,6 +134,7 @@
  * <p>See {@link Trigger} for details on the available triggers.
  */
 @AutoValue
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public abstract class Window<T> extends PTransform<PCollection<T>, PCollection<T>> {
 
   /**
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ApiSurface.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ApiSurface.java
index bae8125..418120f 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ApiSurface.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ApiSurface.java
@@ -85,8 +85,11 @@
  * {@link #pruningPrefix} to halt the traversal so it does not uselessly catalog references that are
  * not interesting.
  */
-@SuppressWarnings("rawtypes")
 @Internal
+@SuppressWarnings({
+  "rawtypes",
+  "nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
+})
 public class ApiSurface {
   private static final Logger LOG = LoggerFactory.getLogger(ApiSurface.class);
 
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AppliedCombineFn.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AppliedCombineFn.java
index 4c9341f..09cd513 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AppliedCombineFn.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AppliedCombineFn.java
@@ -41,6 +41,7 @@
  * @param <OutputT> type of output values
  */
 @Internal
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class AppliedCombineFn<K, InputT, AccumT, OutputT> implements Serializable {
 
   private final GlobalCombineFn<InputT, AccumT, OutputT> fn;
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CoderUtils.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CoderUtils.java
index 6aa5d09..6e36eda 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CoderUtils.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CoderUtils.java
@@ -31,6 +31,7 @@
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.io.BaseEncoding;
 
 /** Utilities for working with Coders. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public final class CoderUtils {
   private CoderUtils() {} // Non-instantiable
 
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/EmptyOnDeserializationThreadLocal.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/EmptyOnDeserializationThreadLocal.java
index cacb947..5f9abba 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/EmptyOnDeserializationThreadLocal.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/EmptyOnDeserializationThreadLocal.java
@@ -26,6 +26,7 @@
  * A {@link Serializable} {@link ThreadLocal} which discards any "stored" objects. This allows for
  * Kryo to serialize a {@link Coder} as a final field.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class EmptyOnDeserializationThreadLocal<T> extends ThreadLocal<T> implements Serializable {
   private void writeObject(java.io.ObjectOutputStream out) throws IOException {}
 
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ExposedByteArrayOutputStream.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ExposedByteArrayOutputStream.java
index fca7491f..572b8d7 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ExposedByteArrayOutputStream.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ExposedByteArrayOutputStream.java
@@ -27,6 +27,7 @@
  * #writeAndOwn(byte[])}, it will return that array directly.
  */
 @Internal
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ExposedByteArrayOutputStream extends ByteArrayOutputStream {
 
   private byte[] swappedBuffer;
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/FilePatternMatchingShardedFile.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/FilePatternMatchingShardedFile.java
index c997e76..f3e8cd7 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/FilePatternMatchingShardedFile.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/FilePatternMatchingShardedFile.java
@@ -46,6 +46,7 @@
  * guaranteed to provide a consistent result during file pattern matching.
  */
 @Internal
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class FilePatternMatchingShardedFile implements ShardedFile {
 
   private static final Logger LOG = LoggerFactory.getLogger(FilePatternMatchingShardedFile.class);
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/FluentBackoff.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/FluentBackoff.java
index 81e8617..e925387 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/FluentBackoff.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/FluentBackoff.java
@@ -27,6 +27,7 @@
  *
  * @see #DEFAULT for the default configuration parameters.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public final class FluentBackoff {
 
   private static final double DEFAULT_EXPONENT = 1.5;
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/InstanceBuilder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/InstanceBuilder.java
index 4e50967..16f675b 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/InstanceBuilder.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/InstanceBuilder.java
@@ -35,6 +35,7 @@
  *
  * @param <T> type type of object returned by this instance builder
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class InstanceBuilder<T> {
 
   /**
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/MutationDetectors.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/MutationDetectors.java
index d82ac6e..b9483b0 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/MutationDetectors.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/MutationDetectors.java
@@ -26,6 +26,7 @@
 
 /** Static methods for creating and working with {@link MutationDetector}. */
 @Internal
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class MutationDetectors {
   private static final Logger LOG = LoggerFactory.getLogger(MutationDetectors.class);
 
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NameUtils.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NameUtils.java
index 52328f5..6be4769 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NameUtils.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NameUtils.java
@@ -30,6 +30,7 @@
 
 /** Helpers for extracting the name of objects and classes. */
 @Internal
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class NameUtils {
 
   /** Classes may implement this interface to change how names are generated for their instances. */
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NoopLock.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NoopLock.java
index 12a7483..1e49c6e 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NoopLock.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NoopLock.java
@@ -28,6 +28,7 @@
  * is useful as a performance optimization when locking is not necessary but the code paths have to
  * be shared between the locking and the non-locking variant.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class NoopLock implements Lock, Serializable {
 
   private static NoopLock instance;
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NumberedShardedFile.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NumberedShardedFile.java
index 8b70f84..4ecc75c 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NumberedShardedFile.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NumberedShardedFile.java
@@ -46,6 +46,7 @@
  * hardcoded to allow existing uses to work OK.
  */
 @Internal
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class NumberedShardedFile implements ShardedFile {
 
   private static final Logger LOG = LoggerFactory.getLogger(NumberedShardedFile.class);
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Preconditions.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Preconditions.java
index 0597496..e657423 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Preconditions.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Preconditions.java
@@ -31,6 +31,7 @@
  * appropriate exception classes while being static analysis friendly.
  */
 @Internal
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class Preconditions {
   /**
    * Ensures that an object reference passed as a parameter to the calling method is not null.
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReleaseInfo.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReleaseInfo.java
index de537e9..524799d 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReleaseInfo.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReleaseInfo.java
@@ -35,6 +35,7 @@
  */
 @AutoValue
 @Internal
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public abstract class ReleaseInfo implements Serializable {
   private static final String PROPERTIES_PATH = "/org/apache/beam/sdk/sdk.properties";
 
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/RowJson.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/RowJson.java
index eb895f6..9714303 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/RowJson.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/RowJson.java
@@ -81,6 +81,7 @@
  * </ul>
  */
 @Experimental(Kind.SCHEMAS)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class RowJson {
   private static final ImmutableSet<TypeName> SUPPORTED_TYPES =
       ImmutableSet.of(BYTE, INT16, INT32, INT64, FLOAT, DOUBLE, BOOLEAN, STRING, DECIMAL);
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/SerializableThrowable.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/SerializableThrowable.java
index 382236f..536b3d4 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/SerializableThrowable.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/SerializableThrowable.java
@@ -27,6 +27,7 @@
  * A wrapper around {@link Throwable} that preserves the stack trace on serialization, unlike
  * regular {@link Throwable}.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public final class SerializableThrowable implements Serializable {
   private final @Nullable Throwable throwable;
   private final StackTraceElement @Nullable [] stackTrace;
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/SerializableUtils.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/SerializableUtils.java
index 46c16e5..7bf2a7a 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/SerializableUtils.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/SerializableUtils.java
@@ -38,6 +38,7 @@
 import org.xerial.snappy.SnappyOutputStream;
 
 /** Utilities for working with Serializables. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SerializableUtils {
   /**
    * Serializes the argument into an array of bytes, and returns it.
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ShardedKey.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ShardedKey.java
new file mode 100644
index 0000000..bb7de17
--- /dev/null
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ShardedKey.java
@@ -0,0 +1,150 @@
+/*
+ * 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.util;
+
+import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import org.apache.beam.sdk.coders.ByteArrayCoder;
+import org.apache.beam.sdk.coders.StructuredCoder;
+import org.apache.beam.sdk.util.common.ElementByteSizeObserver;
+
+/** A sharded key consisting of a user key and an opaque shard id represented by bytes. */
+public class ShardedKey<K> {
+
+  private final K key;
+  private final byte[] shardId;
+
+  protected ShardedKey(K key, byte[] shardId) {
+    this.key = key;
+    this.shardId = shardId;
+  }
+
+  /**
+   * Creates a ShardedKey with given key and shard id. Shard id must not be null and must not be
+   * mutated.
+   */
+  public static <K> ShardedKey<K> of(K key, byte[] shardId) {
+    checkArgument(key != null, "Key should not be null!");
+    checkArgument(shardId != null, "Shard id should not be null!");
+    return new ShardedKey<K>(key, shardId);
+  }
+
+  public K getKey() {
+    return key;
+  }
+
+  @Override
+  public String toString() {
+    return "ShardedKey{key=" + key + ", shardId=" + Arrays.toString(shardId) + "}";
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (o == this) {
+      return true;
+    }
+    if (o instanceof ShardedKey) {
+      ShardedKey<?> that = (ShardedKey<?>) o;
+      return this.key.equals(that.key) && Arrays.equals(this.shardId, that.shardId);
+    }
+    return false;
+  }
+
+  @Override
+  public int hashCode() {
+    int hash = 1;
+    hash *= 1000003;
+    hash ^= key.hashCode();
+    hash *= 1000003;
+    hash ^= Arrays.hashCode(shardId);
+    return hash;
+  }
+
+  public static class Coder<K> extends StructuredCoder<ShardedKey<K>> {
+
+    private final ByteArrayCoder shardCoder = ByteArrayCoder.of();
+    private final org.apache.beam.sdk.coders.Coder<K> keyCoder;
+
+    private Coder(org.apache.beam.sdk.coders.Coder<K> coder) {
+      keyCoder = coder;
+    }
+
+    public static <K> ShardedKey.Coder<K> of(org.apache.beam.sdk.coders.Coder<K> keyCoder) {
+      return new ShardedKey.Coder<K>(keyCoder);
+    }
+
+    public org.apache.beam.sdk.coders.Coder<K> getKeyCoder() {
+      return keyCoder;
+    }
+
+    @Override
+    public void encode(ShardedKey<K> shardedKey, OutputStream outStream) throws IOException {
+      // The encoding should follow the order:
+      //   shard id byte string
+      //   encoded user key
+      shardCoder.encode(shardedKey.shardId, outStream);
+      keyCoder.encode(shardedKey.key, outStream);
+    }
+
+    @Override
+    public ShardedKey<K> decode(InputStream inStream) throws IOException {
+      byte[] shardId = shardCoder.decode(inStream);
+      K key = keyCoder.decode(inStream);
+      return ShardedKey.of(key, shardId);
+    }
+
+    @Override
+    public List<? extends org.apache.beam.sdk.coders.Coder<?>> getCoderArguments() {
+      return Collections.singletonList(keyCoder);
+    }
+
+    @Override
+    public void verifyDeterministic() throws NonDeterministicException {
+      verifyDeterministic(this, "Key coder must be deterministic", keyCoder);
+    }
+
+    @Override
+    public boolean consistentWithEquals() {
+      return keyCoder.consistentWithEquals();
+    }
+
+    @Override
+    public boolean isRegisterByteSizeObserverCheap(ShardedKey<K> shardedKey) {
+      return shardCoder.isRegisterByteSizeObserverCheap(shardedKey.shardId)
+          && keyCoder.isRegisterByteSizeObserverCheap(shardedKey.key);
+    }
+
+    @Override
+    public Object structuralValue(ShardedKey<K> shardedKey) {
+      return ShardedKey.of(keyCoder.structuralValue(shardedKey.key), shardedKey.shardId);
+    }
+
+    @Override
+    public void registerByteSizeObserver(ShardedKey<K> shardedKey, ElementByteSizeObserver observer)
+        throws Exception {
+      shardCoder.registerByteSizeObserver(shardedKey.shardId, observer);
+      keyCoder.registerByteSizeObserver(shardedKey.key, observer);
+    }
+  }
+}
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/StreamUtils.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/StreamUtils.java
index 7cc4d74..c21717b 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/StreamUtils.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/StreamUtils.java
@@ -26,6 +26,7 @@
 
 /** Utility functions for stream operations. */
 @Internal
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class StreamUtils {
 
   private StreamUtils() {}
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/UserCodeException.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/UserCodeException.java
index 8657155..286e1cb 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/UserCodeException.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/UserCodeException.java
@@ -26,6 +26,7 @@
  * enters user code down through the rest of the user's stack frames until the exception is reached.
  */
 @Internal
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class UserCodeException extends RuntimeException {
 
   public static UserCodeException wrap(Throwable t) {
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ZipFiles.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ZipFiles.java
index fee4a3b..f4bfb97 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ZipFiles.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ZipFiles.java
@@ -45,6 +45,7 @@
  * again.
  */
 @Internal
+@SuppressWarnings({"nullness", "keyfor"}) // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public final class ZipFiles {
   private ZipFiles() {}
 
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/ElementByteSizeObserver.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/ElementByteSizeObserver.java
index 50b62d2..0284d6b 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/ElementByteSizeObserver.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/ElementByteSizeObserver.java
@@ -23,6 +23,7 @@
 
 /** An observer that gets notified when additional bytes are read and/or used. */
 @Internal
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public abstract class ElementByteSizeObserver implements Observer {
   private boolean isLazy = false;
   private long totalSize = 0;
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/ReflectHelpers.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/ReflectHelpers.java
index 8f123d3..bd47037 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/ReflectHelpers.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/ReflectHelpers.java
@@ -42,6 +42,7 @@
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Queues;
 
 /** Utilities for working with with {@link Class Classes} and {@link Method Methods}. */
+@SuppressWarnings({"nullness", "keyfor"}) // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ReflectHelpers {
 
   private static final Joiner COMMA_SEPARATOR = Joiner.on(", ");
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/FailsafeValueInSingleWindow.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/FailsafeValueInSingleWindow.java
new file mode 100644
index 0000000..078299c
--- /dev/null
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/FailsafeValueInSingleWindow.java
@@ -0,0 +1,143 @@
+/*
+ * 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.values;
+
+import com.google.auto.value.AutoValue;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.List;
+import org.apache.beam.sdk.annotations.Internal;
+import org.apache.beam.sdk.coders.InstantCoder;
+import org.apache.beam.sdk.coders.StructuredCoder;
+import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
+import org.checkerframework.checker.nullness.qual.Nullable;
+import org.joda.time.Instant;
+
+/**
+ * An immutable tuple of value, timestamp, window, and pane.
+ *
+ * @param <T> the type of the value
+ */
+@AutoValue
+@Internal
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
+public abstract class FailsafeValueInSingleWindow<T, ErrorT> {
+  /** Returns the value of this {@code FailsafeValueInSingleWindow}. */
+  public abstract @Nullable T getValue();
+
+  /** Returns the timestamp of this {@code FailsafeValueInSingleWindow}. */
+  public abstract Instant getTimestamp();
+
+  /** Returns the window of this {@code FailsafeValueInSingleWindow}. */
+  public abstract BoundedWindow getWindow();
+
+  /** Returns the pane of this {@code FailsafeValueInSingleWindow} in its window. */
+  public abstract PaneInfo getPane();
+
+  /** Returns the failsafe value of this {@code FailsafeValueInSingleWindow}. */
+  public abstract @Nullable ErrorT getFailsafeValue();
+
+  public static <T, ErrorT> FailsafeValueInSingleWindow<T, ErrorT> of(
+      T value, Instant timestamp, BoundedWindow window, PaneInfo paneInfo, ErrorT failsafeValue) {
+    return new AutoValue_FailsafeValueInSingleWindow<>(
+        value, timestamp, window, paneInfo, failsafeValue);
+  }
+
+  /** A coder for {@link FailsafeValueInSingleWindow}. */
+  public static class Coder<T, ErrorT>
+      extends StructuredCoder<FailsafeValueInSingleWindow<T, ErrorT>> {
+    private final org.apache.beam.sdk.coders.Coder<T> valueCoder;
+    private final org.apache.beam.sdk.coders.Coder<ErrorT> failsafeValueCoder;
+    private final org.apache.beam.sdk.coders.Coder<BoundedWindow> windowCoder;
+
+    public static <T, ErrorT> Coder<T, ErrorT> of(
+        org.apache.beam.sdk.coders.Coder<T> valueCoder,
+        org.apache.beam.sdk.coders.Coder<ErrorT> failsafeValueCoder,
+        org.apache.beam.sdk.coders.Coder<? extends BoundedWindow> windowCoder) {
+      return new Coder<>(valueCoder, failsafeValueCoder, windowCoder);
+    }
+
+    @SuppressWarnings({"unchecked", "rawtypes"})
+    Coder(
+        org.apache.beam.sdk.coders.Coder<T> valueCoder,
+        org.apache.beam.sdk.coders.Coder<ErrorT> failsafeValueCoder,
+        org.apache.beam.sdk.coders.Coder<? extends BoundedWindow> windowCoder) {
+      this.valueCoder = valueCoder;
+      this.failsafeValueCoder = failsafeValueCoder;
+      this.windowCoder = (org.apache.beam.sdk.coders.Coder) windowCoder;
+    }
+
+    @Override
+    public void encode(FailsafeValueInSingleWindow<T, ErrorT> windowedElem, OutputStream outStream)
+        throws IOException {
+      encode(windowedElem, outStream, Context.NESTED);
+    }
+
+    @Override
+    public void encode(
+        FailsafeValueInSingleWindow<T, ErrorT> windowedElem,
+        OutputStream outStream,
+        Context context)
+        throws IOException {
+      InstantCoder.of().encode(windowedElem.getTimestamp(), outStream);
+      windowCoder.encode(windowedElem.getWindow(), outStream);
+      PaneInfo.PaneInfoCoder.INSTANCE.encode(windowedElem.getPane(), outStream);
+      valueCoder.encode(windowedElem.getValue(), outStream);
+      failsafeValueCoder.encode(windowedElem.getFailsafeValue(), outStream);
+    }
+
+    @Override
+    public FailsafeValueInSingleWindow<T, ErrorT> decode(InputStream inStream) throws IOException {
+      return decode(inStream, Context.NESTED);
+    }
+
+    @Override
+    public FailsafeValueInSingleWindow<T, ErrorT> decode(InputStream inStream, Context context)
+        throws IOException {
+      Instant timestamp = InstantCoder.of().decode(inStream);
+      BoundedWindow window = windowCoder.decode(inStream);
+      PaneInfo paneInfo = PaneInfo.PaneInfoCoder.INSTANCE.decode(inStream);
+      T value = valueCoder.decode(inStream);
+      ErrorT failsafeValue = failsafeValueCoder.decode(inStream);
+      return new AutoValue_FailsafeValueInSingleWindow<>(
+          value, timestamp, window, paneInfo, failsafeValue);
+    }
+
+    @Override
+    public List<? extends org.apache.beam.sdk.coders.Coder<?>> getCoderArguments() {
+      // Coder arguments are coders for the type parameters of the coder - i.e. T and ErrorT
+      return ImmutableList.of(valueCoder, failsafeValueCoder);
+    }
+
+    @Override
+    public List<? extends org.apache.beam.sdk.coders.Coder<?>> getComponents() {
+      // Coder components are all inner coders that it uses - i.e. both T, ErrorT and BoundedWindow.
+      return ImmutableList.of(valueCoder, failsafeValueCoder, windowCoder);
+    }
+
+    @Override
+    public void verifyDeterministic() throws NonDeterministicException {
+      valueCoder.verifyDeterministic();
+      failsafeValueCoder.verifyDeterministic();
+      windowCoder.verifyDeterministic();
+    }
+  }
+}
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollection.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollection.java
index 443a3f9..02055c1 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollection.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollection.java
@@ -76,6 +76,7 @@
  *
  * @param <T> the type of the elements of this {@link PCollection}
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class PCollection<T> extends PValueBase implements PValue {
 
   /**
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionView.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionView.java
index e7e8884..b729f6a 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionView.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionView.java
@@ -47,6 +47,7 @@
  *
  * @param <T> the type of the value(s) accessible via this {@link PCollectionView}
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public interface PCollectionView<T> extends PValue, Serializable {
   /**
    * <b>For internal use only.</b>
@@ -55,8 +56,8 @@
    *
    * <p>The {@link PCollection} may not be available in all contexts.
    */
-  @Nullable
   @Internal
+  @Nullable
   PCollection<?> getPCollection();
   /**
    * <b>For internal use only.</b>
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionViews.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionViews.java
index fdf6116b..2c44e6b 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionViews.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionViews.java
@@ -81,6 +81,7 @@
  * <p>Implementations of {@link PCollectionView} shared across the SDK.
  */
 @Internal
+@SuppressWarnings({"nullness", "keyfor"}) // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class PCollectionViews {
   public interface TypeDescriptorSupplier<T> extends Supplier<TypeDescriptor<T>>, Serializable {}
 
@@ -196,6 +197,26 @@
   }
 
   /**
+   * Returns a {@code PCollectionView<List<T>>} capable of processing elements windowed using the
+   * provided {@link WindowingStrategy}.
+   *
+   * @deprecated See {@link #listView}.
+   */
+  @Deprecated
+  public static <T, W extends BoundedWindow> PCollectionView<List<T>> listViewUsingVoidKey(
+      PCollection<KV<Void, T>> pCollection,
+      TupleTag<MultimapView<Void, T>> tag,
+      TypeDescriptorSupplier<T> typeDescriptorSupplier,
+      WindowingStrategy<?, W> windowingStrategy) {
+    return new SimplePCollectionView<>(
+        pCollection,
+        tag,
+        new ListViewFn<>(typeDescriptorSupplier),
+        windowingStrategy.getWindowFn().getDefaultWindowMappingFn(),
+        windowingStrategy);
+  }
+
+  /**
    * Returns a {@code PCollectionView<Map<K, V>>} capable of processing elements windowed using the
    * provided {@link WindowingStrategy}.
    */
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PValue.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PValue.java
index 4ac7aa1..04c4ee2 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PValue.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PValue.java
@@ -27,6 +27,7 @@
  * <p>A primitive value within Beam.
  */
 @Internal
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public interface PValue extends POutput, PInput {
 
   /** Returns the name of this {@link PValue}. */
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PValueBase.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PValueBase.java
index 353bb77..ed0b8d2 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PValueBase.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PValueBase.java
@@ -32,6 +32,7 @@
  * PValue}.
  */
 @Internal
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public abstract class PValueBase implements PValue {
 
   private final transient @Nullable Pipeline pipeline;
@@ -105,7 +106,12 @@
 
   @Override
   public String toString() {
-    return (name == null ? "<unnamed>" : getName()) + " [" + getKindString() + "]";
+    return (name == null ? "<unnamed>" : getName())
+        + " ["
+        + getKindString()
+        + "@"
+        + hashCode()
+        + "]";
   }
 
   /**
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PValues.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PValues.java
new file mode 100644
index 0000000..c4183a0
--- /dev/null
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PValues.java
@@ -0,0 +1,127 @@
+/*
+ * 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.values;
+
+import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument;
+
+import java.util.LinkedHashMap;
+import java.util.Map;
+import org.apache.beam.sdk.annotations.Internal;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables;
+
+/**
+ * <b><i>For internal use. No backwards compatibility guarantees.</i></b>
+ *
+ * <p>A primitive value within Beam.
+ */
+@Internal
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
+public class PValues {
+
+  // Do not instantiate
+  private PValues() {}
+
+  /**
+   * Returns all the tagged {@link PCollection PCollections} represented in the given {@link
+   * PValue}.
+   *
+   * <p>For backwards-compatibility, PCollectionView is still a "PValue" to users, which occurs in
+   * only these places:
+   *
+   * <ul>
+   *   <li>{@link POutput#expand} (users can write custom POutputs)
+   *   <li>{@link PInput#expand} (users can write custom PInputs)
+   *   <li>{@link PTransform#getAdditionalInputs} (users can have their composites report inputs not
+   *       passed by {@link PCollection#apply})
+   * </ul>
+   *
+   * <p>These all return {@code Map<TupleTag<?> PValue>}. A user's implementation of these methods
+   * is permitted to return either a {@link PCollection} or a {@link PCollectionView} for each
+   * PValue. PCollection's expand to themselves and {@link PCollectionView} expands to the {@link
+   * PCollection} that it is a view of.
+   */
+  public static Map<TupleTag<?>, PCollection<?>> fullyExpand(
+      Map<TupleTag<?>, PValue> partiallyExpanded) {
+    Map<TupleTag<?>, PCollection<?>> result = new LinkedHashMap<>();
+    for (Map.Entry<TupleTag<?>, PValue> pvalue : partiallyExpanded.entrySet()) {
+      if (pvalue.getValue() instanceof PCollection) {
+        PCollection<?> previous = result.put(pvalue.getKey(), (PCollection<?>) pvalue.getValue());
+        checkArgument(
+            previous == null,
+            "Found conflicting %ss in flattened expansion of %s: %s maps to %s and %s",
+            partiallyExpanded,
+            TupleTag.class.getSimpleName(),
+            pvalue.getKey(),
+            previous,
+            pvalue.getValue());
+      } else {
+        if (pvalue.getValue().expand().size() == 1
+            && Iterables.getOnlyElement(pvalue.getValue().expand().values())
+                .equals(pvalue.getValue())) {
+          throw new IllegalStateException(
+              String.format(
+                  "Non %s %s that expands into itself %s",
+                  PCollection.class.getSimpleName(),
+                  PValue.class.getSimpleName(),
+                  pvalue.getValue()));
+        }
+        /* At this point we know it is a PCollectionView or some internal hacked PValue. To be
+        liberal, we
+        allow it to expand into any number of PCollections, but do not allow structures that
+        require
+        further recursion. */
+        for (Map.Entry<TupleTag<?>, PValue> valueComponent :
+            pvalue.getValue().expand().entrySet()) {
+          if (!(valueComponent.getValue() instanceof PCollection)) {
+            throw new IllegalStateException(
+                String.format(
+                    "A %s contained in %s expanded to a non-%s: %s",
+                    PValue.class.getSimpleName(),
+                    partiallyExpanded,
+                    PCollection.class.getSimpleName(),
+                    valueComponent.getValue()));
+          }
+          PCollection<?> previous =
+              result.put(valueComponent.getKey(), (PCollection<?>) valueComponent.getValue());
+          checkArgument(
+              previous == null,
+              "Found conflicting %ss in flattened expansion of %s: %s maps to %s and %s",
+              partiallyExpanded,
+              TupleTag.class.getSimpleName(),
+              valueComponent.getKey(),
+              previous,
+              valueComponent.getValue());
+        }
+      }
+    }
+    return result;
+  }
+
+  public static Map<TupleTag<?>, PCollection<?>> expandOutput(POutput output) {
+    return fullyExpand(output.expand());
+  }
+
+  public static Map<TupleTag<?>, PCollection<?>> expandInput(PInput input) {
+    return fullyExpand(input.expand());
+  }
+
+  public static Map<TupleTag<?>, PCollection<?>> expandValue(PValue value) {
+    return fullyExpand(value.expand());
+  }
+}
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 ee79832..1a3ab62 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
@@ -86,6 +86,7 @@
  * }</pre>
  */
 @Experimental(Kind.SCHEMAS)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public abstract class Row implements Serializable {
   private final Schema schema;
 
@@ -96,9 +97,8 @@
   // Abstract methods to be implemented by subclasses that handle object access.
 
   /** Get value by field index, {@link ClassCastException} is thrown if schema doesn't match. */
-  @Nullable
   @SuppressWarnings("TypeParameterUnusedInFormals")
-  public abstract <T> T getValue(int fieldIdx);
+  public abstract <T> @Nullable T getValue(int fieldIdx);
 
   /** Return the size of data fields. */
   public abstract int getFieldCount();
@@ -114,9 +114,8 @@
   }
 
   /** Get value by field name, {@link ClassCastException} is thrown if type doesn't match. */
-  @Nullable
   @SuppressWarnings("TypeParameterUnusedInFormals")
-  public <T> T getValue(String fieldName) {
+  public <T> @Nullable T getValue(String fieldName) {
     return getValue(getSchema().indexOf(fieldName));
   }
 
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/RowUtils.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/RowUtils.java
index 8812d67..fa40a2c 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/RowUtils.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/RowUtils.java
@@ -39,6 +39,7 @@
 import org.joda.time.Instant;
 import org.joda.time.base.AbstractInstant;
 
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class RowUtils {
   static class RowPosition {
     FieldAccessDescriptor descriptor;
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 fff0e5d..c9296c0 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
@@ -45,6 +45,7 @@
  * the appropriate fields from the POJO.
  */
 @Experimental(Kind.SCHEMAS)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class RowWithGetters extends Row {
   private final Factory<List<FieldValueGetter>> fieldValueGetterFactory;
   private final Object getterTarget;
@@ -62,10 +63,9 @@
     this.getters = fieldValueGetterFactory.create(getterTarget.getClass(), schema);
   }
 
-  @Nullable
   @Override
   @SuppressWarnings({"TypeParameterUnusedInFormals", "unchecked"})
-  public <T> T getValue(int fieldIdx) {
+  public <T> @Nullable T getValue(int fieldIdx) {
     Field field = getSchema().getField(fieldIdx);
     FieldType type = field.getType();
     Object fieldValue = getters.get(fieldIdx).get(getterTarget);
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/RowWithStorage.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/RowWithStorage.java
index a4f6ae6..6ca5868 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/RowWithStorage.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/RowWithStorage.java
@@ -34,9 +34,8 @@
   }
 
   @Override
-  @Nullable
   @SuppressWarnings("TypeParameterUnusedInFormals")
-  public <T> T getValue(int fieldIdx) {
+  public <T> @Nullable T getValue(int fieldIdx) {
     if (values.size() > fieldIdx) {
       return (T) values.get(fieldIdx);
     } else {
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TaggedPValue.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TaggedPValue.java
index 6b1193c..972bcc8 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TaggedPValue.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TaggedPValue.java
@@ -28,18 +28,19 @@
  */
 @AutoValue
 @Internal
+@SuppressWarnings("keyfor")
 public abstract class TaggedPValue {
-  public static TaggedPValue of(TupleTag<?> tag, PValue value) {
+  public static TaggedPValue of(TupleTag<?> tag, PCollection<?> value) {
     return new AutoValue_TaggedPValue(tag, value);
   }
 
-  public static TaggedPValue ofExpandedValue(PValue value) {
+  public static TaggedPValue ofExpandedValue(PCollection<?> value) {
     return of(Iterables.getOnlyElement(value.expand().keySet()), value);
   }
 
   /** Returns the local tag associated with the {@link PValue}. */
   public abstract TupleTag<?> getTag();
 
-  /** Returns the {@link PValue}. */
-  public abstract PValue getValue();
+  /** Returns the {@link PCollection}. */
+  public abstract PCollection<?> getValue();
 }
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TimestampedValue.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TimestampedValue.java
index 8543ff9..5404325 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TimestampedValue.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TimestampedValue.java
@@ -41,6 +41,7 @@
  *
  * @param <V> the type of the value
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class TimestampedValue<V> {
   /**
    * Returns a new {@link TimestampedValue} with the {@link BoundedWindow#TIMESTAMP_MIN_VALUE
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TypeDescriptor.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TypeDescriptor.java
index 6d5ca6d..1379a05 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TypeDescriptor.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TypeDescriptor.java
@@ -46,6 +46,7 @@
  *
  * @param <T> the type represented by this {@link TypeDescriptor}
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public abstract class TypeDescriptor<T> implements Serializable {
 
   // This class is just a wrapper for TypeToken
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TypeDescriptors.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TypeDescriptors.java
index 8c25748..a0f8066 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TypeDescriptors.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TypeDescriptors.java
@@ -34,6 +34,7 @@
  * extracting type variables of parameterized types (e.g. extracting the {@code OutputT} type
  * variable of a {@code DoFn<InputT, OutputT>}).
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class TypeDescriptors {
   /**
    * The {@link TypeDescriptor} for Boolean. This is the equivalent of:
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TypeParameter.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TypeParameter.java
index b49f2ef..2fa8477 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TypeParameter.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TypeParameter.java
@@ -34,6 +34,7 @@
  * }
  * }</pre>
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public abstract class TypeParameter<T> {
   final TypeVariable<?> typeVariable;
 
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/ValueInSingleWindow.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/ValueInSingleWindow.java
index c200619..9f6684c 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/ValueInSingleWindow.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/ValueInSingleWindow.java
@@ -38,6 +38,7 @@
  */
 @AutoValue
 @Internal
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public abstract class ValueInSingleWindow<T> {
   /** Returns the value of this {@code ValueInSingleWindow}. */
   public abstract @Nullable T getValue();
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/PipelineTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/PipelineTest.java
index 1560f0f..b44a436 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/PipelineTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/PipelineTest.java
@@ -65,7 +65,6 @@
 import org.apache.beam.sdk.values.PCollectionTuple;
 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.TaggedPValue;
 import org.apache.beam.sdk.values.TupleTag;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
@@ -82,6 +81,7 @@
 
 /** Tests for Pipeline. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class PipelineTest {
 
   @Rule public final TestPipeline pipeline = TestPipeline.create();
@@ -470,8 +470,8 @@
       }
 
       @Override
-      public Map<PValue, ReplacementOutput> mapOutputs(
-          Map<TupleTag<?>, PValue> outputs, PCollection<Integer> newOutput) {
+      public Map<PCollection<?>, ReplacementOutput> mapOutputs(
+          Map<TupleTag<?>, PCollection<?>> outputs, PCollection<Integer> newOutput) {
         return Collections.singletonMap(
             newOutput,
             ReplacementOutput.of(
@@ -523,11 +523,12 @@
     }
 
     @Override
-    public Map<PValue, ReplacementOutput> mapOutputs(
-        Map<TupleTag<?>, PValue> outputs, PCollection<Long> newOutput) {
-      Map.Entry<TupleTag<?>, PValue> original = Iterables.getOnlyElement(outputs.entrySet());
-      Map.Entry<TupleTag<?>, PValue> replacement =
-          Iterables.getOnlyElement(newOutput.expand().entrySet());
+    public Map<PCollection<?>, ReplacementOutput> mapOutputs(
+        Map<TupleTag<?>, PCollection<?>> outputs, PCollection<Long> newOutput) {
+      Map.Entry<TupleTag<?>, PCollection<?>> original =
+          Iterables.getOnlyElement(outputs.entrySet());
+      Map.Entry<TupleTag<?>, PCollection<?>> replacement =
+          (Map.Entry) Iterables.getOnlyElement(newOutput.expand().entrySet());
       return Collections.singletonMap(
           newOutput,
           ReplacementOutput.of(
@@ -554,11 +555,12 @@
     }
 
     @Override
-    public Map<PValue, ReplacementOutput> mapOutputs(
-        Map<TupleTag<?>, PValue> outputs, PCollection<T> newOutput) {
-      Map.Entry<TupleTag<?>, PValue> original = Iterables.getOnlyElement(outputs.entrySet());
-      Map.Entry<TupleTag<?>, PValue> replacement =
-          Iterables.getOnlyElement(newOutput.expand().entrySet());
+    public Map<PCollection<?>, ReplacementOutput> mapOutputs(
+        Map<TupleTag<?>, PCollection<?>> outputs, PCollection<T> newOutput) {
+      Map.Entry<TupleTag<?>, PCollection<?>> original =
+          Iterables.getOnlyElement(outputs.entrySet());
+      Map.Entry<TupleTag<?>, PCollection<?>> replacement =
+          (Map.Entry) Iterables.getOnlyElement(newOutput.expand().entrySet());
       return Collections.singletonMap(
           newOutput,
           ReplacementOutput.of(
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/TestUtils.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/TestUtils.java
index d37b50e..8f2ae06 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/TestUtils.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/TestUtils.java
@@ -25,6 +25,7 @@
 import org.hamcrest.TypeSafeMatcher;
 
 /** Utilities for tests. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class TestUtils {
   // Do not instantiate.
   private TestUtils() {}
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/AvroCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/AvroCoderTest.java
index 8c0a685..cdea937 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/AvroCoderTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/AvroCoderTest.java
@@ -85,6 +85,7 @@
 
 /** Tests for {@link AvroCoder}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class AvroCoderTest {
 
   public static final DateTime DATETIME_A =
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/AvroCoderTestPojo.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/AvroCoderTestPojo.java
index d48ae90..1d7d779 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/AvroCoderTestPojo.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/AvroCoderTestPojo.java
@@ -22,6 +22,7 @@
 import org.checkerframework.checker.nullness.qual.Nullable;
 
 /** A Pojo at the top level for use in tests. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class AvroCoderTestPojo {
 
   public String text;
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/BigDecimalCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/BigDecimalCoderTest.java
index 9e53399..871edfc 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/BigDecimalCoderTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/BigDecimalCoderTest.java
@@ -33,6 +33,7 @@
 
 /** Test case for {@link BigDecimalCoder}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BigDecimalCoderTest {
 
   @Rule public ExpectedException thrown = ExpectedException.none();
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/BigEndianIntegerCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/BigEndianIntegerCoderTest.java
index b9730ad..2de47ab 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/BigEndianIntegerCoderTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/BigEndianIntegerCoderTest.java
@@ -33,6 +33,7 @@
 
 /** Test case for {@link BigEndianIntegerCoder}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BigEndianIntegerCoderTest {
 
   private static final Coder<Integer> TEST_CODER = BigEndianIntegerCoder.of();
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/BigEndianLongCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/BigEndianLongCoderTest.java
index 873cac8..df43a70 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/BigEndianLongCoderTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/BigEndianLongCoderTest.java
@@ -33,6 +33,7 @@
 
 /** Test case for {@link BigEndianLongCoder}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BigEndianLongCoderTest {
 
   private static final Coder<Long> TEST_CODER = BigEndianLongCoder.of();
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/BigIntegerCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/BigIntegerCoderTest.java
index d94451d..6a5321c 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/BigIntegerCoderTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/BigIntegerCoderTest.java
@@ -33,6 +33,7 @@
 
 /** Test case for {@link BigIntegerCoder}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BigIntegerCoderTest {
 
   @Rule public ExpectedException thrown = ExpectedException.none();
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/BitSetCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/BitSetCoderTest.java
index 7d87303..194fc13 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/BitSetCoderTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/BitSetCoderTest.java
@@ -34,6 +34,7 @@
 
 /** Tests for {@link BitSetCoder}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BitSetCoderTest {
   private static final Coder<BitSet> TEST_CODER = BitSetCoder.of();
 
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/ByteArrayCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/ByteArrayCoderTest.java
index 2aaa83a..1cb4a68 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/ByteArrayCoderTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/ByteArrayCoderTest.java
@@ -36,6 +36,7 @@
 
 /** Unit tests for {@link ByteArrayCoder}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ByteArrayCoderTest {
 
   private static final ByteArrayCoder TEST_CODER = ByteArrayCoder.of();
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/ByteCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/ByteCoderTest.java
index ad11eb8..9c1c69e 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/ByteCoderTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/ByteCoderTest.java
@@ -33,6 +33,7 @@
 
 /** Test case for {@link ByteCoder}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ByteCoderTest {
 
   private static final Coder<Byte> TEST_CODER = ByteCoder.of();
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CoderRegistryTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CoderRegistryTest.java
index f06d791..3da026b 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CoderRegistryTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CoderRegistryTest.java
@@ -54,6 +54,7 @@
 
 /** Tests for CoderRegistry. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class CoderRegistryTest {
 
   @Rule public TestPipeline pipeline = TestPipeline.create();
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CoderTest.java
index 4ff6130..e3b928c 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CoderTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CoderTest.java
@@ -37,6 +37,7 @@
 
 /** Tests for constructs defined within {@link Coder}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class CoderTest {
   @Rule public ExpectedException expectedException = ExpectedException.none();
 
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CollectionCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CollectionCoderTest.java
index ab81790..dc75f9b 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CollectionCoderTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CollectionCoderTest.java
@@ -38,6 +38,7 @@
 
 /** Test case for {@link CollectionCoder}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class CollectionCoderTest {
 
   private static final Coder<Collection<Integer>> TEST_CODER = CollectionCoder.of(VarIntCoder.of());
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/DoubleCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/DoubleCoderTest.java
index 3e7dd4a..0481dcc 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/DoubleCoderTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/DoubleCoderTest.java
@@ -33,6 +33,7 @@
 
 /** Test case for {@link DoubleCoder}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DoubleCoderTest {
 
   private static final Coder<Double> TEST_CODER = DoubleCoder.of();
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/DurationCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/DurationCoderTest.java
index e49658d..09b7cf6 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/DurationCoderTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/DurationCoderTest.java
@@ -36,6 +36,7 @@
 
 /** Unit tests for {@link DurationCoder}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DurationCoderTest {
 
   private static final DurationCoder TEST_CODER = DurationCoder.of();
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/FloatCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/FloatCoderTest.java
index 17f2629..4cba3b5 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/FloatCoderTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/FloatCoderTest.java
@@ -33,6 +33,7 @@
 
 /** Test case for {@link FloatCoder}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class FloatCoderTest {
 
   private static final Coder<Float> TEST_CODER = FloatCoder.of();
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/InstantCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/InstantCoderTest.java
index 082ea80..5228898 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/InstantCoderTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/InstantCoderTest.java
@@ -39,6 +39,7 @@
 
 /** Unit tests for {@link InstantCoder}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class InstantCoderTest {
 
   private static final InstantCoder TEST_CODER = InstantCoder.of();
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/IterableCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/IterableCoderTest.java
index baa38de..6a91275 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/IterableCoderTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/IterableCoderTest.java
@@ -36,6 +36,7 @@
 
 /** Unit tests for {@link IterableCoder}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class IterableCoderTest {
 
   private static final Coder<Iterable<Integer>> TEST_CODER = IterableCoder.of(VarIntCoder.of());
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/KvCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/KvCoderTest.java
index 3d75eb2..a6c573f 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/KvCoderTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/KvCoderTest.java
@@ -36,6 +36,7 @@
 
 /** Test case for {@link KvCoder}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class KvCoderTest {
   private static class CoderAndData<T> {
     Coder<T> coder;
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/ListCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/ListCoderTest.java
index e793e40..11c0061 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/ListCoderTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/ListCoderTest.java
@@ -38,6 +38,7 @@
 
 /** Unit tests for {@link ListCoder}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ListCoderTest {
 
   private static final Coder<List<Integer>> TEST_CODER = ListCoder.of(VarIntCoder.of());
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/MapCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/MapCoderTest.java
index 788bfd9..4b376bb 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/MapCoderTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/MapCoderTest.java
@@ -40,6 +40,7 @@
 
 /** Unit tests for {@link MapCoder}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class MapCoderTest {
 
   private static final Coder<Map<Integer, String>> TEST_CODER =
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/NullableCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/NullableCoderTest.java
index 461bac1..926bc3f 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/NullableCoderTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/NullableCoderTest.java
@@ -45,6 +45,7 @@
 
 /** Unit tests for {@link NullableCoder}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class NullableCoderTest {
 
   private static final Coder<String> TEST_CODER = NullableCoder.of(StringUtf8Coder.of());
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/PCollectionCustomCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/PCollectionCustomCoderTest.java
index 065a8bf..9d0dd40 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/PCollectionCustomCoderTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/PCollectionCustomCoderTest.java
@@ -53,6 +53,7 @@
 
 /** Tests for coder exception handling in runners. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class PCollectionCustomCoderTest {
   private static final Logger LOG = LoggerFactory.getLogger(PCollectionCustomCoderTest.class);
   /**
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/PrintBase64Encodings.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/PrintBase64Encodings.java
index 5b97ecf..76c2b5c 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/PrintBase64Encodings.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/PrintBase64Encodings.java
@@ -35,6 +35,7 @@
  * -Dexec.args='org.apache.beam.sdk.coders.BigEndianIntegerCoderTest.TEST_CODER \
  * org.apache.beam.sdk.coders.BigEndianIntegerCoderTest.TEST_VALUES' }
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class PrintBase64Encodings {
 
   /** Gets a field even if it is private, which the test data generally will be. */
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/SerializableCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/SerializableCoderTest.java
index 006935b..4adb089 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/SerializableCoderTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/SerializableCoderTest.java
@@ -59,6 +59,7 @@
 
 /** Tests SerializableCoder. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SerializableCoderTest implements Serializable {
   @Rule public ExpectedLogs expectedLogs = ExpectedLogs.none(SerializableCoder.class);
 
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/SetCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/SetCoderTest.java
index 07fe5d8..908e695 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/SetCoderTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/SetCoderTest.java
@@ -37,6 +37,7 @@
 
 /** Test case for {@link SetCoder}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SetCoderTest {
 
   private static final Coder<Set<Integer>> TEST_CODER = SetCoder.of(VarIntCoder.of());
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/StringUtf8CoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/StringUtf8CoderTest.java
index d85f2d5..9b5799b 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/StringUtf8CoderTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/StringUtf8CoderTest.java
@@ -33,6 +33,7 @@
 
 /** Test case for {@link StringUtf8Coder}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class StringUtf8CoderTest {
 
   private static final Coder<String> TEST_CODER = StringUtf8Coder.of();
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/StructuredCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/StructuredCoderTest.java
index e1cd229..acad5fc 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/StructuredCoderTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/StructuredCoderTest.java
@@ -35,6 +35,7 @@
 
 /** Test case for {@link StructuredCoder}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class StructuredCoderTest {
 
   /** A coder for nullable {@code Boolean} values that is consistent with equals. */
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/TextualIntegerCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/TextualIntegerCoderTest.java
index 654826d..0aaab29 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/TextualIntegerCoderTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/TextualIntegerCoderTest.java
@@ -33,6 +33,7 @@
 
 /** Test case for {@link TextualIntegerCoder}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class TextualIntegerCoderTest {
 
   private static final Coder<Integer> TEST_CODER = TextualIntegerCoder.of();
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/VarIntCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/VarIntCoderTest.java
index 718c9b6..ce86a25 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/VarIntCoderTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/VarIntCoderTest.java
@@ -33,6 +33,7 @@
 
 /** Test case for {@link VarIntCoder}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class VarIntCoderTest {
 
   private static final Coder<Integer> TEST_CODER = VarIntCoder.of();
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/VarLongCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/VarLongCoderTest.java
index ca84262..bcb7488 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/VarLongCoderTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/VarLongCoderTest.java
@@ -33,6 +33,7 @@
 
 /** Test case for {@link VarLongCoder}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class VarLongCoderTest {
 
   private static final Coder<Long> TEST_CODER = VarLongCoder.of();
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/AvroIOTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/AvroIOTest.java
index c8b3e56..28f30bd 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/AvroIOTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/AvroIOTest.java
@@ -117,6 +117,7 @@
 import org.junit.runners.Parameterized;
 
 /** Tests for AvroIO Read and Write transforms. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class AvroIOTest implements Serializable {
   /** Unit tests. */
   @RunWith(JUnit4.class)
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/AvroSourceTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/AvroSourceTest.java
index 88dbfd9..5a05174 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/AvroSourceTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/AvroSourceTest.java
@@ -74,6 +74,7 @@
 
 /** Tests for AvroSource. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class AvroSourceTest {
   @Rule public TemporaryFolder tmpFolder = new TemporaryFolder();
 
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/BoundedReadFromUnboundedSourceTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/BoundedReadFromUnboundedSourceTest.java
index 81359f5..36b8410 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/BoundedReadFromUnboundedSourceTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/BoundedReadFromUnboundedSourceTest.java
@@ -57,6 +57,7 @@
 
 /** Unit tests for {@link BoundedReadFromUnboundedSource}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BoundedReadFromUnboundedSourceTest implements Serializable {
   private static final int NUM_RECORDS = 100;
 
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/ClassLoaderFileSystemTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/ClassLoaderFileSystemTest.java
index 2667196..3982102 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/ClassLoaderFileSystemTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/ClassLoaderFileSystemTest.java
@@ -31,6 +31,7 @@
 import org.junit.runners.JUnit4;
 
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ClassLoaderFileSystemTest {
 
   private static final String SOME_CLASS =
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/CompressedSourceTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/CompressedSourceTest.java
index 5c73349..36fba0e 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/CompressedSourceTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/CompressedSourceTest.java
@@ -80,6 +80,7 @@
 
 /** Tests for CompressedSource. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class CompressedSourceTest {
 
   private final double delta = 1e-6;
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/CountingSourceTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/CountingSourceTest.java
index 720dcb9..3f8d8ef 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/CountingSourceTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/CountingSourceTest.java
@@ -56,6 +56,7 @@
 
 /** Tests of {@link CountingSource}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class CountingSourceTest {
 
   public static void addCountingAsserts(PCollection<Long> input, long numElements) {
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/DefaultFilenamePolicyTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/DefaultFilenamePolicyTest.java
index ec959f6..b435209 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/DefaultFilenamePolicyTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/DefaultFilenamePolicyTest.java
@@ -31,6 +31,7 @@
 
 /** Tests of {@link DefaultFilenamePolicy}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DefaultFilenamePolicyTest {
 
   @Before
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/FileBasedSinkTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/FileBasedSinkTest.java
index ec358e4..872d535 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/FileBasedSinkTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/FileBasedSinkTest.java
@@ -73,6 +73,7 @@
 
 /** Tests for {@link FileBasedSink}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class FileBasedSinkTest {
   @Rule public TemporaryFolder tmpFolder = new TemporaryFolder();
 
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/FileBasedSourceTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/FileBasedSourceTest.java
index cd3db3d..d3915ac 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/FileBasedSourceTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/FileBasedSourceTest.java
@@ -66,6 +66,7 @@
 
 /** Tests code common to all file-based sources. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class FileBasedSourceTest {
 
   private Random random = new Random(0L);
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/FileIOTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/FileIOTest.java
index f219ee8..2eb1264 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/FileIOTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/FileIOTest.java
@@ -68,6 +68,7 @@
 
 /** Tests for {@link FileIO}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class FileIOTest implements Serializable {
   @Rule public transient TestPipeline p = TestPipeline.create();
 
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/FileSystemsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/FileSystemsTest.java
index d95d613..bbd94e9 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/FileSystemsTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/FileSystemsTest.java
@@ -50,6 +50,7 @@
 
 /** Tests for {@link FileSystems}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class FileSystemsTest {
 
   @Rule public TemporaryFolder temporaryFolder = new TemporaryFolder();
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/LocalFileSystemTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/LocalFileSystemTest.java
index 8bbe1fb..de8093a 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/LocalFileSystemTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/LocalFileSystemTest.java
@@ -60,6 +60,7 @@
 
 /** Tests for {@link LocalFileSystem}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class LocalFileSystemTest {
   @Rule public ExpectedException thrown = ExpectedException.none();
   @Rule public TemporaryFolder temporaryFolder = new TemporaryFolder();
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/LocalResourceIdTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/LocalResourceIdTest.java
index effcd9d..6463a21 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/LocalResourceIdTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/LocalResourceIdTest.java
@@ -48,6 +48,7 @@
  * <p>TODO: re-enable unicode tests when BEAM-1453 is resolved.
  */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class LocalResourceIdTest {
 
   @Rule public ExpectedException thrown = ExpectedException.none();
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/OffsetBasedSourceTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/OffsetBasedSourceTest.java
index 10de308..b795275 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/OffsetBasedSourceTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/OffsetBasedSourceTest.java
@@ -41,6 +41,7 @@
 
 /** Tests code common to all offset-based sources. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class OffsetBasedSourceTest {
   // An offset-based source with 4 bytes per offset that yields its own current offset
   // and rounds the start and end offset to the nearest multiple of a given number,
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/ReadTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/ReadTest.java
index d1671fd..0b13f50 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/ReadTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/ReadTest.java
@@ -41,6 +41,7 @@
 
 /** Tests for {@link Read}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ReadTest implements Serializable {
   @Rule public transient ExpectedException thrown = ExpectedException.none();
 
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/SimpleSink.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/SimpleSink.java
index 1501a81..58b792d 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/SimpleSink.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/SimpleSink.java
@@ -29,6 +29,7 @@
 /**
  * A simple {@link FileBasedSink} that writes {@link String} values as lines with header and footer.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class SimpleSink<DestinationT> extends FileBasedSink<String, DestinationT, String> {
   public SimpleSink(
       ResourceId tempDirectory,
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TFRecordIOTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TFRecordIOTest.java
index 0be211f..ac8a69c 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TFRecordIOTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TFRecordIOTest.java
@@ -82,6 +82,7 @@
 
 /** Tests for TFRecordIO Read and Write transforms. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class TFRecordIOTest {
 
   /*
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOReadTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOReadTest.java
index 4dd694b..9ef2d3a 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOReadTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOReadTest.java
@@ -95,6 +95,7 @@
 
 /** Tests for {@link TextIO.Read}. */
 @RunWith(Enclosed.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class TextIOReadTest {
   private static final int LINES_NUMBER_FOR_LARGE = 1000;
   private static final List<String> EMPTY = Collections.emptyList();
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOWriteTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOWriteTest.java
index d7846e5..59899e0 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOWriteTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOWriteTest.java
@@ -88,6 +88,7 @@
 
 /** Tests for {@link TextIO.Write}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class TextIOWriteTest {
   private static final String MY_HEADER = "myHeader";
   private static final String MY_FOOTER = "myFooter";
@@ -122,9 +123,8 @@
       return "";
     }
 
-    @Nullable
     @Override
-    public Coder<String> getDestinationCoder() {
+    public @Nullable Coder<String> getDestinationCoder() {
       return StringUtf8Coder.of();
     }
 
@@ -466,9 +466,8 @@
   private static Function<List<String>, List<String>> removeHeaderAndFooter(
       final String header, final String footer) {
     return new Function<List<String>, List<String>>() {
-      @Nullable
       @Override
-      public List<String> apply(List<String> lines) {
+      public @Nullable List<String> apply(List<String> lines) {
         ArrayList<String> newLines = Lists.newArrayList(lines);
         if (header != null) {
           newLines.remove(0);
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextSourceTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextSourceTest.java
index 36a3f68..c7e8aae 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextSourceTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextSourceTest.java
@@ -42,6 +42,7 @@
 
 /** Tests for TextSource class. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class TextSourceTest {
   @Rule public transient TestPipeline pipeline = TestPipeline.create();
 
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/WriteFilesTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/WriteFilesTest.java
index 01bb331..f1d3885 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/WriteFilesTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/WriteFilesTest.java
@@ -105,6 +105,7 @@
 
 /** Tests for the WriteFiles PTransform. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class WriteFilesTest {
   @Rule public TemporaryFolder tmpFolder = new TemporaryFolder();
   @Rule public final TestPipeline p = TestPipeline.create();
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/range/ByteKeyRangeInterpolateKeyTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/range/ByteKeyRangeInterpolateKeyTest.java
index 0835402..fae6ff9 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/range/ByteKeyRangeInterpolateKeyTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/range/ByteKeyRangeInterpolateKeyTest.java
@@ -34,6 +34,7 @@
  * ByteKeyRange#estimateFractionForKey} by converting the interpolated keys back to fractions.
  */
 @RunWith(Parameterized.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ByteKeyRangeInterpolateKeyTest {
   private static final ByteKey[] TEST_KEYS = ByteKeyRangeTest.RANGE_TEST_KEYS;
 
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/range/ByteKeyRangeTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/range/ByteKeyRangeTest.java
index 66a7d5b..1d1ea64 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/range/ByteKeyRangeTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/range/ByteKeyRangeTest.java
@@ -35,6 +35,7 @@
 
 /** Tests for {@link ByteKeyRange}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ByteKeyRangeTest {
   // A set of ranges for testing.
   private static final ByteKeyRange RANGE_1_10 = ByteKeyRange.of(ByteKey.of(1), ByteKey.of(10));
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/metrics/MetricResultsMatchers.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/metrics/MetricResultsMatchers.java
index 0519255..2b3d97b 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/metrics/MetricResultsMatchers.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/metrics/MetricResultsMatchers.java
@@ -23,6 +23,7 @@
 import org.hamcrest.TypeSafeMatcher;
 
 /** Matchers for {@link MetricResults}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class MetricResultsMatchers {
 
   /**
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/metrics/MetricsEnvironmentTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/metrics/MetricsEnvironmentTest.java
index bbf09a6..78de75a 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/metrics/MetricsEnvironmentTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/metrics/MetricsEnvironmentTest.java
@@ -30,6 +30,7 @@
 
 /** Tests for {@link MetricsEnvironment}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class MetricsEnvironmentTest {
   @After
   public void teardown() {
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/metrics/MetricsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/metrics/MetricsTest.java
index 284cba7..00491e8 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/metrics/MetricsTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/metrics/MetricsTest.java
@@ -54,6 +54,7 @@
 import org.mockito.Mockito;
 
 /** Tests for {@link Metrics}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class MetricsTest implements Serializable {
 
   private static final String NS = "test";
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsFactoryTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsFactoryTest.java
index 56d8d60..ccbe139 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsFactoryTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsFactoryTest.java
@@ -86,6 +86,7 @@
 
 /** Tests for {@link PipelineOptionsFactory}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class PipelineOptionsFactoryTest {
   private static final String DEFAULT_RUNNER_NAME = "DirectRunner";
   private static final Class<? extends PipelineRunner<?>> REGISTERED_RUNNER =
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsValidatorTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsValidatorTest.java
index a98d027..533946e 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsValidatorTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsValidatorTest.java
@@ -26,6 +26,7 @@
 
 /** Tests for {@link PipelineOptionsValidator}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class PipelineOptionsValidatorTest {
   @Rule public ExpectedException expectedException = ExpectedException.none();
 
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ProxyInvocationHandlerTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ProxyInvocationHandlerTest.java
index 8bb0a93..7f13556 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ProxyInvocationHandlerTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ProxyInvocationHandlerTest.java
@@ -72,6 +72,7 @@
 
 /** Tests for {@link ProxyInvocationHandler}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ProxyInvocationHandlerTest {
   @Rule public ExpectedException expectedException = ExpectedException.none();
 
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/RemoteEnvironmentOptionsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/RemoteEnvironmentOptionsTest.java
index ef4b164..64d2066 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/RemoteEnvironmentOptionsTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/RemoteEnvironmentOptionsTest.java
@@ -25,6 +25,7 @@
 
 /** Tests for {@link RemoteEnvironmentOptions}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class RemoteEnvironmentOptionsTest {
 
   @Test
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/SdkHarnessOptionsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/SdkHarnessOptionsTest.java
index c574495..a959dae 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/SdkHarnessOptionsTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/SdkHarnessOptionsTest.java
@@ -32,6 +32,7 @@
 
 /** Tests for {@link SdkHarnessOptions}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SdkHarnessOptionsTest {
   private static final ObjectMapper MAPPER =
       new ObjectMapper()
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformHierarchyTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformHierarchyTest.java
index 4c3bc35..8156fc5 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformHierarchyTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformHierarchyTest.java
@@ -71,6 +71,7 @@
 
 /** Tests for {@link TransformHierarchy}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class TransformHierarchyTest implements Serializable {
   @Rule
   public final transient TestPipeline pipeline =
@@ -162,7 +163,7 @@
           }
         });
     thrown.expect(IllegalArgumentException.class);
-    thrown.expectMessage("contains a primitive POutput produced by it");
+    thrown.expectMessage("contains a PCollection produced by it");
     thrown.expectMessage("AddPc");
     thrown.expectMessage("Create");
     thrown.expectMessage(appended.expand().toString());
@@ -189,7 +190,7 @@
     hierarchy.popNode();
 
     thrown.expect(IllegalArgumentException.class);
-    thrown.expectMessage("contains a primitive POutput produced by it");
+    thrown.expectMessage("contains a PCollection produced by it");
     thrown.expectMessage("primitive transforms are permitted to produce");
     thrown.expectMessage("Composite");
     hierarchy.setOutput(comp);
@@ -227,7 +228,7 @@
     hierarchy.setOutput(replacementOutput);
 
     TaggedPValue taggedReplacement = TaggedPValue.ofExpandedValue(replacementOutput);
-    Map<PValue, ReplacementOutput> replacementOutputs =
+    Map<PCollection<?>, ReplacementOutput> replacementOutputs =
         Collections.singletonMap(
             replacementOutput,
             ReplacementOutput.of(TaggedPValue.ofExpandedValue(originalOutput), taggedReplacement));
@@ -294,8 +295,8 @@
     hierarchy.popNode();
     hierarchy.setOutput(replacementOutput.get(longs));
 
-    Entry<TupleTag<?>, PValue> replacementLongs =
-        Iterables.getOnlyElement(replacementOutput.expand().entrySet());
+    Entry<TupleTag<?>, PCollection<?>> replacementLongs =
+        (Map.Entry) Iterables.getOnlyElement(replacementOutput.expand().entrySet());
     hierarchy.replaceOutputs(
         Collections.singletonMap(
             replacementOutput.get(longs),
@@ -456,8 +457,9 @@
     hierarchy.popNode();
     hierarchy.setOutput(replacementOutput.get(longs));
 
-    Entry<TupleTag<?>, PValue> replacementLongs =
-        Iterables.getOnlyElement(replacementOutput.expand().entrySet());
+    Map<TupleTag<?>, PCollection<?>> expandedReplacementOutput = (Map) replacementOutput.expand();
+    Entry<TupleTag<?>, PCollection<?>> replacementLongs =
+        Iterables.getOnlyElement(expandedReplacementOutput.entrySet());
     hierarchy.replaceOutputs(
         Collections.singletonMap(
             replacementOutput.get(longs),
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformTreeTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformTreeTest.java
index 03149fa..a5d8324 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformTreeTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/TransformTreeTest.java
@@ -54,6 +54,7 @@
 
 /** Tests for {@link TransformHierarchy.Node} and {@link TransformHierarchy}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class TransformTreeTest {
 
   @Rule public final TestPipeline p = TestPipeline.create();
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/AutoValueSchemaTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/AutoValueSchemaTest.java
index 0b6aa0b..feffba7 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/AutoValueSchemaTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/AutoValueSchemaTest.java
@@ -41,6 +41,7 @@
 
 /** Tests for {@link AutoValueSchema}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class AutoValueSchemaTest {
   static final DateTime DATE = DateTime.parse("1979-03-14");
   static final byte[] BYTE_ARRAY = "bytearray".getBytes(Charset.defaultCharset());
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 c776d18..d7daa8e 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
@@ -58,6 +58,7 @@
 import org.junit.experimental.categories.Category;
 
 /** Tests for AVRO schema classes. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class AvroSchemaTest {
   /** A test POJO that corresponds to our AVRO schema. */
   public static class AvroSubPojo {
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/FieldAccessDescriptorTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/FieldAccessDescriptorTest.java
index 4b3cf15..a804614 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/FieldAccessDescriptorTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/FieldAccessDescriptorTest.java
@@ -30,6 +30,7 @@
 import org.junit.rules.ExpectedException;
 
 /** Tests for {@link FieldAccessDescriptor}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class FieldAccessDescriptorTest {
   private static final Schema SIMPLE_SCHEMA =
       Schema.builder()
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 81c5ad9..990a8d8 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
@@ -62,6 +62,7 @@
 import org.junit.rules.ExpectedException;
 
 /** Tests for the {@link JavaBeanSchema} schema provider. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class JavaBeanSchemaTest {
   static final DateTime DATE = DateTime.parse("1979-03-14");
   static final byte[] BYTE_ARRAY = "bytearray".getBytes(Charset.defaultCharset());
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 b642109..b077302 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
@@ -71,6 +71,7 @@
 import org.junit.Test;
 
 /** Tests for the {@link JavaFieldSchema} schema provider. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class JavaFieldSchemaTest {
   static final DateTime DATE = DateTime.parse("1979-03-14");
   static final Instant INSTANT = DateTime.parse("1979-03-15").toInstant();
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/SchemaCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/SchemaCoderTest.java
index 8d71180..0e5c072 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/SchemaCoderTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/SchemaCoderTest.java
@@ -49,6 +49,7 @@
 
 /** Unit tests for {@link Schema}. */
 @RunWith(Enclosed.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SchemaCoderTest {
 
   private static final Schema INT32_SCHEMA =
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/SchemaOptionsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/SchemaOptionsTest.java
index 81897a8..f8fcbd5 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/SchemaOptionsTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/SchemaOptionsTest.java
@@ -40,6 +40,7 @@
 import org.junit.rules.ExpectedException;
 
 /** Unit tests for {@link Schema.Options}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SchemaOptionsTest {
 
   private static final String OPTION_NAME = "beam:test:field_i1";
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/SchemaRegistryTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/SchemaRegistryTest.java
index bf39e0e..949bbe2 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/SchemaRegistryTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/SchemaRegistryTest.java
@@ -37,6 +37,7 @@
 import org.junit.rules.ExpectedException;
 
 /** Unit tests for {@link SchemaRegistry}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SchemaRegistryTest {
   static final Schema EMPTY_SCHEMA = Schema.builder().build();
   static final Schema STRING_SCHEMA = Schema.builder().addStringField("string").build();
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 05ed20d..d6be6d9 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
@@ -32,6 +32,7 @@
 import org.junit.rules.ExpectedException;
 
 /** Unit tests for {@link Schema}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SchemaTest {
   @Rule public ExpectedException thrown = ExpectedException.none();
 
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
index f52f360..1dd5dd3 100644
--- 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
@@ -31,6 +31,7 @@
 import org.junit.Test;
 
 /** Unit tests for logical types. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class LogicalTypesTest {
   @Test
   public void testEnumeration() {
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/transforms/AddFieldsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/transforms/AddFieldsTest.java
index 6f9590d..307d99c 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/transforms/AddFieldsTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/transforms/AddFieldsTest.java
@@ -37,6 +37,7 @@
 import org.junit.rules.ExpectedException;
 
 /** Tests for {@link AddFields}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class AddFieldsTest {
   @Rule public final transient TestPipeline pipeline = TestPipeline.create();
   @Rule public transient ExpectedException thrown = ExpectedException.none();
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/transforms/CastValidatorTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/transforms/CastValidatorTest.java
index 46533c3..8b4e2bc 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/transforms/CastValidatorTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/transforms/CastValidatorTest.java
@@ -42,6 +42,7 @@
 /** Tests for {@link Cast.Widening}, {@link Cast.Narrowing}. */
 @Category(UsesSchema.class)
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class CastValidatorTest {
 
   public static final Map<TypeName, Number> NUMERICS =
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 5f2e28d..891aa15 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
@@ -52,6 +52,7 @@
 /** Tests for {@link CoGroup}. */
 @RunWith(JUnit4.class)
 @Category(UsesSchema.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class CoGroupTest {
   @Rule public final transient TestPipeline pipeline = TestPipeline.create();
   @Rule public transient ExpectedException thrown = ExpectedException.none();
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/transforms/ConvertTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/transforms/ConvertTest.java
index 10858be..11cb0a6 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/transforms/ConvertTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/transforms/ConvertTest.java
@@ -45,6 +45,7 @@
 /** Tests for the {@link Convert} class. */
 @RunWith(JUnit4.class)
 @Category(UsesSchema.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ConvertTest {
   @Rule public final transient TestPipeline pipeline = TestPipeline.create();
 
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/transforms/FilterTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/transforms/FilterTest.java
index 9a5e2e4..d8cd1fa 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/transforms/FilterTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/transforms/FilterTest.java
@@ -37,6 +37,7 @@
 /** Test for {@link Filter}. * */
 @RunWith(JUnit4.class)
 @Category(UsesSchema.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class FilterTest {
   @Rule public final transient TestPipeline pipeline = TestPipeline.create();
   @Rule public transient ExpectedException thrown = ExpectedException.none();
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/transforms/GroupTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/transforms/GroupTest.java
index 57c0301..086249a 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/transforms/GroupTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/transforms/GroupTest.java
@@ -68,6 +68,7 @@
 /** Test for {@link Group}. */
 @RunWith(JUnit4.class)
 @Category(UsesSchema.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class GroupTest implements Serializable {
   @Rule public final transient TestPipeline pipeline = TestPipeline.create();
 
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/transforms/JoinTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/transforms/JoinTest.java
index ef8daf1..2e94299 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/transforms/JoinTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/transforms/JoinTest.java
@@ -40,6 +40,7 @@
 /** Tests for {@link org.apache.beam.sdk.schemas.transforms.Join}. */
 @RunWith(JUnit4.class)
 @Category(UsesSchema.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class JoinTest {
   @Rule public final transient TestPipeline pipeline = TestPipeline.create();
 
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 471248d..f3c5de9 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
@@ -46,6 +46,7 @@
 /** Test for {@link Select}. */
 @RunWith(JUnit4.class)
 @Category(UsesSchema.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SelectTest {
   @Rule public final transient TestPipeline pipeline = TestPipeline.create();
   @Rule public transient ExpectedException thrown = ExpectedException.none();
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/AvroGenerators.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/AvroGenerators.java
index cbeceab..d55fcc7 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/AvroGenerators.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/AvroGenerators.java
@@ -34,6 +34,7 @@
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ObjectArrays;
 
 /** QuickCheck generators for AVRO. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class AvroGenerators {
 
   /** Generates arbitrary AVRO schemas. */
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/AvroUtilsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/AvroUtilsTest.java
index 12e4016..94f2313 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/AvroUtilsTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/AvroUtilsTest.java
@@ -64,6 +64,7 @@
 
 /** Tests for conversion between AVRO records and Beam rows. */
 @RunWith(JUnitQuickcheck.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class AvroUtilsTest {
 
   private static final org.apache.avro.Schema NULL_SCHEMA =
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/JavaBeanUtilsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/JavaBeanUtilsTest.java
index 0980ead..c157435 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/JavaBeanUtilsTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/JavaBeanUtilsTest.java
@@ -56,6 +56,7 @@
 import org.junit.Test;
 
 /** Tests for the {@link JavaBeanUtils} class. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class JavaBeanUtilsTest {
   @Test
   public void testNullable() {
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/POJOUtilsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/POJOUtilsTest.java
index 3a675b1..850a550 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/POJOUtilsTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/POJOUtilsTest.java
@@ -55,6 +55,7 @@
 import org.junit.Test;
 
 /** Tests for the {@link POJOUtils} class. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class POJOUtilsTest {
   static final DateTime DATE = DateTime.parse("1979-03-14");
   static final Instant INSTANT = DateTime.parse("1979-03-15").toInstant();
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 7b541d0..b8ac26e 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
@@ -38,6 +38,7 @@
 import org.hamcrest.Description;
 
 /** Utilities for testing schemas. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SchemaTestUtils {
   // 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.
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/SelectHelpersTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/SelectHelpersTest.java
index baff537..c33ba50 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/SelectHelpersTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/schemas/utils/SelectHelpersTest.java
@@ -37,6 +37,7 @@
 
 /** Tests for {@link SelectHelpers}. */
 @RunWith(Parameterized.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SelectHelpersTest {
   @Parameterized.Parameter public boolean useOptimizedSelect;
 
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 65f1f71..b9bc47c 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
@@ -35,6 +35,7 @@
 import org.joda.time.Instant;
 
 /** Various Java Beans and associated schemas used in tests. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class TestJavaBeans {
   /** A Bean containing one nullable and one non-nullable type. */
   @DefaultSchema(JavaBeanSchema.class)
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 4d29dba..f1a78aa 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
@@ -36,6 +36,7 @@
 import org.joda.time.Instant;
 
 /** Various Java POJOs and associated schemas used in tests. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class TestPOJOs {
   /** A POJO containing one nullable and one non-nullable type. */
   @DefaultSchema(JavaFieldSchema.class)
@@ -910,7 +911,7 @@
   /** A simple POJO containing nullable basic types. * */
   @DefaultSchema(JavaFieldSchema.class)
   public static class NullablePOJO {
-    @Nullable public String str;
+    public @Nullable String str;
     public @Nullable Byte aByte;
     public @Nullable Short aShort;
     public @Nullable Integer anInt;
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/state/StateContextsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/state/StateContextsTest.java
index 8233df8..d14b521 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/state/StateContextsTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/state/StateContextsTest.java
@@ -37,6 +37,7 @@
 
 /** Tests for {@link StateContexts}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class StateContextsTest {
   @Rule public ExpectedException thrown = ExpectedException.none();
 
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/CoderPropertiesTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/CoderPropertiesTest.java
index d0d2643..e0f54b1 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/CoderPropertiesTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/CoderPropertiesTest.java
@@ -40,6 +40,7 @@
 
 /** Unit tests for {@link CoderProperties}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class CoderPropertiesTest {
   @Rule public ExpectedException expectedException = ExpectedException.none();
 
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/ExpectedLogs.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/ExpectedLogs.java
index ad97653..d3fcc96 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/ExpectedLogs.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/ExpectedLogs.java
@@ -40,6 +40,7 @@
  * certain log levels. For logs generated via the SLF4J logging frontend, the JUL backend must be
  * used.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ExpectedLogs extends ExternalResource {
   /**
    * Returns a {@link TestRule} that captures logs for the given logger name.
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/ExpectedLogsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/ExpectedLogsTest.java
index 896ca69..495df7a 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/ExpectedLogsTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/ExpectedLogsTest.java
@@ -41,6 +41,7 @@
 
 /** Tests for {@link ExpectedLogs}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ExpectedLogsTest {
   private static final Logger LOG = LoggerFactory.getLogger(ExpectedLogsTest.class);
   private Random random = new Random();
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/FailsafeValueInSingleWindowCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/FailsafeValueInSingleWindowCoderTest.java
new file mode 100644
index 0000000..4c8237b
--- /dev/null
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/FailsafeValueInSingleWindowCoderTest.java
@@ -0,0 +1,64 @@
+/*
+ * 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;
+
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
+import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
+import org.apache.beam.sdk.transforms.windowing.PaneInfo;
+import org.apache.beam.sdk.values.FailsafeValueInSingleWindow;
+import org.joda.time.Duration;
+import org.joda.time.Instant;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/** Tests for {@link FailsafeValueInSingleWindow.Coder}. */
+@RunWith(JUnit4.class)
+public class FailsafeValueInSingleWindowCoderTest {
+  @Test
+  public void testDecodeEncodeEqual() throws Exception {
+    Instant now = Instant.now();
+    FailsafeValueInSingleWindow<String, String> value =
+        FailsafeValueInSingleWindow.of(
+            "foo",
+            now,
+            new IntervalWindow(now, now.plus(Duration.standardSeconds(10))),
+            PaneInfo.NO_FIRING,
+            "bar");
+
+    CoderProperties.coderDecodeEncodeEqual(
+        FailsafeValueInSingleWindow.Coder.of(
+            StringUtf8Coder.of(), StringUtf8Coder.of(), IntervalWindow.getCoder()),
+        value);
+  }
+
+  @Test
+  public void testCoderSerializable() throws Exception {
+    CoderProperties.coderSerializable(
+        FailsafeValueInSingleWindow.Coder.of(
+            StringUtf8Coder.of(), StringUtf8Coder.of(), IntervalWindow.getCoder()));
+  }
+
+  @Test
+  public void testCoderIsSerializableWithWellKnownCoderType() {
+    CoderProperties.coderSerializable(
+        FailsafeValueInSingleWindow.Coder.of(
+            GlobalWindow.Coder.INSTANCE, GlobalWindow.Coder.INSTANCE, GlobalWindow.Coder.INSTANCE));
+  }
+}
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/FileChecksumMatcherTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/FileChecksumMatcherTest.java
index b44036d..6ae347f 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/FileChecksumMatcherTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/FileChecksumMatcherTest.java
@@ -38,6 +38,7 @@
 
 /** Tests for {@link FileChecksumMatcher}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class FileChecksumMatcherTest {
   @Rule public TemporaryFolder tmpFolder = new TemporaryFolder();
   @Rule public ExpectedException thrown = ExpectedException.none();
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/InterceptingUrlClassLoader.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/InterceptingUrlClassLoader.java
index 39e11ef..ea15a17 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/InterceptingUrlClassLoader.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/InterceptingUrlClassLoader.java
@@ -28,6 +28,7 @@
  * original classes definition and is useful for testing code which needs to validate usage with
  * multiple classloaders..
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class InterceptingUrlClassLoader extends ClassLoader {
   private final Predicate<String> test;
 
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/PAssertTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/PAssertTest.java
index 46b0a37..d8a0c47 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/PAssertTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/PAssertTest.java
@@ -71,6 +71,7 @@
 
 /** Test case for {@link PAssert}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class PAssertTest implements Serializable {
 
   @Rule public final transient TestPipeline pipeline = TestPipeline.create();
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/RestoreSystemProperties.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/RestoreSystemProperties.java
index 9190361..68be2a0 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/RestoreSystemProperties.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/RestoreSystemProperties.java
@@ -24,6 +24,7 @@
 import org.junit.rules.TestRule;
 
 /** Saves and restores the current system properties for tests. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class RestoreSystemProperties extends ExternalResource implements TestRule {
   private byte[] originalProperties;
 
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/RestoreSystemPropertiesTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/RestoreSystemPropertiesTest.java
index 1a36586..a6f9651 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/RestoreSystemPropertiesTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/RestoreSystemPropertiesTest.java
@@ -28,6 +28,7 @@
 
 /** Tests for {@link RestoreSystemProperties}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class RestoreSystemPropertiesTest {
   @Rule public TestRule restoreSystemProperties = new RestoreSystemProperties();
 
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/SerializableMatchersTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/SerializableMatchersTest.java
index 6e21cb6..9ffe527 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/SerializableMatchersTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/SerializableMatchersTest.java
@@ -49,6 +49,7 @@
  * boilerplate that is identical to each is considered thoroughly tested.
  */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SerializableMatchersTest implements Serializable {
   @Rule public transient ExpectedException thrown = ExpectedException.none();
 
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/SourceTestUtilsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/SourceTestUtilsTest.java
index 81c002b..4724059 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/SourceTestUtilsTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/SourceTestUtilsTest.java
@@ -34,6 +34,7 @@
 
 /** Tests for {@link SourceTestUtils}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SourceTestUtilsTest {
 
   @Test
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestStreamTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestStreamTest.java
index 841a75c..a9990bf 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestStreamTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestStreamTest.java
@@ -78,6 +78,7 @@
 
 /** Tests for {@link TestStream}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class TestStreamTest implements Serializable {
   @Rule public transient TestPipeline p = TestPipeline.create();
   @Rule public transient ExpectedException thrown = ExpectedException.none();
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/ThreadLeakTracker.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/ThreadLeakTracker.java
index 1ab1127..adf58d8 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/ThreadLeakTracker.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/ThreadLeakTracker.java
@@ -32,6 +32,7 @@
  * Tracks the threads created during a test method execution (or class using @ClassRule) and fails
  * if some still exists after the test method execution.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ThreadLeakTracker implements TestRule {
   private final Field groupField;
 
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ApproximateUniqueTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ApproximateUniqueTest.java
index bfa4f8d..fa47a93 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ApproximateUniqueTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ApproximateUniqueTest.java
@@ -57,6 +57,7 @@
 import org.junit.runners.Parameterized;
 
 /** Tests for the ApproximateUnique transform. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ApproximateUniqueTest implements Serializable {
   // implements Serializable just to make it easy to use anonymous inner DoFn subclasses
 
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineFnsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineFnsTest.java
index 7182bf8..9dc6599 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineFnsTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineFnsTest.java
@@ -60,6 +60,7 @@
 
 /** Unit tests for {@link CombineFns}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class CombineFnsTest {
   @Rule public final TestPipeline p = TestPipeline.create();
   @Rule public ExpectedException expectedException = ExpectedException.none();
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineTest.java
index 28554ac..7deb2e2 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CombineTest.java
@@ -96,6 +96,7 @@
 import org.junit.runners.JUnit4;
 
 /** Tests for {@link Combine} transforms. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class CombineTest implements Serializable {
   // This test is Serializable, just so that it's easy to have
   // anonymous inner classes inside the non-static test methods.
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CreateTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CreateTest.java
index 975aa04..ce6538b 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CreateTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CreateTest.java
@@ -78,7 +78,10 @@
 
 /** Tests for Create. */
 @RunWith(JUnit4.class)
-@SuppressWarnings("unchecked")
+@SuppressWarnings({
+  "unchecked",
+  "nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
+})
 public class CreateTest {
   @Rule public final ExpectedException thrown = ExpectedException.none();
   @Rule public final TestPipeline p = TestPipeline.create();
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DistinctTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DistinctTest.java
index f2e7517..39242ce 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DistinctTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DistinctTest.java
@@ -62,6 +62,7 @@
 
 /** Tests for {@link Distinct}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DistinctTest {
 
   @Rule public final TestPipeline p = TestPipeline.create();
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FilterTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FilterTest.java
index 091c28f..3da1a5e 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FilterTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FilterTest.java
@@ -36,6 +36,7 @@
 
 /** Tests for {@link Filter}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class FilterTest implements Serializable {
 
   static class TrivialFn implements SerializableFunction<Integer, Boolean> {
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FlatMapElementsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FlatMapElementsTest.java
index 9e606b3..f07f24c 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FlatMapElementsTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FlatMapElementsTest.java
@@ -54,6 +54,7 @@
 
 /** Tests for {@link FlatMapElements}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class FlatMapElementsTest implements Serializable {
 
   @Rule public final transient TestPipeline pipeline = TestPipeline.create();
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FlattenTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FlattenTest.java
index 07c8a53..cc79986 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FlattenTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/FlattenTest.java
@@ -77,6 +77,7 @@
 
 /** Tests for Flatten. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class FlattenTest implements Serializable {
 
   @Rule public final transient TestPipeline p = TestPipeline.create();
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/JsonToRowTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/JsonToRowTest.java
index 80c2a5b..4c739b0 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/JsonToRowTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/JsonToRowTest.java
@@ -44,6 +44,7 @@
 /** Unit tests for {@link JsonToRow}. */
 @RunWith(JUnit4.class)
 @Category(UsesSchema.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class JsonToRowTest implements Serializable {
 
   @Rule public transient TestPipeline pipeline = TestPipeline.create();
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/KvSwapTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/KvSwapTest.java
index 725c10b..0fbf89f 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/KvSwapTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/KvSwapTest.java
@@ -36,7 +36,11 @@
 
 /** Tests for KvSwap transform. */
 @RunWith(JUnit4.class)
-@SuppressWarnings({"rawtypes", "unchecked"})
+@SuppressWarnings({
+  "rawtypes",
+  "unchecked",
+  "nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
+})
 public class KvSwapTest {
   private static final KV<String, Integer>[] TABLE =
       new KV[] {
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/LatestFnTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/LatestFnTest.java
index 7e90578..a4e255d 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/LatestFnTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/LatestFnTest.java
@@ -39,6 +39,7 @@
 
 /** Unit tests for {@link Latest.LatestFn}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class LatestFnTest {
   private static final Instant INSTANT = new Instant(100);
   private static final long VALUE = 100 * INSTANT.getMillis();
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/LatestTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/LatestTest.java
index a8aed85..2abc21b 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/LatestTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/LatestTest.java
@@ -46,6 +46,7 @@
 
 /** Unit tests for {@link Latest} {@link PTransform} and {@link Combine.CombineFn}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class LatestTest implements Serializable {
 
   @Rule public final transient TestPipeline p = TestPipeline.create();
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/MapElementsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/MapElementsTest.java
index 390a613..a666e11 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/MapElementsTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/MapElementsTest.java
@@ -55,6 +55,7 @@
 
 /** Tests for {@link MapElements}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class MapElementsTest implements Serializable {
 
   @Rule public final transient TestPipeline pipeline = TestPipeline.create();
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/PTransformTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/PTransformTest.java
index ae63d5b..efbddd1 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/PTransformTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/PTransformTest.java
@@ -37,6 +37,7 @@
 
 /** Tests for {@link PTransform} base class. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class PTransformTest implements Serializable {
 
   @Rule public final transient TestPipeline pipeline = TestPipeline.create();
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoLifecycleTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoLifecycleTest.java
index 5f71f86..39acfa4 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoLifecycleTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoLifecycleTest.java
@@ -62,6 +62,7 @@
 
 /** Tests that {@link ParDo} exercises {@link DoFn} methods in the appropriate sequence. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ParDoLifecycleTest implements Serializable {
 
   @Rule public final transient TestPipeline p = TestPipeline.create();
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoSchemaTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoSchemaTest.java
index 9eee454..1eee1e2 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoSchemaTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoSchemaTest.java
@@ -69,6 +69,7 @@
 /** Test {@link Schema} support. */
 @RunWith(JUnit4.class)
 @Category(UsesSchema.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ParDoSchemaTest implements Serializable {
   @Rule public final transient TestPipeline pipeline = TestPipeline.create();
   @Rule public transient ExpectedException thrown = ExpectedException.none();
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java
index 8d19535..4e757c1 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoTest.java
@@ -162,6 +162,7 @@
 import org.junit.runners.JUnit4;
 
 /** Tests for ParDo. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ParDoTest implements Serializable {
   // This test is Serializable, just so that it's easy to have
   // anonymous inner classes inside the non-static test methods.
@@ -2374,7 +2375,17 @@
 
     @Test
     @Category({ValidatesRunner.class, UsesStatefulParDo.class, UsesOrderedListState.class})
-    public void testOrderedListState() {
+    public void testOrderedListStateBounded() {
+      testOrderedListStateImpl(false);
+    }
+
+    @Test
+    @Category({ValidatesRunner.class, UsesStatefulParDo.class, UsesOrderedListState.class})
+    public void testOrderedListStateUnbounded() {
+      testOrderedListStateImpl(true);
+    }
+
+    void testOrderedListStateImpl(boolean unbounded) {
       final String stateId = "foo";
 
       DoFn<KV<String, TimestampedValue<String>>, Iterable<TimestampedValue<String>>> fn =
@@ -2408,6 +2419,7 @@
                       KV.of("hello", TimestampedValue.of("b", Instant.ofEpochMilli(42))),
                       KV.of("hello", TimestampedValue.of("b", Instant.ofEpochMilli(52))),
                       KV.of("hello", TimestampedValue.of("c", Instant.ofEpochMilli(12)))))
+              .setIsBoundedInternal(unbounded ? IsBounded.UNBOUNDED : IsBounded.BOUNDED)
               .apply(ParDo.of(fn));
 
       List<TimestampedValue<String>> expected =
@@ -2423,7 +2435,17 @@
 
     @Test
     @Category({ValidatesRunner.class, UsesStatefulParDo.class, UsesOrderedListState.class})
-    public void testOrderedListStateRangeFetch() {
+    public void testOrderedListStateRangeFetchBounded() {
+      testOrderedListStateRangeFetchImpl(false);
+    }
+
+    @Test
+    @Category({ValidatesRunner.class, UsesStatefulParDo.class, UsesOrderedListState.class})
+    public void testOrderedListStateRangeFetchUnbounded() {
+      testOrderedListStateRangeFetchImpl(true);
+    }
+
+    void testOrderedListStateRangeFetchImpl(boolean unbounded) {
       final String stateId = "foo";
 
       DoFn<KV<String, TimestampedValue<String>>, Iterable<TimestampedValue<String>>> fn =
@@ -2459,6 +2481,7 @@
                       KV.of("hello", TimestampedValue.of("b", Instant.ofEpochMilli(42))),
                       KV.of("hello", TimestampedValue.of("b", Instant.ofEpochMilli(52))),
                       KV.of("hello", TimestampedValue.of("c", Instant.ofEpochMilli(12)))))
+              .setIsBoundedInternal(unbounded ? IsBounded.UNBOUNDED : IsBounded.BOUNDED)
               .apply(ParDo.of(fn));
 
       List<TimestampedValue<String>> expected1 = Lists.newArrayList();
@@ -2482,7 +2505,17 @@
 
     @Test
     @Category({ValidatesRunner.class, UsesStatefulParDo.class, UsesOrderedListState.class})
-    public void testOrderedListStateRangeDelete() {
+    public void testOrderedListStateRangeDeleteBounded() {
+      testOrderedListStateRangeDeleteImpl(false);
+    }
+
+    @Test
+    @Category({ValidatesRunner.class, UsesStatefulParDo.class, UsesOrderedListState.class})
+    public void testOrderedListStateRangeDeleteUnbounded() {
+      testOrderedListStateRangeDeleteImpl(true);
+    }
+
+    void testOrderedListStateRangeDeleteImpl(boolean unbounded) {
       final String stateId = "foo";
       DoFn<KV<String, TimestampedValue<String>>, Iterable<TimestampedValue<String>>> fn =
           new DoFn<KV<String, TimestampedValue<String>>, Iterable<TimestampedValue<String>>>() {
@@ -2525,6 +2558,7 @@
                       KV.of("hello", TimestampedValue.of("b", Instant.ofEpochMilli(42))),
                       KV.of("hello", TimestampedValue.of("b", Instant.ofEpochMilli(52))),
                       KV.of("hello", TimestampedValue.of("c", Instant.ofEpochMilli(12)))))
+              .setIsBoundedInternal(unbounded ? IsBounded.UNBOUNDED : IsBounded.BOUNDED)
               .apply(ParDo.of(fn));
 
       List<TimestampedValue<String>> expected =
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/PartitionTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/PartitionTest.java
index 7f16cc0..49bb4d3 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/PartitionTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/PartitionTest.java
@@ -44,6 +44,7 @@
 
 /** Tests for {@link Partition}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class PartitionTest implements Serializable {
 
   @Rule public final transient TestPipeline pipeline = TestPipeline.create();
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SampleTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SampleTest.java
index 9f7e89f..61a998e 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SampleTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SampleTest.java
@@ -62,6 +62,7 @@
 import org.junit.runners.Parameterized;
 
 /** Tests for Sample transform. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SampleTest {
   private static final Integer[] EMPTY = new Integer[] {};
   private static final Integer[] DATA = new Integer[] {1, 2, 3, 4, 5};
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SetsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SetsTest.java
index ed07a65..3da327d 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SetsTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SetsTest.java
@@ -36,6 +36,7 @@
 import org.junit.runners.JUnit4;
 
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SetsTest {
 
   @Rule public final TestPipeline p = TestPipeline.create();
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SimpleStatsFnsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SimpleStatsFnsTest.java
index b04beb4..7a5502a 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SimpleStatsFnsTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SimpleStatsFnsTest.java
@@ -29,6 +29,7 @@
 
 /** Tests of Min, Max, Mean, and Sum. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SimpleStatsFnsTest {
   static final double DOUBLE_COMPARISON_ACCURACY = 1e-7;
 
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SplittableDoFnTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SplittableDoFnTest.java
index 8d85601..7037698 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SplittableDoFnTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SplittableDoFnTest.java
@@ -83,6 +83,7 @@
  * Tests for <a href="https://s.apache.org/splittable-do-fn">splittable</a> {@link DoFn} behavior.
  */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SplittableDoFnTest implements Serializable {
 
   static class PairStringWithIndexToLengthBase extends DoFn<String, KV<String, Integer>> {
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ViewTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ViewTest.java
index 4014a3c..0a2edf4 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ViewTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ViewTest.java
@@ -78,6 +78,7 @@
  */
 @RunWith(JUnit4.class)
 @Category(UsesSideInputs.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ViewTest implements Serializable {
   // This test is Serializable, just so that it's easy to have
   // anonymous inner classes inside the non-static test methods.
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/WaitTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/WaitTest.java
index 914d503..13cfbe1 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/WaitTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/WaitTest.java
@@ -52,6 +52,7 @@
 
 /** Tests for {@link Wait}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class WaitTest implements Serializable {
   @Rule public transient TestPipeline p = TestPipeline.create();
 
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 5d8154b..de15b02 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
@@ -81,6 +81,7 @@
 
 /** Tests for {@link Watch}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class WatchTest implements Serializable {
 
   @Rule public transient TestPipeline p = TestPipeline.create();
@@ -334,17 +335,15 @@
             outputs -> {
               Function<TimestampedValue<Integer>, Integer> extractValueFn =
                   new Function<TimestampedValue<Integer>, Integer>() {
-                    @Nullable
                     @Override
-                    public Integer apply(@Nullable TimestampedValue<Integer> input) {
+                    public @Nullable Integer apply(@Nullable TimestampedValue<Integer> input) {
                       return input.getValue();
                     }
                   };
               Function<TimestampedValue<Integer>, Instant> extractTimestampFn =
                   new Function<TimestampedValue<Integer>, Instant>() {
-                    @Nullable
                     @Override
-                    public Instant apply(@Nullable TimestampedValue<Integer> input) {
+                    public @Nullable Instant apply(@Nullable TimestampedValue<Integer> input) {
                       return input.getTimestamp();
                     }
                   };
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/WithFailuresTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/WithFailuresTest.java
index 827331b..50f6929 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/WithFailuresTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/WithFailuresTest.java
@@ -43,6 +43,7 @@
 
 /** Tests for {@link WithFailures}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class WithFailuresTest implements Serializable {
 
   @Rule public final transient TestPipeline pipeline = TestPipeline.create();
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/WithTimestampsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/WithTimestampsTest.java
index 93b0c48..b2f801b8 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/WithTimestampsTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/WithTimestampsTest.java
@@ -38,6 +38,7 @@
 
 /** Tests for {@link WithTimestamps}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class WithTimestampsTest implements Serializable {
 
   @Rule public final transient TestPipeline p = TestPipeline.create();
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataEvaluator.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataEvaluator.java
index 5cdfc41..f6ebc0b 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataEvaluator.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataEvaluator.java
@@ -36,6 +36,7 @@
 /**
  * Test utilities to evaluate the {@link DisplayData} in the context of a {@link PipelineRunner}.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DisplayDataEvaluator {
   private final PipelineOptions options;
 
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataMatchers.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataMatchers.java
index de5d0ab..852f09c 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataMatchers.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataMatchers.java
@@ -34,6 +34,7 @@
 import org.joda.time.Instant;
 
 /** Hamcrest matcher for making assertions on {@link DisplayData} instances. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DisplayDataMatchers {
   /** Do not instantiate. */
   private DisplayDataMatchers() {}
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataTest.java
index 8914500..9513a35 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataTest.java
@@ -79,6 +79,7 @@
 
 /** Tests for {@link DisplayData} class. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DisplayDataTest implements Serializable {
   @Rule public transient ExpectedException thrown = ExpectedException.none();
 
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/join/CoGroupByKeyTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/join/CoGroupByKeyTest.java
index 09553d1..c2ca308 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/join/CoGroupByKeyTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/join/CoGroupByKeyTest.java
@@ -56,6 +56,7 @@
 
 /** Tests for CoGroupByKeyTest. Implements Serializable for anonymous DoFns. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class CoGroupByKeyTest implements Serializable {
 
   /**
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 c95596a..a87247f 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
@@ -86,6 +86,7 @@
 
 /** Tests for {@link DoFnInvokers}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DoFnInvokersTest {
   @Rule public ExpectedException thrown = ExpectedException.none();
 
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnSignaturesSplittableDoFnTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnSignaturesSplittableDoFnTest.java
index 26a0867..3b56e48 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnSignaturesSplittableDoFnTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnSignaturesSplittableDoFnTest.java
@@ -68,8 +68,8 @@
  * Tests for {@link DoFnSignatures} focused on methods related to <a
  * href="https://s.apache.org/splittable-do-fn">splittable</a> {@link DoFn}.
  */
-@SuppressWarnings("unused")
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DoFnSignaturesSplittableDoFnTest {
   @Rule public ExpectedException thrown = ExpectedException.none();
 
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnSignaturesTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnSignaturesTest.java
index df64637..277c0f1 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnSignaturesTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnSignaturesTest.java
@@ -94,6 +94,7 @@
 
 /** Tests for {@link DoFnSignatures}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DoFnSignaturesTest {
 
   @Rule public ExpectedException thrown = ExpectedException.none();
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/OnTimerInvokersTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/OnTimerInvokersTest.java
index dbf2e2e..1126b9f 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/OnTimerInvokersTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/OnTimerInvokersTest.java
@@ -39,6 +39,7 @@
 
 /** Tests for {@link DoFnInvokers}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class OnTimerInvokersTest {
   @Rule public ExpectedException thrown = ExpectedException.none();
 
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/testhelper/DoFnInvokersTestHelper.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/testhelper/DoFnInvokersTestHelper.java
index 05f066c..ff7c199 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/testhelper/DoFnInvokersTestHelper.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/testhelper/DoFnInvokersTestHelper.java
@@ -32,6 +32,7 @@
  * Test helper for {@link DoFnInvokersTest}, which needs to test package-private access to DoFns in
  * other packages.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DoFnInvokersTestHelper {
 
   private static class StaticPrivateDoFn extends DoFn<String, String> {
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 1ebb4fe..7dac768 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
@@ -35,6 +35,7 @@
 
 /** Tests for {@link ByteKeyRangeTrackerTest}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ByteKeyRangeTrackerTest {
   @Rule public final ExpectedException expected = ExpectedException.none();
 
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/splittabledofn/GrowableOffsetRangeTrackerTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/splittabledofn/GrowableOffsetRangeTrackerTest.java
index c2446f6..9e7afaa 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/splittabledofn/GrowableOffsetRangeTrackerTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/splittabledofn/GrowableOffsetRangeTrackerTest.java
@@ -35,6 +35,7 @@
 
 /** Tests for {@link GrowableOffsetRangeTracker}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class GrowableOffsetRangeTrackerTest {
   private static class SimpleEstimator implements GrowableOffsetRangeTracker.RangeEndEstimator {
     private long estimateRangeEnd = 0;
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 b2c381c..b703735 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
@@ -34,6 +34,7 @@
 
 /** Tests for {@link OffsetRangeTracker}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class OffsetRangeTrackerTest {
   @Rule public final ExpectedException expected = ExpectedException.none();
 
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
index 47c54bc..bdad4fa 100644
--- 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
@@ -24,6 +24,7 @@
 import org.junit.runners.JUnit4;
 
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SplitResultTest {
   @Test
   public void testPrimaryAndResidualAreSet() {
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/RepeatedlyTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/RepeatedlyTest.java
index 55cb77e..80baf08 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/RepeatedlyTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/RepeatedlyTest.java
@@ -33,6 +33,7 @@
 
 /** Tests for {@link Repeatedly}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class RepeatedlyTest {
 
   @Mock private Trigger mockTrigger;
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/StubTrigger.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/StubTrigger.java
index d4ee23b..c758713 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/StubTrigger.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/StubTrigger.java
@@ -22,6 +22,7 @@
 import org.joda.time.Instant;
 
 /** No-op {@link OnceTrigger} implementation for testing. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 abstract class StubTrigger extends Trigger.OnceTrigger {
   /**
    * Create a stub {@link Trigger} instance which returns the specified name on {@link #toString()}.
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/TriggerTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/TriggerTest.java
index 335d967..e8967e8 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/TriggerTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/TriggerTest.java
@@ -30,6 +30,7 @@
 
 /** Tests for {@link Trigger}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class TriggerTest {
 
   @Test
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/WindowTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/WindowTest.java
index b0483d2..93b3ac3 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/WindowTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/WindowTest.java
@@ -84,6 +84,7 @@
 
 /** Tests for {@link Window}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class WindowTest implements Serializable {
 
   @Rule
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/CombineFnUtilTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/CombineFnUtilTest.java
index 084b067..f848753 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/CombineFnUtilTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/CombineFnUtilTest.java
@@ -40,6 +40,7 @@
 
 /** Unit tests for {@link CombineFnUtil}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class CombineFnUtilTest {
 
   @Rule public ExpectedException expectedException = ExpectedException.none();
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/FilePatternMatchingShardedFileTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/FilePatternMatchingShardedFileTest.java
index 87044cb..4d1364e 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/FilePatternMatchingShardedFileTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/FilePatternMatchingShardedFileTest.java
@@ -43,6 +43,7 @@
 
 /** Tests for {@link FilePatternMatchingShardedFile}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class FilePatternMatchingShardedFileTest {
   @Rule public TemporaryFolder tmpFolder = new TemporaryFolder();
   @Rule public ExpectedException thrown = ExpectedException.none();
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/NameUtilsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/NameUtilsTest.java
index f327c3c..cf8d9f9 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/NameUtilsTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/NameUtilsTest.java
@@ -33,6 +33,7 @@
 
 /** Tests for {@link NameUtils}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class NameUtilsTest {
 
   @Rule public ExpectedException thrown = ExpectedException.none();
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/NumberedShardedFileTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/NumberedShardedFileTest.java
index 5f0faeb..05d2c95 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/NumberedShardedFileTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/NumberedShardedFileTest.java
@@ -44,6 +44,7 @@
 
 /** Tests for {@link NumberedShardedFile}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class NumberedShardedFileTest {
   @Rule public TemporaryFolder tmpFolder = new TemporaryFolder();
   @Rule public ExpectedException thrown = ExpectedException.none();
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 d614295..303e51f 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
@@ -53,6 +53,7 @@
 
 /** Unit tests for {@link RowJson.RowJsonDeserializer} and {@link RowJson.RowJsonSerializer}. */
 @RunWith(Enclosed.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class RowJsonTest {
   @RunWith(Parameterized.class)
   public static class ValueTests {
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/SerializableUtilsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/SerializableUtilsTest.java
index 9f96bce..e919fdb 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/SerializableUtilsTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/SerializableUtilsTest.java
@@ -39,6 +39,7 @@
 
 /** Tests for {@link SerializableUtils}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SerializableUtilsTest {
   @Rule public ExpectedException expectedException = ExpectedException.none();
 
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ShardedKeyTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ShardedKeyTest.java
new file mode 100644
index 0000000..553b51c
--- /dev/null
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ShardedKeyTest.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.util;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.junit.Assert.assertEquals;
+
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.testing.CoderProperties;
+import org.junit.Test;
+
+public class ShardedKeyTest {
+
+  private static final String KEY = "key";
+  private static final byte[] EMPTY_SHARD = new byte[0];
+  private static final byte[] SHARD = "shard_id".getBytes(UTF_8);
+
+  @Test
+  public void testStructuralValueEqual() throws Exception {
+    Coder<ShardedKey<String>> coder = ShardedKey.Coder.of(StringUtf8Coder.of());
+    CoderProperties.coderSerializable(coder);
+    CoderProperties.structuralValueDecodeEncodeEqual(coder, ShardedKey.of(KEY, SHARD));
+    CoderProperties.structuralValueDecodeEncodeEqual(coder, ShardedKey.of(KEY, EMPTY_SHARD));
+    CoderProperties.structuralValueConsistentWithEquals(
+        coder,
+        ShardedKey.of(KEY, "shard_id".getBytes(UTF_8)),
+        ShardedKey.of(KEY, "shard_id".getBytes(UTF_8)));
+    CoderProperties.structuralValueConsistentWithEquals(
+        coder, ShardedKey.of(KEY, new byte[0]), ShardedKey.of(KEY, new byte[0]));
+  }
+
+  @Test
+  public void testDecodeEncodeEqual() throws Exception {
+    Coder<ShardedKey<String>> coder = ShardedKey.Coder.of(StringUtf8Coder.of());
+    CoderProperties.coderDecodeEncodeEqual(coder, ShardedKey.of(KEY, SHARD));
+    CoderProperties.coderDecodeEncodeEqual(coder, ShardedKey.of(KEY, EMPTY_SHARD));
+    CoderProperties.coderConsistentWithEquals(
+        coder,
+        ShardedKey.of(KEY, "shard_id".getBytes(UTF_8)),
+        ShardedKey.of(KEY, "shard_id".getBytes(UTF_8)));
+    CoderProperties.coderConsistentWithEquals(
+        coder, ShardedKey.of(KEY, new byte[0]), ShardedKey.of(KEY, new byte[0]));
+    CoderProperties.coderDeterministic(
+        coder,
+        ShardedKey.of(KEY, "shard_id".getBytes(UTF_8)),
+        ShardedKey.of(KEY, "shard_id".getBytes(UTF_8)));
+    CoderProperties.coderDeterministic(
+        coder, ShardedKey.of(KEY, new byte[0]), ShardedKey.of(KEY, new byte[0]));
+  }
+
+  @Test
+  public void testEquality() {
+    assertEquals(ShardedKey.of("key", new byte[0]), ShardedKey.of("key", new byte[0]));
+    assertEquals(
+        ShardedKey.of("key", "shard_id".getBytes(UTF_8)),
+        ShardedKey.of("key", "shard_id".getBytes(UTF_8)));
+  }
+}
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/StreamUtilsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/StreamUtilsTest.java
index 68f87d7..7b686ca 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/StreamUtilsTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/StreamUtilsTest.java
@@ -33,6 +33,7 @@
 
 /** Unit tests for {@link ExposedByteArrayInputStream}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class StreamUtilsTest {
 
   private byte[] testData = null;
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/UnownedInputStreamTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/UnownedInputStreamTest.java
index b265780..33e3c63 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/UnownedInputStreamTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/UnownedInputStreamTest.java
@@ -30,6 +30,7 @@
 
 /** Unit tests for {@link UnownedInputStream}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class UnownedInputStreamTest {
   @Rule public ExpectedException expectedException = ExpectedException.none();
   private ByteArrayInputStream bais;
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/UnownedOutputStreamTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/UnownedOutputStreamTest.java
index 2aa7b52..91920c5 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/UnownedOutputStreamTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/UnownedOutputStreamTest.java
@@ -33,6 +33,7 @@
 
 /** Unit tests for {@link UnownedOutputStream}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class UnownedOutputStreamTest {
   @Rule public ExpectedException expectedException = ExpectedException.none();
   private ByteArrayOutputStream baos;
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/UserCodeExceptionTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/UserCodeExceptionTest.java
index 08c7aa4..1a0e23c 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/UserCodeExceptionTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/UserCodeExceptionTest.java
@@ -37,6 +37,7 @@
 
 /** Tests for {@link UserCodeException} functionality. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class UserCodeExceptionTest {
 
   @Rule public ExpectedException thrown = ExpectedException.none();
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ZipFilesTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ZipFilesTest.java
index f9e2e12..dd79fa0 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ZipFilesTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ZipFilesTest.java
@@ -50,6 +50,7 @@
  * fine.
  */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ZipFilesTest {
   @Rule public TemporaryFolder tmpFolder = new TemporaryFolder();
   private File tmpDir;
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/common/ReflectHelpersTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/common/ReflectHelpersTest.java
index 3a6830b..30aca1a 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/common/ReflectHelpersTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/common/ReflectHelpersTest.java
@@ -36,6 +36,7 @@
 
 /** Tests for {@link ReflectHelpers}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ReflectHelpersTest {
 
   @Test
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/values/KVTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/values/KVTest.java
index 44c0824..6554ff5 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/values/KVTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/values/KVTest.java
@@ -30,6 +30,7 @@
 
 /** Tests for KV. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class KVTest {
   private static final Integer[] TEST_VALUES = {
     null, Integer.MIN_VALUE, -1, 0, 1, Integer.MAX_VALUE
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/values/RowTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/values/RowTest.java
index 043c19d..ab322b4 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/values/RowTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/values/RowTest.java
@@ -46,6 +46,7 @@
 import org.junit.rules.ExpectedException;
 
 /** Unit tests for {@link Row}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class RowTest {
 
   @Rule public ExpectedException thrown = ExpectedException.none();
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/values/TimestampedValueTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/values/TimestampedValueTest.java
index 2a246fe..ca2ef44 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/values/TimestampedValueTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/values/TimestampedValueTest.java
@@ -33,6 +33,7 @@
 
 /** Unit tests for {@link TimestampedValue}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class TimestampedValueTest {
   @Rule public ExpectedException thrown = ExpectedException.none();
 
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/values/TypeDescriptorTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/values/TypeDescriptorTest.java
index dbad49c..26b7a09 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/values/TypeDescriptorTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/values/TypeDescriptorTest.java
@@ -32,6 +32,7 @@
 
 /** Tests for TypeDescriptor. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class TypeDescriptorTest {
 
   @Rule public transient ExpectedException thrown = ExpectedException.none();
diff --git a/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/AssignEventTime.java b/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/AssignEventTime.java
index b29e198..5bf53629 100644
--- a/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/AssignEventTime.java
+++ b/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/AssignEventTime.java
@@ -48,6 +48,7 @@
  */
 @Audience(Audience.Type.CLIENT)
 @Derived(state = StateComplexity.ZERO, repartitions = 0)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class AssignEventTime<InputT> extends Operator<InputT>
     implements CompositeOperator<InputT, InputT> {
 
diff --git a/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/CountByKey.java b/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/CountByKey.java
index c8fffe6..8aa742b 100644
--- a/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/CountByKey.java
+++ b/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/CountByKey.java
@@ -64,6 +64,7 @@
  */
 @Audience(Audience.Type.CLIENT)
 @Derived(state = StateComplexity.CONSTANT, repartitions = 1)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class CountByKey<InputT, KeyT> extends ShuffleOperator<InputT, KeyT, KV<KeyT, Long>>
     implements CompositeOperator<InputT, KV<KeyT, Long>> {
 
diff --git a/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/Distinct.java b/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/Distinct.java
index 3cac655..21d2e9c 100644
--- a/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/Distinct.java
+++ b/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/Distinct.java
@@ -73,6 +73,7 @@
             + "(e.g. using bloom filters), which might reduce the space complexity",
     state = StateComplexity.CONSTANT,
     repartitions = 1)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class Distinct<InputT, KeyT> extends ShuffleOperator<InputT, KeyT, InputT>
     implements CompositeOperator<InputT, InputT> {
 
diff --git a/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/Filter.java b/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/Filter.java
index 21c53e7..a6ad307 100644
--- a/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/Filter.java
+++ b/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/Filter.java
@@ -49,6 +49,7 @@
  */
 @Audience(Audience.Type.CLIENT)
 @Derived(state = StateComplexity.ZERO, repartitions = 0)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class Filter<InputT> extends Operator<InputT> implements CompositeOperator<InputT, InputT> {
 
   /**
diff --git a/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/FlatMap.java b/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/FlatMap.java
index 1a59630..60d2500 100644
--- a/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/FlatMap.java
+++ b/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/FlatMap.java
@@ -82,6 +82,7 @@
  */
 @Audience(Audience.Type.CLIENT)
 @Basic(state = StateComplexity.ZERO, repartitions = 0)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class FlatMap<InputT, OutputT> extends Operator<OutputT>
     implements TypeAware.Output<OutputT> {
 
diff --git a/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/FullJoin.java b/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/FullJoin.java
index e97098d..20b10d4 100644
--- a/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/FullJoin.java
+++ b/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/FullJoin.java
@@ -54,6 +54,7 @@
  * </ol>
  */
 @Audience(Audience.Type.CLIENT)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class FullJoin {
 
   /**
diff --git a/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/Join.java b/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/Join.java
index d5a0bdc..218c357 100644
--- a/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/Join.java
+++ b/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/Join.java
@@ -76,6 +76,7 @@
             + "complexity",
     state = StateComplexity.LINEAR,
     repartitions = 1)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class Join<LeftT, RightT, KeyT, OutputT>
     extends ShuffleOperator<Object, KeyT, KV<KeyT, OutputT>> {
 
diff --git a/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/LeftJoin.java b/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/LeftJoin.java
index ffd9717..cfabfcf 100644
--- a/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/LeftJoin.java
+++ b/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/LeftJoin.java
@@ -53,6 +53,7 @@
  * </ol>
  */
 @Audience(Audience.Type.CLIENT)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class LeftJoin {
 
   /**
diff --git a/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/MapElements.java b/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/MapElements.java
index 9ec50b7..fb91ed9 100644
--- a/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/MapElements.java
+++ b/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/MapElements.java
@@ -50,6 +50,7 @@
  */
 @Audience(Audience.Type.CLIENT)
 @Derived(state = StateComplexity.ZERO, repartitions = 0)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class MapElements<InputT, OutputT> extends Operator<OutputT>
     implements CompositeOperator<InputT, OutputT>, TypeAware.Output<OutputT> {
 
diff --git a/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/ReduceByKey.java b/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/ReduceByKey.java
index 7b33aa6..01a4e53 100644
--- a/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/ReduceByKey.java
+++ b/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/ReduceByKey.java
@@ -96,6 +96,7 @@
             + "can be efficiently used in the executor-specific implementation",
     state = StateComplexity.CONSTANT_IF_COMBINABLE,
     repartitions = 1)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ReduceByKey<InputT, KeyT, ValueT, AccT, OutputT>
     extends ShuffleOperator<InputT, KeyT, KV<KeyT, OutputT>> implements TypeAware.Value<ValueT> {
 
diff --git a/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/ReduceWindow.java b/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/ReduceWindow.java
index c343499..2f9d48c 100644
--- a/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/ReduceWindow.java
+++ b/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/ReduceWindow.java
@@ -77,6 +77,7 @@
  */
 @Audience(Audience.Type.CLIENT)
 @Derived(state = StateComplexity.CONSTANT_IF_COMBINABLE, repartitions = 1)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ReduceWindow<InputT, ValueT, AccT, OutputT>
     extends ShuffleOperator<InputT, Byte, OutputT>
     implements TypeAware.Value<ValueT>, CompositeOperator<InputT, OutputT> {
diff --git a/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/RightJoin.java b/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/RightJoin.java
index ae27bf6..55e1292 100644
--- a/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/RightJoin.java
+++ b/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/RightJoin.java
@@ -54,6 +54,7 @@
  * </ol>
  */
 @Audience(Audience.Type.CLIENT)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class RightJoin {
 
   /**
diff --git a/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/SumByKey.java b/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/SumByKey.java
index c05840d..b48ac0b 100644
--- a/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/SumByKey.java
+++ b/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/SumByKey.java
@@ -76,6 +76,7 @@
  */
 @Audience(Audience.Type.CLIENT)
 @Derived(state = StateComplexity.CONSTANT, repartitions = 1)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SumByKey<InputT, KeyT> extends ShuffleOperator<InputT, KeyT, KV<KeyT, Long>>
     implements CompositeOperator<InputT, KV<KeyT, Long>> {
 
diff --git a/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/TopPerKey.java b/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/TopPerKey.java
index 3c9edd6..ba51b60 100644
--- a/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/TopPerKey.java
+++ b/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/TopPerKey.java
@@ -81,6 +81,7 @@
  */
 @Audience(Audience.Type.CLIENT)
 @Derived(state = StateComplexity.CONSTANT, repartitions = 1)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class TopPerKey<InputT, KeyT, ValueT, ScoreT extends Comparable<ScoreT>>
     extends ShuffleOperator<InputT, KeyT, Triple<KeyT, ValueT, ScoreT>>
     implements TypeAware.Value<ValueT>, CompositeOperator<InputT, Triple<KeyT, ValueT, ScoreT>> {
diff --git a/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/Union.java b/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/Union.java
index 9dbefa8..7beae3b 100644
--- a/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/Union.java
+++ b/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/Union.java
@@ -66,6 +66,7 @@
  */
 @Audience(Audience.Type.CLIENT)
 @Basic(state = StateComplexity.ZERO, repartitions = 0)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class Union<InputT> extends Operator<InputT> {
 
   /**
diff --git a/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/WindowBuilder.java b/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/WindowBuilder.java
index 57b8d51..5cef366f 100644
--- a/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/WindowBuilder.java
+++ b/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/WindowBuilder.java
@@ -36,6 +36,7 @@
  *
  * @param <T> type of windowed element
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class WindowBuilder<T>
     implements Builders.WindowBy<WindowBuilder<T>>,
         Builders.TriggeredBy<WindowBuilder<T>>,
diff --git a/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/base/Builders.java b/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/base/Builders.java
index eb488c7..a59fe49 100644
--- a/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/base/Builders.java
+++ b/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/base/Builders.java
@@ -47,6 +47,7 @@
  * <p>For internal usage only.
  */
 @Audience(Audience.Type.INTERNAL)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class Builders {
 
   /**
diff --git a/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/type/TypeUtils.java b/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/type/TypeUtils.java
index cfdb94f..6faea5b 100644
--- a/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/type/TypeUtils.java
+++ b/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/client/type/TypeUtils.java
@@ -25,6 +25,7 @@
 import org.checkerframework.checker.nullness.qual.Nullable;
 
 /** A collections of {@link TypeDescriptor} construction methods. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class TypeUtils {
 
   /**
diff --git a/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/BroadcastHashJoinTranslator.java b/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/BroadcastHashJoinTranslator.java
index c0f1585..2e7b298 100644
--- a/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/BroadcastHashJoinTranslator.java
+++ b/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/BroadcastHashJoinTranslator.java
@@ -48,6 +48,7 @@
  *   <li>Key extractor of broadcast side has to be the same {@link UnaryFunction} instance
  * </ul>
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BroadcastHashJoinTranslator<LeftT, RightT, KeyT, OutputT>
     extends AbstractJoinTranslator<LeftT, RightT, KeyT, OutputT> {
 
diff --git a/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/ExtractKey.java b/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/ExtractKey.java
index 654172a..3b7c4aa 100644
--- a/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/ExtractKey.java
+++ b/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/ExtractKey.java
@@ -30,6 +30,7 @@
 import org.apache.beam.sdk.values.TypeDescriptors;
 
 /** Key extracting utility transformation shared among operator translators. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class ExtractKey<KeyT, ValueT>
     extends PTransform<PCollection<ValueT>, PCollection<KV<KeyT, ValueT>>> {
 
diff --git a/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/JoinTranslator.java b/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/JoinTranslator.java
index c5599eb..c8d1236 100644
--- a/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/JoinTranslator.java
+++ b/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/JoinTranslator.java
@@ -34,6 +34,7 @@
 import org.checkerframework.checker.nullness.qual.Nullable;
 
 /** {@link OperatorTranslator Translator } for Euphoria {@link Join} operator. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class JoinTranslator<LeftT, RightT, KeyT, OutputT>
     extends AbstractJoinTranslator<LeftT, RightT, KeyT, OutputT> {
 
diff --git a/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/LazyAccumulatorProvider.java b/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/LazyAccumulatorProvider.java
index bfe35b9..f91be7a 100644
--- a/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/LazyAccumulatorProvider.java
+++ b/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/LazyAccumulatorProvider.java
@@ -26,6 +26,7 @@
 import org.apache.beam.sdk.extensions.euphoria.core.client.accumulators.Timer;
 
 /** Instantiate accumulator provider on the first usage. Thus {@link Serializable}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class LazyAccumulatorProvider implements AccumulatorProvider, Serializable {
 
   private final Factory factory;
diff --git a/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/OperatorTransform.java b/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/OperatorTransform.java
index 460ea54..af7e1cd 100644
--- a/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/OperatorTransform.java
+++ b/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/OperatorTransform.java
@@ -31,6 +31,7 @@
  * @param <OutputT> type of output elements
  * @param <OperatorT> type of operator to expand
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class OperatorTransform<InputT, OutputT, OperatorT extends Operator<OutputT>>
     extends PTransform<PCollectionList<InputT>, PCollection<OutputT>> {
 
diff --git a/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/ReduceByKeyTranslator.java b/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/ReduceByKeyTranslator.java
index eb3e025..a17f40e 100644
--- a/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/ReduceByKeyTranslator.java
+++ b/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/ReduceByKeyTranslator.java
@@ -50,6 +50,7 @@
 import org.checkerframework.checker.nullness.qual.Nullable;
 
 /** Translator for {@code ReduceByKey} operator. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ReduceByKeyTranslator<InputT, KeyT, ValueT, OutputT>
     implements OperatorTranslator<
         InputT, KV<KeyT, OutputT>, ReduceByKey<InputT, KeyT, ValueT, ?, OutputT>> {
diff --git a/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/SingleValueContext.java b/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/SingleValueContext.java
index 8cff66b..4387763 100644
--- a/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/SingleValueContext.java
+++ b/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/SingleValueContext.java
@@ -31,6 +31,7 @@
  * will free the value as soon as {@code getAndResetValue()} is called.
  */
 @Audience(Audience.Type.EXECUTOR)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SingleValueContext<T> implements Context, Collector<T> {
 
   private final @Nullable Context wrap;
diff --git a/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/TimestampExtractTransform.java b/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/TimestampExtractTransform.java
index 6e159fb..4ebacd6 100644
--- a/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/TimestampExtractTransform.java
+++ b/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/TimestampExtractTransform.java
@@ -33,6 +33,7 @@
  * @param <InputT> input type
  * @param <OutputT> output type
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class TimestampExtractTransform<InputT, OutputT>
     extends PTransform<PCollection<InputT>, PCollection<OutputT>> {
 
diff --git a/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/UnionTranslator.java b/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/UnionTranslator.java
index f111916..ed19e30 100644
--- a/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/UnionTranslator.java
+++ b/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/UnionTranslator.java
@@ -24,6 +24,7 @@
 import org.apache.beam.sdk.values.TypeDescriptor;
 
 /** Euphoria to Beam translation of {@link Union} operator. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class UnionTranslator<InputT> implements OperatorTranslator<InputT, InputT, Union<InputT>> {
 
   @Override
diff --git a/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/collector/AdaptableCollector.java b/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/collector/AdaptableCollector.java
index 115cf94..f602da8 100644
--- a/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/collector/AdaptableCollector.java
+++ b/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/collector/AdaptableCollector.java
@@ -38,6 +38,7 @@
  */
 @NotThreadSafe
 @Audience(Audience.Type.EXECUTOR)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class AdaptableCollector<InputT, OutputT, ElemT>
     implements Collector<ElemT>, Context, Serializable {
 
diff --git a/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/collector/SingleValueCollector.java b/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/collector/SingleValueCollector.java
index eea3498..342e3f9 100644
--- a/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/collector/SingleValueCollector.java
+++ b/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/translate/collector/SingleValueCollector.java
@@ -29,6 +29,7 @@
 import org.checkerframework.checker.nullness.qual.Nullable;
 
 /** {@code Collector} for combinable functors. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SingleValueCollector<T> implements Collector<T>, Serializable {
 
   private static final String UNSUPPORTED = "Accumulators are supported for named operators only.";
diff --git a/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/util/InstanceUtils.java b/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/util/InstanceUtils.java
index b090347..7d9c39b 100644
--- a/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/util/InstanceUtils.java
+++ b/sdks/java/extensions/euphoria/src/main/java/org/apache/beam/sdk/extensions/euphoria/core/util/InstanceUtils.java
@@ -25,6 +25,7 @@
  * purposes only. Should not be used in client code.
  */
 @Audience(Audience.Type.EXECUTOR)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class InstanceUtils {
 
   public static <T> T create(Class<T> cls) {
diff --git a/sdks/java/extensions/euphoria/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/FlatMapTest.java b/sdks/java/extensions/euphoria/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/FlatMapTest.java
index cfec139..ef29299 100644
--- a/sdks/java/extensions/euphoria/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/FlatMapTest.java
+++ b/sdks/java/extensions/euphoria/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/FlatMapTest.java
@@ -33,6 +33,7 @@
 
 /** Test operator FlatMap. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class FlatMapTest {
 
   @Test
diff --git a/sdks/java/extensions/euphoria/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/JoinTest.java b/sdks/java/extensions/euphoria/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/JoinTest.java
index a4d8760..76bede0 100644
--- a/sdks/java/extensions/euphoria/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/JoinTest.java
+++ b/sdks/java/extensions/euphoria/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/JoinTest.java
@@ -44,6 +44,7 @@
 
 /** Test operator Join. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class JoinTest {
 
   @Rule public TestPipeline pipeline = TestPipeline.create();
diff --git a/sdks/java/extensions/euphoria/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/ReduceByKeyTest.java b/sdks/java/extensions/euphoria/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/ReduceByKeyTest.java
index 4dc56e1..482403a 100644
--- a/sdks/java/extensions/euphoria/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/ReduceByKeyTest.java
+++ b/sdks/java/extensions/euphoria/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/ReduceByKeyTest.java
@@ -47,6 +47,7 @@
 
 /** Test operator ReduceByKey. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ReduceByKeyTest {
 
   @Test
diff --git a/sdks/java/extensions/euphoria/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/SumByKeyTest.java b/sdks/java/extensions/euphoria/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/SumByKeyTest.java
index 377055a..e10951a 100644
--- a/sdks/java/extensions/euphoria/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/SumByKeyTest.java
+++ b/sdks/java/extensions/euphoria/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/SumByKeyTest.java
@@ -37,6 +37,7 @@
 
 /** Test behavior of operator {@code SumByKey}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SumByKeyTest {
 
   @Test
diff --git a/sdks/java/extensions/euphoria/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/TestUtils.java b/sdks/java/extensions/euphoria/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/TestUtils.java
index 7263228..5a771f9 100644
--- a/sdks/java/extensions/euphoria/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/TestUtils.java
+++ b/sdks/java/extensions/euphoria/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/operator/TestUtils.java
@@ -39,6 +39,7 @@
 import org.apache.beam.sdk.values.TypeDescriptor;
 
 /** Utility class for easier creating input data sets for operator testing. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class TestUtils {
 
   private static class PrimitiveOutputTranslatorProvider implements TranslatorProvider {
diff --git a/sdks/java/extensions/euphoria/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/type/TypePropagationAssert.java b/sdks/java/extensions/euphoria/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/type/TypePropagationAssert.java
index e03aa5f..22768a0 100644
--- a/sdks/java/extensions/euphoria/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/type/TypePropagationAssert.java
+++ b/sdks/java/extensions/euphoria/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/type/TypePropagationAssert.java
@@ -28,6 +28,7 @@
 import org.checkerframework.checker.nullness.qual.Nullable;
 
 /** Bunch of methods to assert type descriptors in operators. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class TypePropagationAssert {
 
   public static <KeyT, ValueT, OutputT> void assertOperatorTypeAwareness(
diff --git a/sdks/java/extensions/euphoria/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/util/IOUtilsTest.java b/sdks/java/extensions/euphoria/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/util/IOUtilsTest.java
index d881c2b..3fc3f02 100644
--- a/sdks/java/extensions/euphoria/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/util/IOUtilsTest.java
+++ b/sdks/java/extensions/euphoria/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/client/util/IOUtilsTest.java
@@ -30,6 +30,7 @@
 
 /** Test behavior of IOUtils. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class IOUtilsTest {
 
   @Test(expected = IOException.class)
diff --git a/sdks/java/extensions/euphoria/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/FlatMapTest.java b/sdks/java/extensions/euphoria/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/FlatMapTest.java
index 15fe998..f3f9cd0 100644
--- a/sdks/java/extensions/euphoria/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/FlatMapTest.java
+++ b/sdks/java/extensions/euphoria/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/FlatMapTest.java
@@ -35,6 +35,7 @@
 
 /** Test operator {@code FlatMap}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class FlatMapTest extends AbstractOperatorTest {
 
   @Test
diff --git a/sdks/java/extensions/euphoria/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/JoinTest.java b/sdks/java/extensions/euphoria/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/JoinTest.java
index 9a11687..a6bd4f7 100644
--- a/sdks/java/extensions/euphoria/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/JoinTest.java
+++ b/sdks/java/extensions/euphoria/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/JoinTest.java
@@ -53,6 +53,7 @@
 
 /** Test operator {@code Join}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class JoinTest extends AbstractOperatorTest {
 
   @Test
diff --git a/sdks/java/extensions/euphoria/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/MapElementsTest.java b/sdks/java/extensions/euphoria/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/MapElementsTest.java
index 3003824..5f8d147 100644
--- a/sdks/java/extensions/euphoria/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/MapElementsTest.java
+++ b/sdks/java/extensions/euphoria/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/MapElementsTest.java
@@ -35,6 +35,7 @@
 
 /** Tests for operator {@code MapElements}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class MapElementsTest extends AbstractOperatorTest {
 
   @Test
diff --git a/sdks/java/extensions/euphoria/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/ReduceByKeyTest.java b/sdks/java/extensions/euphoria/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/ReduceByKeyTest.java
index f7010ba..01fdf96 100644
--- a/sdks/java/extensions/euphoria/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/ReduceByKeyTest.java
+++ b/sdks/java/extensions/euphoria/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/ReduceByKeyTest.java
@@ -63,6 +63,7 @@
 
 /** Test operator {@code ReduceByKey}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ReduceByKeyTest extends AbstractOperatorTest {
 
   /** Validates the output type upon a `.reduceBy` operation on global window. */
diff --git a/sdks/java/extensions/euphoria/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/SumByKeyTest.java b/sdks/java/extensions/euphoria/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/SumByKeyTest.java
index 89dabdf..eea2ab8 100644
--- a/sdks/java/extensions/euphoria/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/SumByKeyTest.java
+++ b/sdks/java/extensions/euphoria/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/SumByKeyTest.java
@@ -32,6 +32,7 @@
 
 /** Test operator {@code SumByKey}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SumByKeyTest extends AbstractOperatorTest {
 
   @Test
diff --git a/sdks/java/extensions/euphoria/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/translate/SingleJvmAccumulatorProviderTest.java b/sdks/java/extensions/euphoria/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/translate/SingleJvmAccumulatorProviderTest.java
index 4872b8e..32ca349 100644
--- a/sdks/java/extensions/euphoria/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/translate/SingleJvmAccumulatorProviderTest.java
+++ b/sdks/java/extensions/euphoria/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/translate/SingleJvmAccumulatorProviderTest.java
@@ -35,6 +35,7 @@
  * JUnit directly.
  */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SingleJvmAccumulatorProviderTest {
 
   private static final String TEST_COUNTER_NAME = "test-counter";
diff --git a/sdks/java/extensions/euphoria/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/translate/collector/SingleValueCollectorTest.java b/sdks/java/extensions/euphoria/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/translate/collector/SingleValueCollectorTest.java
index ac7f4f7..3f42786 100644
--- a/sdks/java/extensions/euphoria/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/translate/collector/SingleValueCollectorTest.java
+++ b/sdks/java/extensions/euphoria/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/translate/collector/SingleValueCollectorTest.java
@@ -31,6 +31,7 @@
 
 /** {@link SingleValueCollector} unit tests. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SingleValueCollectorTest {
 
   private static final String TEST_COUNTER_NAME = "test-counter";
diff --git a/sdks/java/extensions/euphoria/src/test/java/org/apache/beam/sdk/transforms/windowing/WindowDesc.java b/sdks/java/extensions/euphoria/src/test/java/org/apache/beam/sdk/transforms/windowing/WindowDesc.java
index 14a4ad7a..94b39e5 100644
--- a/sdks/java/extensions/euphoria/src/test/java/org/apache/beam/sdk/transforms/windowing/WindowDesc.java
+++ b/sdks/java/extensions/euphoria/src/test/java/org/apache/beam/sdk/transforms/windowing/WindowDesc.java
@@ -25,6 +25,7 @@
  *
  * @param <T> type of input element
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class WindowDesc<T> {
 
   public static <T> WindowDesc<T> of(Window<T> window) {
diff --git a/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/extensions/gcp/auth/GcpCredentialFactory.java b/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/extensions/gcp/auth/GcpCredentialFactory.java
index a53f6ea..e2a861d 100644
--- a/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/extensions/gcp/auth/GcpCredentialFactory.java
+++ b/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/extensions/gcp/auth/GcpCredentialFactory.java
@@ -28,6 +28,7 @@
  * Construct an oauth credential to be used by the SDK and the SDK workers. Returns a GCP
  * credential.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class GcpCredentialFactory implements CredentialFactory {
   /**
    * The scope cloud-platform provides access to all Cloud Platform resources. cloud-platform isn't
diff --git a/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/extensions/gcp/auth/NoopCredentialFactory.java b/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/extensions/gcp/auth/NoopCredentialFactory.java
index 8585e05..111c5fc 100644
--- a/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/extensions/gcp/auth/NoopCredentialFactory.java
+++ b/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/extensions/gcp/auth/NoopCredentialFactory.java
@@ -28,6 +28,7 @@
  * Construct an oauth credential to be used by the SDK and the SDK workers. Always returns a null
  * Credential object.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class NoopCredentialFactory implements CredentialFactory {
   private static final NoopCredentialFactory INSTANCE = new NoopCredentialFactory();
   private static final NoopCredentials NOOP_CREDENTIALS = new NoopCredentials();
diff --git a/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/extensions/gcp/options/GcpOptions.java b/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/extensions/gcp/options/GcpOptions.java
index cf358e4..0c10d60 100644
--- a/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/extensions/gcp/options/GcpOptions.java
+++ b/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/extensions/gcp/options/GcpOptions.java
@@ -76,6 +76,7 @@
  * mechanisms for creating credentials.
  */
 @Description("Options used to configure Google Cloud Platform project and credentials.")
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public interface GcpOptions extends GoogleApiDebugOptions, PipelineOptions {
   /** Project id to use when launching jobs. */
   @Description(
diff --git a/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/extensions/gcp/options/GoogleApiDebugOptions.java b/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/extensions/gcp/options/GoogleApiDebugOptions.java
index 7b570bf..b2c12ad 100644
--- a/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/extensions/gcp/options/GoogleApiDebugOptions.java
+++ b/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/extensions/gcp/options/GoogleApiDebugOptions.java
@@ -29,6 +29,7 @@
 /**
  * These options configure debug settings for Google API clients created within the Apache Beam SDK.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public interface GoogleApiDebugOptions extends PipelineOptions {
   /**
    * This option enables tracing of API calls to Google services used within the Apache Beam SDK.
diff --git a/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/extensions/gcp/storage/GcsFileSystem.java b/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/extensions/gcp/storage/GcsFileSystem.java
index 8ae2aa3..7c80ee3 100644
--- a/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/extensions/gcp/storage/GcsFileSystem.java
+++ b/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/extensions/gcp/storage/GcsFileSystem.java
@@ -58,6 +58,7 @@
 import org.slf4j.LoggerFactory;
 
 /** {@link FileSystem} implementation for Google Cloud Storage. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class GcsFileSystem extends FileSystem<GcsResourceId> {
   private static final Logger LOG = LoggerFactory.getLogger(GcsFileSystem.class);
 
diff --git a/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/extensions/gcp/util/CustomHttpErrors.java b/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/extensions/gcp/util/CustomHttpErrors.java
index 89089df..5749f3c 100644
--- a/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/extensions/gcp/util/CustomHttpErrors.java
+++ b/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/extensions/gcp/util/CustomHttpErrors.java
@@ -63,6 +63,7 @@
  *
  * <p>
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class CustomHttpErrors {
 
   /**
diff --git a/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/extensions/gcp/util/GcsUtil.java b/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/extensions/gcp/util/GcsUtil.java
index 5c774b0..2626433 100644
--- a/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/extensions/gcp/util/GcsUtil.java
+++ b/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/extensions/gcp/util/GcsUtil.java
@@ -82,6 +82,7 @@
 import org.slf4j.LoggerFactory;
 
 /** Provides operations on GCS. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class GcsUtil {
   /**
    * This is a {@link DefaultValueFactory} able to create a {@link GcsUtil} using any transport
diff --git a/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/extensions/gcp/util/RetryHttpRequestInitializer.java b/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/extensions/gcp/util/RetryHttpRequestInitializer.java
index 0496746..34ec44f 100644
--- a/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/extensions/gcp/util/RetryHttpRequestInitializer.java
+++ b/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/extensions/gcp/util/RetryHttpRequestInitializer.java
@@ -44,6 +44,7 @@
  *
  * <p>Also can take an HttpResponseInterceptor to be applied to the responses.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class RetryHttpRequestInitializer implements HttpRequestInitializer {
 
   private static final Logger LOG = LoggerFactory.getLogger(RetryHttpRequestInitializer.class);
diff --git a/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/extensions/gcp/util/UploadIdResponseInterceptor.java b/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/extensions/gcp/util/UploadIdResponseInterceptor.java
index 7baad4a..09ae043 100644
--- a/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/extensions/gcp/util/UploadIdResponseInterceptor.java
+++ b/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/extensions/gcp/util/UploadIdResponseInterceptor.java
@@ -29,6 +29,7 @@
  * is the first request (does not have upload_id parameter in the request). Only logs if debug level
  * is enabled.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class UploadIdResponseInterceptor implements HttpResponseInterceptor {
 
   private static final Logger LOG = LoggerFactory.getLogger(UploadIdResponseInterceptor.class);
diff --git a/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/extensions/gcp/util/gcsfs/GcsPath.java b/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/extensions/gcp/util/gcsfs/GcsPath.java
index 50dea39..45e6995 100644
--- a/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/extensions/gcp/util/gcsfs/GcsPath.java
+++ b/sdks/java/extensions/google-cloud-platform-core/src/main/java/org/apache/beam/sdk/extensions/gcp/util/gcsfs/GcsPath.java
@@ -65,6 +65,7 @@
  * @see <a href= "http://docs.oracle.com/javase/tutorial/essential/io/pathOps.html" >Java Tutorials:
  *     Path Operations</a>
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class GcsPath implements Path, Serializable {
 
   public static final String SCHEME = "gs";
diff --git a/sdks/java/extensions/google-cloud-platform-core/src/test/java/org/apache/beam/sdk/extensions/gcp/GcpCoreApiSurfaceTest.java b/sdks/java/extensions/google-cloud-platform-core/src/test/java/org/apache/beam/sdk/extensions/gcp/GcpCoreApiSurfaceTest.java
index 9a041e9..807716d 100644
--- a/sdks/java/extensions/google-cloud-platform-core/src/test/java/org/apache/beam/sdk/extensions/gcp/GcpCoreApiSurfaceTest.java
+++ b/sdks/java/extensions/google-cloud-platform-core/src/test/java/org/apache/beam/sdk/extensions/gcp/GcpCoreApiSurfaceTest.java
@@ -31,6 +31,7 @@
 
 /** API surface verification for Google Cloud Platform core components. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class GcpCoreApiSurfaceTest {
 
   @Test
diff --git a/sdks/java/extensions/google-cloud-platform-core/src/test/java/org/apache/beam/sdk/extensions/gcp/options/GcpOptionsTest.java b/sdks/java/extensions/google-cloud-platform-core/src/test/java/org/apache/beam/sdk/extensions/gcp/options/GcpOptionsTest.java
index 0f0b5b3..ab31161 100644
--- a/sdks/java/extensions/google-cloud-platform-core/src/test/java/org/apache/beam/sdk/extensions/gcp/options/GcpOptionsTest.java
+++ b/sdks/java/extensions/google-cloud-platform-core/src/test/java/org/apache/beam/sdk/extensions/gcp/options/GcpOptionsTest.java
@@ -59,6 +59,7 @@
 import org.mockito.MockitoAnnotations;
 
 /** Tests for {@link GcpOptions}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class GcpOptionsTest {
 
   /** Tests for the majority of methods. */
diff --git a/sdks/java/extensions/google-cloud-platform-core/src/test/java/org/apache/beam/sdk/extensions/gcp/storage/GcsFileSystemTest.java b/sdks/java/extensions/google-cloud-platform-core/src/test/java/org/apache/beam/sdk/extensions/gcp/storage/GcsFileSystemTest.java
index 3ae7377..c6a8c55 100644
--- a/sdks/java/extensions/google-cloud-platform-core/src/test/java/org/apache/beam/sdk/extensions/gcp/storage/GcsFileSystemTest.java
+++ b/sdks/java/extensions/google-cloud-platform-core/src/test/java/org/apache/beam/sdk/extensions/gcp/storage/GcsFileSystemTest.java
@@ -53,6 +53,7 @@
 
 /** Tests for {@link GcsFileSystem}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class GcsFileSystemTest {
 
   @Rule public transient ExpectedException thrown = ExpectedException.none();
diff --git a/sdks/java/extensions/google-cloud-platform-core/src/test/java/org/apache/beam/sdk/extensions/gcp/storage/GcsPathValidatorTest.java b/sdks/java/extensions/google-cloud-platform-core/src/test/java/org/apache/beam/sdk/extensions/gcp/storage/GcsPathValidatorTest.java
index 924afe7..468f7ed 100644
--- a/sdks/java/extensions/google-cloud-platform-core/src/test/java/org/apache/beam/sdk/extensions/gcp/storage/GcsPathValidatorTest.java
+++ b/sdks/java/extensions/google-cloud-platform-core/src/test/java/org/apache/beam/sdk/extensions/gcp/storage/GcsPathValidatorTest.java
@@ -36,6 +36,7 @@
 
 /** Tests for {@link GcsPathValidator}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class GcsPathValidatorTest {
   @Rule public ExpectedException expectedException = ExpectedException.none();
 
diff --git a/sdks/java/extensions/google-cloud-platform-core/src/test/java/org/apache/beam/sdk/extensions/gcp/storage/GcsResourceIdTest.java b/sdks/java/extensions/google-cloud-platform-core/src/test/java/org/apache/beam/sdk/extensions/gcp/storage/GcsResourceIdTest.java
index 9df76a8..e795820 100644
--- a/sdks/java/extensions/google-cloud-platform-core/src/test/java/org/apache/beam/sdk/extensions/gcp/storage/GcsResourceIdTest.java
+++ b/sdks/java/extensions/google-cloud-platform-core/src/test/java/org/apache/beam/sdk/extensions/gcp/storage/GcsResourceIdTest.java
@@ -37,6 +37,7 @@
 
 /** Tests for {@link GcsResourceId}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class GcsResourceIdTest {
 
   @Rule public ExpectedException thrown = ExpectedException.none();
diff --git a/sdks/java/extensions/google-cloud-platform-core/src/test/java/org/apache/beam/sdk/extensions/gcp/util/GcsUtilIT.java b/sdks/java/extensions/google-cloud-platform-core/src/test/java/org/apache/beam/sdk/extensions/gcp/util/GcsUtilIT.java
index ff96566..cddbe0a 100644
--- a/sdks/java/extensions/google-cloud-platform-core/src/test/java/org/apache/beam/sdk/extensions/gcp/util/GcsUtilIT.java
+++ b/sdks/java/extensions/google-cloud-platform-core/src/test/java/org/apache/beam/sdk/extensions/gcp/util/GcsUtilIT.java
@@ -44,6 +44,7 @@
  */
 @RunWith(JUnit4.class)
 @Category(UsesKms.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class GcsUtilIT {
   /** Tests a rewrite operation that requires multiple API calls (using a continuation token). */
   @Test
diff --git a/sdks/java/extensions/google-cloud-platform-core/src/test/java/org/apache/beam/sdk/extensions/gcp/util/GcsUtilTest.java b/sdks/java/extensions/google-cloud-platform-core/src/test/java/org/apache/beam/sdk/extensions/gcp/util/GcsUtilTest.java
index 8c0bde0..458e1a5 100644
--- a/sdks/java/extensions/google-cloud-platform-core/src/test/java/org/apache/beam/sdk/extensions/gcp/util/GcsUtilTest.java
+++ b/sdks/java/extensions/google-cloud-platform-core/src/test/java/org/apache/beam/sdk/extensions/gcp/util/GcsUtilTest.java
@@ -91,6 +91,7 @@
 
 /** Test case for {@link GcsUtil}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class GcsUtilTest {
   @Rule public ExpectedException thrown = ExpectedException.none();
 
diff --git a/sdks/java/extensions/google-cloud-platform-core/src/test/java/org/apache/beam/sdk/extensions/gcp/util/LatencyRecordingHttpRequestInitializerTest.java b/sdks/java/extensions/google-cloud-platform-core/src/test/java/org/apache/beam/sdk/extensions/gcp/util/LatencyRecordingHttpRequestInitializerTest.java
index b419e13..fb75759 100644
--- a/sdks/java/extensions/google-cloud-platform-core/src/test/java/org/apache/beam/sdk/extensions/gcp/util/LatencyRecordingHttpRequestInitializerTest.java
+++ b/sdks/java/extensions/google-cloud-platform-core/src/test/java/org/apache/beam/sdk/extensions/gcp/util/LatencyRecordingHttpRequestInitializerTest.java
@@ -51,6 +51,7 @@
 
 /** Tests for LatencyRecordingHttpRequestInitializer. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class LatencyRecordingHttpRequestInitializerTest {
 
   @Rule
diff --git a/sdks/java/extensions/google-cloud-platform-core/src/test/java/org/apache/beam/sdk/extensions/gcp/util/RetryHttpRequestInitializerTest.java b/sdks/java/extensions/google-cloud-platform-core/src/test/java/org/apache/beam/sdk/extensions/gcp/util/RetryHttpRequestInitializerTest.java
index 13a9fd7..eecf357 100644
--- a/sdks/java/extensions/google-cloud-platform-core/src/test/java/org/apache/beam/sdk/extensions/gcp/util/RetryHttpRequestInitializerTest.java
+++ b/sdks/java/extensions/google-cloud-platform-core/src/test/java/org/apache/beam/sdk/extensions/gcp/util/RetryHttpRequestInitializerTest.java
@@ -64,6 +64,7 @@
 
 /** Tests for RetryHttpRequestInitializer. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class RetryHttpRequestInitializerTest {
 
   @Rule public ExpectedLogs expectedLogs = ExpectedLogs.none(RetryHttpRequestInitializer.class);
diff --git a/sdks/java/extensions/google-cloud-platform-core/src/test/java/org/apache/beam/sdk/extensions/gcp/util/UploadIdResponseInterceptorTest.java b/sdks/java/extensions/google-cloud-platform-core/src/test/java/org/apache/beam/sdk/extensions/gcp/util/UploadIdResponseInterceptorTest.java
index 630a084..d0faf6c 100644
--- a/sdks/java/extensions/google-cloud-platform-core/src/test/java/org/apache/beam/sdk/extensions/gcp/util/UploadIdResponseInterceptorTest.java
+++ b/sdks/java/extensions/google-cloud-platform-core/src/test/java/org/apache/beam/sdk/extensions/gcp/util/UploadIdResponseInterceptorTest.java
@@ -32,6 +32,7 @@
 
 /** A test for {@link org.apache.beam.sdk.extensions.gcp.util.UploadIdResponseInterceptor}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class UploadIdResponseInterceptorTest {
 
   @Rule public ExpectedException expectedException = ExpectedException.none();
diff --git a/sdks/java/extensions/jackson/src/main/java/org/apache/beam/sdk/extensions/jackson/AsJsons.java b/sdks/java/extensions/jackson/src/main/java/org/apache/beam/sdk/extensions/jackson/AsJsons.java
index 578610d..0f2ce83 100644
--- a/sdks/java/extensions/jackson/src/main/java/org/apache/beam/sdk/extensions/jackson/AsJsons.java
+++ b/sdks/java/extensions/jackson/src/main/java/org/apache/beam/sdk/extensions/jackson/AsJsons.java
@@ -45,6 +45,7 @@
  * PCollection<InputT>} into a {@link PCollection} of JSON {@link String Strings} representing
  * objects in the original {@link PCollection} using Jackson.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class AsJsons<InputT> extends PTransform<PCollection<InputT>, PCollection<String>> {
   private static final ObjectMapper DEFAULT_MAPPER = new ObjectMapper();
 
diff --git a/sdks/java/extensions/jackson/src/main/java/org/apache/beam/sdk/extensions/jackson/ParseJsons.java b/sdks/java/extensions/jackson/src/main/java/org/apache/beam/sdk/extensions/jackson/ParseJsons.java
index 470b6b4..a2b9fc4 100644
--- a/sdks/java/extensions/jackson/src/main/java/org/apache/beam/sdk/extensions/jackson/ParseJsons.java
+++ b/sdks/java/extensions/jackson/src/main/java/org/apache/beam/sdk/extensions/jackson/ParseJsons.java
@@ -43,6 +43,7 @@
  * String Strings} in JSON format into a {@link PCollection} of objects represented by those JSON
  * {@link String Strings} using Jackson.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ParseJsons<OutputT> extends PTransform<PCollection<String>, PCollection<OutputT>> {
   private static final ObjectMapper DEFAULT_MAPPER = new ObjectMapper();
 
diff --git a/sdks/java/extensions/jackson/src/test/java/org/apache/beam/sdk/extensions/jackson/JacksonTransformsTest.java b/sdks/java/extensions/jackson/src/test/java/org/apache/beam/sdk/extensions/jackson/JacksonTransformsTest.java
index 2e1855c..05e9496 100644
--- a/sdks/java/extensions/jackson/src/test/java/org/apache/beam/sdk/extensions/jackson/JacksonTransformsTest.java
+++ b/sdks/java/extensions/jackson/src/test/java/org/apache/beam/sdk/extensions/jackson/JacksonTransformsTest.java
@@ -49,6 +49,7 @@
 import org.junit.Test;
 
 /** Test Jackson transforms {@link ParseJsons} and {@link AsJsons}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class JacksonTransformsTest implements Serializable {
   private static final List<String> VALID_JSONS =
       Arrays.asList("{\"myString\":\"abc\",\"myInt\":3}", "{\"myString\":\"def\",\"myInt\":4}");
diff --git a/sdks/java/extensions/ml/src/main/java/org/apache/beam/sdk/extensions/ml/AnnotateImages.java b/sdks/java/extensions/ml/src/main/java/org/apache/beam/sdk/extensions/ml/AnnotateImages.java
index 5775ebb..eaf8960 100644
--- a/sdks/java/extensions/ml/src/main/java/org/apache/beam/sdk/extensions/ml/AnnotateImages.java
+++ b/sdks/java/extensions/ml/src/main/java/org/apache/beam/sdk/extensions/ml/AnnotateImages.java
@@ -49,6 +49,7 @@
  * @param <T> Type of input PCollection.
  */
 @Experimental
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 abstract class AnnotateImages<T>
     extends PTransform<PCollection<T>, PCollection<List<AnnotateImageResponse>>> {
 
diff --git a/sdks/java/extensions/ml/src/main/java/org/apache/beam/sdk/extensions/ml/AnnotateText.java b/sdks/java/extensions/ml/src/main/java/org/apache/beam/sdk/extensions/ml/AnnotateText.java
index 6f2c9f3..0aa89bc 100644
--- a/sdks/java/extensions/ml/src/main/java/org/apache/beam/sdk/extensions/ml/AnnotateText.java
+++ b/sdks/java/extensions/ml/src/main/java/org/apache/beam/sdk/extensions/ml/AnnotateText.java
@@ -40,6 +40,7 @@
  */
 @Experimental
 @AutoValue
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public abstract class AnnotateText
     extends PTransform<PCollection<Document>, PCollection<AnnotateTextResponse>> {
 
diff --git a/sdks/java/extensions/ml/src/main/java/org/apache/beam/sdk/extensions/ml/AnnotateVideoBytesWithContextFn.java b/sdks/java/extensions/ml/src/main/java/org/apache/beam/sdk/extensions/ml/AnnotateVideoBytesWithContextFn.java
index c8edcd1..64801df 100644
--- a/sdks/java/extensions/ml/src/main/java/org/apache/beam/sdk/extensions/ml/AnnotateVideoBytesWithContextFn.java
+++ b/sdks/java/extensions/ml/src/main/java/org/apache/beam/sdk/extensions/ml/AnnotateVideoBytesWithContextFn.java
@@ -31,6 +31,7 @@
  * ByteString encoded video contents, values - VideoContext objects.
  */
 @Experimental
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class AnnotateVideoBytesWithContextFn extends AnnotateVideoFn<KV<ByteString, VideoContext>> {
 
   public AnnotateVideoBytesWithContextFn(List<Feature> featureList) {
diff --git a/sdks/java/extensions/ml/src/main/java/org/apache/beam/sdk/extensions/ml/AnnotateVideoFn.java b/sdks/java/extensions/ml/src/main/java/org/apache/beam/sdk/extensions/ml/AnnotateVideoFn.java
index a954ff3..d2be21b 100644
--- a/sdks/java/extensions/ml/src/main/java/org/apache/beam/sdk/extensions/ml/AnnotateVideoFn.java
+++ b/sdks/java/extensions/ml/src/main/java/org/apache/beam/sdk/extensions/ml/AnnotateVideoFn.java
@@ -41,6 +41,7 @@
  *     String or String - a GCS URI of the video to be annotated.
  */
 @Experimental
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 abstract class AnnotateVideoFn<T> extends DoFn<T, List<VideoAnnotationResults>> {
 
   protected final PCollectionView<Map<T, VideoContext>> contextSideInput;
diff --git a/sdks/java/extensions/ml/src/main/java/org/apache/beam/sdk/extensions/ml/AnnotateVideoFromBytesFn.java b/sdks/java/extensions/ml/src/main/java/org/apache/beam/sdk/extensions/ml/AnnotateVideoFromBytesFn.java
index 20aa083..43be5aa 100644
--- a/sdks/java/extensions/ml/src/main/java/org/apache/beam/sdk/extensions/ml/AnnotateVideoFromBytesFn.java
+++ b/sdks/java/extensions/ml/src/main/java/org/apache/beam/sdk/extensions/ml/AnnotateVideoFromBytesFn.java
@@ -32,6 +32,7 @@
  * decoded from the ByteStrings are annotated.
  */
 @Experimental
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class AnnotateVideoFromBytesFn extends AnnotateVideoFn<ByteString> {
 
   public AnnotateVideoFromBytesFn(
diff --git a/sdks/java/extensions/ml/src/main/java/org/apache/beam/sdk/extensions/ml/AnnotateVideoFromURIFn.java b/sdks/java/extensions/ml/src/main/java/org/apache/beam/sdk/extensions/ml/AnnotateVideoFromURIFn.java
index 5dfea0c..cf220f3 100644
--- a/sdks/java/extensions/ml/src/main/java/org/apache/beam/sdk/extensions/ml/AnnotateVideoFromURIFn.java
+++ b/sdks/java/extensions/ml/src/main/java/org/apache/beam/sdk/extensions/ml/AnnotateVideoFromURIFn.java
@@ -31,6 +31,7 @@
  * videos found on GCS based on URIs from input PCollection.
  */
 @Experimental
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class AnnotateVideoFromURIFn extends AnnotateVideoFn<String> {
 
   public AnnotateVideoFromURIFn(
diff --git a/sdks/java/extensions/ml/src/main/java/org/apache/beam/sdk/extensions/ml/AnnotateVideoURIWithContextFn.java b/sdks/java/extensions/ml/src/main/java/org/apache/beam/sdk/extensions/ml/AnnotateVideoURIWithContextFn.java
index a165d5a..377d81a 100644
--- a/sdks/java/extensions/ml/src/main/java/org/apache/beam/sdk/extensions/ml/AnnotateVideoURIWithContextFn.java
+++ b/sdks/java/extensions/ml/src/main/java/org/apache/beam/sdk/extensions/ml/AnnotateVideoURIWithContextFn.java
@@ -30,6 +30,7 @@
  * GCS URIs, values - VideoContext objects.
  */
 @Experimental
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class AnnotateVideoURIWithContextFn extends AnnotateVideoFn<KV<String, VideoContext>> {
 
   public AnnotateVideoURIWithContextFn(List<Feature> featureList) {
diff --git a/sdks/java/extensions/ml/src/main/java/org/apache/beam/sdk/extensions/ml/DLPDeidentifyText.java b/sdks/java/extensions/ml/src/main/java/org/apache/beam/sdk/extensions/ml/DLPDeidentifyText.java
index 45dbd50..9b68021 100644
--- a/sdks/java/extensions/ml/src/main/java/org/apache/beam/sdk/extensions/ml/DLPDeidentifyText.java
+++ b/sdks/java/extensions/ml/src/main/java/org/apache/beam/sdk/extensions/ml/DLPDeidentifyText.java
@@ -60,6 +60,7 @@
  */
 @Experimental
 @AutoValue
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public abstract class DLPDeidentifyText
     extends PTransform<
         PCollection<KV<String, String>>, PCollection<KV<String, DeidentifyContentResponse>>> {
diff --git a/sdks/java/extensions/ml/src/main/java/org/apache/beam/sdk/extensions/ml/DLPInspectText.java b/sdks/java/extensions/ml/src/main/java/org/apache/beam/sdk/extensions/ml/DLPInspectText.java
index 9e10480..427d3cb 100644
--- a/sdks/java/extensions/ml/src/main/java/org/apache/beam/sdk/extensions/ml/DLPInspectText.java
+++ b/sdks/java/extensions/ml/src/main/java/org/apache/beam/sdk/extensions/ml/DLPInspectText.java
@@ -61,6 +61,7 @@
  */
 @Experimental
 @AutoValue
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public abstract class DLPInspectText
     extends PTransform<
         PCollection<KV<String, String>>, PCollection<KV<String, InspectContentResponse>>> {
diff --git a/sdks/java/extensions/ml/src/main/java/org/apache/beam/sdk/extensions/ml/DLPReidentifyText.java b/sdks/java/extensions/ml/src/main/java/org/apache/beam/sdk/extensions/ml/DLPReidentifyText.java
index 79e4c10..39fc2b7 100644
--- a/sdks/java/extensions/ml/src/main/java/org/apache/beam/sdk/extensions/ml/DLPReidentifyText.java
+++ b/sdks/java/extensions/ml/src/main/java/org/apache/beam/sdk/extensions/ml/DLPReidentifyText.java
@@ -66,6 +66,7 @@
  */
 @Experimental
 @AutoValue
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public abstract class DLPReidentifyText
     extends PTransform<
         PCollection<KV<String, String>>, PCollection<KV<String, ReidentifyContentResponse>>> {
diff --git a/sdks/java/extensions/ml/src/test/java/org/apache/beam/sdk/extensions/ml/AnnotateImagesTest.java b/sdks/java/extensions/ml/src/test/java/org/apache/beam/sdk/extensions/ml/AnnotateImagesTest.java
index 341c97e..ba04fa9 100644
--- a/sdks/java/extensions/ml/src/test/java/org/apache/beam/sdk/extensions/ml/AnnotateImagesTest.java
+++ b/sdks/java/extensions/ml/src/test/java/org/apache/beam/sdk/extensions/ml/AnnotateImagesTest.java
@@ -32,6 +32,7 @@
 import org.mockito.junit.MockitoJUnitRunner;
 
 @RunWith(MockitoJUnitRunner.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class AnnotateImagesTest {
   @Mock private ImageAnnotatorClient imageAnnotatorClient;
   @Mock private BatchAnnotateImagesResponse response;
diff --git a/sdks/java/extensions/ml/src/test/java/org/apache/beam/sdk/extensions/ml/AnnotateVideoTest.java b/sdks/java/extensions/ml/src/test/java/org/apache/beam/sdk/extensions/ml/AnnotateVideoTest.java
index 56473aa..cfe1d43 100644
--- a/sdks/java/extensions/ml/src/test/java/org/apache/beam/sdk/extensions/ml/AnnotateVideoTest.java
+++ b/sdks/java/extensions/ml/src/test/java/org/apache/beam/sdk/extensions/ml/AnnotateVideoTest.java
@@ -37,6 +37,7 @@
 import org.mockito.junit.MockitoJUnitRunner;
 
 @RunWith(MockitoJUnitRunner.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class AnnotateVideoTest {
 
   private static final String TEST_URI = "fake_uri";
diff --git a/sdks/java/extensions/ml/src/test/java/org/apache/beam/sdk/extensions/ml/BatchRequestForDlpTest.java b/sdks/java/extensions/ml/src/test/java/org/apache/beam/sdk/extensions/ml/BatchRequestForDlpTest.java
index 3e9a741..c8ea07e 100644
--- a/sdks/java/extensions/ml/src/test/java/org/apache/beam/sdk/extensions/ml/BatchRequestForDlpTest.java
+++ b/sdks/java/extensions/ml/src/test/java/org/apache/beam/sdk/extensions/ml/BatchRequestForDlpTest.java
@@ -35,6 +35,7 @@
 import org.junit.runners.JUnit4;
 
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BatchRequestForDlpTest {
 
   @Rule public TestPipeline testPipeline = TestPipeline.create();
diff --git a/sdks/java/extensions/ml/src/test/java/org/apache/beam/sdk/extensions/ml/CloudVisionIT.java b/sdks/java/extensions/ml/src/test/java/org/apache/beam/sdk/extensions/ml/CloudVisionIT.java
index 5ee7298..89ed54c 100644
--- a/sdks/java/extensions/ml/src/test/java/org/apache/beam/sdk/extensions/ml/CloudVisionIT.java
+++ b/sdks/java/extensions/ml/src/test/java/org/apache/beam/sdk/extensions/ml/CloudVisionIT.java
@@ -33,6 +33,7 @@
 import org.junit.runners.JUnit4;
 
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class CloudVisionIT {
   @Rule public TestPipeline testPipeline = TestPipeline.create();
   private static final String TEST_IMAGE_URI = "gs://cloud-samples-data/vision/label/setagaya.jpeg";
diff --git a/sdks/java/extensions/ml/src/test/java/org/apache/beam/sdk/extensions/ml/CloudVisionTest.java b/sdks/java/extensions/ml/src/test/java/org/apache/beam/sdk/extensions/ml/CloudVisionTest.java
index 27f68b4..80170eb 100644
--- a/sdks/java/extensions/ml/src/test/java/org/apache/beam/sdk/extensions/ml/CloudVisionTest.java
+++ b/sdks/java/extensions/ml/src/test/java/org/apache/beam/sdk/extensions/ml/CloudVisionTest.java
@@ -30,6 +30,7 @@
 import org.junit.runners.JUnit4;
 
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class CloudVisionTest {
 
   private static final String TEST_URI = "test_uri";
diff --git a/sdks/java/extensions/ml/src/test/java/org/apache/beam/sdk/extensions/ml/MapStringToDlpRowTest.java b/sdks/java/extensions/ml/src/test/java/org/apache/beam/sdk/extensions/ml/MapStringToDlpRowTest.java
index 577a5dc..6929a91 100644
--- a/sdks/java/extensions/ml/src/test/java/org/apache/beam/sdk/extensions/ml/MapStringToDlpRowTest.java
+++ b/sdks/java/extensions/ml/src/test/java/org/apache/beam/sdk/extensions/ml/MapStringToDlpRowTest.java
@@ -31,6 +31,7 @@
 import org.junit.runners.JUnit4;
 
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class MapStringToDlpRowTest {
   @Rule public TestPipeline testPipeline = TestPipeline.create();
 
diff --git a/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoByteBuddyUtils.java b/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoByteBuddyUtils.java
index f47deed..eed9083 100644
--- a/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoByteBuddyUtils.java
+++ b/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoByteBuddyUtils.java
@@ -111,6 +111,7 @@
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Multimap;
 import org.checkerframework.checker.nullness.qual.Nullable;
 
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class ProtoByteBuddyUtils {
   private static final ByteBuddy BYTE_BUDDY = new ByteBuddy();
   private static final TypeDescriptor<ByteString> BYTE_STRING_TYPE_DESCRIPTOR =
diff --git a/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoCoder.java b/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoCoder.java
index 62c34c10..1b41f80 100644
--- a/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoCoder.java
+++ b/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoCoder.java
@@ -105,6 +105,7 @@
  *
  * @param <T> the Protocol Buffers {@link Message} handled by this {@link Coder}.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ProtoCoder<T extends Message> extends CustomCoder<T> {
 
   public static final long serialVersionUID = -5043999806040629525L;
diff --git a/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoDomain.java b/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoDomain.java
index 2f33010..3a02732 100644
--- a/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoDomain.java
+++ b/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoDomain.java
@@ -56,6 +56,7 @@
  * <p>ProtoDomains aren't assumed to be used on with normal Message descriptors, only with
  * DynamicMessage descriptors.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public final class ProtoDomain implements Serializable {
   public static final long serialVersionUID = 1L;
   private transient DescriptorProtos.FileDescriptorSet fileDescriptorSet;
diff --git a/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoDynamicMessageSchema.java b/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoDynamicMessageSchema.java
index 5a993ee..e41089a 100644
--- a/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoDynamicMessageSchema.java
+++ b/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoDynamicMessageSchema.java
@@ -45,6 +45,7 @@
 import org.apache.beam.sdk.values.Row;
 
 @Experimental(Experimental.Kind.SCHEMAS)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ProtoDynamicMessageSchema<T> implements Serializable {
   public static final long serialVersionUID = 1L;
 
diff --git a/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoMessageSchema.java b/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoMessageSchema.java
index 5fc8711..02b3b5c 100644
--- a/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoMessageSchema.java
+++ b/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoMessageSchema.java
@@ -83,9 +83,8 @@
     }
   }
 
-  @Nullable
   @Override
-  public <T> Schema schemaFor(TypeDescriptor<T> typeDescriptor) {
+  public <T> @Nullable Schema schemaFor(TypeDescriptor<T> typeDescriptor) {
     checkForDynamicType(typeDescriptor);
     return ProtoSchemaTranslator.getSchema((Class<Message>) typeDescriptor.getRawType());
   }
diff --git a/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoSchemaLogicalTypes.java b/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoSchemaLogicalTypes.java
index bfb678c..e063f33 100644
--- a/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoSchemaLogicalTypes.java
+++ b/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoSchemaLogicalTypes.java
@@ -29,6 +29,7 @@
 
 /** A set of {@link LogicalType} classes to represent protocol buffer types. */
 @Experimental(Kind.SCHEMAS)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ProtoSchemaLogicalTypes {
 
   /** Compatible schema with the row schema of NanosDuration and NanosInstant. */
diff --git a/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtobufUtil.java b/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtobufUtil.java
index 791ed63..1681b45 100644
--- a/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtobufUtil.java
+++ b/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtobufUtil.java
@@ -36,6 +36,7 @@
  *
  * <p>Used by {@link ProtoCoder}, but in a separate file for testing and isolation.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class ProtobufUtil {
   /**
    * Returns the {@link Descriptor} for the given Protocol Buffers {@link Message}.
diff --git a/sdks/java/extensions/protobuf/src/test/java/org/apache/beam/sdk/extensions/protobuf/ByteStringCoderTest.java b/sdks/java/extensions/protobuf/src/test/java/org/apache/beam/sdk/extensions/protobuf/ByteStringCoderTest.java
index 744de901..578e916 100644
--- a/sdks/java/extensions/protobuf/src/test/java/org/apache/beam/sdk/extensions/protobuf/ByteStringCoderTest.java
+++ b/sdks/java/extensions/protobuf/src/test/java/org/apache/beam/sdk/extensions/protobuf/ByteStringCoderTest.java
@@ -41,6 +41,7 @@
 
 /** Test case for {@link ByteStringCoder}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ByteStringCoderTest {
 
   private static final ByteStringCoder TEST_CODER = ByteStringCoder.of();
diff --git a/sdks/java/extensions/protobuf/src/test/java/org/apache/beam/sdk/extensions/protobuf/DynamicProtoCoderTest.java b/sdks/java/extensions/protobuf/src/test/java/org/apache/beam/sdk/extensions/protobuf/DynamicProtoCoderTest.java
index 1039583..acde2cd 100644
--- a/sdks/java/extensions/protobuf/src/test/java/org/apache/beam/sdk/extensions/protobuf/DynamicProtoCoderTest.java
+++ b/sdks/java/extensions/protobuf/src/test/java/org/apache/beam/sdk/extensions/protobuf/DynamicProtoCoderTest.java
@@ -34,6 +34,7 @@
 
 /** Tests for {@link ProtoCoder}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DynamicProtoCoderTest {
 
   @Rule public ExpectedException thrown = ExpectedException.none();
diff --git a/sdks/java/extensions/protobuf/src/test/java/org/apache/beam/sdk/extensions/protobuf/ProtoCoderTest.java b/sdks/java/extensions/protobuf/src/test/java/org/apache/beam/sdk/extensions/protobuf/ProtoCoderTest.java
index 38aa92b..e0899f6 100644
--- a/sdks/java/extensions/protobuf/src/test/java/org/apache/beam/sdk/extensions/protobuf/ProtoCoderTest.java
+++ b/sdks/java/extensions/protobuf/src/test/java/org/apache/beam/sdk/extensions/protobuf/ProtoCoderTest.java
@@ -43,6 +43,7 @@
 
 /** Tests for {@link ProtoCoder}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ProtoCoderTest {
 
   @Rule public ExpectedException thrown = ExpectedException.none();
diff --git a/sdks/java/extensions/protobuf/src/test/java/org/apache/beam/sdk/extensions/protobuf/ProtoMessageSchemaTest.java b/sdks/java/extensions/protobuf/src/test/java/org/apache/beam/sdk/extensions/protobuf/ProtoMessageSchemaTest.java
index d7c33d4..6146135 100644
--- a/sdks/java/extensions/protobuf/src/test/java/org/apache/beam/sdk/extensions/protobuf/ProtoMessageSchemaTest.java
+++ b/sdks/java/extensions/protobuf/src/test/java/org/apache/beam/sdk/extensions/protobuf/ProtoMessageSchemaTest.java
@@ -80,6 +80,7 @@
 import org.junit.runners.JUnit4;
 
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ProtoMessageSchemaTest {
 
   @Test
diff --git a/sdks/java/extensions/protobuf/src/test/java/org/apache/beam/sdk/extensions/protobuf/ProtoSchemaTranslatorTest.java b/sdks/java/extensions/protobuf/src/test/java/org/apache/beam/sdk/extensions/protobuf/ProtoSchemaTranslatorTest.java
index 4e3cf9a..b1bcbb5 100644
--- a/sdks/java/extensions/protobuf/src/test/java/org/apache/beam/sdk/extensions/protobuf/ProtoSchemaTranslatorTest.java
+++ b/sdks/java/extensions/protobuf/src/test/java/org/apache/beam/sdk/extensions/protobuf/ProtoSchemaTranslatorTest.java
@@ -30,6 +30,7 @@
 
 /** Collection of standard tests for Protobuf Schema support. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ProtoSchemaTranslatorTest {
   @Test
   public void testPrimitiveSchema() {
diff --git a/sdks/java/extensions/sorter/src/main/java/org/apache/beam/sdk/extensions/sorter/BufferedExternalSorter.java b/sdks/java/extensions/sorter/src/main/java/org/apache/beam/sdk/extensions/sorter/BufferedExternalSorter.java
index 7c66485..65ac564 100644
--- a/sdks/java/extensions/sorter/src/main/java/org/apache/beam/sdk/extensions/sorter/BufferedExternalSorter.java
+++ b/sdks/java/extensions/sorter/src/main/java/org/apache/beam/sdk/extensions/sorter/BufferedExternalSorter.java
@@ -42,6 +42,7 @@
     private final SorterType sorterType;
 
     private Options(String tempLocation, int memoryMB, SorterType sorterType) {
+      checkMemoryMB(memoryMB, sorterType);
       this.tempLocation = tempLocation;
       this.memoryMB = memoryMB;
       this.sorterType = sorterType;
@@ -63,14 +64,10 @@
 
     /**
      * Sets the size of the memory buffer in megabytes. This controls both the buffer for initial in
-     * memory sorting and the buffer used when external sorting. Must be greater than zero and less
-     * than 2048.
+     * memory sorting and the buffer used when external sorting. Must be greater than zero. Must be
+     * less than 2048 if sorter type is Hadoop.
      */
     public Options withMemoryMB(int memoryMB) {
-      checkArgument(memoryMB > 0, "memoryMB must be greater than zero");
-      // Hadoop's external sort stores the number of available memory bytes in an int, this prevents
-      // overflow
-      checkArgument(memoryMB < 2048, "memoryMB must be less than 2048");
       return new Options(tempLocation, memoryMB, sorterType);
     }
 
@@ -88,6 +85,15 @@
     public SorterType getExternalSorterType() {
       return sorterType;
     }
+
+    private static void checkMemoryMB(int memoryMB, SorterType sorterType) {
+      checkArgument(memoryMB > 0, "memoryMB must be greater than zero");
+      if (sorterType == SorterType.HADOOP) {
+        // Hadoop's external sort stores the number of available memory bytes in an int, this
+        // prevents overflow
+        checkArgument(memoryMB < 2048, "memoryMB must be less than 2048 for Hadoop sorter");
+      }
+    }
   }
 
   private final ExternalSorter externalSorter;
@@ -102,9 +108,9 @@
 
   public static BufferedExternalSorter create(Options options) {
     ExternalSorter.Options externalSorterOptions = new ExternalSorter.Options();
-    externalSorterOptions.setMemoryMB(options.getMemoryMB());
     externalSorterOptions.setTempLocation(options.getTempLocation());
     externalSorterOptions.setSorterType(options.getExternalSorterType());
+    externalSorterOptions.setMemoryMB(options.getMemoryMB());
 
     InMemorySorter.Options inMemorySorterOptions = new InMemorySorter.Options();
     inMemorySorterOptions.setMemoryMB(options.getMemoryMB());
diff --git a/sdks/java/extensions/sorter/src/main/java/org/apache/beam/sdk/extensions/sorter/ExternalSorter.java b/sdks/java/extensions/sorter/src/main/java/org/apache/beam/sdk/extensions/sorter/ExternalSorter.java
index 1595d5a..d07aeef 100644
--- a/sdks/java/extensions/sorter/src/main/java/org/apache/beam/sdk/extensions/sorter/ExternalSorter.java
+++ b/sdks/java/extensions/sorter/src/main/java/org/apache/beam/sdk/extensions/sorter/ExternalSorter.java
@@ -57,11 +57,8 @@
      * 2048.
      */
     public Options setMemoryMB(int memoryMB) {
-      checkArgument(memoryMB > 0, "memoryMB must be greater than zero");
-      // Hadoop's external sort stores the number of available memory bytes in an int, this prevents
-      // integer overflow
-      checkArgument(memoryMB < 2048, "memoryMB must be less than 2048");
       this.memoryMB = memoryMB;
+      checkMemoryMB();
       return this;
     }
 
@@ -73,6 +70,7 @@
     /** Sets the sorter type. */
     public Options setSorterType(SorterType sorterType) {
       this.sorterType = sorterType;
+      checkMemoryMB();
       return this;
     }
 
@@ -80,6 +78,15 @@
     public SorterType getSorterType() {
       return sorterType;
     }
+
+    private void checkMemoryMB() {
+      checkArgument(memoryMB > 0, "memoryMB must be greater than zero");
+      if (getSorterType() == SorterType.HADOOP) {
+        // Hadoop's external sort stores the number of available memory bytes in an int, this
+        // prevents overflow
+        checkArgument(memoryMB < 2048, "memoryMB must be less than 2048 for Hadoop sorter");
+      }
+    }
   }
 
   /** Returns a {@link Sorter} configured with the given {@link Options}. */
diff --git a/sdks/java/extensions/sorter/src/test/java/org/apache/beam/sdk/extensions/sorter/ExternalSorterTest.java b/sdks/java/extensions/sorter/src/test/java/org/apache/beam/sdk/extensions/sorter/ExternalSorterTest.java
index 5bc751e..152d8f3 100644
--- a/sdks/java/extensions/sorter/src/test/java/org/apache/beam/sdk/extensions/sorter/ExternalSorterTest.java
+++ b/sdks/java/extensions/sorter/src/test/java/org/apache/beam/sdk/extensions/sorter/ExternalSorterTest.java
@@ -178,8 +178,10 @@
 
   @Test
   public void testMemoryTooLarge() {
-    thrown.expect(IllegalArgumentException.class);
-    thrown.expectMessage("memoryMB must be less than 2048");
+    if (sorterType == SorterType.HADOOP) {
+      thrown.expect(IllegalArgumentException.class);
+      thrown.expectMessage("memoryMB must be less than 2048");
+    }
     ExternalSorter.Options options = new ExternalSorter.Options().setSorterType(sorterType);
     options.setMemoryMB(2048);
   }
diff --git a/sdks/java/extensions/sql/build.gradle b/sdks/java/extensions/sql/build.gradle
index fad0da4..a14175c 100644
--- a/sdks/java/extensions/sql/build.gradle
+++ b/sdks/java/extensions/sql/build.gradle
@@ -53,7 +53,7 @@
   compile library.java.jackson_databind
   compile library.java.joda_time
   compile library.java.vendored_calcite_1_20_0
-  compile "com.alibaba:fastjson:1.2.68"
+  compile "com.alibaba:fastjson:1.2.69"
   compile "org.codehaus.janino:janino:3.0.11"
   compile "org.codehaus.janino:commons-compiler:3.0.11"
   provided "org.checkerframework:checker-qual:3.4.1"
diff --git a/sdks/java/extensions/sql/hcatalog/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/hcatalog/DatabaseProvider.java b/sdks/java/extensions/sql/hcatalog/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/hcatalog/DatabaseProvider.java
index ac95478..29eb8ea 100644
--- a/sdks/java/extensions/sql/hcatalog/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/hcatalog/DatabaseProvider.java
+++ b/sdks/java/extensions/sql/hcatalog/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/hcatalog/DatabaseProvider.java
@@ -69,9 +69,8 @@
   }
 
   /** Table metadata to pass the schema to Calcite. */
-  @Nullable
   @Override
-  public Table getTable(String table) {
+  public @Nullable Table getTable(String table) {
     Optional<Schema> tableSchema = metastoreSchema.getTableSchema(db, table);
     if (!tableSchema.isPresent()) {
       return null;
diff --git a/sdks/java/extensions/sql/hcatalog/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/hcatalog/HCatalogTableProvider.java b/sdks/java/extensions/sql/hcatalog/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/hcatalog/HCatalogTableProvider.java
index 5829a2d..4a624e5 100644
--- a/sdks/java/extensions/sql/hcatalog/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/hcatalog/HCatalogTableProvider.java
+++ b/sdks/java/extensions/sql/hcatalog/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/hcatalog/HCatalogTableProvider.java
@@ -36,6 +36,7 @@
  *
  * <p>If only the table name is provided, then 'db' is assumed to be 'default'.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class HCatalogTableProvider implements TableProvider, Serializable {
 
   private HashMap<String, String> configuration;
diff --git a/sdks/java/extensions/sql/hcatalog/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/hcatalog/BeamSqlHiveSchemaTest.java b/sdks/java/extensions/sql/hcatalog/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/hcatalog/BeamSqlHiveSchemaTest.java
index 706cf69..c08c401 100644
--- a/sdks/java/extensions/sql/hcatalog/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/hcatalog/BeamSqlHiveSchemaTest.java
+++ b/sdks/java/extensions/sql/hcatalog/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/hcatalog/BeamSqlHiveSchemaTest.java
@@ -51,6 +51,7 @@
 import org.junit.rules.TemporaryFolder;
 
 /** Test for HCatalogTableProvider. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BeamSqlHiveSchemaTest implements Serializable {
 
   private static final Schema ROW_SCHEMA =
diff --git a/sdks/java/extensions/sql/jdbc/src/test/java/org/apache/beam/sdk/extensions/sql/jdbc/BeamSqlLineIT.java b/sdks/java/extensions/sql/jdbc/src/test/java/org/apache/beam/sdk/extensions/sql/jdbc/BeamSqlLineIT.java
index 17f38ba..ae95739 100644
--- a/sdks/java/extensions/sql/jdbc/src/test/java/org/apache/beam/sdk/extensions/sql/jdbc/BeamSqlLineIT.java
+++ b/sdks/java/extensions/sql/jdbc/src/test/java/org/apache/beam/sdk/extensions/sql/jdbc/BeamSqlLineIT.java
@@ -23,6 +23,8 @@
 import static org.hamcrest.CoreMatchers.everyItem;
 import static org.junit.Assert.assertThat;
 
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.node.ObjectNode;
 import java.io.ByteArrayOutputStream;
 import java.io.Serializable;
 import java.text.ParseException;
@@ -41,8 +43,6 @@
 import org.apache.beam.sdk.testing.TestPipeline;
 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.ImmutableMap;
-import org.codehaus.jackson.map.ObjectMapper;
-import org.codehaus.jackson.node.ObjectNode;
 import org.hamcrest.collection.IsIn;
 import org.joda.time.Duration;
 import org.junit.After;
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/BeamSqlCli.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/BeamSqlCli.java
index 8bdb1bf..d2163e9 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/BeamSqlCli.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/BeamSqlCli.java
@@ -29,6 +29,7 @@
 
 /** {@link BeamSqlCli} provides methods to execute Beam SQL with an interactive client. */
 @Experimental
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BeamSqlCli {
   private BeamSqlEnv env;
   /** The store which persists all the table meta data. */
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/SqlTransform.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/SqlTransform.java
index 82362c7..249f65b 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/SqlTransform.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/SqlTransform.java
@@ -85,6 +85,7 @@
  */
 @AutoValue
 @Experimental
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public abstract class SqlTransform extends PTransform<PInput, PCollection<Row>> {
   static final String PCOLLECTION_NAME = "PCOLLECTION";
 
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/example/model/Customer.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/example/model/Customer.java
index 85162ab..f9228be 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/example/model/Customer.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/example/model/Customer.java
@@ -25,6 +25,7 @@
 
 /** Describes a customer. */
 @DefaultSchema(JavaBeanSchema.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class Customer implements Serializable {
   private int id;
   private String name;
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/BeamCalciteSchema.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/BeamCalciteSchema.java
index 7e2af88..d3d376d 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/BeamCalciteSchema.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/BeamCalciteSchema.java
@@ -34,6 +34,7 @@
 import org.checkerframework.checker.nullness.qual.Nullable;
 
 /** Adapter from {@link TableProvider} to {@link Schema}. */
+@SuppressWarnings({"keyfor", "nullness"}) // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BeamCalciteSchema implements Schema {
   private JdbcConnection connection;
   private TableProvider tableProvider;
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/BeamCalciteSchemaFactory.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/BeamCalciteSchemaFactory.java
index 2c6151a..42b02bc 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/BeamCalciteSchemaFactory.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/BeamCalciteSchemaFactory.java
@@ -54,6 +54,7 @@
  * <p>{@link Empty} is an override used in {@link JdbcDriver#connect(TableProvider,
  * org.apache.beam.sdk.options.PipelineOptions)} to avoid loading all available table providers.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class BeamCalciteSchemaFactory {
 
   /**
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/BeamCalciteTable.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/BeamCalciteTable.java
index 267199b..8415642 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/BeamCalciteTable.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/BeamCalciteTable.java
@@ -44,6 +44,7 @@
 import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.schema.TranslatableTable;
 
 /** Adapter from {@link BeamSqlTable} to a calcite Table. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BeamCalciteTable extends AbstractQueryableTable
     implements ModifiableTable, TranslatableTable {
   private final BeamSqlTable beamTable;
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/BeamSqlEnv.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/BeamSqlEnv.java
index 9c43664..098c946 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/BeamSqlEnv.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/BeamSqlEnv.java
@@ -58,6 +58,7 @@
  */
 @Internal
 @Experimental
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BeamSqlEnv {
   JdbcConnection connection;
   QueryPlanner planner;
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/CalciteQueryPlanner.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/CalciteQueryPlanner.java
index 7e7517b..a2bacb6 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/CalciteQueryPlanner.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/CalciteQueryPlanner.java
@@ -71,6 +71,7 @@
  * The core component to handle through a SQL statement, from explain execution plan, to generate a
  * Beam pipeline.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class CalciteQueryPlanner implements QueryPlanner {
   private static final Logger LOG = LoggerFactory.getLogger(CalciteQueryPlanner.class);
 
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/JdbcConnection.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/JdbcConnection.java
index 46014e0..c650410 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/JdbcConnection.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/JdbcConnection.java
@@ -37,6 +37,7 @@
  * {@link BeamCalciteSchema BeamCalciteSchemas} keep reference to this connection. Pipeline options
  * are stored here.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class JdbcConnection extends CalciteConnectionWrapper {
   /**
    * Connection string parameters that begin with {@code "beam."} will be interpreted as {@link
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/JdbcDriver.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/JdbcDriver.java
index f012fc2..c61f2cb 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/JdbcDriver.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/JdbcDriver.java
@@ -56,6 +56,7 @@
  * <p>The querystring-style parameters are parsed as {@link PipelineOptions}.
  */
 @AutoService(java.sql.Driver.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class JdbcDriver extends Driver {
   public static final JdbcDriver INSTANCE = new JdbcDriver();
   public static final String CONNECT_STRING_PREFIX = "jdbc:beam:";
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/ScalarFunctionImpl.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/ScalarFunctionImpl.java
index b4a9d7e..2c1578a 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/ScalarFunctionImpl.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/ScalarFunctionImpl.java
@@ -52,6 +52,7 @@
  * org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.schema.impl.ScalarFunctionImpl} , to
  * address BEAM-5921.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ScalarFunctionImpl extends UdfImplReflectiveFunctionBase
     implements ScalarFunction, ImplementableFunction {
 
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/TVFSlidingWindowFn.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/TVFSlidingWindowFn.java
index f7f8aef..b03600b 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/TVFSlidingWindowFn.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/TVFSlidingWindowFn.java
@@ -34,6 +34,7 @@
  * timestamps.
  */
 @AutoValue
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public abstract class TVFSlidingWindowFn extends NonMergingWindowFn<Object, IntervalWindow> {
   /** Size of the generated windows. */
   public abstract Duration getSize();
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/TableResolutionUtils.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/TableResolutionUtils.java
index 1659e87..3bd015d 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/TableResolutionUtils.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/TableResolutionUtils.java
@@ -35,6 +35,7 @@
 import org.slf4j.LoggerFactory;
 
 /** Utils to wire up the custom table resolution into Calcite's planner. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class TableResolutionUtils {
 
   private static final Logger LOG = LoggerFactory.getLogger(TableResolutionUtils.class);
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/UdafImpl.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/UdafImpl.java
index 35f5416..8af1eb6 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/UdafImpl.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/UdafImpl.java
@@ -35,6 +35,7 @@
 /** Implement {@link AggregateFunction} to take a {@link CombineFn} as UDAF. */
 @Experimental
 @Internal
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public final class UdafImpl<InputT, AccumT, OutputT>
     implements AggregateFunction, ImplementableAggFunction, Serializable {
   private CombineFn<InputT, AccumT, OutputT> combineFn;
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/UdfImpl.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/UdfImpl.java
index 34f5683..774e59f 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/UdfImpl.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/UdfImpl.java
@@ -23,6 +23,7 @@
 import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.schema.impl.TableMacroImpl;
 
 /** Beam-customized facade behind {@link Function} to address BEAM-5921. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class UdfImpl {
 
   private UdfImpl() {}
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/UdfImplReflectiveFunctionBase.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/UdfImplReflectiveFunctionBase.java
index be13523..ba472ed 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/UdfImplReflectiveFunctionBase.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/UdfImplReflectiveFunctionBase.java
@@ -32,6 +32,7 @@
 import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.util.ReflectUtil;
 
 /** Beam-customized version from {@link ReflectiveFunctionBase}, to address BEAM-5921. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public abstract class UdfImplReflectiveFunctionBase implements Function {
   /** Method that implements the function. */
   public final Method method;
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/cep/CEPLiteral.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/cep/CEPLiteral.java
index 81526b0..518d7cb 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/cep/CEPLiteral.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/cep/CEPLiteral.java
@@ -26,6 +26,7 @@
 /**
  * {@code CEPLiteral} represents a literal node. It corresponds to {@code RexLiteral} in Calcite.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class CEPLiteral extends CEPOperation {
 
   private final Schema.TypeName typeName;
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/cep/CEPPattern.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/cep/CEPPattern.java
index 4224ea5..a068791 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/cep/CEPPattern.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/cep/CEPPattern.java
@@ -26,6 +26,7 @@
 import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexCall;
 
 /** Core pattern class that stores the definition of a single pattern. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class CEPPattern implements Serializable {
 
   private final Schema mySchema;
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/cep/CEPUtils.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/cep/CEPUtils.java
index 4be7591..0475ed1 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/cep/CEPUtils.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/cep/CEPUtils.java
@@ -34,6 +34,7 @@
  * Some utility methods for transforming Calcite's constructs into our own Beam constructs (for
  * serialization purpose).
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class CEPUtils {
 
   private static Quantifier getQuantifier(int start, int end, boolean isReluctant) {
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/nfa/NFA.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/nfa/NFA.java
index 221b888..f9c0eb3 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/nfa/NFA.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/nfa/NFA.java
@@ -49,6 +49,7 @@
  * @see <a
  *     href="https://dl.acm.org/doi/10.1145/1376616.1376634">https://dl.acm.org/doi/10.1145/1376616.1376634</a>
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class NFA implements Serializable {
 
   private final State startState;
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlCreateExternalTable.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlCreateExternalTable.java
index bd331a9..482458e 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlCreateExternalTable.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlCreateExternalTable.java
@@ -43,6 +43,7 @@
 import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.util.Pair;
 
 /** Parse tree for {@code CREATE EXTERNAL TABLE} statement. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SqlCreateExternalTable extends SqlCreate implements SqlExecutableStatement {
   private final SqlIdentifier name;
   private final List<Schema.Field> columnList;
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlSetOptionBeam.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlSetOptionBeam.java
index c74a1fb..6c78980 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlSetOptionBeam.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlSetOptionBeam.java
@@ -31,6 +31,7 @@
 import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.util.Pair;
 
 /** SQL parse tree node to represent {@code SET} and {@code RESET} statements. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SqlSetOptionBeam extends SqlSetOption implements SqlExecutableStatement {
 
   public SqlSetOptionBeam(SqlParserPos pos, String scope, SqlIdentifier name, SqlNode value) {
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/AbstractBeamCalcRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/AbstractBeamCalcRel.java
index 61abbb6..659ca84 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/AbstractBeamCalcRel.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/AbstractBeamCalcRel.java
@@ -32,6 +32,7 @@
 
 /** BeamRelNode to replace {@code Project} and {@code Filter} node. */
 @Internal
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public abstract class AbstractBeamCalcRel extends Calc implements BeamRelNode {
 
   public AbstractBeamCalcRel(
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamAggregationRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamAggregationRel.java
index f5accef..5fbf6ec 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamAggregationRel.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamAggregationRel.java
@@ -64,6 +64,7 @@
 import org.joda.time.Duration;
 
 /** {@link BeamRelNode} to replace a {@link Aggregate} node. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BeamAggregationRel extends Aggregate implements BeamRelNode {
   private @Nullable WindowFn<Row, IntervalWindow> windowFn;
   private final int windowFieldIndex;
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamCalcRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamCalcRel.java
index d9ad401..7e00644 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamCalcRel.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamCalcRel.java
@@ -91,6 +91,7 @@
 import org.joda.time.ReadableInstant;
 
 /** BeamRelNode to replace {@code Project} and {@code Filter} node. */
+@SuppressWarnings({"keyfor", "nullness"}) // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BeamCalcRel extends AbstractBeamCalcRel {
 
   private static final long NANOS_PER_MILLISECOND = 1000000L;
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamEnumerableConverter.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamEnumerableConverter.java
index 48df9cf..aa97268 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamEnumerableConverter.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamEnumerableConverter.java
@@ -82,6 +82,7 @@
 import org.slf4j.LoggerFactory;
 
 /** BeamRelNode to replace a {@code Enumerable} node. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BeamEnumerableConverter extends ConverterImpl implements EnumerableRel {
   private static final Logger LOG = LoggerFactory.getLogger(BeamEnumerableConverter.class);
 
@@ -266,7 +267,7 @@
     // This will only work on the direct runner.
     private static final Map<Long, Queue<Row>> globalValues = new ConcurrentHashMap<>();
 
-    private @Nullable volatile Queue<Row> values;
+    private volatile @Nullable Queue<Row> values;
 
     @StartBundle
     public void startBundle(StartBundleContext context) {
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamMatchRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamMatchRel.java
index ed64e9b..28fbdcc 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamMatchRel.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamMatchRel.java
@@ -68,6 +68,7 @@
  *
  * <p>For now, the underline implementation is based on java.util.regex.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BeamMatchRel extends Match implements BeamRelNode {
 
   public BeamMatchRel(
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamRelNode.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamRelNode.java
index 1b549b4..c48a905 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamRelNode.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamRelNode.java
@@ -30,6 +30,7 @@
 import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.metadata.RelMetadataQuery;
 
 /** A {@link RelNode} that can also give a {@link PTransform} that implements the expression. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public interface BeamRelNode extends RelNode {
 
   /**
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSetOperatorRelBase.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSetOperatorRelBase.java
index ab76a7b..59f1457 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSetOperatorRelBase.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSetOperatorRelBase.java
@@ -35,6 +35,7 @@
  * Delegate for Set operators: {@code BeamUnionRel}, {@code BeamIntersectRel} and {@code
  * BeamMinusRel}.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BeamSetOperatorRelBase extends PTransform<PCollectionList<Row>, PCollection<Row>> {
   /** Set operator type. */
   public enum OpType implements Serializable {
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSortRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSortRel.java
index eced09a..d319f55 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSortRel.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSortRel.java
@@ -91,6 +91,7 @@
  *       make much sense to use ORDER BY with WINDOW.
  * </ul>
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BeamSortRel extends Sort implements BeamRelNode {
   private List<Integer> fieldIndices = new ArrayList<>();
   private List<Boolean> orientation = new ArrayList<>();
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamTableFunctionScanRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamTableFunctionScanRel.java
index 7a309ec..5b1a212 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamTableFunctionScanRel.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamTableFunctionScanRel.java
@@ -70,6 +70,7 @@
  * BeamRelNode to replace {@code TableFunctionScan}. Currently this class limits to support
  * table-valued function for streaming windowing.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BeamTableFunctionScanRel extends TableFunctionScan implements BeamRelNode {
   public BeamTableFunctionScanRel(
       RelOptCluster cluster,
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamUncollectRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamUncollectRel.java
index 24545b6..291da33 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamUncollectRel.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamUncollectRel.java
@@ -37,6 +37,7 @@
 import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.metadata.RelMetadataQuery;
 
 /** {@link BeamRelNode} to implement an uncorrelated {@link Uncollect}, aka UNNEST. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BeamUncollectRel extends Uncollect implements BeamRelNode {
 
   public BeamUncollectRel(
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 23d263b..1fed6c7 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
@@ -47,6 +47,7 @@
  * {@link BeamRelNode} to implement UNNEST, supporting specifically only {@link Correlate} with
  * {@link Uncollect}.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BeamUnnestRel extends Uncollect implements BeamRelNode {
 
   private final RelDataType unnestType;
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamWindowRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamWindowRel.java
index 5d5da8d..432cc6b 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamWindowRel.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamWindowRel.java
@@ -71,6 +71,7 @@
  *   <li>Only Aggregate Analytic Functions are available.
  * </ul>
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BeamWindowRel extends Window implements BeamRelNode {
   public BeamWindowRel(
       RelOptCluster cluster,
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamAggregateProjectMergeRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamAggregateProjectMergeRule.java
index b146b04..ea8f143 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamAggregateProjectMergeRule.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamAggregateProjectMergeRule.java
@@ -37,6 +37,7 @@
  * case when an underlying IO supports project push-down it is more efficient to not merge {@code
  * Project} with an {@code Aggregate}, leaving it for the {@code BeamIOPUshDownRule}.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BeamAggregateProjectMergeRule extends AggregateProjectMergeRule {
   public static final AggregateProjectMergeRule INSTANCE =
       new BeamAggregateProjectMergeRule(
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamAggregationRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamAggregationRule.java
index 6525a57..0e7f634 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamAggregationRule.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamAggregationRule.java
@@ -41,6 +41,7 @@
 import org.joda.time.Duration;
 
 /** Rule to detect the window/trigger settings. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BeamAggregationRule extends RelOptRule {
   public static final BeamAggregationRule INSTANCE =
       new BeamAggregationRule(Aggregate.class, Project.class, RelFactories.LOGICAL_BUILDER);
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamBasicAggregationRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamBasicAggregationRule.java
index 9a8f34b..a851f93 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamBasicAggregationRule.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamBasicAggregationRule.java
@@ -43,6 +43,7 @@
  *
  * <p>{@link BeamAggregationRule} supports projection and windowing.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BeamBasicAggregationRule extends RelOptRule {
   public static final BeamBasicAggregationRule INSTANCE =
       new BeamBasicAggregationRule(Aggregate.class, RelFactories.LOGICAL_BUILDER);
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamIOPushDownRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamIOPushDownRule.java
index 33e9934..b7b5ba6 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamIOPushDownRule.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamIOPushDownRule.java
@@ -58,6 +58,7 @@
 import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.util.Pair;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
 
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BeamIOPushDownRule extends RelOptRule {
   // ~ Static fields/initializers ---------------------------------------------
 
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamTableFunctionScanRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamTableFunctionScanRule.java
index 3fc33a6..5d329de 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamTableFunctionScanRule.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamTableFunctionScanRule.java
@@ -34,6 +34,7 @@
  * This is the conveter rule that converts a Calcite {@code TableFunctionScan} to Beam {@code
  * TableFunctionScanRel}.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BeamTableFunctionScanRule extends ConverterRule {
   public static final BeamTableFunctionScanRule INSTANCE = new BeamTableFunctionScanRule();
 
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/JoinRelOptRuleCall.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/JoinRelOptRuleCall.java
index 27d8168..048fbb8 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/JoinRelOptRuleCall.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/JoinRelOptRuleCall.java
@@ -31,6 +31,7 @@
  * This is a class to catch the built join and check if it is a legal join before passing it to the
  * actual RelOptRuleCall.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class JoinRelOptRuleCall extends RelOptRuleCall {
   private final RelOptRuleCall originalCall;
   private final JoinChecker checker;
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/schema/BeamTableUtils.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/schema/BeamTableUtils.java
index c2a03df..77e6551 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/schema/BeamTableUtils.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/schema/BeamTableUtils.java
@@ -48,6 +48,7 @@
  *
  * <p>TODO: Does not yet support nested types.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public final class BeamTableUtils {
 
   /**
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamBuiltinAggregations.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamBuiltinAggregations.java
index 85e99a3..be310a5 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamBuiltinAggregations.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamBuiltinAggregations.java
@@ -45,6 +45,7 @@
 import org.checkerframework.checker.nullness.qual.Nullable;
 
 /** Built-in aggregations functions for COUNT/MAX/MIN/SUM/AVG/VAR_POP/VAR_SAMP. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BeamBuiltinAggregations {
 
   public static final Map<String, Function<Schema.FieldType, CombineFn<?, ?, ?>>>
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamBuiltinAnalyticFunctions.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamBuiltinAnalyticFunctions.java
index 457e0db..087a6b1 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamBuiltinAnalyticFunctions.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamBuiltinAnalyticFunctions.java
@@ -27,6 +27,7 @@
 import org.apache.beam.vendor.calcite.v1_20_0.com.google.common.collect.ImmutableMap;
 
 /** Built-in Analytic Functions for the aggregation analytics functionality. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BeamBuiltinAnalyticFunctions {
   public static final Map<String, Function<Schema.FieldType, Combine.CombineFn<?, ?, ?>>>
       BUILTIN_ANALYTIC_FACTORIES =
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamJoinTransforms.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamJoinTransforms.java
index 0aa74a2..bdae4d2 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamJoinTransforms.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamJoinTransforms.java
@@ -39,6 +39,7 @@
 import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.util.Pair;
 
 /** Collections of {@code PTransform} and {@code DoFn} used to perform JOIN operation. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BeamJoinTransforms {
 
   public static FieldAccessDescriptor getJoinColumns(
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamSetOperatorsTransforms.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamSetOperatorsTransforms.java
index 4a7985d..d412868 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamSetOperatorsTransforms.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamSetOperatorsTransforms.java
@@ -25,6 +25,7 @@
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables;
 
 /** Collections of {@code PTransform} and {@code DoFn} used to perform Set operations. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public abstract class BeamSetOperatorsTransforms {
   /** Transform a {@code BeamSqlRow} to a {@code KV<BeamSqlRow, BeamSqlRow>}. */
   public static class BeamSqlRow2KvFn extends SimpleFunction<Row, KV<Row, Row>> {
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/agg/AggregationCombineFnAdapter.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/agg/AggregationCombineFnAdapter.java
index b2110af..e151028 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/agg/AggregationCombineFnAdapter.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/agg/AggregationCombineFnAdapter.java
@@ -32,6 +32,7 @@
 import org.checkerframework.checker.nullness.qual.Nullable;
 
 /** Wrapper {@link CombineFn}s for aggregation function calls. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class AggregationCombineFnAdapter<T> {
   private abstract static class WrappedCombinerBase<T> extends CombineFn<T, Object, Object> {
     CombineFn<T, Object, Object> combineFn;
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/agg/CovarianceFn.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/agg/CovarianceFn.java
index ac7edfe..ef4563c 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/agg/CovarianceFn.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/agg/CovarianceFn.java
@@ -43,6 +43,7 @@
  * Statistical Moments".
  */
 @Internal
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class CovarianceFn<T extends Number>
     extends Combine.CombineFn<Row, CovarianceAccumulator, T> {
 
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/udf/BeamBuiltinFunctionProvider.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/udf/BeamBuiltinFunctionProvider.java
index 195ac96..03bb2bd 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/udf/BeamBuiltinFunctionProvider.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/udf/BeamBuiltinFunctionProvider.java
@@ -24,6 +24,7 @@
 import java.util.stream.Collectors;
 
 /** BeamBuiltinFunctionClass interface. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public abstract class BeamBuiltinFunctionProvider {
   public Map<String, List<Method>> getBuiltinMethods() {
     List<Method> methods = Arrays.asList(getClass().getMethods());
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/udf/BuiltinTrigonometricFunctions.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/udf/BuiltinTrigonometricFunctions.java
index 9a5f8ab..010c025 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/udf/BuiltinTrigonometricFunctions.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/udf/BuiltinTrigonometricFunctions.java
@@ -22,6 +22,7 @@
 
 /** TrigonometricFunctions. */
 @AutoService(BeamBuiltinFunctionProvider.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BuiltinTrigonometricFunctions extends BeamBuiltinFunctionProvider {
 
   /**
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 6ded492..369a241 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
@@ -39,6 +39,7 @@
 import org.joda.time.base.AbstractInstant;
 
 /** Utility methods for Calcite related operations. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class CalciteUtils {
   private static final long UNLIMITED_ARRAY_SIZE = -1L;
 
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/FullNameTableProvider.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/FullNameTableProvider.java
index c73294a..9815ca0 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/FullNameTableProvider.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/FullNameTableProvider.java
@@ -34,6 +34,7 @@
  * querying it by parts of the name separately.
  */
 @Experimental
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public abstract class FullNameTableProvider implements TableProvider, CustomTableResolver {
 
   private List<TableName> knownTables;
@@ -132,9 +133,8 @@
       return "google.cloud.datacatalog.subprovider";
     }
 
-    @Nullable
     @Override
-    public Table getTable(String name) {
+    public @Nullable Table getTable(String name) {
 
       // This is called only after getSubProvider() returned null,
       // and since we are tracking the actual parsed table names, this should
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/ReadOnlyTableProvider.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/ReadOnlyTableProvider.java
index ab826fc..f312a60 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/ReadOnlyTableProvider.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/ReadOnlyTableProvider.java
@@ -27,6 +27,7 @@
  * A {@code ReadOnlyTableProvider} provides in-memory read only set of {@code BeamSqlTable
  * BeamSqlTables}.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ReadOnlyTableProvider implements TableProvider {
   private final String typeName;
   private final Map<String, BeamSqlTable> tables;
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/SchemaIOTableProviderWrapper.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/SchemaIOTableProviderWrapper.java
index 46901f5..4f7f4f5 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/SchemaIOTableProviderWrapper.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/SchemaIOTableProviderWrapper.java
@@ -44,6 +44,7 @@
 /** A general {@link TableProvider} for IOs for consumption by Beam SQL. */
 @Internal
 @Experimental
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public abstract class SchemaIOTableProviderWrapper extends InMemoryMetaTableProvider
     implements Serializable {
 
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/TableProvider.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/TableProvider.java
index 4d755f8..944baa9 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/TableProvider.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/TableProvider.java
@@ -36,6 +36,7 @@
  * automatically loaded by CLI or other cases when {@link JdbcDriver} is used with default
  * connection parameters.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public interface TableProvider {
   /** Gets the table type this provider handles. */
   String getTableType();
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BeamBigQuerySqlDialect.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BeamBigQuerySqlDialect.java
index d606b8b..561eba6 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BeamBigQuerySqlDialect.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BeamBigQuerySqlDialect.java
@@ -48,6 +48,7 @@
 
 // TODO(CALCITE-3381): some methods below can be deleted after updating vendor Calcite version.
 // Calcite v1_20_0 does not have type translation implemented, but later (unreleased) versions do.
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BeamBigQuerySqlDialect extends BigQuerySqlDialect {
 
   public static final SqlDialect.Context DEFAULT_CONTEXT =
@@ -87,16 +88,17 @@
           .put("$extract_time", "TIME")
           .put("$extract_datetime", "DATETIME")
           .build();
-  public static final String DOUBLE_POSITIVE_INF_FUNCTION = "double_positive_inf";
-  public static final String DOUBLE_NEGATIVE_INF_FUNCTION = "double_negative_inf";
-  public static final String DOUBLE_NAN_FUNCTION = "double_nan";
-  private static final Map<String, String> DOUBLE_FUNCTIONS =
+  public static final String DOUBLE_POSITIVE_INF_WRAPPER = "double_positive_inf";
+  public static final String DOUBLE_NEGATIVE_INF_WRAPPER = "double_negative_inf";
+  public static final String DOUBLE_NAN_WRAPPER = "double_nan";
+  // ZetaSQL has no literal representation of NaN and infinity, so we need to CAST from strings
+  private static final Map<String, String> DOUBLE_LITERAL_WRAPPERS =
       ImmutableMap.<String, String>builder()
-          .put(DOUBLE_POSITIVE_INF_FUNCTION, "CAST('+inf' AS FLOAT64)")
-          .put(DOUBLE_NEGATIVE_INF_FUNCTION, "CAST('-inf' AS FLOAT64)")
-          .put(DOUBLE_NAN_FUNCTION, "CAST('NaN' AS FLOAT64)")
+          .put(DOUBLE_POSITIVE_INF_WRAPPER, "CAST('+inf' AS FLOAT64)")
+          .put(DOUBLE_NEGATIVE_INF_WRAPPER, "CAST('-inf' AS FLOAT64)")
+          .put(DOUBLE_NAN_WRAPPER, "CAST('NaN' AS FLOAT64)")
           .build();
-  public static final String NUMERIC_LITERAL_FUNCTION = "numeric_literal";
+  public static final String NUMERIC_LITERAL_WRAPPER = "numeric_literal";
 
   public BeamBigQuerySqlDialect(Context context) {
     super(context);
@@ -166,12 +168,12 @@
         break;
       case OTHER_FUNCTION:
         String funName = call.getOperator().getName();
-        if (DOUBLE_FUNCTIONS.containsKey(funName)) {
+        if (DOUBLE_LITERAL_WRAPPERS.containsKey(funName)) {
           // self-designed function dealing with the unparsing of ZetaSQL DOUBLE positive
           // infinity, negative infinity and NaN
-          unparseDoubleWrapperFunction(writer, funName);
+          unparseDoubleLiteralWrapperFunction(writer, funName);
           break;
-        } else if (NUMERIC_LITERAL_FUNCTION.equals(funName)) {
+        } else if (NUMERIC_LITERAL_WRAPPER.equals(funName)) {
           // self-designed function dealing with the unparsing of ZetaSQL NUMERIC literal
           unparseNumericLiteralWrapperFunction(writer, call, leftPrec, rightPrec);
           break;
@@ -253,12 +255,8 @@
     writer.endFunCall(trimFrame);
   }
 
-  /**
-   * As there is no direct ZetaSQL literal representation of NaN or infinity, we cast String "+inf",
-   * "-inf" and "NaN" to FLOAT64 representing positive infinity, negative infinity and NaN.
-   */
-  private void unparseDoubleWrapperFunction(SqlWriter writer, String funName) {
-    writer.literal(DOUBLE_FUNCTIONS.get(funName));
+  private void unparseDoubleLiteralWrapperFunction(SqlWriter writer, String funName) {
+    writer.literal(DOUBLE_LITERAL_WRAPPERS.get(funName));
   }
 
   private void unparseNumericLiteralWrapperFunction(
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BeamSqlUnparseContext.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BeamSqlUnparseContext.java
index ac65eb5..404cb5a 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BeamSqlUnparseContext.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BeamSqlUnparseContext.java
@@ -47,6 +47,7 @@
 import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.util.TimestampString;
 import org.checkerframework.checker.nullness.qual.Nullable;
 
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BeamSqlUnparseContext extends SqlImplementor.SimpleContext {
 
   // More about escape sequences here:
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BigQueryFilter.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BigQueryFilter.java
index 7450036..54ff753 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BigQueryFilter.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BigQueryFilter.java
@@ -42,6 +42,7 @@
 import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableSet;
 
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BigQueryFilter implements BeamSqlTableFilter {
   private static final ImmutableSet<SqlKind> SUPPORTED_OPS =
       ImmutableSet.<SqlKind>builder()
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BigQueryTable.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BigQueryTable.java
index 060c4aa..7524ee3 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BigQueryTable.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BigQueryTable.java
@@ -64,6 +64,7 @@
  * support being a source.
  */
 @Experimental
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class BigQueryTable extends SchemaBaseBeamTable implements Serializable {
   @VisibleForTesting static final String METHOD_PROPERTY = "method";
   @VisibleForTesting static final String WRITE_DISPOSITION_PROPERTY = "writeDisposition";
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/kafka/BeamKafkaTable.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/kafka/BeamKafkaTable.java
index f888ef9..6b71ff1 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/kafka/BeamKafkaTable.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/kafka/BeamKafkaTable.java
@@ -53,6 +53,7 @@
  * {@code BeamKafkaTable} represent a Kafka topic, as source or target. Need to extend to convert
  * between {@code BeamSqlRow} and {@code KV<byte[], byte[]>}.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public abstract class BeamKafkaTable extends SchemaBaseBeamTable {
   private String bootstrapServers;
   private List<String> topics;
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 393162e..3241162 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
@@ -73,6 +73,7 @@
 import org.slf4j.LoggerFactory;
 
 @Experimental
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class MongoDbTable extends SchemaBaseBeamTable implements Serializable {
   private static final Logger LOG = LoggerFactory.getLogger(MongoDbTable.class);
   // Should match: mongodb://username:password@localhost:27017/database/collection
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/test/TestTableFilter.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/test/TestTableFilter.java
index 06082ad..a5f015d 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/test/TestTableFilter.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/test/TestTableFilter.java
@@ -30,6 +30,7 @@
 import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexNode;
 import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.type.SqlTypeName;
 
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class TestTableFilter implements BeamSqlTableFilter {
   private List<RexNode> supported;
   private List<RexNode> unsupported;
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/test/TestTableProvider.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/test/TestTableProvider.java
index 9082d4d..c583990 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/test/TestTableProvider.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/test/TestTableProvider.java
@@ -67,6 +67,7 @@
  * <p>Keeps global state and tracks class instances. Works only in DirectRunner.
  */
 @AutoService(TableProvider.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class TestTableProvider extends InMemoryMetaTableProvider {
   static final Map<Long, Map<String, TableWithRows>> GLOBAL_TABLES = new ConcurrentHashMap<>();
   public static final String PUSH_DOWN_OPTION = "push_down";
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/test/TestTableUtils.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/test/TestTableUtils.java
index 7ebbf91..a4d41f2 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/test/TestTableUtils.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/test/TestTableUtils.java
@@ -31,6 +31,7 @@
 
 /** Utility functions for mock classes. */
 @Experimental
+@SuppressWarnings({"keyfor", "nullness"}) // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class TestTableUtils {
 
   /**
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/test/TestUnboundedTable.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/test/TestUnboundedTable.java
index 22c1bd2..99bfc24 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/test/TestUnboundedTable.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/test/TestUnboundedTable.java
@@ -35,6 +35,7 @@
 
 /** A mocked unbounded table. */
 @Experimental
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class TestUnboundedTable extends TestTable {
   /** rows flow out from this table with the specified watermark instant. */
   private final List<Pair<Duration, List<Row>>> timestampedRows = new ArrayList<>();
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/text/TextTable.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/text/TextTable.java
index d211ab3..a21d493 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/text/TextTable.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/text/TextTable.java
@@ -44,6 +44,7 @@
  * <p>{@link CSVFormat} itself has many dialects, check its javadoc for more info.
  */
 @Internal
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class TextTable extends SchemaBaseBeamTable {
 
   private final PTransform<PCollection<String>, PCollection<Row>> readConverter;
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/text/TextTableProvider.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/text/TextTableProvider.java
index 1d6e50b..9536e2a 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/text/TextTableProvider.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/text/TextTableProvider.java
@@ -74,6 +74,7 @@
  * }</pre>
  */
 @AutoService(TableProvider.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class TextTableProvider extends InMemoryMetaTableProvider {
 
   @Override
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/store/InMemoryMetaStore.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/store/InMemoryMetaStore.java
index 7a3fb1f..a19bcc1 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/store/InMemoryMetaStore.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/store/InMemoryMetaStore.java
@@ -31,6 +31,7 @@
  * which created, you need to create again every time you launch the {@link
  * org.apache.beam.sdk.extensions.sql.BeamSqlCli}.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class InMemoryMetaStore implements MetaStore {
   private Map<String, Table> tables = new HashMap<>();
   private Map<String, TableProvider> providers = new HashMap<>();
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 0519798..a8e06e7 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
@@ -44,6 +44,7 @@
 import org.junit.Test;
 
 /** Unit Tests for ComplexTypes, including nested ROW etc. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BeamComplexTypeTest {
   private static final Schema innerRowSchema =
       Schema.builder().addStringField("string_field").addInt64Field("long_field").build();
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlCliTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlCliTest.java
index 9584972..b78af7d 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlCliTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlCliTest.java
@@ -42,6 +42,7 @@
 import org.junit.Test;
 
 /** UnitTest for {@link BeamSqlCli}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BeamSqlCliTest {
   @Test
   public void testExecute_createTextTable() throws Exception {
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslAggregationCovarianceTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslAggregationCovarianceTest.java
index edb765a..f389811 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslAggregationCovarianceTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslAggregationCovarianceTest.java
@@ -31,6 +31,7 @@
 import org.junit.Test;
 
 /** Integration tests for {@code COVAR_POP} and {@code COVAR_SAMP}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BeamSqlDslAggregationCovarianceTest {
 
   private static final double PRECISION = 1e-7;
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslAggregationNullableTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslAggregationNullableTest.java
index 91925f0..79659ed 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslAggregationNullableTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslAggregationNullableTest.java
@@ -33,6 +33,7 @@
 import org.junit.Test;
 
 /** Integration tests for aggregation nullable columns. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BeamSqlDslAggregationNullableTest {
 
   @Rule public TestPipeline pipeline = TestPipeline.create();
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslAggregationTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslAggregationTest.java
index f7e991d..3a5fc8a 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslAggregationTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslAggregationTest.java
@@ -58,6 +58,7 @@
  * Tests for GROUP-BY/aggregation, with global_window/fix_time_window/sliding_window/session_window
  * with BOUNDED PCollection.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BeamSqlDslAggregationTest extends BeamSqlDslBase {
   public PCollection<Row> boundedInput3;
 
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslAggregationVarianceTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslAggregationVarianceTest.java
index 808b27a..1bacd95 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslAggregationVarianceTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslAggregationVarianceTest.java
@@ -31,6 +31,7 @@
 import org.junit.Test;
 
 /** Integration tests for {@code VAR_POP} and {@code VAR_SAMP}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BeamSqlDslAggregationVarianceTest {
 
   private static final double PRECISION = 1e-7;
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslBase.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslBase.java
index 4298c07..df8d262 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslBase.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslBase.java
@@ -46,6 +46,7 @@
  *
  * <p>Note that, any change in these records would impact tests in this package.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BeamSqlDslBase {
   @Rule public final TestPipeline pipeline = TestPipeline.create();
   @Rule public ExpectedException exceptions = ExpectedException.none();
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslJoinTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslJoinTest.java
index b210bb0..7aed2da 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslJoinTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslJoinTest.java
@@ -42,6 +42,7 @@
 import org.junit.rules.ExpectedException;
 
 /** Tests for joins in queries. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BeamSqlDslJoinTest {
 
   @Rule public final ExpectedException thrown = ExpectedException.none();
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslProjectTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslProjectTest.java
index 998013f..6b736f3 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslProjectTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslProjectTest.java
@@ -35,6 +35,7 @@
 import org.junit.Test;
 
 /** Tests for field-project in queries with BOUNDED PCollection. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BeamSqlDslProjectTest extends BeamSqlDslBase {
   /** select all fields with bounded PCollection. */
   @Test
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslSqlStdOperatorsTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslSqlStdOperatorsTest.java
index 916f229..df84504 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslSqlStdOperatorsTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslSqlStdOperatorsTest.java
@@ -61,6 +61,7 @@
  * DSL compliance tests for the row-level operators of {@link
  * org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.fun.SqlStdOperatorTable}.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BeamSqlDslSqlStdOperatorsTest extends BeamSqlBuiltinFunctionsIntegrationTestBase {
   private static final BigDecimal ZERO = BigDecimal.valueOf(0.0);
   private static final BigDecimal ONE = BigDecimal.valueOf(1.0);
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslUdfUdafTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslUdfUdafTest.java
index 352a3cb..efe7b7f 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslUdfUdafTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslUdfUdafTest.java
@@ -46,6 +46,7 @@
 import org.junit.Test;
 
 /** Tests for UDF/UDAF. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BeamSqlDslUdfUdafTest extends BeamSqlDslBase {
 
   /** GROUP-BY with UDAF. */
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlExplainTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlExplainTest.java
index 3f0d2f2..a0701b0 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlExplainTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlExplainTest.java
@@ -28,6 +28,7 @@
 import org.junit.Ignore;
 
 /** UnitTest for Explain Plan. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BeamSqlExplainTest {
   private InMemoryMetaStore metaStore;
   private BeamSqlCli cli;
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/InferredJavaBeanSqlTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/InferredJavaBeanSqlTest.java
index 45b9601..3417604 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/InferredJavaBeanSqlTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/InferredJavaBeanSqlTest.java
@@ -35,6 +35,7 @@
 import org.junit.Test;
 
 /** Tests for automatic inferring schema from the input {@link PCollection} of JavaBeans. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class InferredJavaBeanSqlTest {
 
   @Rule public final TestPipeline pipeline = TestPipeline.create();
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/TestUtils.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/TestUtils.java
index 3ab0ddd..fc76814 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/TestUtils.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/TestUtils.java
@@ -35,6 +35,7 @@
 import org.joda.time.Instant;
 
 /** Test utilities. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class TestUtils {
   /** A {@code DoFn} to convert a {@code BeamSqlRow} to a comparable {@code String}. */
   public static class BeamSqlRow2StringDoFn extends DoFn<Row, String> {
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/JdbcDriverTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/JdbcDriverTest.java
index 5d01661..43e8179 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/JdbcDriverTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/JdbcDriverTest.java
@@ -65,6 +65,7 @@
 import org.junit.rules.ExpectedException;
 
 /** Test for {@link JdbcDriver}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class JdbcDriverTest {
   public static final DateTime FIRST_DATE = new DateTime(1);
 
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/parser/BeamDDLNestedTypesTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/parser/BeamDDLNestedTypesTest.java
index b472bba..2d47339 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/parser/BeamDDLNestedTypesTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/parser/BeamDDLNestedTypesTest.java
@@ -43,6 +43,7 @@
  * <p>By default quick check runs this test 100 times.
  */
 @RunWith(JUnitQuickcheck.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BeamDDLNestedTypesTest {
 
   @Property
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/parser/BeamDDLTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/parser/BeamDDLTest.java
index f6db1ce..f7172a0 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/parser/BeamDDLTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/parser/BeamDDLTest.java
@@ -36,6 +36,7 @@
 import org.junit.Test;
 
 /** UnitTest for {@link BeamSqlParserImpl}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BeamDDLTest {
 
   @Test
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/planner/NodeStatsTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/planner/NodeStatsTest.java
index 9e442c5..d43d8d3 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/planner/NodeStatsTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/planner/NodeStatsTest.java
@@ -31,6 +31,7 @@
 import org.junit.Test;
 
 /** This tests the NodeStats Metadata handler and the estimations. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class NodeStatsTest extends BaseRelTest {
   static class UnknownRel extends SingleRel {
     protected UnknownRel(RelOptCluster cluster, RelTraitSet traits, RelNode input) {
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BaseRelTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BaseRelTest.java
index 5ba74e8..c3b2b09 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BaseRelTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BaseRelTest.java
@@ -26,6 +26,7 @@
 import org.apache.beam.sdk.values.Row;
 
 /** Base class for rel test. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public abstract class BaseRelTest {
   private static Map<String, BeamSqlTable> tables = new HashMap<>();
   protected static BeamSqlEnv env = BeamSqlEnv.readOnly("test", tables);
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamCoGBKJoinRelBoundedVsBoundedTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamCoGBKJoinRelBoundedVsBoundedTest.java
index 6859f96..7db02b2 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamCoGBKJoinRelBoundedVsBoundedTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamCoGBKJoinRelBoundedVsBoundedTest.java
@@ -34,6 +34,7 @@
 import org.junit.rules.ExpectedException;
 
 /** Bounded + Bounded Test for {@code BeamCoGBKJoinRel}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BeamCoGBKJoinRelBoundedVsBoundedTest extends BaseRelTest {
   @Rule public final TestPipeline pipeline = TestPipeline.create();
   @Rule public ExpectedException thrown = ExpectedException.none();
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamCoGBKJoinRelUnboundedVsUnboundedTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamCoGBKJoinRelUnboundedVsUnboundedTest.java
index f310265..3f9caa9 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamCoGBKJoinRelUnboundedVsUnboundedTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamCoGBKJoinRelUnboundedVsUnboundedTest.java
@@ -37,6 +37,7 @@
 import org.junit.Test;
 
 /** Unbounded + Unbounded Test for {@code BeamCoGBKJoinRel}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BeamCoGBKJoinRelUnboundedVsUnboundedTest extends BaseRelTest {
   @Rule public final TestPipeline pipeline = TestPipeline.create();
   private static final DateTime FIRST_DATE = new DateTime(1);
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamEnumerableConverterTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamEnumerableConverterTest.java
index a0cf404..494203f 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamEnumerableConverterTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamEnumerableConverterTest.java
@@ -58,6 +58,7 @@
 import org.junit.runners.Parameterized;
 
 /** Test for {@code BeamEnumerableConverter}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BeamEnumerableConverterTest {
   static final JavaTypeFactory TYPE_FACTORY = new JavaTypeFactoryImpl(RelDataTypeSystem.DEFAULT);
   static RexBuilder rexBuilder = new RexBuilder(TYPE_FACTORY);
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSideInputJoinRelTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSideInputJoinRelTest.java
index 39e2a73..6644eeb 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSideInputJoinRelTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSideInputJoinRelTest.java
@@ -37,6 +37,7 @@
 import org.junit.Test;
 
 /** Unbounded + Bounded Test for {@code BeamSideInputJoinRel}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BeamSideInputJoinRelTest extends BaseRelTest {
   @Rule public final TestPipeline pipeline = TestPipeline.create();
   public static final DateTime FIRST_DATE = new DateTime(1);
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSideInputLookupJoinRelTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSideInputLookupJoinRelTest.java
index 4b8dc38..6894eb9 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSideInputLookupJoinRelTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSideInputLookupJoinRelTest.java
@@ -41,6 +41,7 @@
 import org.junit.Test;
 import org.junit.rules.ExpectedException;
 
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BeamSideInputLookupJoinRelTest extends BaseRelTest {
 
   @Rule public final TestPipeline pipeline = TestPipeline.create();
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSortRelTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSortRelTest.java
index bba4876..d6d4659 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSortRelTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSortRelTest.java
@@ -34,6 +34,7 @@
 import org.junit.rules.ExpectedException;
 
 /** Test for {@code BeamSortRel}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BeamSortRelTest extends BaseRelTest {
   @Rule public final TestPipeline pipeline = TestPipeline.create();
 
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamAggregateProjectMergeRuleTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamAggregateProjectMergeRuleTest.java
index 2bce48c..5dd8e85 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamAggregateProjectMergeRuleTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamAggregateProjectMergeRuleTest.java
@@ -41,6 +41,7 @@
 import org.junit.runners.JUnit4;
 
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BeamAggregateProjectMergeRuleTest {
   private static final Schema BASIC_SCHEMA =
       Schema.builder()
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rule/IOPushDownRuleTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rule/IOPushDownRuleTest.java
index 5cd2676..cf4d88a 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rule/IOPushDownRuleTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rule/IOPushDownRuleTest.java
@@ -54,6 +54,7 @@
 import org.junit.runners.JUnit4;
 
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class IOPushDownRuleTest {
   private static final Schema BASIC_SCHEMA =
       Schema.builder()
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rule/JoinReorderingTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rule/JoinReorderingTest.java
index a967fc3..ec77127 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rule/JoinReorderingTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rule/JoinReorderingTest.java
@@ -75,6 +75,7 @@
  * This test ensures that we are reordering joins and get a plan similar to Join(large,Join(small,
  * medium)) instead of Join(small, Join(medium,large).
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class JoinReorderingTest {
   private final PipelineOptions defaultPipelineOptions = PipelineOptionsFactory.create();
 
@@ -364,6 +365,7 @@
   }
 }
 
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 final class ThreeTablesSchema extends AbstractSchema {
 
   private final ImmutableMap<String, Table> tables;
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/schema/transform/BeamTransformBaseTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/schema/transform/BeamTransformBaseTest.java
index e37edd8..b32bdca 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/schema/transform/BeamTransformBaseTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/schema/transform/BeamTransformBaseTest.java
@@ -27,6 +27,7 @@
 import org.junit.BeforeClass;
 
 /** shared methods to test PTransforms which execute Beam SQL steps. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BeamTransformBaseTest {
   static final DateTimeFormatter FORMAT = DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ss");
 
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/utils/CalciteUtilsTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/utils/CalciteUtilsTest.java
index 50b6ab2..b49a8fa 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/utils/CalciteUtilsTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/utils/CalciteUtilsTest.java
@@ -33,6 +33,7 @@
 import org.junit.Test;
 
 /** Tests for conversion from Beam schema to Calcite data type. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class CalciteUtilsTest {
 
   RelDataTypeFactory dataTypeFactory;
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/integrationtest/BeamSqlBuiltinFunctionsIntegrationTestBase.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/integrationtest/BeamSqlBuiltinFunctionsIntegrationTestBase.java
index 8ced3f3..40222ee 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/integrationtest/BeamSqlBuiltinFunctionsIntegrationTestBase.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/integrationtest/BeamSqlBuiltinFunctionsIntegrationTestBase.java
@@ -57,6 +57,7 @@
 import org.junit.Rule;
 
 /** Base class for all built-in functions integration tests. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BeamSqlBuiltinFunctionsIntegrationTestBase {
   private static final double PRECISION_DOUBLE = 1e-7;
   private static final float PRECISION_FLOAT = 1e-7f;
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/integrationtest/BeamSqlDateFunctionsIntegrationTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/integrationtest/BeamSqlDateFunctionsIntegrationTest.java
index 0bffb5c..ece19a7 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/integrationtest/BeamSqlDateFunctionsIntegrationTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/integrationtest/BeamSqlDateFunctionsIntegrationTest.java
@@ -35,6 +35,7 @@
 import org.junit.Test;
 
 /** Integration test for date functions. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BeamSqlDateFunctionsIntegrationTest
     extends BeamSqlBuiltinFunctionsIntegrationTestBase {
 
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/CustomTableResolverTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/CustomTableResolverTest.java
index c832251..a0f2ab4 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/CustomTableResolverTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/CustomTableResolverTest.java
@@ -39,6 +39,7 @@
 import org.junit.Test;
 
 /** Test for custom table resolver and full name table provider. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class CustomTableResolverTest implements Serializable {
 
   @Rule public final transient TestPipeline pipeline = TestPipeline.create();
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BigQueryFilterTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BigQueryFilterTest.java
index fa84883..adc2c1b 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BigQueryFilterTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BigQueryFilterTest.java
@@ -41,6 +41,7 @@
 import org.junit.runners.JUnit4;
 
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BigQueryFilterTest {
   // TODO: add date, time, and datetime fields.
   private static final Schema BASIC_SCHEMA =
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BigQueryTestTable.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BigQueryTestTable.java
index a674e40..0709c85 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BigQueryTestTable.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BigQueryTestTable.java
@@ -26,6 +26,7 @@
  * A BigQueryTable that keeps jobName from the pipeline options whenever row count is called. It is
  * made for {@link BigQueryRowCountIT#testPipelineOptionInjection()}
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BigQueryTestTable extends BigQueryTable {
   private String jobName = null;
 
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BigQueryTestTableProvider.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BigQueryTestTableProvider.java
index 92a4424..c193e0a 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BigQueryTestTableProvider.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BigQueryTestTableProvider.java
@@ -27,6 +27,7 @@
 import org.checkerframework.checker.nullness.qual.Nullable;
 
 /** A test table provider for BigQueryRowCountIT. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BigQueryTestTableProvider extends BigQueryTableProvider {
 
   private Map<String, Table> tableSpecMap;
@@ -42,9 +43,8 @@
     tableSpecMap.put(name, table);
   }
 
-  @Nullable
   @Override
-  public Table getTable(String tableName) {
+  public @Nullable Table getTable(String tableName) {
     return tableSpecMap.get(tableName);
   }
 
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/kafka/BeamKafkaTableStatisticsTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/kafka/BeamKafkaTableStatisticsTest.java
index eb291dd..d16b52d 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/kafka/BeamKafkaTableStatisticsTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/kafka/BeamKafkaTableStatisticsTest.java
@@ -26,6 +26,7 @@
 import org.junit.Assert;
 import org.junit.Test;
 
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BeamKafkaTableStatisticsTest {
   @Test
   public void testOrderedArrivalSinglePartitionRate() {
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/kafka/KafkaTableProviderCSVIT.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/kafka/KafkaTableProviderCSVIT.java
index 9dfac1b..d8c92d3 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/kafka/KafkaTableProviderCSVIT.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/kafka/KafkaTableProviderCSVIT.java
@@ -23,6 +23,7 @@
 import org.apache.commons.csv.CSVFormat;
 import org.apache.kafka.clients.producer.ProducerRecord;
 
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class KafkaTableProviderCSVIT extends KafkaTableProviderIT {
   @Override
   protected ProducerRecord<String, byte[]> generateProducerRecord(int i) {
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/kafka/KafkaTableProviderIT.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/kafka/KafkaTableProviderIT.java
index 214ab6d..9c3751e 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/kafka/KafkaTableProviderIT.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/kafka/KafkaTableProviderIT.java
@@ -65,6 +65,7 @@
 import org.testcontainers.utility.DockerImageName;
 
 /** Integration Test utility for KafkaTableProvider implementations. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public abstract class KafkaTableProviderIT {
   private static final String KAFKA_CONTAINER_VERSION = "5.5.2";
 
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/kafka/KafkaTestTable.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/kafka/KafkaTestTable.java
index 6c20b88..c42c486 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/kafka/KafkaTestTable.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/kafka/KafkaTestTable.java
@@ -48,6 +48,7 @@
 import org.apache.kafka.common.record.TimestampType;
 
 /** This is a mock BeamKafkaTable. It will use a Mock Consumer. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class KafkaTestTable extends BeamKafkaTable {
   private final int partitionsPerTopic;
   private final List<KafkaTestRecord> records;
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/mongodb/MongoDbFilterTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/mongodb/MongoDbFilterTest.java
index aafaf9b..dba814e 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/mongodb/MongoDbFilterTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/mongodb/MongoDbFilterTest.java
@@ -44,6 +44,7 @@
 import org.junit.runners.Parameterized.Parameters;
 
 @RunWith(Parameterized.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class MongoDbFilterTest {
   private static final Schema BASIC_SCHEMA =
       Schema.builder()
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 47ad96a..119aff2 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
@@ -77,6 +77,7 @@
  * independent Mongo instance.
  */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class MongoDbReadWriteIT {
   private static final Logger LOG = LoggerFactory.getLogger(MongoDbReadWriteIT.class);
   private static final Schema SOURCE_SCHEMA =
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/parquet/ParquetTableReadTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/parquet/ParquetTableReadTest.java
index efa70a1..15b4fbe 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/parquet/ParquetTableReadTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/parquet/ParquetTableReadTest.java
@@ -40,6 +40,7 @@
 
 /** Test for ParquetTable. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ParquetTableReadTest {
   private static final Logger LOG = LoggerFactory.getLogger(ParquetTableReadTest.class);
 
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/pubsub/PubsubJsonIT.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/pubsub/PubsubJsonIT.java
index 6c1ea51..031e7b0 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/pubsub/PubsubJsonIT.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/pubsub/PubsubJsonIT.java
@@ -80,6 +80,7 @@
 
 /** Integration tests for querying Pubsub JSON messages with SQL. */
 @RunWith(JUnit4.class)
+@SuppressWarnings({"nullness", "keyfor"}) // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class PubsubJsonIT implements Serializable {
   private static final Logger LOG = LoggerFactory.getLogger(PubsubJsonIT.class);
 
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/test/TestTableProviderTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/test/TestTableProviderTest.java
index c15101f..f9ef903 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/test/TestTableProviderTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/test/TestTableProviderTest.java
@@ -37,6 +37,7 @@
 import org.junit.runners.JUnit4;
 
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class TestTableProviderTest {
   private static final Schema BASIC_SCHEMA =
       Schema.builder().addInt32Field("id").addStringField("name").build();
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/test/TestTableProviderWithFilterAndProjectPushDown.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/test/TestTableProviderWithFilterAndProjectPushDown.java
index d3221e0..c543c95 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/test/TestTableProviderWithFilterAndProjectPushDown.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/test/TestTableProviderWithFilterAndProjectPushDown.java
@@ -57,6 +57,7 @@
 import org.junit.runners.JUnit4;
 
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class TestTableProviderWithFilterAndProjectPushDown {
   private static final Schema BASIC_SCHEMA =
       Schema.builder()
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/test/TestTableProviderWithFilterPushDown.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/test/TestTableProviderWithFilterPushDown.java
index 9a67950..a020e95 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/test/TestTableProviderWithFilterPushDown.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/test/TestTableProviderWithFilterPushDown.java
@@ -60,6 +60,7 @@
 import org.junit.runners.JUnit4;
 
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class TestTableProviderWithFilterPushDown {
   private static final Schema BASIC_SCHEMA =
       Schema.builder()
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/test/TestTableProviderWithProjectPushDown.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/test/TestTableProviderWithProjectPushDown.java
index 941e984..eaa0ddf 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/test/TestTableProviderWithProjectPushDown.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/test/TestTableProviderWithProjectPushDown.java
@@ -57,6 +57,7 @@
 import org.junit.runners.JUnit4;
 
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class TestTableProviderWithProjectPushDown {
   private static final Schema BASIC_SCHEMA =
       Schema.builder()
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/store/InMemoryMetaStoreTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/store/InMemoryMetaStoreTest.java
index 92234b4..1018d0e 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/store/InMemoryMetaStoreTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/store/InMemoryMetaStoreTest.java
@@ -36,6 +36,7 @@
 import org.junit.Test;
 
 /** UnitTest for {@link InMemoryMetaStore}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class InMemoryMetaStoreTest {
   private InMemoryMetaStore store;
 
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/utils/RowAsserts.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/utils/RowAsserts.java
index abbb4df..0d94b85 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/utils/RowAsserts.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/utils/RowAsserts.java
@@ -25,6 +25,7 @@
 import org.apache.beam.vendor.calcite.v1_20_0.com.google.common.collect.Iterables;
 
 /** Contain helpers to assert {@link Row}s. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class RowAsserts {
 
   /** Asserts result contains single row with an int field. */
diff --git a/sdks/java/extensions/sql/zetasql/build.gradle b/sdks/java/extensions/sql/zetasql/build.gradle
index 2cf39b2..db409f1 100644
--- a/sdks/java/extensions/sql/zetasql/build.gradle
+++ b/sdks/java/extensions/sql/zetasql/build.gradle
@@ -28,7 +28,7 @@
 description = "Apache Beam :: SDKs :: Java :: Extensions :: SQL :: ZetaSQL"
 ext.summary = "ZetaSQL to Calcite translator"
 
-def zetasql_version = "2020.08.1"
+def zetasql_version = "2020.10.1"
 
 dependencies {
   compile project(":sdks:java:core")
diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/BeamZetaSqlCalcRel.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/BeamZetaSqlCalcRel.java
index 460e600..51759ab 100644
--- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/BeamZetaSqlCalcRel.java
+++ b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/BeamZetaSqlCalcRel.java
@@ -57,6 +57,7 @@
  * expression evaluator.
  */
 @Internal
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BeamZetaSqlCalcRel extends AbstractBeamCalcRel {
 
   private static final SqlDialect DIALECT = BeamBigQuerySqlDialect.DEFAULT;
diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/QueryTrait.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/QueryTrait.java
index 428c360..5aa08ed 100644
--- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/QueryTrait.java
+++ b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/QueryTrait.java
@@ -30,6 +30,7 @@
 import org.apache.beam.sdk.extensions.sql.zetasql.TableResolution.SimpleTableWithPath;
 
 /** QueryTrait. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class QueryTrait {
   public Map<String, ResolvedWithEntry> withEntries = new HashMap<>();
 
diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/SqlAnalyzer.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/SqlAnalyzer.java
index 53cc90b..eea8588 100644
--- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/SqlAnalyzer.java
+++ b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/SqlAnalyzer.java
@@ -65,6 +65,7 @@
 import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.schema.SchemaPlus;
 
 /** Adapter for {@link Analyzer} to simplify the API for parsing the query and resolving the AST. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SqlAnalyzer {
   public static final String PRE_DEFINED_WINDOW_FUNCTIONS = "pre_defined_window_functions";
   public static final String USER_DEFINED_FUNCTIONS = "user_defined_functions";
diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/SupportedZetaSqlBuiltinFunctions.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/SupportedZetaSqlBuiltinFunctions.java
index 06191e8..d0e88ec 100644
--- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/SupportedZetaSqlBuiltinFunctions.java
+++ b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/SupportedZetaSqlBuiltinFunctions.java
@@ -35,6 +35,8 @@
           FunctionSignatureId.FN_ADD_INT64, // $add
           FunctionSignatureId.FN_ADD_NUMERIC, // $add
           // FunctionSignatureId.FN_ADD_BIGNUMERIC, // $add
+          // FunctionSignatureId.FN_ADD_DATE_INT64, // $add
+          // FunctionSignatureId.FN_ADD_INT64_DATE, // $add
           FunctionSignatureId.FN_AND, // $and
           FunctionSignatureId.FN_CASE_NO_VALUE, // $case_no_value
           FunctionSignatureId.FN_CASE_WITH_VALUE, // $case_with_value
@@ -65,6 +67,7 @@
           FunctionSignatureId.FN_SUBTRACT_INT64, // $subtract
           FunctionSignatureId.FN_SUBTRACT_NUMERIC, // $subtract
           // FunctionSignatureId.FN_SUBTRACT_BIGNUMERIC, // $subtract
+          // FunctionSignatureId.FN_SUBTRACT_DATE_INT64, // $subtract
           FunctionSignatureId.FN_UNARY_MINUS_INT64, // $unary_minus
           FunctionSignatureId.FN_UNARY_MINUS_DOUBLE, // $unary_minus
           FunctionSignatureId.FN_UNARY_MINUS_NUMERIC, // $unary_minus
@@ -144,6 +147,7 @@
           // semantically identical to FN_LENGTH_BYTES
           FunctionSignatureId.FN_CHAR_LENGTH_STRING, // char_length(string) -> int64
           // semantically identical to FN_LENGTH_STRING
+          // FunctionSignatureId.FN_FORMAT_STRING, // format(string, ...) -> string
           // FunctionSignatureId.FN_SPLIT_STRING, // split(string, string) -> array of string
           // FunctionSignatureId.FN_SPLIT_BYTES, // split(bytes, bytes) -> array of bytes
           // FunctionSignatureId.FN_REGEXP_CONTAINS_STRING,//regexp_contains(string, string) -> bool
@@ -187,6 +191,8 @@
           // FunctionSignatureId.FN_TRANSLATE_STRING, // translate(string, string, string) -> string
           // FunctionSignatureId.FN_TRANSLATE_BYTES, // soundex(bytes, bytes, bytes) -> bytes
           // FunctionSignatureId.FN_INITCAP_STRING, // initcap(string[, string]) -> string
+          // FunctionSignatureId.FN_UNICODE_STRING, // unicode(string) -> int64
+          // FunctionSignatureId.FN_CHR_STRING, // chr(int64) -> string
 
           // Control flow functions
           FunctionSignatureId.FN_IF, // if
@@ -235,18 +241,28 @@
           // FunctionSignatureId.FN_UNIX_MICROS_FROM_TIMESTAMP,
           FunctionSignatureId.FN_DATE_FROM_TIMESTAMP, // date
           FunctionSignatureId.FN_DATE_FROM_DATETIME, // date
+          // FunctionSignatureId.FN_DATE_FROM_DATE, // date
+          // FunctionSignatureId.FN_DATE_FROM_STRING, // date
           FunctionSignatureId.FN_DATE_FROM_YEAR_MONTH_DAY, // date
           FunctionSignatureId.FN_TIMESTAMP_FROM_STRING, // timestamp
           FunctionSignatureId.FN_TIMESTAMP_FROM_DATE, // timestamp
           FunctionSignatureId.FN_TIMESTAMP_FROM_DATETIME, // timestamp
+          // FunctionSignatureId.FN_TIMESTAMP_FROM_TIMESTAMP, // timestamp
           FunctionSignatureId.FN_TIME_FROM_HOUR_MINUTE_SECOND, // time
           FunctionSignatureId.FN_TIME_FROM_TIMESTAMP, // time
           FunctionSignatureId.FN_TIME_FROM_DATETIME, // time
+          // FunctionSignatureId.FN_TIME_FROM_TIME, // time
+          // FunctionSignatureId.FN_TIME_FROM_STRING, // time
           FunctionSignatureId.FN_DATETIME_FROM_DATE_AND_TIME, // datetime
           FunctionSignatureId.FN_DATETIME_FROM_YEAR_MONTH_DAY_HOUR_MINUTE_SECOND, // datetime
           FunctionSignatureId.FN_DATETIME_FROM_TIMESTAMP, // datetime
           FunctionSignatureId.FN_DATETIME_FROM_DATE, // datetime
+          // FunctionSignatureId.FN_DATETIME_FROM_DATETIME, // datetime
+          // FunctionSignatureId.FN_DATETIME_FROM_STRING, // datetime
+          // FunctionSignatureId.FN_STRING_FROM_DATE, // string
           FunctionSignatureId.FN_STRING_FROM_TIMESTAMP, // string
+          // FunctionSignatureId.FN_STRING_FROM_DATETIME, // string
+          // FunctionSignatureId.FN_STRING_FROM_TIME, // string
 
           // Signatures for extracting date parts, taking a date/timestamp
           // and the target date part as arguments.
@@ -277,29 +293,31 @@
           FunctionSignatureId.FN_PARSE_DATETIME, // parse_datetime
           FunctionSignatureId.FN_PARSE_TIME, // parse_time
           FunctionSignatureId.FN_PARSE_TIMESTAMP, // parse_timestamp
+          // FunctionSignatureId.FN_LAST_DAY_DATE, // last_day date
+          // FunctionSignatureId.FN_LAST_DAY_DATETIME, // last_day datetime
 
           // Math functions
           FunctionSignatureId.FN_ABS_INT64, // abs
-          // FunctionSignatureId.FN_ABS_DOUBLE, // abs
+          FunctionSignatureId.FN_ABS_DOUBLE, // abs
           FunctionSignatureId.FN_ABS_NUMERIC, // abs
           // FunctionSignatureId.FN_ABS_BIGNUMERIC, // abs
-          // FunctionSignatureId.FN_SIGN_INT64, // sign
-          // FunctionSignatureId.FN_SIGN_DOUBLE, // sign
-          // FunctionSignatureId.FN_SIGN_NUMERIC, // sign
+          FunctionSignatureId.FN_SIGN_INT64, // sign
+          FunctionSignatureId.FN_SIGN_DOUBLE, // sign
+          FunctionSignatureId.FN_SIGN_NUMERIC, // sign
           // FunctionSignatureId.FN_SIGN_BIGNUMERIC, // sign
 
-          // FunctionSignatureId.FN_ROUND_DOUBLE, // round(double) -> double
-          // FunctionSignatureId.FN_ROUND_NUMERIC, // round(numeric) -> numeric
+          FunctionSignatureId.FN_ROUND_DOUBLE, // round(double) -> double
+          FunctionSignatureId.FN_ROUND_NUMERIC, // round(numeric) -> numeric
           // FunctionSignatureId.FN_ROUND_BIGNUMERIC, // round(bignumeric) -> bignumeric
-          // FunctionSignatureId.FN_ROUND_WITH_DIGITS_DOUBLE, // round(double, int64) -> double
-          // FunctionSignatureId.FN_ROUND_WITH_DIGITS_NUMERIC, // round(numeric, int64) -> numeric
+          FunctionSignatureId.FN_ROUND_WITH_DIGITS_DOUBLE, // round(double, int64) -> double
+          FunctionSignatureId.FN_ROUND_WITH_DIGITS_NUMERIC, // round(numeric, int64) -> numeric
           // round(bignumeric, int64) -> bignumeric
           // FunctionSignatureId.FN_ROUND_WITH_DIGITS_BIGNUMERIC,
-          // FunctionSignatureId.FN_TRUNC_DOUBLE, // trunc(double) -> double
+          FunctionSignatureId.FN_TRUNC_DOUBLE, // trunc(double) -> double
           FunctionSignatureId.FN_TRUNC_NUMERIC, // trunc(numeric) -> numeric
           // FunctionSignatureId.FN_TRUNC_BIGNUMERIC, // trunc(bignumeric) -> bignumeric
-          // FunctionSignatureId.FN_TRUNC_WITH_DIGITS_DOUBLE, // trunc(double, int64) -> double
-          // FunctionSignatureId.FN_TRUNC_WITH_DIGITS_NUMERIC, // trunc(numeric, int64) -> numeric
+          FunctionSignatureId.FN_TRUNC_WITH_DIGITS_DOUBLE, // trunc(double, int64) -> double
+          FunctionSignatureId.FN_TRUNC_WITH_DIGITS_NUMERIC, // trunc(numeric, int64) -> numeric
           // trunc(bignumeric, int64) -> bignumeric
           // FunctionSignatureId.FN_TRUNC_WITH_DIGITS_BIGNUMERIC,
           FunctionSignatureId.FN_CEIL_DOUBLE, // ceil(double) -> double
@@ -312,52 +330,63 @@
           FunctionSignatureId.FN_MOD_INT64, // mod(int64, int64) -> int64
           FunctionSignatureId.FN_MOD_NUMERIC, // mod(numeric, numeric) -> numeric
           // FunctionSignatureId.FN_MOD_BIGNUMERIC, // mod(bignumeric, bignumeric) -> bignumeric
-          // FunctionSignatureId.FN_DIV_INT64, // div(int64, int64) -> int64
-          // FunctionSignatureId.FN_DIV_NUMERIC, // div(numeric, numeric) -> numeric
+          FunctionSignatureId.FN_DIV_INT64, // div(int64, int64) -> int64
+          FunctionSignatureId.FN_DIV_NUMERIC, // div(numeric, numeric) -> numeric
           // FunctionSignatureId.FN_DIV_BIGNUMERIC, // div(bignumeric, bignumeric) -> bignumeric
 
           FunctionSignatureId.FN_IS_INF, // is_inf
           FunctionSignatureId.FN_IS_NAN, // is_nan
-          // FunctionSignatureId.FN_IEEE_DIVIDE_DOUBLE, // ieee_divide
+          FunctionSignatureId.FN_IEEE_DIVIDE_DOUBLE, // ieee_divide
           FunctionSignatureId.FN_SAFE_DIVIDE_DOUBLE, // safe_divide
           FunctionSignatureId.FN_SAFE_DIVIDE_NUMERIC, // safe_divide
           // FunctionSignatureId.FN_SAFE_DIVIDE_BIGNUMERIC, // safe_divide
-          // FunctionSignatureId.FN_SAFE_ADD_INT64, // safe_add
-          // FunctionSignatureId.FN_SAFE_ADD_DOUBLE, // safe_add
-          // FunctionSignatureId.FN_SAFE_ADD_NUMERIC, // safe_add
+          FunctionSignatureId.FN_SAFE_ADD_INT64, // safe_add
+          FunctionSignatureId.FN_SAFE_ADD_DOUBLE, // safe_add
+          FunctionSignatureId.FN_SAFE_ADD_NUMERIC, // safe_add
           // FunctionSignatureId.FN_SAFE_ADD_BIGNUMERIC, // safe_add
-          // FunctionSignatureId.FN_SAFE_SUBTRACT_INT64, // safe_subtract
-          // FunctionSignatureId.FN_SAFE_SUBTRACT_DOUBLE, // safe_subtract
-          // FunctionSignatureId.FN_SAFE_SUBTRACT_NUMERIC, // safe_subtract
+          FunctionSignatureId.FN_SAFE_SUBTRACT_INT64, // safe_subtract
+          FunctionSignatureId.FN_SAFE_SUBTRACT_DOUBLE, // safe_subtract
+          FunctionSignatureId.FN_SAFE_SUBTRACT_NUMERIC, // safe_subtract
           // FunctionSignatureId.FN_SAFE_SUBTRACT_BIGNUMERIC, // safe_subtract
-          // FunctionSignatureId.FN_SAFE_MULTIPLY_INT64, // safe_multiply
-          // FunctionSignatureId.FN_SAFE_MULTIPLY_DOUBLE, // safe_multiply
-          // FunctionSignatureId.FN_SAFE_MULTIPLY_NUMERIC, // safe_multiply
+          FunctionSignatureId.FN_SAFE_MULTIPLY_INT64, // safe_multiply
+          FunctionSignatureId.FN_SAFE_MULTIPLY_DOUBLE, // safe_multiply
+          FunctionSignatureId.FN_SAFE_MULTIPLY_NUMERIC, // safe_multiply
           // FunctionSignatureId.FN_SAFE_MULTIPLY_BIGNUMERIC, // safe_multiply
-          // FunctionSignatureId.FN_SAFE_UNARY_MINUS_INT64, // safe_negate
-          // FunctionSignatureId.FN_SAFE_UNARY_MINUS_DOUBLE, // safe_negate
-          // FunctionSignatureId.FN_SAFE_UNARY_MINUS_NUMERIC, // safe_negate
+          FunctionSignatureId.FN_SAFE_UNARY_MINUS_INT64, // safe_negate
+          FunctionSignatureId.FN_SAFE_UNARY_MINUS_DOUBLE, // safe_negate
+          FunctionSignatureId.FN_SAFE_UNARY_MINUS_NUMERIC, // safe_negate
           // FunctionSignatureId.FN_SAFE_UNARY_MINUS_BIGNUMERIC, // safe_negate
 
           // FunctionSignatureId.FN_GREATEST, // greatest
           // FunctionSignatureId.FN_LEAST, // least
 
-          // FunctionSignatureId.FN_SQRT_DOUBLE, // sqrt
+          FunctionSignatureId.FN_SQRT_DOUBLE, // sqrt
+          FunctionSignatureId.FN_SQRT_NUMERIC, // sqrt(numeric) -> numeric
+          // FunctionSignatureId.FN_SQRT_BIGNUMERIC, // sqrt(bignumeric) -> bignumeric
           FunctionSignatureId.FN_POW_DOUBLE, // pow
           FunctionSignatureId.FN_POW_NUMERIC, // pow(numeric, numeric) -> numeric
           // FunctionSignatureId.FN_POW_BIGNUMERIC, // pow(bignumeric, bignumeric) -> bignumeric
           FunctionSignatureId.FN_EXP_DOUBLE, // exp
-          // FunctionSignatureId.FN_NATURAL_LOGARITHM_DOUBLE, // ln and log
-          // FunctionSignatureId.FN_DECIMAL_LOGARITHM_DOUBLE, // log10
+          FunctionSignatureId.FN_EXP_NUMERIC, // exp(numeric) -> numeric
+          // FunctionSignatureId.FN_EXP_BIGNUMERIC, // exp(bignumeric) -> bignumeric
+          FunctionSignatureId.FN_NATURAL_LOGARITHM_DOUBLE, // ln
+          FunctionSignatureId.FN_NATURAL_LOGARITHM_NUMERIC, // ln(numeric) -> numeric
+          // FunctionSignatureId.FN_NATURAL_LOGARITHM_BIGNUMERIC, // ln(bignumeric) -> bignumeric
+          FunctionSignatureId.FN_DECIMAL_LOGARITHM_DOUBLE, // log10
+          FunctionSignatureId.FN_DECIMAL_LOGARITHM_NUMERIC, // log10(numeric) -> numeric
+          // FunctionSignatureId.FN_DECIMAL_LOGARITHM_BIGNUMERIC, // log10(bignumeric) -> bignumeric
           FunctionSignatureId.FN_LOGARITHM_DOUBLE, // log
+          FunctionSignatureId.FN_LOGARITHM_NUMERIC, // log(numeric, numeric) -> numeric
+          // FunctionSignatureId.FN_LOGARITHM_BIGNUMERIC,//log(bignumeric, bignumeric) -> bignumeric
+
           FunctionSignatureId.FN_COS_DOUBLE, // cos
           FunctionSignatureId.FN_COSH_DOUBLE, // cosh
           FunctionSignatureId.FN_ACOS_DOUBLE, // acos
-          // FunctionSignatureId.FN_ACOSH_DOUBLE, // acosh
+          FunctionSignatureId.FN_ACOSH_DOUBLE, // acosh
           FunctionSignatureId.FN_SIN_DOUBLE, // sin
-          // FunctionSignatureId.FN_SINH_DOUBLE, // sinh
+          FunctionSignatureId.FN_SINH_DOUBLE, // sinh
           FunctionSignatureId.FN_ASIN_DOUBLE, // asin
-          // FunctionSignatureId.FN_ASINH_DOUBLE, // asinh
+          FunctionSignatureId.FN_ASINH_DOUBLE, // asinh
           FunctionSignatureId.FN_TAN_DOUBLE, // tan
           FunctionSignatureId.FN_TANH_DOUBLE, // tanh
           FunctionSignatureId.FN_ATAN_DOUBLE, // atan
@@ -385,7 +414,7 @@
           // FunctionSignatureId.FN_SUM_BIGNUMERIC, // sum
           // JIRA link: https://issues.apache.org/jira/browse/BEAM-10379
           // FunctionSignatureId.FN_BIT_AND_INT64, // bit_and
-          FunctionSignatureId.FN_BIT_OR_INT64 // bit_or
+          FunctionSignatureId.FN_BIT_OR_INT64, // bit_or
           // FunctionSignatureId.FN_BIT_XOR_INT64, // bit_xor
           // FunctionSignatureId.FN_LOGICAL_AND, // logical_and
           // FunctionSignatureId.FN_LOGICAL_OR, // logical_or
@@ -412,18 +441,25 @@
           // Statistical aggregate functions.
           // FunctionSignatureId.FN_CORR, // corr
           // FunctionSignatureId.FN_CORR_NUMERIC, // corr
+          // FunctionSignatureId.FN_CORR_BIGNUMERIC, // corr
           // FunctionSignatureId.FN_COVAR_POP, // covar_pop
           // FunctionSignatureId.FN_COVAR_POP_NUMERIC, // covar_pop
+          // FunctionSignatureId.FN_COVAR_POP_BIGNUMERIC, // covar_pop
           // FunctionSignatureId.FN_COVAR_SAMP, // covar_samp
           // FunctionSignatureId.FN_COVAR_SAMP_NUMERIC, // covar_samp
+          // FunctionSignatureId.FN_COVAR_SAMP_BIGNUMERIC, // covar_samp
           // FunctionSignatureId.FN_STDDEV_POP, // stddev_pop
           // FunctionSignatureId.FN_STDDEV_POP_NUMERIC, // stddev_pop
+          // FunctionSignatureId.FN_STDDEV_POP_BIGNUMERIC, // stddev_pop
           // FunctionSignatureId.FN_STDDEV_SAMP, // stddev_samp
           // FunctionSignatureId.FN_STDDEV_SAMP_NUMERIC, // stddev_samp
+          // FunctionSignatureId.FN_STDDEV_SAMP_BIGNUMERIC, // stddev_samp
           // FunctionSignatureId.FN_VAR_POP, // var_pop
           // FunctionSignatureId.FN_VAR_POP_NUMERIC, // var_pop
+          // FunctionSignatureId.FN_VAR_POP_BIGNUMERIC, // var_pop
           // FunctionSignatureId.FN_VAR_SAMP, // var_samp
           // FunctionSignatureId.FN_VAR_SAMP_NUMERIC, // var_samp
+          // FunctionSignatureId.FN_VAR_SAMP_BIGNUMERIC, // var_samp
 
           // FunctionSignatureId.FN_COUNTIF, // countif
 
@@ -455,8 +491,10 @@
           // FunctionSignatureId.FN_NTH_VALUE, // nth_value
           // FunctionSignatureId.FN_PERCENTILE_CONT, // percentile_cont
           // FunctionSignatureId.FN_PERCENTILE_CONT_NUMERIC, // percentile_cont
+          // FunctionSignatureId.FN_PERCENTILE_CONT_BIGNUMERIC, // percentile_cont
           // FunctionSignatureId.FN_PERCENTILE_DISC, // percentile_disc
           // FunctionSignatureId.FN_PERCENTILE_DISC_NUMERIC, // percentile_disc
+          // FunctionSignatureId.FN_PERCENTILE_DISC_BIGNUMERIC, // percentile_disc
 
           // Misc functions.
           // FunctionSignatureId.FN_BIT_CAST_INT64_TO_INT64, // bit_cast_to_int64(int64)
@@ -474,17 +512,21 @@
 
           // FunctionSignatureId.FN_RANGE_BUCKET, //  range_bucket(T, array<T>) -> int64
 
-          // FunctionSignatureId.FN_RAND, // rand() -> double
+          // FunctionSignatureId.FN_RAND // rand() -> double
           // FunctionSignatureId.FN_GENERATE_UUID, // generate_uuid() -> string
 
-          // FunctionSignatureId.FN_JSON_EXTRACT, // json_extract(string, string)
-          // FunctionSignatureId.FN_JSON_EXTRACT_SCALAR, // json_extract_scalar(string, string)
+          FunctionSignatureId.FN_JSON_EXTRACT, // json_extract(string, string)
+          // FunctionSignatureId.FN_JSON_EXTRACT_JSON, // json_extract(json, string) -> json
+          FunctionSignatureId.FN_JSON_EXTRACT_SCALAR, // json_extract_scalar(string, string)
+          // json_extract_scalar(json, string) -> string
+          // FunctionSignatureId.FN_JSON_EXTRACT_SCALAR_JSON,
           // json_extract_array(string[, string]) -> array
-          // FunctionSignatureId.FN_JSON_EXTRACT_ARRAY,
-
-          // FunctionSignatureId.FN_TO_JSON_STRING, // to_json_string(any[, bool]) -> string
-          // FunctionSignatureId.FN_JSON_QUERY, // json_query(string, string)
-          // FunctionSignatureId.FN_JSON_VALUE, // json_value(string, string)
+          FunctionSignatureId.FN_JSON_EXTRACT_ARRAY,
+          FunctionSignatureId.FN_TO_JSON_STRING, // to_json_string(any[, bool]) -> string
+          FunctionSignatureId.FN_JSON_QUERY, // json_query(string, string) -> string
+          // FunctionSignatureId.FN_JSON_QUERY_JSON, // json_query(json, string) -> json
+          FunctionSignatureId.FN_JSON_VALUE // json_value(string, string) -> string
+          // FunctionSignatureId.FN_JSON_VALUE_JSON, // json_value(json, string) -> json
 
           // Net functions. These are all found in the "net.*" namespace.
           // FunctionSignatureId.FN_NET_FORMAT_IP,
@@ -595,9 +637,11 @@
           // FunctionSignatureId.FN_ST_UNION_AGG,
           // FunctionSignatureId.FN_ST_ACCUM,
           // FunctionSignatureId.FN_ST_CENTROID_AGG,
+          // FunctionSignatureId.FN_ST_NEAREST_NEIGHBORS,
           // Other geography functions
           // FunctionSignatureId.FN_ST_X,
           // FunctionSignatureId.FN_ST_Y,
+          // FunctionSignatureId.FN_ST_CLUSTERDBSCAN,
 
           // Array functions.
           // FunctionSignatureId.FN_FLATTEN, // flatten(array path) -> array
@@ -613,5 +657,8 @@
           // FunctionSignatureId.FN_MAKE_ARRAY, // $make_array
           // FunctionSignatureId.FN_SAFE_ARRAY_AT_OFFSET, // $safe_array_at_offset
           // FunctionSignatureId.FN_SAFE_ARRAY_AT_ORDINAL, // $safe_array_at_ordinal
+          // FunctionSignatureId.FN_ARRAY_IS_DISTINCT, // array_is_distinct(array) -> bool
+          // FunctionSignatureId.FN_PROTO_MAP_AT_KEY, // $proto_map_at_key
+          // FunctionSignatureId.FN_SAFE_PROTO_MAP_AT_KEY, // $safe_proto_map_at_key
           );
 }
diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/TableResolution.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/TableResolution.java
index d87eaf9..7ca8445 100644
--- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/TableResolution.java
+++ b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/TableResolution.java
@@ -30,6 +30,7 @@
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables;
 
 /** Utility methods to resolve a table, given a top-level Calcite schema and a table path. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class TableResolution {
 
   /**
diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSQLPlannerImpl.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSQLPlannerImpl.java
index dbfc603..05938dc 100644
--- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSQLPlannerImpl.java
+++ b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSQLPlannerImpl.java
@@ -53,6 +53,7 @@
 import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.util.Util;
 
 /** ZetaSQLPlannerImpl. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class ZetaSQLPlannerImpl {
   private final SchemaPlus defaultSchemaPlus;
 
diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSQLQueryPlanner.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSQLQueryPlanner.java
index 334af11..7b2f537 100644
--- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSQLQueryPlanner.java
+++ b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSQLQueryPlanner.java
@@ -62,6 +62,7 @@
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
 
 /** ZetaSQLQueryPlanner. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ZetaSQLQueryPlanner implements QueryPlanner {
   private final ZetaSQLPlannerImpl plannerImpl;
 
diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlBeamTranslationUtils.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlBeamTranslationUtils.java
index d38d597..dd232b0 100644
--- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlBeamTranslationUtils.java
+++ b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlBeamTranslationUtils.java
@@ -48,6 +48,7 @@
  * <p>Unsupported ZetaSQL types: INT32, UINT32, UINT64, FLOAT, ENUM, PROTO, GEOGRAPHY
  */
 @Internal
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public final class ZetaSqlBeamTranslationUtils {
 
   private static final long MICROS_PER_MILLI = 1000L;
diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlCalciteTranslationUtils.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlCalciteTranslationUtils.java
index 6dc3c1e..e40f69b 100644
--- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlCalciteTranslationUtils.java
+++ b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlCalciteTranslationUtils.java
@@ -54,6 +54,7 @@
  * <p>Unsupported ZetaSQL types: INT32, UINT32, UINT64, FLOAT, ENUM (internal), PROTO, GEOGRAPHY
  */
 @Internal
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public final class ZetaSqlCalciteTranslationUtils {
 
   private ZetaSqlCalciteTranslationUtils() {}
@@ -207,24 +208,23 @@
         return rexBuilder.makeExactLiteral(
             new BigDecimal(value.getInt64Value()), toCalciteType(type, false, rexBuilder));
       case TYPE_DOUBLE:
-        // Cannot simply call makeApproxLiteral() for ZetaSQL DOUBLE type because positive infinity,
-        // negative infinity and NaN cannot be directly converted to BigDecimal. So we create three
-        // wrapper functions here for these three cases such that we can later recognize it and
-        // customize its unparsing in BeamBigQuerySqlDialect.
+        // Cannot simply call makeApproxLiteral() because +inf, -inf, and NaN cannot be represented
+        // as BigDecimal. So we create wrapper functions here for these three cases such that we can
+        // later recognize it and customize its unparsing in BeamBigQuerySqlDialect.
         double val = value.getDoubleValue();
         String wrapperFun = null;
         if (val == Double.POSITIVE_INFINITY) {
-          wrapperFun = BeamBigQuerySqlDialect.DOUBLE_POSITIVE_INF_FUNCTION;
+          wrapperFun = BeamBigQuerySqlDialect.DOUBLE_POSITIVE_INF_WRAPPER;
         } else if (val == Double.NEGATIVE_INFINITY) {
-          wrapperFun = BeamBigQuerySqlDialect.DOUBLE_NEGATIVE_INF_FUNCTION;
+          wrapperFun = BeamBigQuerySqlDialect.DOUBLE_NEGATIVE_INF_WRAPPER;
         } else if (Double.isNaN(val)) {
-          wrapperFun = BeamBigQuerySqlDialect.DOUBLE_NAN_FUNCTION;
+          wrapperFun = BeamBigQuerySqlDialect.DOUBLE_NAN_WRAPPER;
         }
 
         RelDataType returnType = toCalciteType(type, false, rexBuilder);
         if (wrapperFun == null) {
           return rexBuilder.makeApproxLiteral(new BigDecimal(val), returnType);
-        } else if (BeamBigQuerySqlDialect.DOUBLE_NAN_FUNCTION.equals(wrapperFun)) {
+        } else if (BeamBigQuerySqlDialect.DOUBLE_NAN_WRAPPER.equals(wrapperFun)) {
           // TODO[BEAM-10550]: Update the temporary workaround below after vendored Calcite version.
           // Adding an additional random parameter for the wrapper function of NaN, to avoid
           // triggering Calcite operation simplification. (e.g. 'NaN == NaN' would be simplify to
@@ -247,14 +247,14 @@
       case TYPE_BYTES:
         return rexBuilder.makeBinaryLiteral(new ByteString(value.getBytesValue().toByteArray()));
       case TYPE_NUMERIC:
-        // Cannot simply call makeExactLiteral() for ZetaSQL NUMERIC type because later it will be
-        // unparsed to the string representation of the BigDecimal itself (e.g. "SELECT NUMERIC '0'"
-        // will be unparsed to "SELECT 0E-9"), and Calcite does not allow customize unparsing of
-        // SqlNumericLiteral. So we create a wrapper function here such that we can later recognize
-        // it and customize its unparsing in BeamBigQuerySqlDialect.
+        // Cannot simply call makeExactLiteral() because later it will be unparsed to the string
+        // representation of the BigDecimal itself (e.g. "SELECT NUMERIC '0'" will be unparsed to
+        // "SELECT 0E-9"), and Calcite does not allow customize unparsing of SqlNumericLiteral.
+        // So we create a wrapper function here such that we can later recognize it and customize
+        // its unparsing in BeamBigQuerySqlDialect.
         return rexBuilder.makeCall(
             SqlOperators.createZetaSqlFunction(
-                BeamBigQuerySqlDialect.NUMERIC_LITERAL_FUNCTION,
+                BeamBigQuerySqlDialect.NUMERIC_LITERAL_WRAPPER,
                 toCalciteType(type, false, rexBuilder).getSqlTypeName()),
             rexBuilder.makeExactLiteral(
                 value.getNumericValue(), toCalciteType(type, false, rexBuilder)));
diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/AggregateScanConverter.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/AggregateScanConverter.java
index e1f6faf..5faa76d 100644
--- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/AggregateScanConverter.java
+++ b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/AggregateScanConverter.java
@@ -47,6 +47,7 @@
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
 
 /** Converts aggregate calls. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class AggregateScanConverter extends RelConverter<ResolvedAggregateScan> {
   private static final String AVG_ILLEGAL_LONG_INPUT_TYPE =
       "AVG(INT64) is not supported. You might want to use AVG(CAST(expression AS FLOAT64).";
diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/ArrayScanColumnRefToUncollect.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/ArrayScanColumnRefToUncollect.java
index c2bf531..42bf9e2 100644
--- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/ArrayScanColumnRefToUncollect.java
+++ b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/ArrayScanColumnRefToUncollect.java
@@ -38,6 +38,7 @@
  * Converts array scan that represents a reference to an array column, or an (possibly nested) array
  * field of an struct column to uncollect.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class ArrayScanColumnRefToUncollect extends RelConverter<ResolvedNodes.ResolvedArrayScan> {
   ArrayScanColumnRefToUncollect(ConversionContext context) {
     super(context);
diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/ExpressionConverter.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/ExpressionConverter.java
index bf8d531..eb7aff0 100644
--- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/ExpressionConverter.java
+++ b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/ExpressionConverter.java
@@ -92,6 +92,7 @@
  * to RexNodes.
  */
 @Internal
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ExpressionConverter {
 
   // Constants of pre-defined functions.
@@ -432,7 +433,7 @@
   public RexCall convertTableValuedFunction(
       RelNode input,
       TableValuedFunction tvf,
-      List<ResolvedNodes.ResolvedTVFArgument> argumentList,
+      List<ResolvedNodes.ResolvedFunctionArgument> argumentList,
       List<ResolvedColumn> inputTableColumns) {
     ResolvedColumn wmCol;
     // Handle builtin windowing TVF.
diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/LimitOffsetScanToLimitConverter.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/LimitOffsetScanToLimitConverter.java
index b58c40a..9b2ed87 100644
--- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/LimitOffsetScanToLimitConverter.java
+++ b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/LimitOffsetScanToLimitConverter.java
@@ -33,6 +33,7 @@
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
 
 /** Converts LIMIT without ORDER BY. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class LimitOffsetScanToLimitConverter extends RelConverter<ResolvedLimitOffsetScan> {
 
   LimitOffsetScanToLimitConverter(ConversionContext context) {
diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/LimitOffsetScanToOrderByLimitConverter.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/LimitOffsetScanToOrderByLimitConverter.java
index 28ee954..b479123 100644
--- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/LimitOffsetScanToOrderByLimitConverter.java
+++ b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/LimitOffsetScanToOrderByLimitConverter.java
@@ -39,6 +39,7 @@
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
 
 /** Converts ORDER BY LIMIT. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class LimitOffsetScanToOrderByLimitConverter extends RelConverter<ResolvedLimitOffsetScan> {
 
   LimitOffsetScanToOrderByLimitConverter(ConversionContext context) {
diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/SqlNullIfOperatorRewriter.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/SqlNullIfOperatorRewriter.java
index 4745f52..09a728c 100644
--- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/SqlNullIfOperatorRewriter.java
+++ b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/SqlNullIfOperatorRewriter.java
@@ -33,6 +33,7 @@
  *   ELSE expression
  *   END</pre></code>
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class SqlNullIfOperatorRewriter implements SqlOperatorRewriter {
   @Override
   public RexNode apply(RexBuilder rexBuilder, List<RexNode> operands) {
diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/SqlOperators.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/SqlOperators.java
index 939bd03..568145d 100644
--- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/SqlOperators.java
+++ b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/SqlOperators.java
@@ -62,6 +62,7 @@
  * Calcite. Most of functions within this class is copied from Calcite.
  */
 @Internal
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SqlOperators {
   public static final SqlOperator ZETASQL_TIMESTAMP_ADD =
       createZetaSqlFunction("timestamp_add", SqlTypeName.TIMESTAMP);
diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/SqlWindowTableFunction.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/SqlWindowTableFunction.java
index 1b8817b..7b62dda 100644
--- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/SqlWindowTableFunction.java
+++ b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/SqlWindowTableFunction.java
@@ -36,6 +36,7 @@
 import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.validate.SqlValidator;
 
 /** Base class for table-valued function windowing operator (TUMBLE, HOP and SESSION). */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class SqlWindowTableFunction extends SqlFunction {
   public SqlWindowTableFunction(String name) {
     super(
diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/TVFScanConverter.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/TVFScanConverter.java
index 4cf058d..0c30c99 100644
--- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/TVFScanConverter.java
+++ b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/TVFScanConverter.java
@@ -22,8 +22,8 @@
 import com.google.zetasql.TableValuedFunction.FixedOutputSchemaTVF;
 import com.google.zetasql.ZetaSQLResolvedNodeKind.ResolvedNodeKind;
 import com.google.zetasql.resolvedast.ResolvedNode;
+import com.google.zetasql.resolvedast.ResolvedNodes.ResolvedFunctionArgument;
 import com.google.zetasql.resolvedast.ResolvedNodes.ResolvedLiteral;
-import com.google.zetasql.resolvedast.ResolvedNodes.ResolvedTVFArgument;
 import com.google.zetasql.resolvedast.ResolvedNodes.ResolvedTVFScan;
 import java.util.ArrayList;
 import java.util.Collections;
@@ -33,6 +33,7 @@
 import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexCall;
 
 /** Converts TVFScan. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class TVFScanConverter extends RelConverter<ResolvedTVFScan> {
 
   TVFScanConverter(ConversionContext context) {
@@ -71,7 +72,7 @@
       inputs.add(context.getUserDefinedTableValuedFunctions().get(zetaNode.getTvf().getNamePath()));
     }
 
-    for (ResolvedTVFArgument argument : zetaNode.getArgumentList()) {
+    for (ResolvedFunctionArgument argument : zetaNode.getArgumentList()) {
       if (argument.getScan() != null) {
         inputs.add(argument.getScan());
       }
@@ -85,8 +86,9 @@
       for (int i = 0; i < tableValuedFunctionProto.getSignature().getArgumentList().size(); i++) {
         String argumentName =
             tableValuedFunctionProto.getSignature().getArgument(i).getOptions().getArgumentName();
-        if (zetaNode.getArgumentList().get(i).nodeKind() == ResolvedNodeKind.RESOLVED_TVFARGUMENT) {
-          ResolvedTVFArgument resolvedTVFArgument = zetaNode.getArgumentList().get(i);
+        if (zetaNode.getArgumentList().get(i).nodeKind()
+            == ResolvedNodeKind.RESOLVED_FUNCTION_ARGUMENT) {
+          ResolvedFunctionArgument resolvedTVFArgument = zetaNode.getArgumentList().get(i);
           if (resolvedTVFArgument.getExpr().nodeKind() == ResolvedNodeKind.RESOLVED_LITERAL) {
             ResolvedLiteral literal = (ResolvedLiteral) resolvedTVFArgument.getExpr();
             context.addToFunctionArgumentRefMapping(
diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/WithRefScanConverter.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/WithRefScanConverter.java
index 6dceeef..8489824 100644
--- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/WithRefScanConverter.java
+++ b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/WithRefScanConverter.java
@@ -24,6 +24,7 @@
 import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
 
 /** Converts a call-site reference to a named WITH subquery. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class WithRefScanConverter extends RelConverter<ResolvedWithRefScan> {
 
   WithRefScanConverter(ConversionContext context) {
diff --git a/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/TableResolutionTest.java b/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/TableResolutionTest.java
index ca20bda..b3a316d 100644
--- a/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/TableResolutionTest.java
+++ b/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/TableResolutionTest.java
@@ -34,6 +34,7 @@
 
 /** Unit tests for {@link TableResolution}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class TableResolutionTest {
 
   // A simple in-memory SchemaPlus would be fine
diff --git a/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/TestInput.java b/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/TestInput.java
index 0c871cc..aa782f4 100644
--- a/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/TestInput.java
+++ b/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/TestInput.java
@@ -33,6 +33,7 @@
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
 
 /** TestInput. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class TestInput {
 
   public static final TestBoundedTable BASIC_TABLE_ONE =
diff --git a/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSQLPushDownTest.java b/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSQLPushDownTest.java
index 80ea234..c4caa56 100644
--- a/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSQLPushDownTest.java
+++ b/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSQLPushDownTest.java
@@ -53,6 +53,7 @@
 import org.junit.runners.JUnit4;
 
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ZetaSQLPushDownTest {
   private static final Long PIPELINE_EXECUTION_WAITTIME_MINUTES = 2L;
   private static final Schema BASIC_SCHEMA =
diff --git a/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlDialectSpecTest.java b/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlDialectSpecTest.java
index 913a8b3..083b236 100644
--- a/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlDialectSpecTest.java
+++ b/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlDialectSpecTest.java
@@ -63,6 +63,7 @@
 
 /** Tests for various operations/functions defined by ZetaSQL dialect. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ZetaSqlDialectSpecTest extends ZetaSqlTestBase {
 
   @Rule public transient TestPipeline pipeline = TestPipeline.create();
@@ -202,21 +203,6 @@
   }
 
   @Test
-  public void testFloat() {
-    String sql = "SELECT 3.0";
-
-    ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
-    BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql);
-    PCollection<Row> stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode);
-
-    final Schema schema = Schema.builder().addNullableField("ColA", FieldType.DOUBLE).build();
-
-    PAssert.that(stream).containsInAnyOrder(Row.withSchema(schema).addValues(3.0).build());
-
-    pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
-  }
-
-  @Test
   public void testStringLiterals() {
     String sql = "SELECT '\"America/Los_Angeles\"\\n'";
 
@@ -801,20 +787,6 @@
   }
 
   @Test
-  public void testMod() {
-    String sql = "SELECT MOD(4, 2)";
-
-    ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
-    BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql);
-    PCollection<Row> stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode);
-
-    final Schema schema = Schema.builder().addInt64Field("field1").build();
-
-    PAssert.that(stream).containsInAnyOrder(Row.withSchema(schema).addValues(0L).build());
-    pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
-  }
-
-  @Test
   public void testSimpleUnionAll() {
     String sql =
         "SELECT CAST (1243 as INT64), "
@@ -2164,376 +2136,6 @@
     pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
   }
 
-  /////////////////////////////////////////////////////////////////////////////
-  // DOUBLE INF and NAN tests
-  /////////////////////////////////////////////////////////////////////////////
-
-  @Test
-  public void testDoubleINF() {
-    String sql = "SELECT CAST('+inf' AS FLOAT64), CAST('-inf' AS FLOAT64)";
-
-    ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
-    BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql);
-    PCollection<Row> stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode);
-
-    PAssert.that(stream)
-        .containsInAnyOrder(
-            Row.withSchema(
-                    Schema.builder()
-                        .addDoubleField("f_double1")
-                        .addDoubleField("f_double2")
-                        .build())
-                .addValues(Double.POSITIVE_INFINITY)
-                .addValues(Double.NEGATIVE_INFINITY)
-                .build());
-    pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
-  }
-
-  @Test
-  public void testDoubleINFEQ() {
-    String sql =
-        "SELECT CAST('+inf' AS FLOAT64) = CAST('+inf' AS FLOAT64), CAST('+inf' AS FLOAT64) = CAST('-inf' AS FLOAT64)";
-
-    ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
-    BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql);
-    PCollection<Row> stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode);
-
-    PAssert.that(stream)
-        .containsInAnyOrder(
-            Row.withSchema(
-                    Schema.builder()
-                        .addBooleanField("f_boolean1")
-                        .addBooleanField("f_boolean2")
-                        .build())
-                .addValues(true)
-                .addValues(false)
-                .build());
-    pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
-  }
-
-  @Test
-  public void testDoubleNAN() {
-    String sql = "SELECT CAST('NaN' AS FLOAT64)";
-
-    ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
-    BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql);
-    PCollection<Row> stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode);
-
-    PAssert.that(stream)
-        .containsInAnyOrder(
-            Row.withSchema(Schema.builder().addDoubleField("f_double").build())
-                .addValues(Double.NaN)
-                .build());
-    pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
-  }
-
-  @Test
-  public void testDoubleNaNEQ() {
-    String sql = "SELECT CAST('NaN' AS FLOAT64) = CAST('NaN' AS FLOAT64)";
-
-    ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
-    BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql);
-    PCollection<Row> stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode);
-
-    PAssert.that(stream)
-        .containsInAnyOrder(
-            Row.withSchema(Schema.builder().addBooleanField("f_boolean").build())
-                .addValues(false)
-                .build());
-    pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
-  }
-
-  @Test
-  public void testDoubleIsINF() {
-    String sql =
-        "SELECT IS_INF(CAST('+inf' AS FLOAT64)), IS_INF(CAST('-inf' AS FLOAT64)), IS_INF(3.0)";
-
-    ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
-    BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql);
-    PCollection<Row> stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode);
-
-    PAssert.that(stream)
-        .containsInAnyOrder(
-            Row.withSchema(
-                    Schema.builder()
-                        .addBooleanField("f_boolean1")
-                        .addBooleanField("f_boolean2")
-                        .addBooleanField("f_boolean3")
-                        .build())
-                .addValues(true)
-                .addValues(true)
-                .addValues(false)
-                .build());
-    pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
-  }
-
-  @Test
-  public void testDoubleIsNAN() {
-    String sql = "SELECT IS_NAN(CAST('NaN' AS FLOAT64)), IS_NAN(3.0)";
-
-    ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
-    BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql);
-    PCollection<Row> stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode);
-
-    PAssert.that(stream)
-        .containsInAnyOrder(
-            Row.withSchema(
-                    Schema.builder()
-                        .addBooleanField("f_boolean1")
-                        .addBooleanField("f_boolean2")
-                        .build())
-                .addValues(true)
-                .addValues(false)
-                .build());
-    pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
-  }
-
-  /////////////////////////////////////////////////////////////////////////////
-  // NUMERIC type tests
-  /////////////////////////////////////////////////////////////////////////////
-
-  @Test
-  public void testNumericLiteral() {
-    String sql =
-        "SELECT NUMERIC '0', "
-            + "NUMERIC '123456', "
-            + "NUMERIC '-3.14', "
-            + "NUMERIC '-0.54321', "
-            + "NUMERIC '1.23456e05', "
-            + "NUMERIC '-9.876e-3', "
-            // min value for ZetaSQL NUMERIC type
-            + "NUMERIC '-99999999999999999999999999999.999999999', "
-            // max value for ZetaSQL NUMERIC type
-            + "NUMERIC '99999999999999999999999999999.999999999'";
-    ;
-
-    ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
-    BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql);
-    PCollection<Row> stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode);
-
-    PAssert.that(stream)
-        .containsInAnyOrder(
-            Row.withSchema(
-                    Schema.builder()
-                        .addDecimalField("f_numeric1")
-                        .addDecimalField("f_numeric2")
-                        .addDecimalField("f_numeric3")
-                        .addDecimalField("f_numeric4")
-                        .addDecimalField("f_numeric5")
-                        .addDecimalField("f_numeric6")
-                        .addDecimalField("f_numeric7")
-                        .addDecimalField("f_numeric8")
-                        .build())
-                .addValues(ZetaSqlTypesUtils.bigDecimalAsNumeric("0"))
-                .addValues(ZetaSqlTypesUtils.bigDecimalAsNumeric("123456"))
-                .addValues(ZetaSqlTypesUtils.bigDecimalAsNumeric("-3.14"))
-                .addValues(ZetaSqlTypesUtils.bigDecimalAsNumeric("-0.54321"))
-                .addValues(ZetaSqlTypesUtils.bigDecimalAsNumeric("123456"))
-                .addValues(ZetaSqlTypesUtils.bigDecimalAsNumeric("-0.009876"))
-                .addValues(
-                    ZetaSqlTypesUtils.bigDecimalAsNumeric(
-                        "-99999999999999999999999999999.999999999"))
-                .addValues(
-                    ZetaSqlTypesUtils.bigDecimalAsNumeric(
-                        "99999999999999999999999999999.999999999"))
-                .build());
-    pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
-  }
-
-  @Test
-  public void testNumericColumn() {
-    String sql = "SELECT numeric_field FROM table_with_numeric";
-
-    ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
-    BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql);
-    PCollection<Row> stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode);
-
-    PAssert.that(stream)
-        .containsInAnyOrder(
-            Row.withSchema(Schema.builder().addDecimalField("f_numeric").build())
-                .addValues(ZetaSqlTypesUtils.bigDecimalAsNumeric("123.4567"))
-                .build(),
-            Row.withSchema(Schema.builder().addDecimalField("f_numeric").build())
-                .addValues(ZetaSqlTypesUtils.bigDecimalAsNumeric("765.4321"))
-                .build(),
-            Row.withSchema(Schema.builder().addDecimalField("f_numeric").build())
-                .addValues(ZetaSqlTypesUtils.bigDecimalAsNumeric("-555.5555"))
-                .build());
-    pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
-  }
-
-  @Test
-  public void testUnaryMinusNumeric() {
-    String sql = "SELECT - NUMERIC '1.23456e05'";
-
-    ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
-    BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql);
-    PCollection<Row> stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode);
-
-    PAssert.that(stream)
-        .containsInAnyOrder(
-            Row.withSchema(Schema.builder().addDecimalField("f_numeric").build())
-                .addValues(ZetaSqlTypesUtils.bigDecimalAsNumeric("-123456"))
-                .build());
-    pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
-  }
-
-  @Test
-  public void testAddNumeric() {
-    String sql = "SELECT NUMERIC '1.23456e05' + NUMERIC '9.876e-3'";
-
-    ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
-    BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql);
-    PCollection<Row> stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode);
-
-    PAssert.that(stream)
-        .containsInAnyOrder(
-            Row.withSchema(Schema.builder().addDecimalField("f_numeric").build())
-                .addValues(ZetaSqlTypesUtils.bigDecimalAsNumeric("123456.009876"))
-                .build());
-    pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
-  }
-
-  @Test
-  public void testSubNumeric() {
-    String sql = "SELECT NUMERIC '1.23456e05' - NUMERIC '-9.876e-3'";
-
-    ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
-    BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql);
-    PCollection<Row> stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode);
-
-    PAssert.that(stream)
-        .containsInAnyOrder(
-            Row.withSchema(Schema.builder().addDecimalField("f_numeric").build())
-                .addValues(ZetaSqlTypesUtils.bigDecimalAsNumeric("123456.009876"))
-                .build());
-    pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
-  }
-
-  @Test
-  public void testMultiNumeric() {
-    String sql = "SELECT NUMERIC '1.23e02' * NUMERIC '-1.001e-3'";
-
-    ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
-    BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql);
-    PCollection<Row> stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode);
-
-    PAssert.that(stream)
-        .containsInAnyOrder(
-            Row.withSchema(Schema.builder().addDecimalField("f_numeric").build())
-                .addValues(ZetaSqlTypesUtils.bigDecimalAsNumeric("-0.123123"))
-                .build());
-    pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
-  }
-
-  @Test
-  public void testDivNumeric() {
-    String sql = "SELECT NUMERIC '-1.23123e-1' / NUMERIC '-1.001e-3'";
-
-    ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
-    BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql);
-    PCollection<Row> stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode);
-
-    PAssert.that(stream)
-        .containsInAnyOrder(
-            Row.withSchema(Schema.builder().addDecimalField("f_numeric").build())
-                .addValues(ZetaSqlTypesUtils.bigDecimalAsNumeric("123"))
-                .build());
-    pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
-  }
-
-  @Test
-  public void testModNumeric() {
-    String sql = "SELECT MOD(NUMERIC '1.23456e05', NUMERIC '5')";
-
-    ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
-    BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql);
-    PCollection<Row> stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode);
-
-    PAssert.that(stream)
-        .containsInAnyOrder(
-            Row.withSchema(Schema.builder().addDecimalField("f_numeric").build())
-                .addValues(ZetaSqlTypesUtils.bigDecimalAsNumeric("1"))
-                .build());
-    pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
-  }
-
-  @Test
-  public void testFloorNumeric() {
-    String sql = "SELECT FLOOR(NUMERIC '1.23456e04'), FLOOR(NUMERIC '-1.23456e04')";
-
-    ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
-    BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql);
-    PCollection<Row> stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode);
-
-    PAssert.that(stream)
-        .containsInAnyOrder(
-            Row.withSchema(
-                    Schema.builder()
-                        .addDecimalField("f_numeric1")
-                        .addDecimalField("f_numeric2")
-                        .build())
-                .addValues(ZetaSqlTypesUtils.bigDecimalAsNumeric("12345"))
-                .addValues(ZetaSqlTypesUtils.bigDecimalAsNumeric("-12346"))
-                .build());
-    pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
-  }
-
-  @Test
-  public void testCeilNumeric() {
-    String sql = "SELECT CEIL(NUMERIC '1.23456e04'), CEIL(NUMERIC '-1.23456e04')";
-
-    ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
-    BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql);
-    PCollection<Row> stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode);
-
-    PAssert.that(stream)
-        .containsInAnyOrder(
-            Row.withSchema(
-                    Schema.builder()
-                        .addDecimalField("f_numeric1")
-                        .addDecimalField("f_numeric2")
-                        .build())
-                .addValues(ZetaSqlTypesUtils.bigDecimalAsNumeric("12346"))
-                .addValues(ZetaSqlTypesUtils.bigDecimalAsNumeric("-12345"))
-                .build());
-    pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
-  }
-
-  @Test
-  @Ignore("https://jira.apache.org/jira/browse/BEAM-10459")
-  public void testSumNumeric() {
-    String sql = "SELECT SUM(numeric_field) FROM table_with_numeric";
-
-    ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
-    BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql);
-    PCollection<Row> stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode);
-
-    PAssert.that(stream)
-        .containsInAnyOrder(
-            Row.withSchema(Schema.builder().addDecimalField("f_numeric").build())
-                .addValues(ZetaSqlTypesUtils.bigDecimalAsNumeric("333.3333"))
-                .build());
-    pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
-  }
-
-  @Test
-  @Ignore("https://jira.apache.org/jira/browse/BEAM-10459")
-  public void testAvgNumeric() {
-    String sql = "SELECT AVG(numeric_field) FROM table_with_numeric";
-
-    ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
-    BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql);
-    PCollection<Row> stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode);
-
-    PAssert.that(stream)
-        .containsInAnyOrder(
-            Row.withSchema(Schema.builder().addDecimalField("f_numeric").build())
-                .addValues(ZetaSqlTypesUtils.bigDecimalAsNumeric("111.1111"))
-                .build());
-    pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
-  }
-
   @Test
   public void testMultipleSelectStatementsThrowsException() {
     String sql = "SELECT 1; SELECT 2;";
diff --git a/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlMathFunctionsTest.java b/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlMathFunctionsTest.java
new file mode 100644
index 0000000..c2ee278
--- /dev/null
+++ b/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlMathFunctionsTest.java
@@ -0,0 +1,1032 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.sql.zetasql;
+
+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.schemas.Schema;
+import org.apache.beam.sdk.testing.PAssert;
+import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.Row;
+import org.joda.time.Duration;
+import org.junit.Before;
+import org.junit.Ignore;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/** Tests for ZetaSQL Math functions (on INT64, DOUBLE, NUMERIC types). */
+@RunWith(JUnit4.class)
+public class ZetaSqlMathFunctionsTest extends ZetaSqlTestBase {
+
+  @Rule public transient TestPipeline pipeline = TestPipeline.create();
+  @Rule public ExpectedException thrown = ExpectedException.none();
+
+  @Before
+  public void setUp() {
+    initialize();
+  }
+
+  /////////////////////////////////////////////////////////////////////////////
+  // INT64 type tests
+  /////////////////////////////////////////////////////////////////////////////
+
+  @Test
+  public void testArithmeticOperatorsInt64() {
+    String sql = "SELECT -1, 1 + 2, 1 - 2, 1 * 2, 1 / 2";
+
+    ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
+    BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql);
+    PCollection<Row> stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode);
+
+    PAssert.that(stream)
+        .containsInAnyOrder(
+            Row.withSchema(
+                    Schema.builder()
+                        .addInt64Field("f_int64_1")
+                        .addInt64Field("f_int64_2")
+                        .addInt64Field("f_int64_3")
+                        .addInt64Field("f_int64_4")
+                        .addDoubleField("f_double")
+                        .build())
+                .addValues(-1L, 3L, -1L, 2L, 0.5)
+                .build());
+    pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
+  }
+
+  @Test
+  public void testAbsInt64() {
+    String sql = "SELECT ABS(1), ABS(-1)";
+
+    ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
+    BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql);
+    PCollection<Row> stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode);
+
+    PAssert.that(stream)
+        .containsInAnyOrder(
+            Row.withSchema(
+                    Schema.builder().addInt64Field("f_int64_1").addInt64Field("f_int64_2").build())
+                .addValues(1L, 1L)
+                .build());
+    pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
+  }
+
+  @Test
+  public void testSignInt64() {
+    String sql = "SELECT SIGN(0), SIGN(5), SIGN(-5)";
+
+    ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
+    BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql);
+    PCollection<Row> stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode);
+
+    PAssert.that(stream)
+        .containsInAnyOrder(
+            Row.withSchema(
+                    Schema.builder()
+                        .addInt64Field("f_int64_1")
+                        .addInt64Field("f_int64_2")
+                        .addInt64Field("f_int64_3")
+                        .build())
+                .addValues(0L, 1L, -1L)
+                .build());
+    pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
+  }
+
+  @Test
+  public void testModInt64() {
+    String sql = "SELECT MOD(4, 2)";
+
+    ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
+    BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql);
+    PCollection<Row> stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode);
+
+    PAssert.that(stream)
+        .containsInAnyOrder(
+            Row.withSchema(Schema.builder().addInt64Field("f_int64").build())
+                .addValues(0L)
+                .build());
+    pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
+  }
+
+  @Test
+  public void testDivInt64() {
+    String sql = "SELECT DIV(1, 2), DIV(2, 1)";
+
+    ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
+    BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql);
+    PCollection<Row> stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode);
+
+    PAssert.that(stream)
+        .containsInAnyOrder(
+            Row.withSchema(
+                    Schema.builder().addInt64Field("f_int64_1").addInt64Field("f_int64_2").build())
+                .addValues(0L, 2L)
+                .build());
+    pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
+  }
+
+  @Test
+  public void testSafeArithmeticFunctionsInt64() {
+    String sql =
+        "SELECT SAFE_ADD(9223372036854775807, 1), "
+            + "SAFE_SUBTRACT(-9223372036854775808, 1), "
+            + "SAFE_MULTIPLY(9223372036854775807, 2), "
+            + "SAFE_DIVIDE(1, 0), "
+            + "SAFE_NEGATE(-9223372036854775808)";
+
+    ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
+    BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql);
+    PCollection<Row> stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode);
+
+    PAssert.that(stream)
+        .containsInAnyOrder(
+            Row.withSchema(
+                    Schema.builder()
+                        .addNullableField("f_int64_1", Schema.FieldType.INT64)
+                        .addNullableField("f_int64_2", Schema.FieldType.INT64)
+                        .addNullableField("f_int64_3", Schema.FieldType.INT64)
+                        .addNullableField("f_int64_4", Schema.FieldType.INT64)
+                        .addNullableField("f_int64_5", Schema.FieldType.INT64)
+                        .build())
+                .addValues(null, null, null, null, null)
+                .build());
+    pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
+  }
+
+  /////////////////////////////////////////////////////////////////////////////
+  // DOUBLE (FLOAT64) type tests
+  /////////////////////////////////////////////////////////////////////////////
+
+  @Test
+  public void testDoubleLiteral() {
+    String sql =
+        "SELECT 3.0, CAST('+inf' AS FLOAT64), CAST('-inf' AS FLOAT64), CAST('NaN' AS FLOAT64)";
+
+    ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
+    BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql);
+    PCollection<Row> stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode);
+
+    PAssert.that(stream)
+        .containsInAnyOrder(
+            Row.withSchema(
+                    Schema.builder()
+                        .addDoubleField("f_double1")
+                        .addDoubleField("f_double2")
+                        .addDoubleField("f_double3")
+                        .addDoubleField("f_double4")
+                        .build())
+                .addValues(3.0, Double.POSITIVE_INFINITY, Double.NEGATIVE_INFINITY, Double.NaN)
+                .build());
+    pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
+  }
+
+  @Test
+  public void testArithmeticOperatorsDouble() {
+    String sql = "SELECT -1.5, 1.5 + 2.5, 1.5 - 2.5, 1.5 * 2.5, 1.5 / 2.5";
+
+    ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
+    BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql);
+    PCollection<Row> stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode);
+
+    PAssert.that(stream)
+        .containsInAnyOrder(
+            Row.withSchema(
+                    Schema.builder()
+                        .addDoubleField("f_double1")
+                        .addDoubleField("f_double2")
+                        .addDoubleField("f_double3")
+                        .addDoubleField("f_double4")
+                        .addDoubleField("f_double5")
+                        .build())
+                .addValues(-1.5, 4.0, -1.0, 3.75, 0.6)
+                .build());
+    pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
+  }
+
+  @Test
+  public void testEqualsInf() {
+    String sql =
+        "SELECT CAST('+inf' AS FLOAT64) = CAST('+inf' AS FLOAT64), "
+            + "CAST('+inf' AS FLOAT64) = CAST('-inf' AS FLOAT64)";
+
+    ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
+    BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql);
+    PCollection<Row> stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode);
+
+    PAssert.that(stream)
+        .containsInAnyOrder(
+            Row.withSchema(
+                    Schema.builder()
+                        .addBooleanField("f_boolean1")
+                        .addBooleanField("f_boolean2")
+                        .build())
+                .addValues(true, false)
+                .build());
+    pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
+  }
+
+  @Test
+  public void testEqualsNaN() {
+    String sql = "SELECT CAST('NaN' AS FLOAT64) = CAST('NaN' AS FLOAT64)";
+
+    ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
+    BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql);
+    PCollection<Row> stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode);
+
+    PAssert.that(stream)
+        .containsInAnyOrder(
+            Row.withSchema(Schema.builder().addBooleanField("f_boolean").build())
+                .addValues(false)
+                .build());
+    pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
+  }
+
+  @Test
+  public void testAbsDouble() {
+    String sql = "SELECT ABS(1.5), ABS(-1.0), ABS(CAST('NaN' AS FLOAT64))";
+
+    ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
+    BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql);
+    PCollection<Row> stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode);
+
+    PAssert.that(stream)
+        .containsInAnyOrder(
+            Row.withSchema(
+                    Schema.builder()
+                        .addDoubleField("f_double1")
+                        .addDoubleField("f_double2")
+                        .addDoubleField("f_double3")
+                        .build())
+                .addValues(1.5, 1.0, Double.NaN)
+                .build());
+    pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
+  }
+
+  @Test
+  public void testSignDouble() {
+    String sql = "SELECT SIGN(-0.0), SIGN(1.5), SIGN(-1.5), SIGN(CAST('NaN' AS FLOAT64))";
+
+    ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
+    BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql);
+    PCollection<Row> stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode);
+
+    PAssert.that(stream)
+        .containsInAnyOrder(
+            Row.withSchema(
+                    Schema.builder()
+                        .addDoubleField("f_double1")
+                        .addDoubleField("f_double2")
+                        .addDoubleField("f_double3")
+                        .addDoubleField("f_double4")
+                        .build())
+                .addValues(0.0, 1.0, -1.0, Double.NaN)
+                .build());
+    pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
+  }
+
+  @Test
+  public void testRoundDouble() {
+    String sql = "SELECT ROUND(1.23), ROUND(-1.27, 1)";
+
+    ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
+    BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql);
+    PCollection<Row> stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode);
+
+    PAssert.that(stream)
+        .containsInAnyOrder(
+            Row.withSchema(
+                    Schema.builder()
+                        .addDoubleField("f_double1")
+                        .addDoubleField("f_double2")
+                        .build())
+                .addValues(1.0, -1.3)
+                .build());
+    pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
+  }
+
+  @Test
+  public void testTruncDouble() {
+    String sql = "SELECT TRUNC(1.23), TRUNC(-1.27, 1)";
+
+    ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
+    BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql);
+    PCollection<Row> stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode);
+
+    PAssert.that(stream)
+        .containsInAnyOrder(
+            Row.withSchema(
+                    Schema.builder()
+                        .addDoubleField("f_double1")
+                        .addDoubleField("f_double2")
+                        .build())
+                .addValues(1.0, -1.2)
+                .build());
+    pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
+  }
+
+  @Test
+  public void testCeilDouble() {
+    String sql = "SELECT CEIL(1.2), CEIL(-1.2)";
+
+    ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
+    BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql);
+    PCollection<Row> stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode);
+
+    PAssert.that(stream)
+        .containsInAnyOrder(
+            Row.withSchema(
+                    Schema.builder()
+                        .addDoubleField("f_double1")
+                        .addDoubleField("f_double2")
+                        .build())
+                .addValues(2.0, -1.0)
+                .build());
+    pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
+  }
+
+  @Test
+  public void testFloorDouble() {
+    String sql = "SELECT FLOOR(1.2), FLOOR(-1.2)";
+
+    ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
+    BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql);
+    PCollection<Row> stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode);
+
+    PAssert.that(stream)
+        .containsInAnyOrder(
+            Row.withSchema(
+                    Schema.builder()
+                        .addDoubleField("f_double1")
+                        .addDoubleField("f_double2")
+                        .build())
+                .addValues(1.0, -2.0)
+                .build());
+    pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
+  }
+
+  @Test
+  public void testIsInf() {
+    String sql =
+        "SELECT IS_INF(CAST('+inf' AS FLOAT64)), IS_INF(CAST('-inf' AS FLOAT64)), IS_INF(3.0)";
+
+    ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
+    BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql);
+    PCollection<Row> stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode);
+
+    PAssert.that(stream)
+        .containsInAnyOrder(
+            Row.withSchema(
+                    Schema.builder()
+                        .addBooleanField("f_boolean1")
+                        .addBooleanField("f_boolean2")
+                        .addBooleanField("f_boolean3")
+                        .build())
+                .addValues(true, true, false)
+                .build());
+    pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
+  }
+
+  @Test
+  public void testIsNaN() {
+    String sql = "SELECT IS_NAN(CAST('NaN' AS FLOAT64)), IS_NAN(3.0)";
+
+    ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
+    BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql);
+    PCollection<Row> stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode);
+
+    PAssert.that(stream)
+        .containsInAnyOrder(
+            Row.withSchema(
+                    Schema.builder()
+                        .addBooleanField("f_boolean1")
+                        .addBooleanField("f_boolean2")
+                        .build())
+                .addValues(true, false)
+                .build());
+    pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
+  }
+
+  @Test
+  public void testIeeeDivide() {
+    String sql = "SELECT IEEE_DIVIDE(1.0, 0.0)";
+
+    ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
+    BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql);
+    PCollection<Row> stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode);
+
+    PAssert.that(stream)
+        .containsInAnyOrder(
+            Row.withSchema(Schema.builder().addDoubleField("f_double").build())
+                .addValues(Double.POSITIVE_INFINITY)
+                .build());
+    pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
+  }
+
+  @Test
+  public void testSafeDivide() {
+    String sql = "SELECT SAFE_DIVIDE(1.0, 0.0)";
+
+    ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
+    BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql);
+    PCollection<Row> stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode);
+
+    PAssert.that(stream)
+        .containsInAnyOrder(
+            Row.withSchema(
+                    Schema.builder().addNullableField("f_double", Schema.FieldType.DOUBLE).build())
+                .addValue(null)
+                .build());
+    pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
+  }
+
+  @Test
+  public void testSqrtDouble() {
+    String sql = "SELECT SQRT(4.0)";
+
+    ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
+    BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql);
+    PCollection<Row> stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode);
+
+    PAssert.that(stream)
+        .containsInAnyOrder(
+            Row.withSchema(Schema.builder().addDoubleField("f_double").build())
+                .addValues(2.0)
+                .build());
+
+    pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
+  }
+
+  @Test
+  public void testPowDouble() {
+    String sql = "SELECT POW(2.0, 3.0)";
+
+    ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
+    BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql);
+    PCollection<Row> stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode);
+
+    PAssert.that(stream)
+        .containsInAnyOrder(
+            Row.withSchema(Schema.builder().addDoubleField("f_double").build())
+                .addValues(8.0)
+                .build());
+
+    pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
+  }
+
+  @Test
+  public void testExpDouble() {
+    String sql = "SELECT EXP(2.0)";
+
+    ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
+    BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql);
+    PCollection<Row> stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode);
+
+    PAssert.that(stream)
+        .containsInAnyOrder(
+            Row.withSchema(Schema.builder().addDoubleField("f_double").build())
+                .addValues(7.38905609893065)
+                .build());
+
+    pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
+  }
+
+  @Test
+  public void testLnDouble() {
+    String sql = "SELECT LN(7.38905609893065)";
+
+    ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
+    BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql);
+    PCollection<Row> stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode);
+
+    PAssert.that(stream)
+        .containsInAnyOrder(
+            Row.withSchema(Schema.builder().addDoubleField("f_double").build())
+                .addValues(2.0)
+                .build());
+
+    pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
+  }
+
+  @Test
+  public void testLog10Double() {
+    String sql = "SELECT LOG10(100.0)";
+
+    ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
+    BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql);
+    PCollection<Row> stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode);
+
+    PAssert.that(stream)
+        .containsInAnyOrder(
+            Row.withSchema(Schema.builder().addDoubleField("f_double").build())
+                .addValues(2.0)
+                .build());
+
+    pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
+  }
+
+  @Test
+  public void testLogDouble() {
+    String sql = "SELECT LOG(2.25, 1.5)";
+
+    ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
+    BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql);
+    PCollection<Row> stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode);
+
+    PAssert.that(stream)
+        .containsInAnyOrder(
+            Row.withSchema(Schema.builder().addDoubleField("f_double").build())
+                .addValues(2.0)
+                .build());
+
+    pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
+  }
+
+  @Test
+  public void testTrigonometricFunctions() {
+    String sql =
+        "SELECT COS(0.0), COSH(0.0), ACOS(1.0), ACOSH(1.0), "
+            + "SIN(0.0), SINH(0.0), ASIN(0.0), ASINH(0.0), "
+            + "TAN(0.0), TANH(0.0), ATAN(0.0), ATANH(0.0), ATAN2(0.0, 0.0)";
+
+    ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
+    BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql);
+    PCollection<Row> stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode);
+
+    PAssert.that(stream)
+        .containsInAnyOrder(
+            Row.withSchema(
+                    Schema.builder()
+                        .addDoubleField("f_double1")
+                        .addDoubleField("f_double2")
+                        .addDoubleField("f_double3")
+                        .addDoubleField("f_double4")
+                        .addDoubleField("f_double5")
+                        .addDoubleField("f_double6")
+                        .addDoubleField("f_double7")
+                        .addDoubleField("f_double8")
+                        .addDoubleField("f_double9")
+                        .addDoubleField("f_double10")
+                        .addDoubleField("f_double11")
+                        .addDoubleField("f_double12")
+                        .addDoubleField("f_double13")
+                        .build())
+                .addValues(1.0, 1.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0.0)
+                .build());
+
+    pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
+  }
+
+  /////////////////////////////////////////////////////////////////////////////
+  // NUMERIC type tests
+  /////////////////////////////////////////////////////////////////////////////
+
+  @Test
+  public void testNumericLiteral() {
+    String sql =
+        "SELECT NUMERIC '0', "
+            + "NUMERIC '123456', "
+            + "NUMERIC '-3.14', "
+            + "NUMERIC '-0.54321', "
+            + "NUMERIC '1.23456e05', "
+            + "NUMERIC '-9.876e-3', "
+            // min value for ZetaSQL NUMERIC type
+            + "NUMERIC '-99999999999999999999999999999.999999999', "
+            // max value for ZetaSQL NUMERIC type
+            + "NUMERIC '99999999999999999999999999999.999999999'";
+    ;
+
+    ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
+    BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql);
+    PCollection<Row> stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode);
+
+    PAssert.that(stream)
+        .containsInAnyOrder(
+            Row.withSchema(
+                    Schema.builder()
+                        .addDecimalField("f_numeric1")
+                        .addDecimalField("f_numeric2")
+                        .addDecimalField("f_numeric3")
+                        .addDecimalField("f_numeric4")
+                        .addDecimalField("f_numeric5")
+                        .addDecimalField("f_numeric6")
+                        .addDecimalField("f_numeric7")
+                        .addDecimalField("f_numeric8")
+                        .build())
+                .addValues(
+                    ZetaSqlTypesUtils.bigDecimalAsNumeric("0"),
+                    ZetaSqlTypesUtils.bigDecimalAsNumeric("123456"),
+                    ZetaSqlTypesUtils.bigDecimalAsNumeric("-3.14"),
+                    ZetaSqlTypesUtils.bigDecimalAsNumeric("-0.54321"),
+                    ZetaSqlTypesUtils.bigDecimalAsNumeric("123456"),
+                    ZetaSqlTypesUtils.bigDecimalAsNumeric("-0.009876"),
+                    ZetaSqlTypesUtils.NUMERIC_MIN_VALUE,
+                    ZetaSqlTypesUtils.NUMERIC_MAX_VALUE)
+                .build());
+    pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
+  }
+
+  @Test
+  public void testNumericColumn() {
+    String sql = "SELECT numeric_field FROM table_with_numeric";
+
+    ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
+    BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql);
+    PCollection<Row> stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode);
+
+    final Schema schema = Schema.builder().addDecimalField("f_numeric").build();
+
+    PAssert.that(stream)
+        .containsInAnyOrder(
+            Row.withSchema(schema)
+                .addValues(ZetaSqlTypesUtils.bigDecimalAsNumeric("123.4567"))
+                .build(),
+            Row.withSchema(schema)
+                .addValues(ZetaSqlTypesUtils.bigDecimalAsNumeric("765.4321"))
+                .build(),
+            Row.withSchema(schema)
+                .addValues(ZetaSqlTypesUtils.bigDecimalAsNumeric("-555.5555"))
+                .build());
+    pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
+  }
+
+  @Test
+  public void testArithmeticOperatorsNumeric() {
+    String sql =
+        "SELECT - NUMERIC '1.23456e05', "
+            + "NUMERIC '1.23456e05' + NUMERIC '9.876e-3', "
+            + "NUMERIC '1.23456e05' - NUMERIC '-9.876e-3', "
+            + "NUMERIC '1.23e02' * NUMERIC '-1.001e-3', "
+            + "NUMERIC '-1.23123e-1' / NUMERIC '-1.001e-3', ";
+
+    ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
+    BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql);
+    PCollection<Row> stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode);
+
+    PAssert.that(stream)
+        .containsInAnyOrder(
+            Row.withSchema(
+                    Schema.builder()
+                        .addDecimalField("f_numeric1")
+                        .addDecimalField("f_numeric2")
+                        .addDecimalField("f_numeric3")
+                        .addDecimalField("f_numeric4")
+                        .addDecimalField("f_numeric5")
+                        .build())
+                .addValues(
+                    ZetaSqlTypesUtils.bigDecimalAsNumeric("-123456"),
+                    ZetaSqlTypesUtils.bigDecimalAsNumeric("123456.009876"),
+                    ZetaSqlTypesUtils.bigDecimalAsNumeric("123456.009876"),
+                    ZetaSqlTypesUtils.bigDecimalAsNumeric("-0.123123"),
+                    ZetaSqlTypesUtils.bigDecimalAsNumeric("123"))
+                .build());
+    pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
+  }
+
+  @Test
+  public void testAbsNumeric() {
+    String sql = "SELECT ABS(NUMERIC '1.23456e04'), ABS(NUMERIC '-1.23456e04')";
+
+    ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
+    BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql);
+    PCollection<Row> stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode);
+
+    PAssert.that(stream)
+        .containsInAnyOrder(
+            Row.withSchema(
+                    Schema.builder()
+                        .addDecimalField("f_numeric1")
+                        .addDecimalField("f_numeric2")
+                        .build())
+                .addValues(
+                    ZetaSqlTypesUtils.bigDecimalAsNumeric("12345.6"),
+                    ZetaSqlTypesUtils.bigDecimalAsNumeric("12345.6"))
+                .build());
+    pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
+  }
+
+  @Test
+  public void testSignNumeric() {
+    String sql = "SELECT SIGN(NUMERIC '0'), SIGN(NUMERIC '1.23e01'), SIGN(NUMERIC '-1.23e01')";
+
+    ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
+    BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql);
+    PCollection<Row> stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode);
+
+    PAssert.that(stream)
+        .containsInAnyOrder(
+            Row.withSchema(
+                    Schema.builder()
+                        .addDecimalField("f_numeric1")
+                        .addDecimalField("f_numeric2")
+                        .addDecimalField("f_numeric3")
+                        .build())
+                .addValues(
+                    ZetaSqlTypesUtils.bigDecimalAsNumeric("0"),
+                    ZetaSqlTypesUtils.bigDecimalAsNumeric("1"),
+                    ZetaSqlTypesUtils.bigDecimalAsNumeric("-1"))
+                .build());
+    pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
+  }
+
+  @Test
+  public void testRoundNumeric() {
+    String sql = "SELECT ROUND(NUMERIC '1.23456e04'), ROUND(NUMERIC '-1.234567e04', 1)";
+
+    ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
+    BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql);
+    PCollection<Row> stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode);
+
+    PAssert.that(stream)
+        .containsInAnyOrder(
+            Row.withSchema(
+                    Schema.builder()
+                        .addDecimalField("f_numeric1")
+                        .addDecimalField("f_numeric2")
+                        .build())
+                .addValues(
+                    ZetaSqlTypesUtils.bigDecimalAsNumeric("12346"),
+                    ZetaSqlTypesUtils.bigDecimalAsNumeric("-12345.7"))
+                .build());
+    pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
+  }
+
+  @Test
+  public void testTruncNumeric() {
+    String sql = "SELECT TRUNC(NUMERIC '1.23456e04'), TRUNC(NUMERIC '-1.234567e04', 1)";
+
+    ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
+    BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql);
+    PCollection<Row> stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode);
+
+    PAssert.that(stream)
+        .containsInAnyOrder(
+            Row.withSchema(
+                    Schema.builder()
+                        .addDecimalField("f_numeric1")
+                        .addDecimalField("f_numeric2")
+                        .build())
+                .addValues(
+                    ZetaSqlTypesUtils.bigDecimalAsNumeric("12345"),
+                    ZetaSqlTypesUtils.bigDecimalAsNumeric("-12345.6"))
+                .build());
+    pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
+  }
+
+  @Test
+  public void testCeilNumeric() {
+    String sql = "SELECT CEIL(NUMERIC '1.23456e04'), CEIL(NUMERIC '-1.23456e04')";
+
+    ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
+    BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql);
+    PCollection<Row> stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode);
+
+    PAssert.that(stream)
+        .containsInAnyOrder(
+            Row.withSchema(
+                    Schema.builder()
+                        .addDecimalField("f_numeric1")
+                        .addDecimalField("f_numeric2")
+                        .build())
+                .addValues(
+                    ZetaSqlTypesUtils.bigDecimalAsNumeric("12346"),
+                    ZetaSqlTypesUtils.bigDecimalAsNumeric("-12345"))
+                .build());
+    pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
+  }
+
+  @Test
+  public void testFloorNumeric() {
+    String sql = "SELECT FLOOR(NUMERIC '1.23456e04'), FLOOR(NUMERIC '-1.23456e04')";
+
+    ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
+    BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql);
+    PCollection<Row> stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode);
+
+    PAssert.that(stream)
+        .containsInAnyOrder(
+            Row.withSchema(
+                    Schema.builder()
+                        .addDecimalField("f_numeric1")
+                        .addDecimalField("f_numeric2")
+                        .build())
+                .addValues(
+                    ZetaSqlTypesUtils.bigDecimalAsNumeric("12345"),
+                    ZetaSqlTypesUtils.bigDecimalAsNumeric("-12346"))
+                .build());
+    pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
+  }
+
+  @Test
+  public void testModNumeric() {
+    String sql = "SELECT MOD(NUMERIC '1.23456e05', NUMERIC '5')";
+
+    ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
+    BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql);
+    PCollection<Row> stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode);
+
+    PAssert.that(stream)
+        .containsInAnyOrder(
+            Row.withSchema(Schema.builder().addDecimalField("f_numeric").build())
+                .addValues(ZetaSqlTypesUtils.bigDecimalAsNumeric("1"))
+                .build());
+    pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
+  }
+
+  @Test
+  public void testDivNumeric() {
+    String sql = "SELECT DIV(NUMERIC '1.23456e05', NUMERIC '5')";
+
+    ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
+    BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql);
+    PCollection<Row> stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode);
+
+    PAssert.that(stream)
+        .containsInAnyOrder(
+            Row.withSchema(Schema.builder().addDecimalField("f_numeric").build())
+                .addValues(ZetaSqlTypesUtils.bigDecimalAsNumeric("24691"))
+                .build());
+    pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
+  }
+
+  @Test
+  public void testSafeArithmeticFunctionsNumeric() {
+    String sql =
+        "SELECT SAFE_ADD(NUMERIC '99999999999999999999999999999.999999999', NUMERIC '1'), "
+            + "SAFE_SUBTRACT(NUMERIC '-99999999999999999999999999999.999999999', NUMERIC '1'), "
+            + "SAFE_MULTIPLY(NUMERIC '99999999999999999999999999999.999999999', NUMERIC '2'), "
+            + "SAFE_DIVIDE(NUMERIC '1.23456e05', NUMERIC '0'), "
+            + "SAFE_NEGATE(NUMERIC '99999999999999999999999999999.999999999')";
+
+    ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
+    BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql);
+    PCollection<Row> stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode);
+
+    PAssert.that(stream)
+        .containsInAnyOrder(
+            Row.withSchema(
+                    Schema.builder()
+                        .addNullableField("f_numeric1", Schema.FieldType.DECIMAL)
+                        .addNullableField("f_numeric2", Schema.FieldType.DECIMAL)
+                        .addNullableField("f_numeric3", Schema.FieldType.DECIMAL)
+                        .addNullableField("f_numeric4", Schema.FieldType.DECIMAL)
+                        .addNullableField("f_numeric5", Schema.FieldType.DECIMAL)
+                        .build())
+                .addValues(null, null, null, null, ZetaSqlTypesUtils.NUMERIC_MIN_VALUE)
+                .build());
+    pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
+  }
+
+  @Test
+  public void testSqrtNumeric() {
+    String sql = "SELECT SQRT(NUMERIC '4')";
+
+    ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
+    BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql);
+    PCollection<Row> stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode);
+
+    PAssert.that(stream)
+        .containsInAnyOrder(
+            Row.withSchema(Schema.builder().addDecimalField("f_numeric").build())
+                .addValues(ZetaSqlTypesUtils.bigDecimalAsNumeric("2"))
+                .build());
+
+    pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
+  }
+
+  @Test
+  public void testPowNumeric() {
+    String sql = "SELECT POW(NUMERIC '2', NUMERIC '3')";
+
+    ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
+    BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql);
+    PCollection<Row> stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode);
+
+    PAssert.that(stream)
+        .containsInAnyOrder(
+            Row.withSchema(Schema.builder().addDecimalField("f_numeric").build())
+                .addValues(ZetaSqlTypesUtils.bigDecimalAsNumeric("8"))
+                .build());
+
+    pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
+  }
+
+  @Test
+  public void testExpNumeric() {
+    String sql = "SELECT EXP(NUMERIC '2')";
+
+    ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
+    BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql);
+    PCollection<Row> stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode);
+
+    PAssert.that(stream)
+        .containsInAnyOrder(
+            Row.withSchema(Schema.builder().addDecimalField("f_numeric").build())
+                .addValues(ZetaSqlTypesUtils.bigDecimalAsNumeric("7.389056099"))
+                .build());
+
+    pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
+  }
+
+  @Test
+  public void testLnNumeric() {
+    String sql = "SELECT LN(NUMERIC '7.389056099')";
+
+    ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
+    BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql);
+    PCollection<Row> stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode);
+
+    PAssert.that(stream)
+        .containsInAnyOrder(
+            Row.withSchema(Schema.builder().addDecimalField("f_numeric").build())
+                .addValues(ZetaSqlTypesUtils.bigDecimalAsNumeric("2"))
+                .build());
+
+    pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
+  }
+
+  @Test
+  public void testLog10Numeric() {
+    String sql = "SELECT LOG10(NUMERIC '100')";
+
+    ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
+    BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql);
+    PCollection<Row> stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode);
+
+    PAssert.that(stream)
+        .containsInAnyOrder(
+            Row.withSchema(Schema.builder().addDecimalField("f_numeric").build())
+                .addValues(ZetaSqlTypesUtils.bigDecimalAsNumeric("2"))
+                .build());
+
+    pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
+  }
+
+  @Test
+  public void testLogNumeric() {
+    String sql = "SELECT LOG(NUMERIC '2.25', NUMERIC '1.5')";
+
+    ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
+    BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql);
+    PCollection<Row> stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode);
+
+    PAssert.that(stream)
+        .containsInAnyOrder(
+            Row.withSchema(Schema.builder().addDecimalField("f_numeric").build())
+                .addValues(ZetaSqlTypesUtils.bigDecimalAsNumeric("2"))
+                .build());
+
+    pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
+  }
+
+  @Test
+  @Ignore("[BEAM-10459] Aggregation functions on NUMERIC is not supported yet")
+  public void testSumNumeric() {
+    String sql = "SELECT SUM(numeric_field) FROM table_with_numeric";
+
+    ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
+    BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql);
+    PCollection<Row> stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode);
+
+    PAssert.that(stream)
+        .containsInAnyOrder(
+            Row.withSchema(Schema.builder().addDecimalField("f_numeric").build())
+                .addValues(ZetaSqlTypesUtils.bigDecimalAsNumeric("333.3333"))
+                .build());
+    pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
+  }
+
+  @Test
+  @Ignore("[BEAM-10459] Aggregation functions on NUMERIC is not supported yet")
+  public void testAvgNumeric() {
+    String sql = "SELECT AVG(numeric_field) FROM table_with_numeric";
+
+    ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
+    BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql);
+    PCollection<Row> stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode);
+
+    PAssert.that(stream)
+        .containsInAnyOrder(
+            Row.withSchema(Schema.builder().addDecimalField("f_numeric").build())
+                .addValues(ZetaSqlTypesUtils.bigDecimalAsNumeric("111.1111"))
+                .build());
+    pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
+  }
+}
diff --git a/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlTestBase.java b/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlTestBase.java
index d3c54c7..dee7d67 100644
--- a/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlTestBase.java
+++ b/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlTestBase.java
@@ -34,6 +34,7 @@
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
 
 /** Common setup for ZetaSQL tests. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public abstract class ZetaSqlTestBase {
   protected static final long PIPELINE_EXECUTION_WAITTIME_MINUTES = 2L;
 
diff --git a/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlTimeFunctionsTest.java b/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlTimeFunctionsTest.java
index ca37715..38f7ed7 100644
--- a/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlTimeFunctionsTest.java
+++ b/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlTimeFunctionsTest.java
@@ -49,6 +49,7 @@
 
 /** Tests for ZetaSQL time functions (DATE, TIME, DATETIME, and TIMESTAMP functions). */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ZetaSqlTimeFunctionsTest extends ZetaSqlTestBase {
 
   @Rule public transient TestPipeline pipeline = TestPipeline.create();
diff --git a/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlTypesUtils.java b/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlTypesUtils.java
index 3d8d478..02893b7 100644
--- a/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlTypesUtils.java
+++ b/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlTypesUtils.java
@@ -24,6 +24,11 @@
 @Internal
 public class ZetaSqlTypesUtils {
 
+  public static final BigDecimal NUMERIC_MAX_VALUE =
+      bigDecimalAsNumeric("99999999999999999999999999999.999999999");
+  public static final BigDecimal NUMERIC_MIN_VALUE =
+      bigDecimalAsNumeric("-99999999999999999999999999999.999999999");
+
   private ZetaSqlTypesUtils() {}
 
   /**
diff --git a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/data/BeamFnDataSizeBasedBufferingOutboundObserver.java b/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/data/BeamFnDataSizeBasedBufferingOutboundObserver.java
index 1cd9772..f0e16f4 100644
--- a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/data/BeamFnDataSizeBasedBufferingOutboundObserver.java
+++ b/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/data/BeamFnDataSizeBasedBufferingOutboundObserver.java
@@ -34,6 +34,7 @@
  * <p>TODO: Handle outputting elements that are zero bytes by outputting a single byte as a marker,
  * detect on the input side that no bytes were read and force reading a single byte.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BeamFnDataSizeBasedBufferingOutboundObserver<T>
     implements BeamFnDataBufferingOutboundObserver<T> {
   private static final Logger LOG =
diff --git a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/data/BeamFnDataTimeBasedBufferingOutboundObserver.java b/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/data/BeamFnDataTimeBasedBufferingOutboundObserver.java
index b339e33..77cbc3b 100644
--- a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/data/BeamFnDataTimeBasedBufferingOutboundObserver.java
+++ b/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/data/BeamFnDataTimeBasedBufferingOutboundObserver.java
@@ -33,6 +33,7 @@
  * A buffering outbound {@link FnDataReceiver} with both size-based buffer and time-based buffer
  * enabled for the Beam Fn Data API.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BeamFnDataTimeBasedBufferingOutboundObserver<T>
     extends BeamFnDataSizeBasedBufferingOutboundObserver<T> {
 
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 533c7aa..f3dcbd9 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
@@ -23,6 +23,7 @@
 import org.apache.beam.sdk.transforms.splittabledofn.SplitResult;
 
 /** Support utilities for interacting with {@link RestrictionTracker RestrictionTrackers}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class RestrictionTrackers {
 
   /** Interface allowing a runner to observe the calls to {@link RestrictionTracker#tryClaim}. */
diff --git a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/stream/BufferingStreamObserver.java b/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/stream/BufferingStreamObserver.java
index 12f042d..e182740 100644
--- a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/stream/BufferingStreamObserver.java
+++ b/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/stream/BufferingStreamObserver.java
@@ -41,6 +41,7 @@
  * becomes ready.
  */
 @ThreadSafe
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public final class BufferingStreamObserver<T> implements StreamObserver<T> {
   private static final Object POISON_PILL = new Object();
   private final LinkedBlockingDeque<T> queue;
diff --git a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/stream/DataStreams.java b/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/stream/DataStreams.java
index 140f508..cdb9325 100644
--- a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/stream/DataStreams.java
+++ b/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/stream/DataStreams.java
@@ -36,6 +36,7 @@
  * {@link #outbound(OutputChunkConsumer)} treats a single {@link OutputStream} as multiple {@link
  * ByteString}s.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DataStreams {
   public static final int DEFAULT_OUTBOUND_BUFFER_LIMIT_BYTES = 1_000_000;
 
diff --git a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/test/TestExecutors.java b/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/test/TestExecutors.java
index 4e46390..b453807 100644
--- a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/test/TestExecutors.java
+++ b/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/test/TestExecutors.java
@@ -29,6 +29,7 @@
  * A {@link TestRule} that validates that all submitted tasks finished and were completed. This
  * allows for testing that tasks have exercised the appropriate shutdown logic.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class TestExecutors {
   public static TestExecutorService from(final ExecutorService staticExecutorService) {
     return from(() -> staticExecutorService);
diff --git a/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/JvmInitializersTest.java b/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/JvmInitializersTest.java
index a20b3a9..a1780a3 100644
--- a/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/JvmInitializersTest.java
+++ b/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/JvmInitializersTest.java
@@ -31,6 +31,7 @@
 
 /** Tests for {@link JvmInitializers}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public final class JvmInitializersTest {
 
   private static Boolean onStartupRan;
diff --git a/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/data/BeamFnDataGrpcMultiplexerTest.java b/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/data/BeamFnDataGrpcMultiplexerTest.java
index 56109eb..ac850f4 100644
--- a/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/data/BeamFnDataGrpcMultiplexerTest.java
+++ b/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/data/BeamFnDataGrpcMultiplexerTest.java
@@ -37,6 +37,7 @@
 import org.junit.Test;
 
 /** Tests for {@link BeamFnDataGrpcMultiplexer}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BeamFnDataGrpcMultiplexerTest {
   private static final Endpoints.ApiServiceDescriptor DESCRIPTOR =
       Endpoints.ApiServiceDescriptor.newBuilder().setUrl("test").build();
diff --git a/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/data/BeamFnDataSizeBasedBufferingOutboundObserverTest.java b/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/data/BeamFnDataSizeBasedBufferingOutboundObserverTest.java
index ff8d499..c71d472 100644
--- a/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/data/BeamFnDataSizeBasedBufferingOutboundObserverTest.java
+++ b/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/data/BeamFnDataSizeBasedBufferingOutboundObserverTest.java
@@ -46,6 +46,7 @@
 
 /** Tests for {@link BeamFnDataSizeBasedBufferingOutboundObserver}. */
 @RunWith(Parameterized.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BeamFnDataSizeBasedBufferingOutboundObserverTest {
   private static final LogicalEndpoint DATA_OUTPUT_LOCATION = LogicalEndpoint.data("777L", "555L");
   private static final LogicalEndpoint TIMER_OUTPUT_LOCATION =
diff --git a/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/data/BeamFnDataTimeBasedBufferingOutboundObserverTest.java b/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/data/BeamFnDataTimeBasedBufferingOutboundObserverTest.java
index 48ee753..0ff770a 100644
--- a/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/data/BeamFnDataTimeBasedBufferingOutboundObserverTest.java
+++ b/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/data/BeamFnDataTimeBasedBufferingOutboundObserverTest.java
@@ -44,6 +44,7 @@
 
 /** Tests for {@link BeamFnDataTimeBasedBufferingOutboundObserver}. */
 @RunWith(Parameterized.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BeamFnDataTimeBasedBufferingOutboundObserverTest {
   private static final LogicalEndpoint DATA_OUTPUT_LOCATION = LogicalEndpoint.data("777L", "555L");
   private static final LogicalEndpoint TIMER_OUTPUT_LOCATION =
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 4b23395..bdcedd5 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
@@ -34,6 +34,7 @@
 
 /** Tests for {@link RestrictionTrackers}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class RestrictionTrackersTest {
   @Test
   public void testObservingClaims() {
diff --git a/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/stream/DataStreamsTest.java b/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/stream/DataStreamsTest.java
index 8dd5819..06168aa 100644
--- a/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/stream/DataStreamsTest.java
+++ b/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/stream/DataStreamsTest.java
@@ -53,6 +53,7 @@
 import org.junit.runners.JUnit4;
 
 /** Tests for {@link DataStreams}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DataStreamsTest {
 
   /** Tests for {@link DataStreams.Inbound}. */
diff --git a/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/stream/OutboundObserverFactoryTest.java b/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/stream/OutboundObserverFactoryTest.java
index de56d01..0a23b2d 100644
--- a/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/stream/OutboundObserverFactoryTest.java
+++ b/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/stream/OutboundObserverFactoryTest.java
@@ -33,6 +33,7 @@
 
 /** Tests for {@link OutboundObserverFactory}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class OutboundObserverFactoryTest {
   @Mock private StreamObserver<Integer> mockRequestObserver;
   @Mock private CallStreamObserver<String> mockResponseObserver;
diff --git a/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/test/TestExecutorsTest.java b/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/test/TestExecutorsTest.java
index 11c4425..9e18f0d 100644
--- a/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/test/TestExecutorsTest.java
+++ b/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/test/TestExecutorsTest.java
@@ -33,6 +33,7 @@
 
 /** Tests for {@link TestExecutors}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class TestExecutorsTest {
   @Test
   public void testSuccessfulTermination() throws Throwable {
diff --git a/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/test/TestStreamsTest.java b/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/test/TestStreamsTest.java
index 9fa8de1..88ab871 100644
--- a/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/test/TestStreamsTest.java
+++ b/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/test/TestStreamsTest.java
@@ -31,6 +31,7 @@
 
 /** Tests for {@link TestStreams}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class TestStreamsTest {
   @Test
   public void testOnNextIsCalled() {
diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/BeamFnDataReadRunner.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/BeamFnDataReadRunner.java
index aebcdb6..351b5cf 100644
--- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/BeamFnDataReadRunner.java
+++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/BeamFnDataReadRunner.java
@@ -70,6 +70,7 @@
  * <p>Can be re-used serially across {@link BeamFnApi.ProcessBundleRequest}s. For each request, call
  * {@link #registerInputLocation()} to start and call {@link #blockTillReadFinishes()} to finish.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BeamFnDataReadRunner<OutputT> {
 
   private static final Logger LOG = LoggerFactory.getLogger(BeamFnDataReadRunner.class);
diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/BeamFnDataWriteRunner.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/BeamFnDataWriteRunner.java
index 0de2d3f..0cb8419 100644
--- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/BeamFnDataWriteRunner.java
+++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/BeamFnDataWriteRunner.java
@@ -60,6 +60,7 @@
  * <p>Can be re-used serially across {@link BeamFnApi.ProcessBundleRequest}s. For each request, call
  * {@link #registerForOutput()} to start and call {@link #close()} to finish.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BeamFnDataWriteRunner<InputT> {
 
   private static final Logger LOG = LoggerFactory.getLogger(BeamFnDataWriteRunner.class);
diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/CombineRunners.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/CombineRunners.java
index e32ff38..a1f16b1 100644
--- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/CombineRunners.java
+++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/CombineRunners.java
@@ -51,6 +51,7 @@
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables;
 
 /** Executes different components of Combine PTransforms. */
+@SuppressWarnings({"nullness", "keyfor"}) // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class CombineRunners {
 
   /** A registrar which provides a factory to handle combine component PTransforms. */
diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnApiDoFnRunner.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnApiDoFnRunner.java
index f77d8d6..93d2fed 100644
--- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnApiDoFnRunner.java
+++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnApiDoFnRunner.java
@@ -131,6 +131,7 @@
  * abstraction caused by StateInternals/TimerInternals since they model state and timer concepts
  * differently.
  */
+@SuppressWarnings({"nullness", "keyfor"}) // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class FnApiDoFnRunner<InputT, RestrictionT, PositionT, WatermarkEstimatorStateT, OutputT> {
   /** A registrar which provides a factory to handle Java {@link DoFn}s. */
   @AutoService(PTransformRunnerFactory.Registrar.class)
diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java
index bdf3096..cb55eb1 100644
--- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java
+++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java
@@ -73,6 +73,7 @@
  *       for further details.
  * </ul>
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class FnHarness {
   private static final String HARNESS_ID = "HARNESS_ID";
   private static final String CONTROL_API_SERVICE_DESCRIPTOR = "CONTROL_API_SERVICE_DESCRIPTOR";
diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/PrecombineGroupingTable.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/PrecombineGroupingTable.java
index 515f064..dc87956 100644
--- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/PrecombineGroupingTable.java
+++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/PrecombineGroupingTable.java
@@ -39,6 +39,7 @@
 import org.joda.time.Instant;
 
 /** Static utility methods that provide {@link GroupingTable} implementations. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class PrecombineGroupingTable<K, InputT, AccumT>
     implements GroupingTable<K, InputT, AccumT> {
   private static long getGroupingTableSizeBytes(PipelineOptions options) {
diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/FinalizeBundleHandler.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/FinalizeBundleHandler.java
index 5434b51..4c95a8e 100644
--- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/FinalizeBundleHandler.java
+++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/FinalizeBundleHandler.java
@@ -47,6 +47,7 @@
  * <p>See <a href="https://s.apache.org/beam-finalizing-bundles">Apache Beam Portability API: How to
  * Finalize Bundles</a> for further details.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class FinalizeBundleHandler {
 
   /** A {@link BundleFinalizer.Callback} and expiry time pair. */
diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ProcessBundleHandler.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ProcessBundleHandler.java
index a664ea2..81ee4b1 100644
--- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ProcessBundleHandler.java
+++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ProcessBundleHandler.java
@@ -101,6 +101,7 @@
  * perform a split request. See <a href="https://s.apache.org/beam-breaking-fusion">breaking the
  * fusion barrier</a> for further details.
  */
+@SuppressWarnings({"nullness", "keyfor"}) // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ProcessBundleHandler {
 
   // TODO: What should the initial set of URNs be?
diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/data/PCollectionConsumerRegistry.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/data/PCollectionConsumerRegistry.java
index 3276c46..6ceed4e 100644
--- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/data/PCollectionConsumerRegistry.java
+++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/data/PCollectionConsumerRegistry.java
@@ -50,6 +50,7 @@
  * count counter for every pCollection. A combined MultiplexingConsumer (Wrapped with an
  * ElementCountFnDataReceiver) is returned by calling getMultiplexingConsumer.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class PCollectionConsumerRegistry {
 
   /** Stores metadata about each consumer so that the appropriate metrics tracking can occur. */
diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/data/QueueingBeamFnDataClient.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/data/QueueingBeamFnDataClient.java
index 8fae554..d371a0d 100644
--- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/data/QueueingBeamFnDataClient.java
+++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/data/QueueingBeamFnDataClient.java
@@ -35,6 +35,7 @@
  * A {@link BeamFnDataClient} that queues elements so that they can be consumed and processed in the
  * thread which calls @{link #drainAndBlock}.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class QueueingBeamFnDataClient implements BeamFnDataClient {
 
   private static final int QUEUE_SIZE = 1000;
diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/logging/BeamFnLoggingClient.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/logging/BeamFnLoggingClient.java
index e17f5eb..1323c95 100644
--- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/logging/BeamFnLoggingClient.java
+++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/logging/BeamFnLoggingClient.java
@@ -58,6 +58,7 @@
 /**
  * Configures {@link java.util.logging} to send all {@link LogRecord}s via the Beam Fn Logging API.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BeamFnLoggingClient implements AutoCloseable {
   private static final String ROOT_LOGGER_NAME = "";
   private static final ImmutableMap<Level, BeamFnApi.LogEntry.Severity.Enum> LOG_LEVEL_MAP =
diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/BagUserState.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/BagUserState.java
index cb3c195..7bf3ee9 100644
--- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/BagUserState.java
+++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/BagUserState.java
@@ -42,6 +42,7 @@
  *
  * <p>TODO: Support block level caching and prefetch.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BagUserState<T> {
   private final BeamFnStateClient beamFnStateClient;
   private final StateRequest request;
diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/BeamFnStateGrpcClientCache.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/BeamFnStateGrpcClientCache.java
index 313cf62..254cdc4 100644
--- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/BeamFnStateGrpcClientCache.java
+++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/BeamFnStateGrpcClientCache.java
@@ -40,6 +40,7 @@
  *
  * <p>TODO: Add the ability to close which cancels any pending and stops any future requests.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BeamFnStateGrpcClientCache {
   private static final Logger LOG = LoggerFactory.getLogger(BeamFnStateGrpcClientCache.class);
 
diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/FnApiStateAccessor.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/FnApiStateAccessor.java
index d6b4a90..6e40e96 100644
--- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/FnApiStateAccessor.java
+++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/FnApiStateAccessor.java
@@ -59,6 +59,7 @@
 import org.checkerframework.checker.nullness.qual.Nullable;
 
 /** Provides access to side inputs and state via a {@link BeamFnStateClient}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class FnApiStateAccessor<K> implements SideInputReader, StateBinder {
   private final PipelineOptions pipelineOptions;
   private final Map<StateKey, Object> stateKeyObjectCache;
@@ -287,9 +288,8 @@
                   @Override
                   public ReadableState<Boolean> isEmpty() {
                     return new ReadableState<Boolean>() {
-                      @Nullable
                       @Override
-                      public Boolean read() {
+                      public @Nullable Boolean read() {
                         return !impl.get().iterator().hasNext();
                       }
 
diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/LazyCachingIteratorToIterable.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/LazyCachingIteratorToIterable.java
index e01a695..d954385 100644
--- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/LazyCachingIteratorToIterable.java
+++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/LazyCachingIteratorToIterable.java
@@ -28,6 +28,7 @@
  * Converts an iterator to an iterable lazily loading values from the underlying iterator and
  * caching them to support reiteration.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class LazyCachingIteratorToIterable<T> implements Iterable<T> {
   private final List<T> cachedElements;
   private final Iterator<T> iterator;
diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/MultimapSideInput.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/MultimapSideInput.java
index 7955610..feae294 100644
--- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/MultimapSideInput.java
+++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/MultimapSideInput.java
@@ -29,6 +29,7 @@
  *
  * <p>TODO: Support block level caching and prefetch.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class MultimapSideInput<K, V> implements MultimapView<K, V> {
 
   private final BeamFnStateClient beamFnStateClient;
diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/StateFetchingIterators.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/StateFetchingIterators.java
index 7af431d..8a8b37f 100644
--- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/StateFetchingIterators.java
+++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/StateFetchingIterators.java
@@ -31,6 +31,7 @@
  * Adapters which convert a a logical series of chunks using continuation tokens over the Beam Fn
  * State API into an {@link Iterator} of {@link ByteString}s.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class StateFetchingIterators {
 
   // do not instantiate
diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/BeamFnDataReadRunnerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/BeamFnDataReadRunnerTest.java
index ed46188..a80a34e 100644
--- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/BeamFnDataReadRunnerTest.java
+++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/BeamFnDataReadRunnerTest.java
@@ -95,6 +95,7 @@
 
 /** Tests for {@link BeamFnDataReadRunner}. */
 @RunWith(Enclosed.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BeamFnDataReadRunnerTest {
   private static final Coder<String> ELEMENT_CODER = StringUtf8Coder.of();
   private static final String ELEMENT_CODER_SPEC_ID = "string-coder-id";
diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/BeamFnDataWriteRunnerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/BeamFnDataWriteRunnerTest.java
index c59e97d..4cba8ae 100644
--- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/BeamFnDataWriteRunnerTest.java
+++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/BeamFnDataWriteRunnerTest.java
@@ -73,6 +73,7 @@
 
 /** Tests for {@link BeamFnDataWriteRunner}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BeamFnDataWriteRunnerTest {
 
   private static final String ELEM_CODER_ID = "string-coder-id";
diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/CombineRunnersTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/CombineRunnersTest.java
index 90457fa..54ae8e3 100644
--- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/CombineRunnersTest.java
+++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/CombineRunnersTest.java
@@ -56,6 +56,7 @@
 
 /** Tests for {@link CombineRunners}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class CombineRunnersTest {
   // CombineFn that converts strings to ints and sums them up to an accumulator, and negates the
   // value of the accumulator when extracting outputs. These operations are chosen to avoid
diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FlattenRunnerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FlattenRunnerTest.java
index f9f093d..40d6776 100644
--- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FlattenRunnerTest.java
+++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FlattenRunnerTest.java
@@ -42,6 +42,7 @@
 
 /** Tests for {@link FlattenRunner}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class FlattenRunnerTest {
 
   /**
diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FnApiDoFnRunnerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FnApiDoFnRunnerTest.java
index 27a7a76..586e1be 100644
--- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FnApiDoFnRunnerTest.java
+++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FnApiDoFnRunnerTest.java
@@ -158,6 +158,7 @@
 
 /** Tests for {@link FnApiDoFnRunner}. */
 @RunWith(Enclosed.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class FnApiDoFnRunnerTest implements Serializable {
 
   @RunWith(JUnit4.class)
diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/MapFnRunnersTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/MapFnRunnersTest.java
index 559a71c..269074a 100644
--- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/MapFnRunnersTest.java
+++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/MapFnRunnersTest.java
@@ -52,6 +52,7 @@
 
 /** Tests for {@link MapFnRunners}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class MapFnRunnersTest {
   private static final String EXPECTED_ID = "pTransformId";
   private static final RunnerApi.PTransform EXPECTED_PTRANSFORM =
diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/PrecombineGroupingTableTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/PrecombineGroupingTableTest.java
index 0b1c871..c854d3d 100644
--- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/PrecombineGroupingTableTest.java
+++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/PrecombineGroupingTableTest.java
@@ -44,6 +44,7 @@
 
 /** Unit tests for {@link PrecombineGroupingTable}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class PrecombineGroupingTableTest {
 
   private static class TestOutputReceiver implements Receiver {
diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/control/BeamFnControlClientTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/control/BeamFnControlClientTest.java
index 5adb001..efdf026 100644
--- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/control/BeamFnControlClientTest.java
+++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/control/BeamFnControlClientTest.java
@@ -54,6 +54,7 @@
 
 /** Tests for {@link BeamFnControlClient}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BeamFnControlClientTest {
   private static final BeamFnApi.InstructionRequest SUCCESSFUL_REQUEST =
       BeamFnApi.InstructionRequest.newBuilder()
diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/control/FinalizeBundleHandlerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/control/FinalizeBundleHandlerTest.java
index a760d22..4e86d2b 100644
--- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/control/FinalizeBundleHandlerTest.java
+++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/control/FinalizeBundleHandlerTest.java
@@ -40,6 +40,7 @@
 
 /** Tests for {@link FinalizeBundleHandler}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class FinalizeBundleHandlerTest {
   private static final String INSTRUCTION_ID = "instructionId";
   private static final InstructionResponse SUCCESSFUL_RESPONSE =
diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/control/ProcessBundleHandlerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/control/ProcessBundleHandlerTest.java
index 4ff83d1..cc81c0f 100644
--- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/control/ProcessBundleHandlerTest.java
+++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/control/ProcessBundleHandlerTest.java
@@ -111,6 +111,7 @@
 
 /** Tests for {@link ProcessBundleHandler}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ProcessBundleHandlerTest {
   private static final String DATA_INPUT_URN = "beam:runner:source:v1";
   private static final String DATA_OUTPUT_URN = "beam:runner:sink:v1";
diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/data/BeamFnDataGrpcClientTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/data/BeamFnDataGrpcClientTest.java
index e1ce20e..8a86b3d 100644
--- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/data/BeamFnDataGrpcClientTest.java
+++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/data/BeamFnDataGrpcClientTest.java
@@ -59,6 +59,7 @@
 
 /** Tests for {@link BeamFnDataGrpcClient}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BeamFnDataGrpcClientTest {
   private static final Coder<WindowedValue<String>> CODER =
       LengthPrefixCoder.of(
diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/data/FakeBeamFnTimerClient.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/data/FakeBeamFnTimerClient.java
index e025f03..d6dc49d 100644
--- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/data/FakeBeamFnTimerClient.java
+++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/data/FakeBeamFnTimerClient.java
@@ -28,6 +28,7 @@
 import org.apache.beam.sdk.fn.data.LogicalEndpoint;
 
 /** An implementation of a {@link BeamFnTimerClient} that can be used for testing. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class FakeBeamFnTimerClient implements BeamFnTimerClient {
   private final ConcurrentMap<LogicalEndpoint, TimerHandler<?>> timerHandlers;
   private final ConcurrentMap<LogicalEndpoint, List<Timer<?>>> setTimers;
diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/data/PCollectionConsumerRegistryTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/data/PCollectionConsumerRegistryTest.java
index 0efdd42..4d68a37 100644
--- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/data/PCollectionConsumerRegistryTest.java
+++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/data/PCollectionConsumerRegistryTest.java
@@ -52,6 +52,7 @@
 /** Tests for {@link PCollectionConsumerRegistryTest}. */
 @RunWith(PowerMockRunner.class)
 @PrepareForTest(MetricsEnvironment.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class PCollectionConsumerRegistryTest {
 
   @Rule public ExpectedException expectedException = ExpectedException.none();
diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/logging/BeamFnLoggingClientTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/logging/BeamFnLoggingClientTest.java
index dc49275..6ed8cd8 100644
--- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/logging/BeamFnLoggingClientTest.java
+++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/logging/BeamFnLoggingClientTest.java
@@ -53,6 +53,7 @@
 
 /** Tests for {@link BeamFnLoggingClient}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BeamFnLoggingClientTest {
 
   private static final LogRecord FILTERED_RECORD;
diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/state/BagUserStateTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/state/BagUserStateTest.java
index dbf9885..f5dd0f1 100644
--- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/state/BagUserStateTest.java
+++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/state/BagUserStateTest.java
@@ -36,6 +36,7 @@
 
 /** Tests for {@link BagUserState}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BagUserStateTest {
   @Rule public ExpectedException thrown = ExpectedException.none();
 
diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/state/BeamFnStateGrpcClientCacheTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/state/BeamFnStateGrpcClientCacheTest.java
index e8f1780..0bf7c2d 100644
--- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/state/BeamFnStateGrpcClientCacheTest.java
+++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/state/BeamFnStateGrpcClientCacheTest.java
@@ -53,6 +53,7 @@
 
 /** Tests for {@link BeamFnStateGrpcClientCache}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BeamFnStateGrpcClientCacheTest {
   private static final String SUCCESS = "SUCCESS";
   private static final String FAIL = "FAIL";
diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/state/StateBackedIterableTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/state/StateBackedIterableTest.java
index fc6549b..f27fdde 100644
--- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/state/StateBackedIterableTest.java
+++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/state/StateBackedIterableTest.java
@@ -43,6 +43,7 @@
 
 /** Tests for {@link StateBackedIterable}. */
 @RunWith(Enclosed.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class StateBackedIterableTest {
 
   @RunWith(Parameterized.class)
diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/stream/HarnessStreamObserverFactoriesTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/stream/HarnessStreamObserverFactoriesTest.java
index 76294ee..ea63b5e 100644
--- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/stream/HarnessStreamObserverFactoriesTest.java
+++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/stream/HarnessStreamObserverFactoriesTest.java
@@ -36,6 +36,7 @@
 
 /** Tests for {@link HarnessStreamObserverFactoriesTest}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class HarnessStreamObserverFactoriesTest {
   @Mock private StreamObserver<Integer> mockRequestObserver;
   @Mock private CallStreamObserver<String> mockResponseObserver;
diff --git a/sdks/java/io/amazon-web-services/build.gradle b/sdks/java/io/amazon-web-services/build.gradle
index e523d0f..87afdeb 100644
--- a/sdks/java/io/amazon-web-services/build.gradle
+++ b/sdks/java/io/amazon-web-services/build.gradle
@@ -56,7 +56,7 @@
   testCompile library.java.junit
   testCompile library.java.testcontainers_localstack
   testCompile "org.assertj:assertj-core:3.11.1"
-  testCompile 'org.elasticmq:elasticmq-rest-sqs_2.12:0.14.1'
+  testCompile 'org.elasticmq:elasticmq-rest-sqs_2.12:0.15.6'
   testRuntimeOnly library.java.slf4j_jdk14
   testRuntimeOnly project(path: ":runners:direct-java", configuration: "shadow")
 }
diff --git a/sdks/java/io/amazon-web-services/src/main/java/org/apache/beam/sdk/io/aws/coders/AwsCoders.java b/sdks/java/io/amazon-web-services/src/main/java/org/apache/beam/sdk/io/aws/coders/AwsCoders.java
index 01b54ce..aa33167 100644
--- a/sdks/java/io/amazon-web-services/src/main/java/org/apache/beam/sdk/io/aws/coders/AwsCoders.java
+++ b/sdks/java/io/amazon-web-services/src/main/java/org/apache/beam/sdk/io/aws/coders/AwsCoders.java
@@ -36,6 +36,7 @@
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
 
 /** {@link Coder}s for common AWS SDK objects. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public final class AwsCoders {
 
   private AwsCoders() {}
diff --git a/sdks/java/io/amazon-web-services/src/main/java/org/apache/beam/sdk/io/aws/dynamodb/DynamoDBIO.java b/sdks/java/io/amazon-web-services/src/main/java/org/apache/beam/sdk/io/aws/dynamodb/DynamoDBIO.java
index f25d6ba..8c8e868 100644
--- a/sdks/java/io/amazon-web-services/src/main/java/org/apache/beam/sdk/io/aws/dynamodb/DynamoDBIO.java
+++ b/sdks/java/io/amazon-web-services/src/main/java/org/apache/beam/sdk/io/aws/dynamodb/DynamoDBIO.java
@@ -121,6 +121,7 @@
  * </ul>
  */
 @Experimental(Kind.SOURCE_SINK)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public final class DynamoDBIO {
   public static <T> Read<T> read() {
     return new AutoValue_DynamoDBIO_Read.Builder().build();
diff --git a/sdks/java/io/amazon-web-services/src/main/java/org/apache/beam/sdk/io/aws/options/AwsModule.java b/sdks/java/io/amazon-web-services/src/main/java/org/apache/beam/sdk/io/aws/options/AwsModule.java
index 2416d30..b71b895 100644
--- a/sdks/java/io/amazon-web-services/src/main/java/org/apache/beam/sdk/io/aws/options/AwsModule.java
+++ b/sdks/java/io/amazon-web-services/src/main/java/org/apache/beam/sdk/io/aws/options/AwsModule.java
@@ -61,6 +61,7 @@
  * SSEAwsKeyManagementParams}.
  */
 @AutoService(Module.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class AwsModule extends SimpleModule {
 
   private static final String AWS_ACCESS_KEY_ID = "awsAccessKeyId";
diff --git a/sdks/java/io/amazon-web-services/src/main/java/org/apache/beam/sdk/io/aws/s3/S3FileSystem.java b/sdks/java/io/amazon-web-services/src/main/java/org/apache/beam/sdk/io/aws/s3/S3FileSystem.java
index a09e3e9..255729e 100644
--- a/sdks/java/io/amazon-web-services/src/main/java/org/apache/beam/sdk/io/aws/s3/S3FileSystem.java
+++ b/sdks/java/io/amazon-web-services/src/main/java/org/apache/beam/sdk/io/aws/s3/S3FileSystem.java
@@ -85,6 +85,7 @@
 import org.slf4j.LoggerFactory;
 
 /** {@link FileSystem} implementation for Amazon S3. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class S3FileSystem extends FileSystem<S3ResourceId> {
 
   private static final Logger LOG = LoggerFactory.getLogger(S3FileSystem.class);
diff --git a/sdks/java/io/amazon-web-services/src/main/java/org/apache/beam/sdk/io/aws/s3/S3ReadableSeekableByteChannel.java b/sdks/java/io/amazon-web-services/src/main/java/org/apache/beam/sdk/io/aws/s3/S3ReadableSeekableByteChannel.java
index 44a7820..a40177d 100644
--- a/sdks/java/io/amazon-web-services/src/main/java/org/apache/beam/sdk/io/aws/s3/S3ReadableSeekableByteChannel.java
+++ b/sdks/java/io/amazon-web-services/src/main/java/org/apache/beam/sdk/io/aws/s3/S3ReadableSeekableByteChannel.java
@@ -35,6 +35,7 @@
 import org.apache.beam.sdk.io.aws.options.S3Options;
 
 /** A readable S3 object, as a {@link SeekableByteChannel}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class S3ReadableSeekableByteChannel implements SeekableByteChannel {
 
   private final AmazonS3 amazonS3;
diff --git a/sdks/java/io/amazon-web-services/src/main/java/org/apache/beam/sdk/io/aws/s3/S3ResourceId.java b/sdks/java/io/amazon-web-services/src/main/java/org/apache/beam/sdk/io/aws/s3/S3ResourceId.java
index 60be93e..867c858 100644
--- a/sdks/java/io/amazon-web-services/src/main/java/org/apache/beam/sdk/io/aws/s3/S3ResourceId.java
+++ b/sdks/java/io/amazon-web-services/src/main/java/org/apache/beam/sdk/io/aws/s3/S3ResourceId.java
@@ -32,6 +32,7 @@
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Strings;
 import org.checkerframework.checker.nullness.qual.Nullable;
 
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class S3ResourceId implements ResourceId {
 
   static final String SCHEME = "s3";
@@ -154,9 +155,8 @@
     return SCHEME;
   }
 
-  @Nullable
   @Override
-  public String getFilename() {
+  public @Nullable String getFilename() {
     if (!isDirectory()) {
       return key.substring(key.lastIndexOf('/') + 1);
     }
diff --git a/sdks/java/io/amazon-web-services/src/main/java/org/apache/beam/sdk/io/aws/s3/S3WritableByteChannel.java b/sdks/java/io/amazon-web-services/src/main/java/org/apache/beam/sdk/io/aws/s3/S3WritableByteChannel.java
index ac0bb3a..4667989 100644
--- a/sdks/java/io/amazon-web-services/src/main/java/org/apache/beam/sdk/io/aws/s3/S3WritableByteChannel.java
+++ b/sdks/java/io/amazon-web-services/src/main/java/org/apache/beam/sdk/io/aws/s3/S3WritableByteChannel.java
@@ -44,6 +44,7 @@
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
 
 /** A writable S3 object, as a {@link WritableByteChannel}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class S3WritableByteChannel implements WritableByteChannel {
   private final AmazonS3 amazonS3;
   private final S3Options options;
diff --git a/sdks/java/io/amazon-web-services/src/main/java/org/apache/beam/sdk/io/aws/sns/PublishResultCoders.java b/sdks/java/io/amazon-web-services/src/main/java/org/apache/beam/sdk/io/aws/sns/PublishResultCoders.java
index 297c1c8..4cd9772 100644
--- a/sdks/java/io/amazon-web-services/src/main/java/org/apache/beam/sdk/io/aws/sns/PublishResultCoders.java
+++ b/sdks/java/io/amazon-web-services/src/main/java/org/apache/beam/sdk/io/aws/sns/PublishResultCoders.java
@@ -31,6 +31,7 @@
 import org.apache.beam.sdk.io.aws.coders.AwsCoders;
 
 /** Coders for SNS {@link PublishResult}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public final class PublishResultCoders {
 
   private static final Coder<String> MESSAGE_ID_CODER = StringUtf8Coder.of();
diff --git a/sdks/java/io/amazon-web-services/src/main/java/org/apache/beam/sdk/io/aws/sns/SnsIO.java b/sdks/java/io/amazon-web-services/src/main/java/org/apache/beam/sdk/io/aws/sns/SnsIO.java
index a7dfeca..72fc97f 100644
--- a/sdks/java/io/amazon-web-services/src/main/java/org/apache/beam/sdk/io/aws/sns/SnsIO.java
+++ b/sdks/java/io/amazon-web-services/src/main/java/org/apache/beam/sdk/io/aws/sns/SnsIO.java
@@ -87,6 +87,7 @@
  * can call {@link Write#withFullPublishResultWithoutHeaders}.
  */
 @Experimental(Kind.SOURCE_SINK)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public final class SnsIO {
 
   // Write data tp SNS
diff --git a/sdks/java/io/amazon-web-services/src/main/java/org/apache/beam/sdk/io/aws/sqs/SqsConfiguration.java b/sdks/java/io/amazon-web-services/src/main/java/org/apache/beam/sdk/io/aws/sqs/SqsConfiguration.java
index f1128b9..00df099 100644
--- a/sdks/java/io/amazon-web-services/src/main/java/org/apache/beam/sdk/io/aws/sqs/SqsConfiguration.java
+++ b/sdks/java/io/amazon-web-services/src/main/java/org/apache/beam/sdk/io/aws/sqs/SqsConfiguration.java
@@ -26,6 +26,7 @@
 import org.apache.beam.sdk.io.aws.options.AwsModule;
 import org.apache.beam.sdk.io.aws.options.AwsOptions;
 
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class SqsConfiguration implements Serializable {
 
   private String awsRegion;
diff --git a/sdks/java/io/amazon-web-services/src/main/java/org/apache/beam/sdk/io/aws/sqs/SqsIO.java b/sdks/java/io/amazon-web-services/src/main/java/org/apache/beam/sdk/io/aws/sqs/SqsIO.java
index 08989b8..250a3ac 100644
--- a/sdks/java/io/amazon-web-services/src/main/java/org/apache/beam/sdk/io/aws/sqs/SqsIO.java
+++ b/sdks/java/io/amazon-web-services/src/main/java/org/apache/beam/sdk/io/aws/sqs/SqsIO.java
@@ -78,6 +78,7 @@
  * <p>For more information on the available options see {@link AwsOptions}.
  */
 @Experimental(Kind.SOURCE_SINK)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SqsIO {
 
   public static Read read() {
diff --git a/sdks/java/io/amazon-web-services/src/main/java/org/apache/beam/sdk/io/aws/sqs/SqsUnboundedReader.java b/sdks/java/io/amazon-web-services/src/main/java/org/apache/beam/sdk/io/aws/sqs/SqsUnboundedReader.java
index 29d2106..cb07ae0 100644
--- a/sdks/java/io/amazon-web-services/src/main/java/org/apache/beam/sdk/io/aws/sqs/SqsUnboundedReader.java
+++ b/sdks/java/io/amazon-web-services/src/main/java/org/apache/beam/sdk/io/aws/sqs/SqsUnboundedReader.java
@@ -35,6 +35,7 @@
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
 import org.joda.time.Instant;
 
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class SqsUnboundedReader extends UnboundedSource.UnboundedReader<Message> implements Serializable {
 
   public static final int MAX_NUMBER_OF_MESSAGES = 10;
diff --git a/sdks/java/io/amazon-web-services/src/main/java/org/apache/beam/sdk/io/aws/sqs/SqsUnboundedSource.java b/sdks/java/io/amazon-web-services/src/main/java/org/apache/beam/sdk/io/aws/sqs/SqsUnboundedSource.java
index dbfe490..75ac6fd 100644
--- a/sdks/java/io/amazon-web-services/src/main/java/org/apache/beam/sdk/io/aws/sqs/SqsUnboundedSource.java
+++ b/sdks/java/io/amazon-web-services/src/main/java/org/apache/beam/sdk/io/aws/sqs/SqsUnboundedSource.java
@@ -32,6 +32,7 @@
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Suppliers;
 import org.checkerframework.checker.nullness.qual.Nullable;
 
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class SqsUnboundedSource extends UnboundedSource<Message, SqsCheckpointMark> {
 
   private final Read read;
diff --git a/sdks/java/io/amazon-web-services/src/test/java/org/apache/beam/sdk/io/aws/coders/AwsCodersTest.java b/sdks/java/io/amazon-web-services/src/test/java/org/apache/beam/sdk/io/aws/coders/AwsCodersTest.java
index 2baaa21..62fde8e 100644
--- a/sdks/java/io/amazon-web-services/src/test/java/org/apache/beam/sdk/io/aws/coders/AwsCodersTest.java
+++ b/sdks/java/io/amazon-web-services/src/test/java/org/apache/beam/sdk/io/aws/coders/AwsCodersTest.java
@@ -29,6 +29,7 @@
 import org.junit.Test;
 
 /** Tests for AWS coders. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class AwsCodersTest {
 
   @Test
diff --git a/sdks/java/io/amazon-web-services/src/test/java/org/apache/beam/sdk/io/aws/dynamodb/AwsClientsProviderMock.java b/sdks/java/io/amazon-web-services/src/test/java/org/apache/beam/sdk/io/aws/dynamodb/AwsClientsProviderMock.java
index dfcf302..924a49f 100644
--- a/sdks/java/io/amazon-web-services/src/test/java/org/apache/beam/sdk/io/aws/dynamodb/AwsClientsProviderMock.java
+++ b/sdks/java/io/amazon-web-services/src/test/java/org/apache/beam/sdk/io/aws/dynamodb/AwsClientsProviderMock.java
@@ -22,6 +22,7 @@
 import org.mockito.Mockito;
 
 /** Mocking AwsClientProvider. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class AwsClientsProviderMock implements AwsClientsProvider {
 
   private static AwsClientsProviderMock instance = new AwsClientsProviderMock();
diff --git a/sdks/java/io/amazon-web-services/src/test/java/org/apache/beam/sdk/io/aws/dynamodb/DynamoDBIOTest.java b/sdks/java/io/amazon-web-services/src/test/java/org/apache/beam/sdk/io/aws/dynamodb/DynamoDBIOTest.java
index 7491536..85b7bc6 100644
--- a/sdks/java/io/amazon-web-services/src/test/java/org/apache/beam/sdk/io/aws/dynamodb/DynamoDBIOTest.java
+++ b/sdks/java/io/amazon-web-services/src/test/java/org/apache/beam/sdk/io/aws/dynamodb/DynamoDBIOTest.java
@@ -49,6 +49,7 @@
 
 /** Test Coverage for the IO. */
 @Ignore("[BEAM-7794] DynamoDBIOTest is blocking forever")
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DynamoDBIOTest implements Serializable {
   @Rule public final transient TestPipeline pipeline = TestPipeline.create();
   @Rule public final transient ExpectedLogs expectedLogs = ExpectedLogs.none(DynamoDBIO.class);
diff --git a/sdks/java/io/amazon-web-services/src/test/java/org/apache/beam/sdk/io/aws/dynamodb/DynamoDBIOTestHelper.java b/sdks/java/io/amazon-web-services/src/test/java/org/apache/beam/sdk/io/aws/dynamodb/DynamoDBIOTestHelper.java
index 7087a68..89ac593 100644
--- a/sdks/java/io/amazon-web-services/src/test/java/org/apache/beam/sdk/io/aws/dynamodb/DynamoDBIOTestHelper.java
+++ b/sdks/java/io/amazon-web-services/src/test/java/org/apache/beam/sdk/io/aws/dynamodb/DynamoDBIOTestHelper.java
@@ -45,6 +45,7 @@
 import org.testcontainers.utility.DockerImageName;
 
 /** A utility to generate test table and data for {@link DynamoDBIOTest}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class DynamoDBIOTestHelper implements Serializable {
   private static final String LOCALSTACK_VERSION = "0.11.3";
 
diff --git a/sdks/java/io/amazon-web-services/src/test/java/org/apache/beam/sdk/io/aws/options/AwsModuleTest.java b/sdks/java/io/amazon-web-services/src/test/java/org/apache/beam/sdk/io/aws/options/AwsModuleTest.java
index b069f1e..b358ae8 100644
--- a/sdks/java/io/amazon-web-services/src/test/java/org/apache/beam/sdk/io/aws/options/AwsModuleTest.java
+++ b/sdks/java/io/amazon-web-services/src/test/java/org/apache/beam/sdk/io/aws/options/AwsModuleTest.java
@@ -47,6 +47,7 @@
 
 /** Tests {@link AwsModule}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class AwsModuleTest {
 
   private final ObjectMapper objectMapper = new ObjectMapper().registerModule(new AwsModule());
diff --git a/sdks/java/io/amazon-web-services/src/test/java/org/apache/beam/sdk/io/aws/s3/MatchResultMatcher.java b/sdks/java/io/amazon-web-services/src/test/java/org/apache/beam/sdk/io/aws/s3/MatchResultMatcher.java
index 2766a64..388a030 100644
--- a/sdks/java/io/amazon-web-services/src/test/java/org/apache/beam/sdk/io/aws/s3/MatchResultMatcher.java
+++ b/sdks/java/io/amazon-web-services/src/test/java/org/apache/beam/sdk/io/aws/s3/MatchResultMatcher.java
@@ -33,6 +33,7 @@
  * Hamcrest {@link Matcher} to match {@link MatchResult}. Necessary because {@link
  * MatchResult#metadata()} throws an exception under normal circumstances.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class MatchResultMatcher extends BaseMatcher<MatchResult> {
 
   private final MatchResult.Status expectedStatus;
diff --git a/sdks/java/io/amazon-web-services/src/test/java/org/apache/beam/sdk/io/aws/s3/S3FileSystemTest.java b/sdks/java/io/amazon-web-services/src/test/java/org/apache/beam/sdk/io/aws/s3/S3FileSystemTest.java
index 00d9c5a..5b8d4e1 100644
--- a/sdks/java/io/amazon-web-services/src/test/java/org/apache/beam/sdk/io/aws/s3/S3FileSystemTest.java
+++ b/sdks/java/io/amazon-web-services/src/test/java/org/apache/beam/sdk/io/aws/s3/S3FileSystemTest.java
@@ -80,6 +80,7 @@
 
 /** Test case for {@link S3FileSystem}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class S3FileSystemTest {
   private static S3Mock api;
   private static AmazonS3 client;
diff --git a/sdks/java/io/amazon-web-services/src/test/java/org/apache/beam/sdk/io/aws/s3/S3ResourceIdTest.java b/sdks/java/io/amazon-web-services/src/test/java/org/apache/beam/sdk/io/aws/s3/S3ResourceIdTest.java
index a5b148d..5fd683c 100644
--- a/sdks/java/io/amazon-web-services/src/test/java/org/apache/beam/sdk/io/aws/s3/S3ResourceIdTest.java
+++ b/sdks/java/io/amazon-web-services/src/test/java/org/apache/beam/sdk/io/aws/s3/S3ResourceIdTest.java
@@ -41,6 +41,7 @@
 
 /** Tests {@link S3ResourceId}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class S3ResourceIdTest {
 
   @Rule public ExpectedException thrown = ExpectedException.none();
diff --git a/sdks/java/io/amazon-web-services/src/test/java/org/apache/beam/sdk/io/aws/s3/S3WritableByteChannelTest.java b/sdks/java/io/amazon-web-services/src/test/java/org/apache/beam/sdk/io/aws/s3/S3WritableByteChannelTest.java
index d79f6b2..ded9394 100644
--- a/sdks/java/io/amazon-web-services/src/test/java/org/apache/beam/sdk/io/aws/s3/S3WritableByteChannelTest.java
+++ b/sdks/java/io/amazon-web-services/src/test/java/org/apache/beam/sdk/io/aws/s3/S3WritableByteChannelTest.java
@@ -55,6 +55,7 @@
 
 /** Tests {@link S3WritableByteChannel}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class S3WritableByteChannelTest {
   @Rule public ExpectedException expected = ExpectedException.none();
 
diff --git a/sdks/java/io/amazon-web-services/src/test/java/org/apache/beam/sdk/io/aws/sns/PublishResultCodersTest.java b/sdks/java/io/amazon-web-services/src/test/java/org/apache/beam/sdk/io/aws/sns/PublishResultCodersTest.java
index e21456d..ef1700d 100644
--- a/sdks/java/io/amazon-web-services/src/test/java/org/apache/beam/sdk/io/aws/sns/PublishResultCodersTest.java
+++ b/sdks/java/io/amazon-web-services/src/test/java/org/apache/beam/sdk/io/aws/sns/PublishResultCodersTest.java
@@ -31,6 +31,7 @@
 import org.junit.Test;
 
 /** Tests for PublishResult coders. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class PublishResultCodersTest {
 
   @Test
diff --git a/sdks/java/io/amazon-web-services/src/test/java/org/apache/beam/sdk/io/aws/sns/SnsIOTest.java b/sdks/java/io/amazon-web-services/src/test/java/org/apache/beam/sdk/io/aws/sns/SnsIOTest.java
index db28f3d..ad81ba1 100644
--- a/sdks/java/io/amazon-web-services/src/test/java/org/apache/beam/sdk/io/aws/sns/SnsIOTest.java
+++ b/sdks/java/io/amazon-web-services/src/test/java/org/apache/beam/sdk/io/aws/sns/SnsIOTest.java
@@ -56,6 +56,7 @@
 
 /** Tests to verify writes to Sns. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SnsIOTest implements Serializable {
 
   private static final String topicName = "arn:aws:sns:us-west-2:5880:topic-FMFEHJ47NRFO";
diff --git a/sdks/java/io/amazon-web-services/src/test/java/org/apache/beam/sdk/io/aws/sqs/SqsIOTest.java b/sdks/java/io/amazon-web-services/src/test/java/org/apache/beam/sdk/io/aws/sqs/SqsIOTest.java
index 5ce832d..3e3fc6d 100644
--- a/sdks/java/io/amazon-web-services/src/test/java/org/apache/beam/sdk/io/aws/sqs/SqsIOTest.java
+++ b/sdks/java/io/amazon-web-services/src/test/java/org/apache/beam/sdk/io/aws/sqs/SqsIOTest.java
@@ -45,6 +45,7 @@
 
 /** Tests on {@link SqsIO}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SqsIOTest {
 
   @Rule public TestPipeline pipeline = TestPipeline.create();
diff --git a/sdks/java/io/amazon-web-services2/build.gradle b/sdks/java/io/amazon-web-services2/build.gradle
index 321aa2b..0b62359 100644
--- a/sdks/java/io/amazon-web-services2/build.gradle
+++ b/sdks/java/io/amazon-web-services2/build.gradle
@@ -55,7 +55,7 @@
   testCompile library.java.guava_testlib
   testCompile library.java.hamcrest_core
   testCompile library.java.junit
-  testCompile 'org.elasticmq:elasticmq-rest-sqs_2.12:0.14.1'
+  testCompile 'org.elasticmq:elasticmq-rest-sqs_2.12:0.15.6'
   testCompile library.java.hamcrest_library
   testCompile library.java.powermock
   testCompile library.java.powermock_mockito
diff --git a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/dynamodb/DynamoDBIO.java b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/dynamodb/DynamoDBIO.java
index 857b164..b041bc5 100644
--- a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/dynamodb/DynamoDBIO.java
+++ b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/dynamodb/DynamoDBIO.java
@@ -128,6 +128,7 @@
  * </ul>
  */
 @Experimental(Kind.SOURCE_SINK)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public final class DynamoDBIO {
   public static <T> Read<T> read() {
     return new AutoValue_DynamoDBIO_Read.Builder().build();
diff --git a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/KinesisIO.java b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/KinesisIO.java
index f4b173b..5240a88 100644
--- a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/KinesisIO.java
+++ b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/KinesisIO.java
@@ -219,6 +219,7 @@
  * }</pre>
  */
 @Experimental(Kind.SOURCE_SINK)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public final class KinesisIO {
 
   private static final Logger LOG = LoggerFactory.getLogger(KinesisIO.class);
diff --git a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/KinesisReader.java b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/KinesisReader.java
index 824fbe2..588774f 100644
--- a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/KinesisReader.java
+++ b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/KinesisReader.java
@@ -32,6 +32,7 @@
  * Reads data from multiple kinesis shards in a single thread. It uses simple round robin algorithm
  * when fetching data from shards.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class KinesisReader extends UnboundedSource.UnboundedReader<KinesisRecord> {
 
   private static final Logger LOG = LoggerFactory.getLogger(KinesisReader.class);
diff --git a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/KinesisRecord.java b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/KinesisRecord.java
index 20b0db7..08976d9 100644
--- a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/KinesisRecord.java
+++ b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/KinesisRecord.java
@@ -29,6 +29,7 @@
 import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber;
 
 /** {@link KinesisClientRecord} enhanced with utility methods. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class KinesisRecord {
 
   private Instant readTime;
diff --git a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/KinesisSource.java b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/KinesisSource.java
index 5f54782..a56bfe8 100644
--- a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/KinesisSource.java
+++ b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/KinesisSource.java
@@ -30,6 +30,7 @@
 import org.slf4j.LoggerFactory;
 
 /** Represents source for single stream in Kinesis. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class KinesisSource extends UnboundedSource<KinesisRecord, KinesisReaderCheckpoint> {
 
   private static final Logger LOG = LoggerFactory.getLogger(KinesisSource.class);
diff --git a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/ShardCheckpoint.java b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/ShardCheckpoint.java
index a8b7d06..e310286 100644
--- a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/ShardCheckpoint.java
+++ b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/ShardCheckpoint.java
@@ -42,6 +42,7 @@
  *
  * This class is immutable.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class ShardCheckpoint implements Serializable {
 
   private final String streamName;
diff --git a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/ShardReadersPool.java b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/ShardReadersPool.java
index a3376d6..195ef39 100644
--- a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/ShardReadersPool.java
+++ b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/ShardReadersPool.java
@@ -45,6 +45,7 @@
  * Internal shard iterators pool. It maintains the thread pool for reading Kinesis shards in
  * separate threads. Read records are stored in a blocking queue of limited capacity.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class ShardReadersPool {
 
   private static final Logger LOG = LoggerFactory.getLogger(ShardReadersPool.class);
diff --git a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/SimplifiedKinesisClient.java b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/SimplifiedKinesisClient.java
index eaa09ba..e93d029 100644
--- a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/SimplifiedKinesisClient.java
+++ b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/SimplifiedKinesisClient.java
@@ -55,6 +55,7 @@
 import software.amazon.kinesis.retrieval.KinesisClientRecord;
 
 /** Wraps {@link KinesisClient} class providing much simpler interface and proper error handling. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class SimplifiedKinesisClient {
 
   private static final String KINESIS_NAMESPACE = "AWS/Kinesis";
diff --git a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/StartingPoint.java b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/StartingPoint.java
index ab065d9..e7f1fbc 100644
--- a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/StartingPoint.java
+++ b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/StartingPoint.java
@@ -31,6 +31,7 @@
  * expressed either as an {@link InitialPositionInStream} enum constant or a timestamp, in which
  * case the reader will start reading at the specified point in time.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class StartingPoint implements Serializable {
 
   private final InitialPositionInStream position;
diff --git a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/StartingPointShardsFinder.java b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/StartingPointShardsFinder.java
index 5c16638..dc9afd1 100644
--- a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/StartingPointShardsFinder.java
+++ b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/StartingPointShardsFinder.java
@@ -32,6 +32,7 @@
  * This class is responsible for establishing the initial set of shards that existed at the given
  * starting point.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class StartingPointShardsFinder implements Serializable {
 
   private static final Logger LOG = LoggerFactory.getLogger(StartingPointShardsFinder.class);
diff --git a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/TimeUtil.java b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/TimeUtil.java
index 95026c6..5d6ed41 100644
--- a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/TimeUtil.java
+++ b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/kinesis/TimeUtil.java
@@ -21,6 +21,7 @@
 import org.joda.time.Instant;
 
 /** Time conversion utilities. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public final class TimeUtil {
 
   public static java.time.Instant toJava(Instant timestamp) {
diff --git a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/options/AwsModule.java b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/options/AwsModule.java
index 6cb83fa..add64b4 100644
--- a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/options/AwsModule.java
+++ b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/options/AwsModule.java
@@ -59,6 +59,7 @@
  */
 @Experimental(Kind.SOURCE_SINK)
 @AutoService(Module.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class AwsModule extends SimpleModule {
   private static final String ACCESS_KEY_ID = "accessKeyId";
   private static final String SECRET_ACCESS_KEY = "secretAccessKey";
diff --git a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/sns/SnsIO.java b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/sns/SnsIO.java
index 0ff02f5..ed07d4e 100644
--- a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/sns/SnsIO.java
+++ b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/sns/SnsIO.java
@@ -117,6 +117,7 @@
  * </ul>
  */
 @Experimental(Kind.SOURCE_SINK)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public final class SnsIO {
 
   // Write data to SNS (synchronous)
diff --git a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/sns/SnsResponseCoder.java b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/sns/SnsResponseCoder.java
index b627822..b82a528 100644
--- a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/sns/SnsResponseCoder.java
+++ b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/sns/SnsResponseCoder.java
@@ -31,6 +31,7 @@
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
 
 /** Custom Coder for WrappedSnsResponse. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class SnsResponseCoder<T> extends StructuredCoder<SnsResponse<T>> {
 
   private final Coder<T> elementCoder;
diff --git a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/sqs/SqsIO.java b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/sqs/SqsIO.java
index c3a190c..02ec8a9 100644
--- a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/sqs/SqsIO.java
+++ b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/sqs/SqsIO.java
@@ -78,6 +78,7 @@
  * <p>For more information on the available options see {@link AwsOptions}.
  */
 @Experimental(Kind.SOURCE_SINK)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SqsIO {
 
   public static Read read() {
diff --git a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/sqs/SqsUnboundedReader.java b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/sqs/SqsUnboundedReader.java
index 72e8d7a..9f1bec8 100644
--- a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/sqs/SqsUnboundedReader.java
+++ b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/sqs/SqsUnboundedReader.java
@@ -35,6 +35,7 @@
 import software.amazon.awssdk.services.sqs.model.ReceiveMessageRequest;
 import software.amazon.awssdk.services.sqs.model.ReceiveMessageResponse;
 
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class SqsUnboundedReader extends UnboundedSource.UnboundedReader<SqsMessage>
     implements Serializable {
 
diff --git a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/sqs/SqsUnboundedSource.java b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/sqs/SqsUnboundedSource.java
index 3b09d47..d2e8cc3 100644
--- a/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/sqs/SqsUnboundedSource.java
+++ b/sdks/java/io/amazon-web-services2/src/main/java/org/apache/beam/sdk/io/aws2/sqs/SqsUnboundedSource.java
@@ -30,6 +30,7 @@
 import org.checkerframework.checker.nullness.qual.Nullable;
 import software.amazon.awssdk.services.sqs.SqsClient;
 
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class SqsUnboundedSource extends UnboundedSource<SqsMessage, SqsCheckpointMark> {
   private final Read read;
   private final Supplier<SqsClient> sqs;
diff --git a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/dynamodb/DynamoDBIOTest.java b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/dynamodb/DynamoDBIOTest.java
index a1be352..b4bdd87 100644
--- a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/dynamodb/DynamoDBIOTest.java
+++ b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/dynamodb/DynamoDBIOTest.java
@@ -60,6 +60,7 @@
 
 /** Test Coverage for the IO. */
 @Ignore("[BEAM-7794] DynamoDBIOTest is blocking forever")
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DynamoDBIOTest implements Serializable {
   @Rule public final transient TestPipeline pipeline = TestPipeline.create();
   @Rule public final transient ExpectedLogs expectedLogs = ExpectedLogs.none(DynamoDBIO.class);
diff --git a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/dynamodb/DynamoDBIOTestHelper.java b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/dynamodb/DynamoDBIOTestHelper.java
index e4c516c..81e7a8b 100644
--- a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/dynamodb/DynamoDBIOTestHelper.java
+++ b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/dynamodb/DynamoDBIOTestHelper.java
@@ -53,6 +53,7 @@
 import software.amazon.awssdk.services.dynamodb.model.WriteRequest;
 
 /** A utility to generate test table and data for {@link DynamoDBIOTest}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class DynamoDBIOTestHelper implements Serializable {
   private static final String DYNAMODB_LOCAL_VERSION = "1.13.3";
 
diff --git a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/dynamodb/DynamoDbClientProviderMock.java b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/dynamodb/DynamoDbClientProviderMock.java
index a017966..a95c227 100644
--- a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/dynamodb/DynamoDbClientProviderMock.java
+++ b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/dynamodb/DynamoDbClientProviderMock.java
@@ -20,6 +20,7 @@
 import software.amazon.awssdk.services.dynamodb.DynamoDbClient;
 
 /** Mocking AwsClientProvider. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DynamoDbClientProviderMock implements DynamoDbClientProvider {
 
   private static DynamoDbClientProviderMock instance = new DynamoDbClientProviderMock();
diff --git a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/AmazonKinesisMock.java b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/AmazonKinesisMock.java
index 6d7c134..c9589f8 100644
--- a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/AmazonKinesisMock.java
+++ b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/AmazonKinesisMock.java
@@ -94,6 +94,7 @@
 import software.amazon.awssdk.services.kinesis.model.UpdateShardCountResponse;
 
 /** Mock implementation of {@link KinesisClient} for testing. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class AmazonKinesisMock implements KinesisClient {
 
   static class TestData implements Serializable {
diff --git a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/DynamicCheckpointGeneratorTest.java b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/DynamicCheckpointGeneratorTest.java
index 3bbf39a..a8e9ab6 100644
--- a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/DynamicCheckpointGeneratorTest.java
+++ b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/DynamicCheckpointGeneratorTest.java
@@ -33,6 +33,7 @@
 /** * */
 @RunWith(PowerMockRunner.class)
 @PrepareForTest(Shard.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DynamicCheckpointGeneratorTest {
 
   @Mock private SimplifiedKinesisClient kinesisClient;
diff --git a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/KinesisIOIT.java b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/KinesisIOIT.java
index 565e03c..9e29c50 100644
--- a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/KinesisIOIT.java
+++ b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/KinesisIOIT.java
@@ -48,6 +48,7 @@
  * KinesisTestOptions} in order to run this.
  */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class KinesisIOIT implements Serializable {
   private static int numberOfShards;
   private static int numberOfRows;
diff --git a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/KinesisReaderCheckpointTest.java b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/KinesisReaderCheckpointTest.java
index aab5c1d..92fb304 100644
--- a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/KinesisReaderCheckpointTest.java
+++ b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/KinesisReaderCheckpointTest.java
@@ -31,6 +31,7 @@
 
 /** * */
 @RunWith(MockitoJUnitRunner.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class KinesisReaderCheckpointTest {
 
   @Mock private ShardCheckpoint a, b, c;
diff --git a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/KinesisReaderTest.java b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/KinesisReaderTest.java
index ab15be0..69e790f 100644
--- a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/KinesisReaderTest.java
+++ b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/KinesisReaderTest.java
@@ -38,6 +38,7 @@
 
 /** Tests {@link KinesisReader}. */
 @RunWith(MockitoJUnitRunner.Silent.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class KinesisReaderTest {
 
   @Mock private SimplifiedKinesisClient kinesis;
diff --git a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/KinesisServiceMock.java b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/KinesisServiceMock.java
index ede988a..ea31b61 100644
--- a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/KinesisServiceMock.java
+++ b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/KinesisServiceMock.java
@@ -26,6 +26,7 @@
 import org.joda.time.DateTime;
 
 /** Simple mock implementation of Kinesis service for testing, singletone. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class KinesisServiceMock {
   private static KinesisServiceMock instance;
 
diff --git a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/RecordFilterTest.java b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/RecordFilterTest.java
index 0058e2e..53cfd9c 100644
--- a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/RecordFilterTest.java
+++ b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/RecordFilterTest.java
@@ -30,6 +30,7 @@
 
 /** * */
 @RunWith(MockitoJUnitRunner.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class RecordFilterTest {
 
   @Mock private ShardCheckpoint checkpoint;
diff --git a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/ShardCheckpointTest.java b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/ShardCheckpointTest.java
index aa4b21b..07d580c 100644
--- a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/ShardCheckpointTest.java
+++ b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/ShardCheckpointTest.java
@@ -42,6 +42,7 @@
 
 /** */
 @RunWith(MockitoJUnitRunner.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ShardCheckpointTest {
 
   private static final String AT_SEQUENCE_SHARD_IT = "AT_SEQUENCE_SHARD_IT";
diff --git a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/ShardReadersPoolTest.java b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/ShardReadersPoolTest.java
index 462ae65..8e11f74 100644
--- a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/ShardReadersPoolTest.java
+++ b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/ShardReadersPoolTest.java
@@ -47,6 +47,7 @@
 
 /** Tests {@link ShardReadersPool}. */
 @RunWith(MockitoJUnitRunner.Silent.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ShardReadersPoolTest {
 
   private static final int TIMEOUT_IN_MILLIS = (int) TimeUnit.SECONDS.toMillis(10);
diff --git a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/ShardRecordsIteratorTest.java b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/ShardRecordsIteratorTest.java
index a1f01cd..f674f5d 100644
--- a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/ShardRecordsIteratorTest.java
+++ b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/ShardRecordsIteratorTest.java
@@ -39,6 +39,7 @@
 
 /** Tests {@link ShardRecordsIterator}. */
 @RunWith(MockitoJUnitRunner.Silent.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ShardRecordsIteratorTest {
 
   private static final String INITIAL_ITERATOR = "INITIAL_ITERATOR";
diff --git a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/SimplifiedKinesisClientTest.java b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/SimplifiedKinesisClientTest.java
index 83b3e54..ff66c85 100644
--- a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/SimplifiedKinesisClientTest.java
+++ b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/SimplifiedKinesisClientTest.java
@@ -60,6 +60,7 @@
 
 /** * */
 @RunWith(MockitoJUnitRunner.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SimplifiedKinesisClientTest {
 
   private static final String STREAM = "stream";
diff --git a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/StartingPointShardsFinderTest.java b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/StartingPointShardsFinderTest.java
index 51d97bb..8d3d011 100644
--- a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/StartingPointShardsFinderTest.java
+++ b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/StartingPointShardsFinderTest.java
@@ -35,6 +35,7 @@
 
 /** Tests StartingPointShardsFinder. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class StartingPointShardsFinderTest {
 
   private static final String STREAM_NAME = "streamName";
diff --git a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/TimeUtilTest.java b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/TimeUtilTest.java
index 02fa7ed..2e02916 100644
--- a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/TimeUtilTest.java
+++ b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/kinesis/TimeUtilTest.java
@@ -24,6 +24,7 @@
 import org.junit.runners.JUnit4;
 
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class TimeUtilTest {
 
   @Test
diff --git a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/sns/MockSnsAsyncBaseClient.java b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/sns/MockSnsAsyncBaseClient.java
index 697ae8b..313192d 100644
--- a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/sns/MockSnsAsyncBaseClient.java
+++ b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/sns/MockSnsAsyncBaseClient.java
@@ -20,6 +20,7 @@
 import java.io.Serializable;
 import software.amazon.awssdk.services.sns.SnsAsyncClient;
 
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class MockSnsAsyncBaseClient implements SnsAsyncClient, Serializable {
   @Override
   public String serviceName() {
diff --git a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/sns/SnsClientMockErrors.java b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/sns/SnsClientMockErrors.java
index 0557b64..0b0d1e9 100644
--- a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/sns/SnsClientMockErrors.java
+++ b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/sns/SnsClientMockErrors.java
@@ -27,6 +27,7 @@
 import software.amazon.awssdk.services.sns.model.PublishResponse;
 
 /** Mock class to test a failed publish of a msg. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SnsClientMockErrors implements SnsClient {
 
   @Override
diff --git a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/sns/SnsClientMockSuccess.java b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/sns/SnsClientMockSuccess.java
index ca49b1f..7035c93 100644
--- a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/sns/SnsClientMockSuccess.java
+++ b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/sns/SnsClientMockSuccess.java
@@ -30,6 +30,7 @@
 // import static org.mockito.BDDMockito.given;
 
 /** Mock class to test a successful publish of a msg. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SnsClientMockSuccess implements SnsClient {
 
   @Override
diff --git a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/sns/SnsIOTest.java b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/sns/SnsIOTest.java
index 9aae09e..29f982d 100644
--- a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/sns/SnsIOTest.java
+++ b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/sns/SnsIOTest.java
@@ -41,6 +41,7 @@
 /** Tests to verify writes to Sns. */
 @RunWith(PowerMockRunner.class)
 @PrepareForTest({PublishResponse.class, GetTopicAttributesResponse.class})
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SnsIOTest implements Serializable {
 
   private static final String topicArn = "arn:aws:sns:us-west-2:5880:topic-FMFEHJ47NRFO";
diff --git a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/sns/SnsIOWriteTest.java b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/sns/SnsIOWriteTest.java
index 8b9f795..11e1859 100644
--- a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/sns/SnsIOWriteTest.java
+++ b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/sns/SnsIOWriteTest.java
@@ -40,6 +40,7 @@
 import software.amazon.awssdk.services.sns.model.PublishRequest;
 
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SnsIOWriteTest implements Serializable {
   private static final String TOPIC = "test";
   private static final int FAILURE_STATUS_CODE = 400;
diff --git a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/sqs/EmbeddedSqsServer.java b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/sqs/EmbeddedSqsServer.java
index 2b82693..0d77336 100644
--- a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/sqs/EmbeddedSqsServer.java
+++ b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/sqs/EmbeddedSqsServer.java
@@ -27,6 +27,7 @@
 import software.amazon.awssdk.services.sqs.model.CreateQueueRequest;
 import software.amazon.awssdk.services.sqs.model.CreateQueueResponse;
 
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class EmbeddedSqsServer {
   private static SQSRestServer sqsRestServer;
   private static SqsClient client;
diff --git a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/sqs/SqsClientProviderMock.java b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/sqs/SqsClientProviderMock.java
index bd34152..b20478d 100644
--- a/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/sqs/SqsClientProviderMock.java
+++ b/sdks/java/io/amazon-web-services2/src/test/java/org/apache/beam/sdk/io/aws2/sqs/SqsClientProviderMock.java
@@ -20,6 +20,7 @@
 import software.amazon.awssdk.services.sqs.SqsClient;
 
 /** Mocking AwsClientProvider. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SqsClientProviderMock implements SqsClientProvider {
 
   private static SqsClientProviderMock instance = new SqsClientProviderMock();
diff --git a/sdks/java/io/amqp/src/main/java/org/apache/beam/sdk/io/amqp/AmqpIO.java b/sdks/java/io/amqp/src/main/java/org/apache/beam/sdk/io/amqp/AmqpIO.java
index f9e968e..a36243a 100644
--- a/sdks/java/io/amqp/src/main/java/org/apache/beam/sdk/io/amqp/AmqpIO.java
+++ b/sdks/java/io/amqp/src/main/java/org/apache/beam/sdk/io/amqp/AmqpIO.java
@@ -91,6 +91,7 @@
  * }</pre>
  */
 @Experimental(Kind.SOURCE_SINK)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class AmqpIO {
 
   public static Read read() {
diff --git a/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/blobstore/AzfsResourceId.java b/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/blobstore/AzfsResourceId.java
index 7905264..3ecbce2 100644
--- a/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/blobstore/AzfsResourceId.java
+++ b/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/blobstore/AzfsResourceId.java
@@ -30,6 +30,7 @@
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Strings;
 import org.checkerframework.checker.nullness.qual.Nullable;
 
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class AzfsResourceId implements ResourceId {
 
   static final String SCHEME = "azfs";
@@ -151,9 +152,8 @@
     return fromComponents(account, container, blob.substring(0, blob.lastIndexOf('/') + 1));
   }
 
-  @Nullable
   @Override
-  public String getFilename() {
+  public @Nullable String getFilename() {
     if (blob == null) {
       return null;
     }
diff --git a/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/blobstore/AzureBlobStoreFileSystem.java b/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/blobstore/AzureBlobStoreFileSystem.java
index 77f8e6c..4e291f1 100644
--- a/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/blobstore/AzureBlobStoreFileSystem.java
+++ b/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/blobstore/AzureBlobStoreFileSystem.java
@@ -64,6 +64,7 @@
 import org.slf4j.LoggerFactory;
 
 /** {@link FileSystem} implementation for Azure Blob Storage. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class AzureBlobStoreFileSystem extends FileSystem<AzfsResourceId> {
 
   private static final Logger LOG = LoggerFactory.getLogger(AzureBlobStoreFileSystem.class);
diff --git a/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/blobstore/DefaultBlobstoreClientBuilderFactory.java b/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/blobstore/DefaultBlobstoreClientBuilderFactory.java
index d8b44e2..0cd6ea4 100644
--- a/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/blobstore/DefaultBlobstoreClientBuilderFactory.java
+++ b/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/blobstore/DefaultBlobstoreClientBuilderFactory.java
@@ -24,6 +24,7 @@
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Strings;
 
 /** Construct BlobServiceClientBuilder with given values of Azure client properties. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DefaultBlobstoreClientBuilderFactory implements BlobstoreClientBuilderFactory {
 
   @Override
diff --git a/sdks/java/io/azure/src/test/java/org/apache/beam/sdk/io/azure/blobstore/AzfsResourceIdTest.java b/sdks/java/io/azure/src/test/java/org/apache/beam/sdk/io/azure/blobstore/AzfsResourceIdTest.java
index 6a1379b..6e449d8 100644
--- a/sdks/java/io/azure/src/test/java/org/apache/beam/sdk/io/azure/blobstore/AzfsResourceIdTest.java
+++ b/sdks/java/io/azure/src/test/java/org/apache/beam/sdk/io/azure/blobstore/AzfsResourceIdTest.java
@@ -42,6 +42,7 @@
 import org.junit.runners.Parameterized;
 
 @RunWith(Enclosed.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class AzfsResourceIdTest {
   @RunWith(Parameterized.class)
   public static class ResolveTest {
diff --git a/sdks/java/io/azure/src/test/java/org/apache/beam/sdk/io/azure/blobstore/AzureBlobStoreFileSystemTest.java b/sdks/java/io/azure/src/test/java/org/apache/beam/sdk/io/azure/blobstore/AzureBlobStoreFileSystemTest.java
index d169e78..7f89119 100644
--- a/sdks/java/io/azure/src/test/java/org/apache/beam/sdk/io/azure/blobstore/AzureBlobStoreFileSystemTest.java
+++ b/sdks/java/io/azure/src/test/java/org/apache/beam/sdk/io/azure/blobstore/AzureBlobStoreFileSystemTest.java
@@ -62,7 +62,10 @@
 import org.mockito.Mockito;
 
 @RunWith(JUnit4.class)
-@SuppressWarnings("CannotMockFinalClass") // Mockito 2 and above can mock final classes
+@SuppressWarnings({
+  "CannotMockFinalClass",
+  "nullness"
+}) // Mockito 2 and above can mock final classes
 public class AzureBlobStoreFileSystemTest {
 
   private static AzureBlobStoreFileSystem azureBlobStoreFileSystem;
diff --git a/sdks/java/io/bigquery-io-perf-tests/src/test/java/org/apache/beam/sdk/bigqueryioperftests/BigQueryIOIT.java b/sdks/java/io/bigquery-io-perf-tests/src/test/java/org/apache/beam/sdk/bigqueryioperftests/BigQueryIOIT.java
index 41cad8d..5c118c3 100644
--- a/sdks/java/io/bigquery-io-perf-tests/src/test/java/org/apache/beam/sdk/bigqueryioperftests/BigQueryIOIT.java
+++ b/sdks/java/io/bigquery-io-perf-tests/src/test/java/org/apache/beam/sdk/bigqueryioperftests/BigQueryIOIT.java
@@ -78,6 +78,7 @@
  * </pre>
  */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BigQueryIOIT {
   private static final String NAMESPACE = BigQueryIOIT.class.getName();
   private static final String TEST_ID = UUID.randomUUID().toString();
diff --git a/sdks/java/io/cassandra/src/main/java/org/apache/beam/sdk/io/cassandra/CassandraIO.java b/sdks/java/io/cassandra/src/main/java/org/apache/beam/sdk/io/cassandra/CassandraIO.java
index e5f7ae4..133f9dc 100644
--- a/sdks/java/io/cassandra/src/main/java/org/apache/beam/sdk/io/cassandra/CassandraIO.java
+++ b/sdks/java/io/cassandra/src/main/java/org/apache/beam/sdk/io/cassandra/CassandraIO.java
@@ -122,6 +122,7 @@
  * }</pre>
  */
 @Experimental(Kind.SOURCE_SINK)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class CassandraIO {
 
   private static final Logger LOG = LoggerFactory.getLogger(CassandraIO.class);
diff --git a/sdks/java/io/cassandra/src/main/java/org/apache/beam/sdk/io/cassandra/DefaultObjectMapperFactory.java b/sdks/java/io/cassandra/src/main/java/org/apache/beam/sdk/io/cassandra/DefaultObjectMapperFactory.java
index a21e5a4..cecad67 100644
--- a/sdks/java/io/cassandra/src/main/java/org/apache/beam/sdk/io/cassandra/DefaultObjectMapperFactory.java
+++ b/sdks/java/io/cassandra/src/main/java/org/apache/beam/sdk/io/cassandra/DefaultObjectMapperFactory.java
@@ -27,6 +27,7 @@
  *
  * @see org.apache.beam.sdk.io.cassandra.DefaultObjectMapper
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class DefaultObjectMapperFactory<T> implements SerializableFunction<Session, Mapper> {
 
   private transient MappingManager mappingManager;
diff --git a/sdks/java/io/cassandra/src/test/java/org/apache/beam/sdk/io/cassandra/CassandraIOIT.java b/sdks/java/io/cassandra/src/test/java/org/apache/beam/sdk/io/cassandra/CassandraIOIT.java
index 1f93676..4535eef 100644
--- a/sdks/java/io/cassandra/src/test/java/org/apache/beam/sdk/io/cassandra/CassandraIOIT.java
+++ b/sdks/java/io/cassandra/src/test/java/org/apache/beam/sdk/io/cassandra/CassandraIOIT.java
@@ -60,6 +60,7 @@
  * </pre>
  */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class CassandraIOIT implements Serializable {
 
   /** CassandraIOIT options. */
diff --git a/sdks/java/io/cassandra/src/test/java/org/apache/beam/sdk/io/cassandra/CassandraIOTest.java b/sdks/java/io/cassandra/src/test/java/org/apache/beam/sdk/io/cassandra/CassandraIOTest.java
index f368fb4..a0b801e 100644
--- a/sdks/java/io/cassandra/src/test/java/org/apache/beam/sdk/io/cassandra/CassandraIOTest.java
+++ b/sdks/java/io/cassandra/src/test/java/org/apache/beam/sdk/io/cassandra/CassandraIOTest.java
@@ -95,6 +95,7 @@
 
 /** Tests of {@link CassandraIO}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class CassandraIOTest implements Serializable {
   private static final long NUM_ROWS = 20L;
   private static final String CASSANDRA_KEYSPACE = "beam_ks";
diff --git a/sdks/java/io/clickhouse/build.gradle b/sdks/java/io/clickhouse/build.gradle
index 83c7948..0c202b7 100644
--- a/sdks/java/io/clickhouse/build.gradle
+++ b/sdks/java/io/clickhouse/build.gradle
@@ -47,7 +47,7 @@
   }
 }
 
-def clickhouse_jdbc_version = "0.1.47"
+def clickhouse_jdbc_version = "0.2.4"
 
 dependencies {
   javacc "net.java.dev.javacc:javacc:4.0"
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 904349c..ce3baaa 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
@@ -118,6 +118,7 @@
  * done using {@link org.apache.beam.sdk.schemas.transforms.Cast} before {@link ClickHouseIO}.
  */
 @Experimental(Kind.SOURCE_SINK)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ClickHouseIO {
 
   public static final long DEFAULT_MAX_INSERT_BLOCK_SIZE = 1000000;
diff --git a/sdks/java/io/clickhouse/src/main/java/org/apache/beam/sdk/io/clickhouse/ClickHouseWriter.java b/sdks/java/io/clickhouse/src/main/java/org/apache/beam/sdk/io/clickhouse/ClickHouseWriter.java
index d0ed33f..5b10425 100644
--- a/sdks/java/io/clickhouse/src/main/java/org/apache/beam/sdk/io/clickhouse/ClickHouseWriter.java
+++ b/sdks/java/io/clickhouse/src/main/java/org/apache/beam/sdk/io/clickhouse/ClickHouseWriter.java
@@ -32,6 +32,7 @@
 
 /** Writes Rows and field values using {@link ClickHouseRowBinaryStream}. */
 @Experimental(Kind.SOURCE_SINK)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ClickHouseWriter {
   private static final Instant EPOCH_INSTANT = new Instant(0L);
 
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 8a693f2..21fb985 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
@@ -33,6 +33,7 @@
 /** A descriptor for ClickHouse table schema. */
 @Experimental(Kind.SOURCE_SINK)
 @AutoValue
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public abstract class TableSchema implements Serializable {
 
   public abstract List<Column> columns();
diff --git a/sdks/java/io/clickhouse/src/test/java/org/apache/beam/sdk/io/clickhouse/BaseClickHouseTest.java b/sdks/java/io/clickhouse/src/test/java/org/apache/beam/sdk/io/clickhouse/BaseClickHouseTest.java
index 6d6213d..9f6b269 100644
--- a/sdks/java/io/clickhouse/src/test/java/org/apache/beam/sdk/io/clickhouse/BaseClickHouseTest.java
+++ b/sdks/java/io/clickhouse/src/test/java/org/apache/beam/sdk/io/clickhouse/BaseClickHouseTest.java
@@ -44,7 +44,10 @@
 import org.testcontainers.utility.DockerImageName;
 
 /** Base setup for ClickHouse containers. */
-@SuppressWarnings("unchecked")
+@SuppressWarnings({
+  "unchecked",
+  "nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
+})
 public class BaseClickHouseTest {
 
   public static Network network;
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 d557e33..c44baee 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
@@ -43,6 +43,7 @@
 
 /** Tests for {@link ClickHouseIO}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ClickHouseIOTest extends BaseClickHouseTest {
 
   @Rule public TestPipeline pipeline = TestPipeline.create();
diff --git a/sdks/java/io/common/src/test/java/org/apache/beam/sdk/io/common/HashingFn.java b/sdks/java/io/common/src/test/java/org/apache/beam/sdk/io/common/HashingFn.java
index 5107f26..d57ab50 100644
--- a/sdks/java/io/common/src/test/java/org/apache/beam/sdk/io/common/HashingFn.java
+++ b/sdks/java/io/common/src/test/java/org/apache/beam/sdk/io/common/HashingFn.java
@@ -34,6 +34,7 @@
 import org.checkerframework.checker.nullness.qual.Nullable;
 
 /** Custom Function for Hashing. The combiner is combineUnordered, and accumulator is a HashCode. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class HashingFn extends CombineFn<String, HashingFn.Accum, String> {
 
   /** Serializable Class to store the HashCode of input String. */
diff --git a/sdks/java/io/common/src/test/java/org/apache/beam/sdk/io/common/IOITHelper.java b/sdks/java/io/common/src/test/java/org/apache/beam/sdk/io/common/IOITHelper.java
index d14eacb..22d302e 100644
--- a/sdks/java/io/common/src/test/java/org/apache/beam/sdk/io/common/IOITHelper.java
+++ b/sdks/java/io/common/src/test/java/org/apache/beam/sdk/io/common/IOITHelper.java
@@ -26,6 +26,7 @@
 import org.slf4j.LoggerFactory;
 
 /** Methods common to all types of IOITs. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class IOITHelper {
   private static final Logger LOG = LoggerFactory.getLogger(IOITHelper.class);
   private static final int maxAttempts = 3;
diff --git a/sdks/java/io/common/src/test/java/org/apache/beam/sdk/io/common/IOITHelperTest.java b/sdks/java/io/common/src/test/java/org/apache/beam/sdk/io/common/IOITHelperTest.java
index 7901209..dfdfd1b 100644
--- a/sdks/java/io/common/src/test/java/org/apache/beam/sdk/io/common/IOITHelperTest.java
+++ b/sdks/java/io/common/src/test/java/org/apache/beam/sdk/io/common/IOITHelperTest.java
@@ -31,6 +31,7 @@
 
 /** Tests for functions in {@link IOITHelper}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class IOITHelperTest {
   private static long startTimeMeasure;
   private static String message = "";
diff --git a/sdks/java/io/contextualtextio/src/main/java/org/apache/beam/sdk/io/contextualtextio/ContextualTextIO.java b/sdks/java/io/contextualtextio/src/main/java/org/apache/beam/sdk/io/contextualtextio/ContextualTextIO.java
index c446afe..5979f81 100644
--- a/sdks/java/io/contextualtextio/src/main/java/org/apache/beam/sdk/io/contextualtextio/ContextualTextIO.java
+++ b/sdks/java/io/contextualtextio/src/main/java/org/apache/beam/sdk/io/contextualtextio/ContextualTextIO.java
@@ -180,9 +180,7 @@
  *      .apply(ContextualTextIO.readFiles());
  * }</pre>
  *
- * <p>Example 6: reading with recordNum metadata. (the Objects still contain recordNums, but these
- * recordNums would correspond to their positions in their respective offsets rather than their
- * positions within the entire file).
+ * <p>Example 6: reading with recordNum metadata.
  *
  * <pre>{@code
  * Pipeline p = ...;
@@ -192,14 +190,15 @@
  *      .setWithRecordNumMetadata(true));
  * }</pre>
  *
- * <p>NOTE: When using {@link ContextualTextIO.Read#withHasMultilineCSVRecords(Boolean)} this
- * option, a single reader will be used to process the file, rather than multiple readers which can
- * read from different offsets. For a large file this can result in lower performance.
+ * <p>NOTE: When using {@link ContextualTextIO.Read#withHasMultilineCSVRecords(Boolean)}, a single
+ * reader will be used to process the file, rather than multiple readers which can read from
+ * different offsets. For a large file this can result in lower performance.
  *
  * <p>NOTE: Use {@link Read#withRecordNumMetadata()} when recordNum metadata is required. Computing
- * record positions currently introduces a grouping step, which increases the resources used by the
- * pipeline. By default withRecordNumMetadata is set to false, so the shuffle step is not performed.
- * <b> this option is only supported with default triggers.</b>
+ * absolute record positions currently introduces a grouping step, which increases the resources
+ * used by the pipeline. By default withRecordNumMetadata is set to false, in this case record
+ * objects will not contain absolute record positions within the entire file, but will still contain
+ * relative positions in respective offsets.
  *
  * <h3>Reading a very large number of files</h3>
  *
@@ -208,6 +207,7 @@
  * performance and scalability. Note that it may decrease performance if the filepattern matches
  * only a small number of files.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ContextualTextIO {
   private static final long DEFAULT_BUNDLE_SIZE_BYTES = 64 * 1024 * 1024L;
   private static final Logger LOG = LoggerFactory.getLogger(ContextualTextIO.class);
@@ -340,9 +340,10 @@
     }
 
     /**
-     * Allows the user to opt into getting recordNums associated with each record.
+     * Allows the user to opt into getting recordNums associated with each record. <b> This option
+     * is only supported with default triggers.</b>
      *
-     * <p>When set to true, it will introduce a shuffle step to assemble the recordNums for each
+     * <p>When set to true, it will introduce a grouping step to assemble the recordNums for each
      * record, which will increase the resources used by the pipeline.
      *
      * <p>Use this when you need metadata like fileNames and you need processed position/order
diff --git a/sdks/java/io/contextualtextio/src/main/java/org/apache/beam/sdk/io/contextualtextio/ContextualTextIOSource.java b/sdks/java/io/contextualtextio/src/main/java/org/apache/beam/sdk/io/contextualtextio/ContextualTextIOSource.java
index 396a0c1..cdf9fd7 100644
--- a/sdks/java/io/contextualtextio/src/main/java/org/apache/beam/sdk/io/contextualtextio/ContextualTextIOSource.java
+++ b/sdks/java/io/contextualtextio/src/main/java/org/apache/beam/sdk/io/contextualtextio/ContextualTextIOSource.java
@@ -51,6 +51,7 @@
  * representing the beginning of the first record to be decoded.
  */
 @VisibleForTesting
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class ContextualTextIOSource extends FileBasedSource<Row> {
   byte[] delimiter;
 
diff --git a/sdks/java/io/contextualtextio/src/main/java/org/apache/beam/sdk/io/contextualtextio/RecordWithMetadata.java b/sdks/java/io/contextualtextio/src/main/java/org/apache/beam/sdk/io/contextualtextio/RecordWithMetadata.java
index 5b99e6d..521992c 100644
--- a/sdks/java/io/contextualtextio/src/main/java/org/apache/beam/sdk/io/contextualtextio/RecordWithMetadata.java
+++ b/sdks/java/io/contextualtextio/src/main/java/org/apache/beam/sdk/io/contextualtextio/RecordWithMetadata.java
@@ -46,6 +46,7 @@
  *       RecordWithMetadata#RANGE_OFFSET}
  * </ul>
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class RecordWithMetadata {
 
   public static final String VALUE = "value";
diff --git a/sdks/java/io/contextualtextio/src/test/java/org/apache/beam/sdk/io/contextualtextio/ContextualTextIOTest.java b/sdks/java/io/contextualtextio/src/test/java/org/apache/beam/sdk/io/contextualtextio/ContextualTextIOTest.java
index b329f32..06e2672 100644
--- a/sdks/java/io/contextualtextio/src/test/java/org/apache/beam/sdk/io/contextualtextio/ContextualTextIOTest.java
+++ b/sdks/java/io/contextualtextio/src/test/java/org/apache/beam/sdk/io/contextualtextio/ContextualTextIOTest.java
@@ -103,6 +103,7 @@
 import org.junit.runners.Parameterized;
 
 /** Tests for {@link ContextualTextIO.Read}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ContextualTextIOTest {
   private static final int NUM_LINES_FOR_LARGE = 1024;
 
diff --git a/sdks/java/io/elasticsearch-tests/elasticsearch-tests-2/build.gradle b/sdks/java/io/elasticsearch-tests/elasticsearch-tests-2/build.gradle
index 2beea5e..cdd47a5 100644
--- a/sdks/java/io/elasticsearch-tests/elasticsearch-tests-2/build.gradle
+++ b/sdks/java/io/elasticsearch-tests/elasticsearch-tests-2/build.gradle
@@ -27,7 +27,6 @@
 description = "Apache Beam :: SDKs :: Java :: IO :: Elasticsearch-Tests :: 2.x"
 ext.summary = "Tests of ElasticsearchIO on Elasticsearch 2.x"
 
-def jna_version = "4.1.0"
 def log4j_version = "2.6.2"
 def elastic_search_version = "2.4.1"
 
@@ -37,7 +36,6 @@
   testCompile project(":sdks:java:io:elasticsearch")
   testCompile project(path: ":sdks:java:io:common", configuration: "testRuntime")
   testCompile library.java.slf4j_api
-  testCompile "net.java.dev.jna:jna:$jna_version"
   testCompile "org.apache.logging.log4j:log4j-api:$log4j_version"
   testCompile "org.apache.logging.log4j:log4j-core:$log4j_version"
   testCompile library.java.hamcrest_core
diff --git a/sdks/java/io/elasticsearch-tests/elasticsearch-tests-2/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOIT.java b/sdks/java/io/elasticsearch-tests/elasticsearch-tests-2/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOIT.java
index 6c8aa3c..6d54e06 100644
--- a/sdks/java/io/elasticsearch-tests/elasticsearch-tests-2/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOIT.java
+++ b/sdks/java/io/elasticsearch-tests/elasticsearch-tests-2/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOIT.java
@@ -49,6 +49,7 @@
  * <p>It is likely that you will need to configure <code>thread_pool.bulk.queue_size: 250</code> (or
  * higher) in the backend Elasticsearch server for this test to run.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ElasticsearchIOIT {
   private static RestClient restClient;
   private static ElasticsearchPipelineOptions options;
diff --git a/sdks/java/io/elasticsearch-tests/elasticsearch-tests-2/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTest.java b/sdks/java/io/elasticsearch-tests/elasticsearch-tests-2/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTest.java
index d1b43f0..74eeb57 100644
--- a/sdks/java/io/elasticsearch-tests/elasticsearch-tests-2/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTest.java
+++ b/sdks/java/io/elasticsearch-tests/elasticsearch-tests-2/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTest.java
@@ -44,6 +44,7 @@
 
 /** Tests for {@link ElasticsearchIO} version 2.x. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ElasticsearchIOTest implements Serializable {
 
   private static final Logger LOG = LoggerFactory.getLogger(ElasticsearchIOTest.class);
diff --git a/sdks/java/io/elasticsearch-tests/elasticsearch-tests-5/build.gradle b/sdks/java/io/elasticsearch-tests/elasticsearch-tests-5/build.gradle
index 750faf3..a394979 100644
--- a/sdks/java/io/elasticsearch-tests/elasticsearch-tests-5/build.gradle
+++ b/sdks/java/io/elasticsearch-tests/elasticsearch-tests-5/build.gradle
@@ -32,7 +32,6 @@
   systemProperty "tests.security.manager", "false"
 }
 
-def jna_version = "4.1.0"
 def log4j_version = "2.6.2"
 def elastic_search_version = "5.6.3"
 
@@ -58,7 +57,6 @@
   testCompile "org.apache.logging.log4j:log4j-core:$log4j_version"
   testCompile "org.apache.logging.log4j:log4j-api:$log4j_version"
   testCompile library.java.slf4j_api
-  testCompile "net.java.dev.jna:jna:$jna_version"
   testCompile library.java.hamcrest_core
   testCompile library.java.hamcrest_library
   testCompile library.java.junit
diff --git a/sdks/java/io/elasticsearch-tests/elasticsearch-tests-5/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOIT.java b/sdks/java/io/elasticsearch-tests/elasticsearch-tests-5/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOIT.java
index c9032dd..923f58d 100644
--- a/sdks/java/io/elasticsearch-tests/elasticsearch-tests-5/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOIT.java
+++ b/sdks/java/io/elasticsearch-tests/elasticsearch-tests-5/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOIT.java
@@ -52,6 +52,7 @@
  * higher) in the backend Elasticsearch server for this test to run.
  */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ElasticsearchIOIT {
   private static RestClient restClient;
   private static ElasticsearchPipelineOptions options;
diff --git a/sdks/java/io/elasticsearch-tests/elasticsearch-tests-5/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTest.java b/sdks/java/io/elasticsearch-tests/elasticsearch-tests-5/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTest.java
index e675d43..5ed409f 100644
--- a/sdks/java/io/elasticsearch-tests/elasticsearch-tests-5/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTest.java
+++ b/sdks/java/io/elasticsearch-tests/elasticsearch-tests-5/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTest.java
@@ -47,6 +47,7 @@
 @ThreadLeakScope(ThreadLeakScope.Scope.NONE)
 // use cluster of 1 node that has data + master roles
 @ESIntegTestCase.ClusterScope(scope = SUITE, numDataNodes = 1, supportsDedicatedMasters = false)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ElasticsearchIOTest extends ESIntegTestCase implements Serializable {
 
   private ElasticsearchIOTestCommon elasticsearchIOTestCommon;
diff --git a/sdks/java/io/elasticsearch-tests/elasticsearch-tests-6/build.gradle b/sdks/java/io/elasticsearch-tests/elasticsearch-tests-6/build.gradle
index f961e9c..291c77a 100644
--- a/sdks/java/io/elasticsearch-tests/elasticsearch-tests-6/build.gradle
+++ b/sdks/java/io/elasticsearch-tests/elasticsearch-tests-6/build.gradle
@@ -32,7 +32,6 @@
   systemProperty "tests.security.manager", "false"
 }
 
-def jna_version = "4.1.0"
 def log4j_version = "2.6.2"
 def elastic_search_version = "6.4.0"
 
@@ -58,7 +57,6 @@
   testCompile "org.apache.logging.log4j:log4j-core:$log4j_version"
   testCompile "org.apache.logging.log4j:log4j-api:$log4j_version"
   testCompile library.java.slf4j_api
-  testCompile "net.java.dev.jna:jna:$jna_version"
   testCompile library.java.hamcrest_core
   testCompile library.java.hamcrest_library
   testCompile library.java.junit
diff --git a/sdks/java/io/elasticsearch-tests/elasticsearch-tests-6/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOIT.java b/sdks/java/io/elasticsearch-tests/elasticsearch-tests-6/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOIT.java
index f9ae9f6..4319144 100644
--- a/sdks/java/io/elasticsearch-tests/elasticsearch-tests-6/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOIT.java
+++ b/sdks/java/io/elasticsearch-tests/elasticsearch-tests-6/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOIT.java
@@ -52,6 +52,7 @@
  * higher) in the backend Elasticsearch server for this test to run.
  */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ElasticsearchIOIT {
   private static RestClient restClient;
   private static ElasticsearchPipelineOptions options;
diff --git a/sdks/java/io/elasticsearch-tests/elasticsearch-tests-6/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTest.java b/sdks/java/io/elasticsearch-tests/elasticsearch-tests-6/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTest.java
index 0de7398..5774996 100644
--- a/sdks/java/io/elasticsearch-tests/elasticsearch-tests-6/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTest.java
+++ b/sdks/java/io/elasticsearch-tests/elasticsearch-tests-6/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTest.java
@@ -47,6 +47,7 @@
 @ThreadLeakScope(ThreadLeakScope.Scope.NONE)
 // use cluster of 1 node that has data + master roles
 @ESIntegTestCase.ClusterScope(scope = SUITE, numDataNodes = 1, supportsDedicatedMasters = false)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ElasticsearchIOTest extends ESIntegTestCase implements Serializable {
 
   private ElasticsearchIOTestCommon elasticsearchIOTestCommon;
diff --git a/sdks/java/io/elasticsearch-tests/elasticsearch-tests-7/build.gradle b/sdks/java/io/elasticsearch-tests/elasticsearch-tests-7/build.gradle
index 10e239c..4d8b1fb 100644
--- a/sdks/java/io/elasticsearch-tests/elasticsearch-tests-7/build.gradle
+++ b/sdks/java/io/elasticsearch-tests/elasticsearch-tests-7/build.gradle
@@ -32,7 +32,6 @@
   systemProperty "tests.security.manager", "false"
 }
 
-def jna_version = "4.1.0"
 def log4j_version = "2.11.1"
 def elastic_search_version = "7.9.2"
 
@@ -58,7 +57,6 @@
   testCompile "org.apache.logging.log4j:log4j-core:$log4j_version"
   testCompile "org.apache.logging.log4j:log4j-api:$log4j_version"
   testCompile library.java.slf4j_api
-  testCompile "net.java.dev.jna:jna:$jna_version"
   testCompile library.java.hamcrest_core
   testCompile library.java.hamcrest_library
   testCompile library.java.junit
diff --git a/sdks/java/io/elasticsearch-tests/elasticsearch-tests-7/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOIT.java b/sdks/java/io/elasticsearch-tests/elasticsearch-tests-7/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOIT.java
index ce4a041..b98b885 100644
--- a/sdks/java/io/elasticsearch-tests/elasticsearch-tests-7/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOIT.java
+++ b/sdks/java/io/elasticsearch-tests/elasticsearch-tests-7/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOIT.java
@@ -52,6 +52,7 @@
  * (or higher) in the backend Elasticsearch server for this test to run.
  */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ElasticsearchIOIT {
   private static RestClient restClient;
   private static ElasticsearchPipelineOptions options;
diff --git a/sdks/java/io/elasticsearch-tests/elasticsearch-tests-7/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTest.java b/sdks/java/io/elasticsearch-tests/elasticsearch-tests-7/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTest.java
index e9dceb7..8d57603 100644
--- a/sdks/java/io/elasticsearch-tests/elasticsearch-tests-7/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTest.java
+++ b/sdks/java/io/elasticsearch-tests/elasticsearch-tests-7/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTest.java
@@ -47,6 +47,7 @@
 @ThreadLeakScope(ThreadLeakScope.Scope.NONE)
 // use cluster of 1 node that has data + master roles
 @ESIntegTestCase.ClusterScope(scope = SUITE, numDataNodes = 1, supportsDedicatedMasters = false)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ElasticsearchIOTest extends ESIntegTestCase implements Serializable {
 
   private ElasticsearchIOTestCommon elasticsearchIOTestCommon;
diff --git a/sdks/java/io/elasticsearch-tests/elasticsearch-tests-common/build.gradle b/sdks/java/io/elasticsearch-tests/elasticsearch-tests-common/build.gradle
index 8c0a4dc..6895a2f 100644
--- a/sdks/java/io/elasticsearch-tests/elasticsearch-tests-common/build.gradle
+++ b/sdks/java/io/elasticsearch-tests/elasticsearch-tests-common/build.gradle
@@ -25,7 +25,6 @@
 description = "Apache Beam :: SDKs :: Java :: IO :: Elasticsearch-Tests :: Common"
 ext.summary = "Common test classes for ElasticsearchIO"
 
-def jna_version = "4.1.0"
 def log4j_version = "2.11.1"
 def elastic_search_version = "7.9.2"
 
@@ -44,7 +43,6 @@
   testCompile project(":sdks:java:io:elasticsearch")
   testCompile project(path: ":sdks:java:io:common", configuration: "testRuntime")
   testCompile library.java.slf4j_api
-  testCompile "net.java.dev.jna:jna:$jna_version"
   testCompile "org.apache.logging.log4j:log4j-api:$log4j_version"
   testCompile "org.apache.logging.log4j:log4j-core:$log4j_version"
   testCompile library.java.hamcrest_core
diff --git a/sdks/java/io/elasticsearch-tests/elasticsearch-tests-common/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTestCommon.java b/sdks/java/io/elasticsearch-tests/elasticsearch-tests-common/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTestCommon.java
index a7b2ca4..65dbf5d 100644
--- a/sdks/java/io/elasticsearch-tests/elasticsearch-tests-common/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTestCommon.java
+++ b/sdks/java/io/elasticsearch-tests/elasticsearch-tests-common/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTestCommon.java
@@ -74,6 +74,7 @@
 import org.slf4j.LoggerFactory;
 
 /** Common test class for {@link ElasticsearchIO}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class ElasticsearchIOTestCommon implements Serializable {
 
   private static final Logger LOG = LoggerFactory.getLogger(ElasticsearchIOTestCommon.class);
diff --git a/sdks/java/io/elasticsearch-tests/elasticsearch-tests-common/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTestUtils.java b/sdks/java/io/elasticsearch-tests/elasticsearch-tests-common/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTestUtils.java
index cc5a18e..d781277 100644
--- a/sdks/java/io/elasticsearch-tests/elasticsearch-tests-common/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTestUtils.java
+++ b/sdks/java/io/elasticsearch-tests/elasticsearch-tests-common/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIOTestUtils.java
@@ -34,6 +34,7 @@
 import org.elasticsearch.client.RestClient;
 
 /** Test utilities to use with {@link ElasticsearchIO}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class ElasticsearchIOTestUtils {
   static final String[] FAMOUS_SCIENTISTS = {
     "Einstein",
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 a9887c6..54e8b48 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
@@ -153,6 +153,7 @@
  * connect timeout of 1000ms.
  */
 @Experimental(Kind.SOURCE_SINK)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ElasticsearchIO {
 
   private static final Logger LOG = LoggerFactory.getLogger(ElasticsearchIO.class);
diff --git a/sdks/java/io/file-based-io-tests/src/test/java/org/apache/beam/sdk/io/avro/AvroIOIT.java b/sdks/java/io/file-based-io-tests/src/test/java/org/apache/beam/sdk/io/avro/AvroIOIT.java
index 0d48480..d8a0c83 100644
--- a/sdks/java/io/file-based-io-tests/src/test/java/org/apache/beam/sdk/io/avro/AvroIOIT.java
+++ b/sdks/java/io/file-based-io-tests/src/test/java/org/apache/beam/sdk/io/avro/AvroIOIT.java
@@ -78,6 +78,7 @@
  * performance testing framework.
  */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class AvroIOIT {
 
   private static final Schema AVRO_SCHEMA =
diff --git a/sdks/java/io/file-based-io-tests/src/test/java/org/apache/beam/sdk/io/parquet/ParquetIOIT.java b/sdks/java/io/file-based-io-tests/src/test/java/org/apache/beam/sdk/io/parquet/ParquetIOIT.java
index 7a36ac8..aa5e818 100644
--- a/sdks/java/io/file-based-io-tests/src/test/java/org/apache/beam/sdk/io/parquet/ParquetIOIT.java
+++ b/sdks/java/io/file-based-io-tests/src/test/java/org/apache/beam/sdk/io/parquet/ParquetIOIT.java
@@ -78,6 +78,7 @@
  * performance testing framework.
  */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ParquetIOIT {
 
   private static final Schema SCHEMA =
diff --git a/sdks/java/io/file-based-io-tests/src/test/java/org/apache/beam/sdk/io/text/TextIOIT.java b/sdks/java/io/file-based-io-tests/src/test/java/org/apache/beam/sdk/io/text/TextIOIT.java
index c28bd93..94b45aa 100644
--- a/sdks/java/io/file-based-io-tests/src/test/java/org/apache/beam/sdk/io/text/TextIOIT.java
+++ b/sdks/java/io/file-based-io-tests/src/test/java/org/apache/beam/sdk/io/text/TextIOIT.java
@@ -77,6 +77,7 @@
  * performance testing framework.
  */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class TextIOIT {
   private static final Logger LOG = LoggerFactory.getLogger(TextIOIT.class);
 
diff --git a/sdks/java/io/file-based-io-tests/src/test/java/org/apache/beam/sdk/io/tfrecord/TFRecordIOIT.java b/sdks/java/io/file-based-io-tests/src/test/java/org/apache/beam/sdk/io/tfrecord/TFRecordIOIT.java
index 395000b..c88fad0 100644
--- a/sdks/java/io/file-based-io-tests/src/test/java/org/apache/beam/sdk/io/tfrecord/TFRecordIOIT.java
+++ b/sdks/java/io/file-based-io-tests/src/test/java/org/apache/beam/sdk/io/tfrecord/TFRecordIOIT.java
@@ -78,6 +78,7 @@
  * performance testing framework.
  */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class TFRecordIOIT {
   private static final String TFRECORD_NAMESPACE = TFRecordIOIT.class.getName();
 
diff --git a/sdks/java/io/file-based-io-tests/src/test/java/org/apache/beam/sdk/io/xml/XmlIOIT.java b/sdks/java/io/file-based-io-tests/src/test/java/org/apache/beam/sdk/io/xml/XmlIOIT.java
index 2b44274..e4dedc8 100644
--- a/sdks/java/io/file-based-io-tests/src/test/java/org/apache/beam/sdk/io/xml/XmlIOIT.java
+++ b/sdks/java/io/file-based-io-tests/src/test/java/org/apache/beam/sdk/io/xml/XmlIOIT.java
@@ -81,6 +81,7 @@
  * performance testing framework.
  */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class XmlIOIT {
 
   /** XmlIOIT options. */
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/AvroRowWriter.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/AvroRowWriter.java
index 74a0bb4..ed32869 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/AvroRowWriter.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/AvroRowWriter.java
@@ -24,6 +24,7 @@
 import org.apache.beam.sdk.transforms.SerializableFunction;
 import org.apache.beam.sdk.util.MimeTypes;
 
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class AvroRowWriter<AvroT, T> extends BigQueryRowWriter<T> {
   private final DataFileWriter<AvroT> writer;
   private final Schema schema;
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BatchLoads.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BatchLoads.java
index b39399e..53eccc8 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BatchLoads.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BatchLoads.java
@@ -78,6 +78,7 @@
 import org.slf4j.LoggerFactory;
 
 /** PTransform that uses BigQuery batch-load jobs to write a PCollection to BigQuery. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class BatchLoads<DestinationT, ElementT>
     extends PTransform<PCollection<KV<DestinationT, ElementT>>, WriteResult> {
   private static final Logger LOG = LoggerFactory.getLogger(BatchLoads.class);
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryAvroUtils.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryAvroUtils.java
index e289f53..a29cbfb 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryAvroUtils.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryAvroUtils.java
@@ -59,6 +59,7 @@
  * <p>These utilities are based on the <a href="https://avro.apache.org/docs/1.8.1/spec.html">Avro
  * 1.8.1</a> specification.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class BigQueryAvroUtils {
 
   /**
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryHelpers.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryHelpers.java
index 1f00723..4c70ae4 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryHelpers.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryHelpers.java
@@ -53,6 +53,7 @@
 import org.slf4j.LoggerFactory;
 
 /** A set of helper functions and classes used by {@link BigQueryIO}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BigQueryHelpers {
   private static final String RESOURCE_NOT_FOUND_ERROR =
       "BigQuery %1$s not found for table \"%2$s\" . Please create the %1$s before pipeline"
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java
index 2f22af3..ec38603 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java
@@ -455,6 +455,7 @@
  * <p>Please see <a href="https://cloud.google.com/bigquery/access-control">BigQuery Access Control
  * </a> for security and permission related information specific to BigQuery.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BigQueryIO {
 
   /**
@@ -823,17 +824,14 @@
 
     abstract @Nullable String getKmsKey();
 
-    @Nullable
     @Experimental(Kind.SCHEMAS)
-    abstract TypeDescriptor<T> getTypeDescriptor();
+    abstract @Nullable TypeDescriptor<T> getTypeDescriptor();
 
-    @Nullable
     @Experimental(Kind.SCHEMAS)
-    abstract ToBeamRowFunction<T> getToBeamRowFn();
+    abstract @Nullable ToBeamRowFunction<T> getToBeamRowFn();
 
-    @Nullable
     @Experimental(Kind.SCHEMAS)
-    abstract FromBeamRowFunction<T> getFromBeamRowFn();
+    abstract @Nullable FromBeamRowFunction<T> getFromBeamRowFn();
 
     abstract Boolean getUseAvroLogicalTypes();
 
@@ -1723,6 +1721,8 @@
 
     abstract @Nullable SerializableFunction<T, TableRow> getFormatFunction();
 
+    abstract @Nullable SerializableFunction<T, TableRow> getFormatRecordOnFailureFunction();
+
     abstract RowWriterFactory.@Nullable AvroRowWriterFactory<T, ?, ?> getAvroRowWriterFactory();
 
     abstract @Nullable SerializableFunction<TableSchema, org.apache.avro.Schema>
@@ -1798,6 +1798,9 @@
 
       abstract Builder<T> setFormatFunction(SerializableFunction<T, TableRow> formatFunction);
 
+      abstract Builder<T> setFormatRecordOnFailureFunction(
+          SerializableFunction<T, TableRow> formatFunction);
+
       abstract Builder<T> setAvroRowWriterFactory(
           RowWriterFactory.AvroRowWriterFactory<T, ?, ?> avroRowWriterFactory);
 
@@ -2003,6 +2006,16 @@
     }
 
     /**
+     * If an insert failure occurs, this function is applied to the originally supplied row T. The
+     * resulting {@link TableRow} will be accessed via {@link
+     * WriteResult#getFailedInsertsWithErr()}.
+     */
+    public Write<T> withFormatRecordOnFailureFunction(
+        SerializableFunction<T, TableRow> formatFunction) {
+      return toBuilder().setFormatRecordOnFailureFunction(formatFunction).build();
+    }
+
+    /**
      * Formats the user's type into a {@link GenericRecord} to be written to BigQuery. The
      * GenericRecords are written as avro using the standard {@link GenericDatumWriter}.
      *
@@ -2521,6 +2534,8 @@
         PCollection<T> input, DynamicDestinations<T, DestinationT> dynamicDestinations) {
       boolean optimizeWrites = getOptimizeWrites();
       SerializableFunction<T, TableRow> formatFunction = getFormatFunction();
+      SerializableFunction<T, TableRow> formatRecordOnFailureFunction =
+          getFormatRecordOnFailureFunction();
       RowWriterFactory.AvroRowWriterFactory<T, ?, DestinationT> avroRowWriterFactory =
           (RowWriterFactory.AvroRowWriterFactory<T, ?, DestinationT>) getAvroRowWriterFactory();
 
@@ -2566,8 +2581,8 @@
       }
 
       Method method = resolveMethod(input);
+      RowWriterFactory<T, DestinationT> rowWriterFactory;
       if (optimizeWrites) {
-        RowWriterFactory<T, DestinationT> rowWriterFactory;
         if (avroRowWriterFactory != null) {
           checkArgument(
               formatFunction == null,
@@ -2584,7 +2599,8 @@
           }
           rowWriterFactory = avroRowWriterFactory.prepare(dynamicDestinations, avroSchemaFactory);
         } else if (formatFunction != null) {
-          rowWriterFactory = RowWriterFactory.tableRows(formatFunction);
+          rowWriterFactory =
+              RowWriterFactory.tableRows(formatFunction, formatRecordOnFailureFunction);
         } else {
           throw new IllegalArgumentException(
               "A function must be provided to convert the input type into a TableRow or "
@@ -2592,20 +2608,6 @@
                   + "BigQueryIO.Write.withAvroFormatFunction to provide a formatting function. "
                   + "A format function is not required if Beam schemas are used.");
         }
-
-        PCollection<KV<DestinationT, T>> rowsWithDestination =
-            input
-                .apply(
-                    "PrepareWrite",
-                    new PrepareWrite<>(dynamicDestinations, SerializableFunctions.identity()))
-                .setCoder(KvCoder.of(destinationCoder, input.getCoder()));
-        return continueExpandTyped(
-            rowsWithDestination,
-            input.getCoder(),
-            destinationCoder,
-            dynamicDestinations,
-            rowWriterFactory,
-            method);
       } else {
         checkArgument(avroRowWriterFactory == null);
         checkArgument(
@@ -2615,22 +2617,22 @@
                 + "BigQueryIO.Write.withAvroFormatFunction to provide a formatting function. "
                 + "A format function is not required if Beam schemas are used.");
 
-        PCollection<KV<DestinationT, TableRow>> rowsWithDestination =
-            input
-                .apply("PrepareWrite", new PrepareWrite<>(dynamicDestinations, formatFunction))
-                .setCoder(KvCoder.of(destinationCoder, TableRowJsonCoder.of()));
-
-        RowWriterFactory<TableRow, DestinationT> rowWriterFactory =
-            RowWriterFactory.tableRows(SerializableFunctions.identity());
-
-        return continueExpandTyped(
-            rowsWithDestination,
-            TableRowJsonCoder.of(),
-            destinationCoder,
-            dynamicDestinations,
-            rowWriterFactory,
-            method);
+        rowWriterFactory =
+            RowWriterFactory.tableRows(formatFunction, formatRecordOnFailureFunction);
       }
+      PCollection<KV<DestinationT, T>> rowsWithDestination =
+          input
+              .apply(
+                  "PrepareWrite",
+                  new PrepareWrite<>(dynamicDestinations, SerializableFunctions.identity()))
+              .setCoder(KvCoder.of(destinationCoder, input.getCoder()));
+      return continueExpandTyped(
+          rowsWithDestination,
+          input.getCoder(),
+          destinationCoder,
+          dynamicDestinations,
+          rowWriterFactory,
+          method);
     }
 
     private <DestinationT, ElementT> WriteResult continueExpandTyped(
@@ -2659,7 +2661,8 @@
                     getCreateDisposition(),
                     dynamicDestinations,
                     elementCoder,
-                    tableRowWriterFactory.getToRowFn())
+                    tableRowWriterFactory.getToRowFn(),
+                    tableRowWriterFactory.getToFailsafeRowFn())
                 .withInsertRetryPolicy(retryPolicy)
                 .withTestServices(getBigQueryServices())
                 .withExtendedErrorInfo(getExtendedErrorInfo())
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOMetadata.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOMetadata.java
index e2d058d..0bc976c 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOMetadata.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOMetadata.java
@@ -23,6 +23,7 @@
 import org.apache.beam.sdk.extensions.gcp.util.GceMetadataUtil;
 
 /** Metadata class for BigQueryIO. i.e. to use as BQ job labels. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 final class BigQueryIOMetadata {
 
   private String beamJobId;
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryQueryHelper.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryQueryHelper.java
index 367617f..40a1ec5 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryQueryHelper.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryQueryHelper.java
@@ -47,6 +47,7 @@
  * <p>This object is not serializable, and its state can be safely discarded across serialization
  * boundaries for any associated source objects.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class BigQueryQueryHelper {
 
   private static final Integer JOB_POLL_MAX_RETRIES = Integer.MAX_VALUE;
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryResourceNaming.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryResourceNaming.java
index aa4ad89..d842344 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryResourceNaming.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryResourceNaming.java
@@ -37,6 +37,7 @@
  *
  * <p>This class has no backwards compatibility guaantees. It is considered internal.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class BigQueryResourceNaming {
 
   /**
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServices.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServices.java
index 12f0029..f6894ca 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServices.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServices.java
@@ -38,6 +38,7 @@
 import java.io.Serializable;
 import java.util.List;
 import org.apache.beam.sdk.util.Histogram;
+import org.apache.beam.sdk.values.FailsafeValueInSingleWindow;
 import org.apache.beam.sdk.values.ValueInSingleWindow;
 import org.checkerframework.checker.nullness.qual.Nullable;
 
@@ -154,7 +155,7 @@
      */
     <T> long insertAll(
         TableReference ref,
-        List<ValueInSingleWindow<TableRow>> rowList,
+        List<FailsafeValueInSingleWindow<TableRow, TableRow>> rowList,
         @Nullable List<String> insertIdList,
         InsertRetryPolicy retryPolicy,
         List<ValueInSingleWindow<T>> failedInserts,
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java
index 5935a76..0c94bcf 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java
@@ -44,10 +44,11 @@
 import com.google.api.services.bigquery.model.JobReference;
 import com.google.api.services.bigquery.model.JobStatistics;
 import com.google.api.services.bigquery.model.JobStatus;
+import com.google.api.services.bigquery.model.QueryRequest;
+import com.google.api.services.bigquery.model.QueryResponse;
 import com.google.api.services.bigquery.model.Table;
 import com.google.api.services.bigquery.model.TableDataInsertAllRequest;
 import com.google.api.services.bigquery.model.TableDataInsertAllResponse;
-import com.google.api.services.bigquery.model.TableDataList;
 import com.google.api.services.bigquery.model.TableReference;
 import com.google.api.services.bigquery.model.TableRow;
 import com.google.auth.Credentials;
@@ -92,6 +93,7 @@
 import org.apache.beam.sdk.util.FluentBackoff;
 import org.apache.beam.sdk.util.Histogram;
 import org.apache.beam.sdk.util.ReleaseInfo;
+import org.apache.beam.sdk.values.FailsafeValueInSingleWindow;
 import org.apache.beam.sdk.values.ValueInSingleWindow;
 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.collect.ImmutableList;
@@ -105,6 +107,7 @@
  * An implementation of {@link BigQueryServices} that actually communicates with the cloud BigQuery
  * service.
  */
+@SuppressWarnings({"keyfor", "nullness"}) // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class BigQueryServicesImpl implements BigQueryServices {
 
   private static final Logger LOG = LoggerFactory.getLogger(BigQueryServicesImpl.class);
@@ -624,11 +627,22 @@
     @VisibleForTesting
     boolean isTableEmpty(TableReference tableRef, BackOff backoff, Sleeper sleeper)
         throws IOException, InterruptedException {
-      TableDataList dataList =
+      QueryResponse response =
           executeWithRetries(
               client
-                  .tabledata()
-                  .list(tableRef.getProjectId(), tableRef.getDatasetId(), tableRef.getTableId())
+                  .jobs()
+                  .query(
+                      tableRef.getProjectId(),
+                      new QueryRequest()
+                          .setQuery(
+                              // Attempts to fetch a single row, if found returns false,
+                              // otherwise empty result. Runs quickly on large datasets.
+                              "SELECT false FROM (SELECT AS STRUCT * FROM `"
+                                  + tableRef.getDatasetId()
+                                  + "`.`"
+                                  + tableRef.getTableId()
+                                  + "` LIMIT 1) AS i WHERE i IS NOT NULL")
+                          .setUseLegacySql(false))
                   .setPrettyPrint(false),
               String.format(
                   "Unable to list table data: %s, aborting after %d retries.",
@@ -636,7 +650,7 @@
               sleeper,
               backoff,
               DONT_RETRY_NOT_FOUND);
-      return dataList.getRows() == null || dataList.getRows().isEmpty();
+      return response.getRows() == null || response.getRows().isEmpty();
     }
 
     /**
@@ -752,7 +766,7 @@
     @VisibleForTesting
     <T> long insertAll(
         TableReference ref,
-        List<ValueInSingleWindow<TableRow>> rowList,
+        List<FailsafeValueInSingleWindow<TableRow, TableRow>> rowList,
         @Nullable List<String> insertIdList,
         BackOff backoff,
         FluentBackoff rateLimitBackoffFactory,
@@ -781,13 +795,13 @@
       List<TableDataInsertAllResponse.InsertErrors> allErrors = new ArrayList<>();
       // These lists contain the rows to publish. Initially the contain the entire list.
       // If there are failures, they will contain only the failed rows to be retried.
-      List<ValueInSingleWindow<TableRow>> rowsToPublish = rowList;
+      List<FailsafeValueInSingleWindow<TableRow, TableRow>> rowsToPublish = rowList;
       List<String> idsToPublish = null;
       if (!ignoreInsertIds) {
         idsToPublish = insertIdList;
       }
       while (true) {
-        List<ValueInSingleWindow<TableRow>> retryRows = new ArrayList<>();
+        List<FailsafeValueInSingleWindow<TableRow, TableRow>> retryRows = new ArrayList<>();
         List<String> retryIds = (idsToPublish != null) ? new ArrayList<>() : null;
 
         int strideIndex = 0;
@@ -944,7 +958,7 @@
     @Override
     public <T> long insertAll(
         TableReference ref,
-        List<ValueInSingleWindow<TableRow>> rowList,
+        List<FailsafeValueInSingleWindow<TableRow, TableRow>> rowList,
         @Nullable List<String> insertIdList,
         InsertRetryPolicy retryPolicy,
         List<ValueInSingleWindow<T>> failedInserts,
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySourceBase.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySourceBase.java
index 26d200b..74b5cd5 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySourceBase.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySourceBase.java
@@ -64,6 +64,7 @@
  *
  * ...
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 abstract class BigQuerySourceBase<T> extends BoundedSource<T> {
   private static final Logger LOG = LoggerFactory.getLogger(BigQuerySourceBase.class);
 
@@ -225,9 +226,8 @@
   }
 
   private static class TableSchemaFunction implements Serializable, Function<String, TableSchema> {
-    @Nullable
     @Override
-    public TableSchema apply(@Nullable String input) {
+    public @Nullable TableSchema apply(@Nullable String input) {
       return BigQueryHelpers.fromJsonString(input, TableSchema.class);
     }
   }
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageQuerySource.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageQuerySource.java
index f1c9daf..4e43f4c 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageQuerySource.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageQuerySource.java
@@ -34,6 +34,7 @@
 import org.checkerframework.checker.nullness.qual.Nullable;
 
 /** A {@link org.apache.beam.sdk.io.Source} representing reading the results of a query. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BigQueryStorageQuerySource<T> extends BigQueryStorageSourceBase<T> {
 
   public static <T> BigQueryStorageQuerySource<T> create(
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageSourceBase.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageSourceBase.java
index 06627f4..f26d282 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageSourceBase.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageSourceBase.java
@@ -44,6 +44,7 @@
  * A base class for {@link BoundedSource} implementations which read from BigQuery using the
  * BigQuery storage API.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 abstract class BigQueryStorageSourceBase<T> extends BoundedSource<T> {
 
   private static final Logger LOG = LoggerFactory.getLogger(BigQueryStorageSourceBase.class);
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamSource.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamSource.java
index 936e11a..085d7f5 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamSource.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageStreamSource.java
@@ -53,6 +53,7 @@
 import org.slf4j.LoggerFactory;
 
 /** A {@link org.apache.beam.sdk.io.Source} representing a single stream in a read session. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BigQueryStorageStreamSource<T> extends BoundedSource<T> {
 
   private static final Logger LOG = LoggerFactory.getLogger(BigQueryStorageStreamSource.class);
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageTableSource.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageTableSource.java
index fe6cb1a..9135290 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageTableSource.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryStorageTableSource.java
@@ -37,6 +37,7 @@
 import org.slf4j.LoggerFactory;
 
 /** A {@link org.apache.beam.sdk.io.Source} representing reading from a table. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BigQueryStorageTableSource<T> extends BigQueryStorageSourceBase<T> {
 
   private static final Logger LOG = LoggerFactory.getLogger(BigQueryStorageTableSource.class);
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTableSource.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTableSource.java
index c6717fe..782f452 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTableSource.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTableSource.java
@@ -31,6 +31,7 @@
 
 /** A {@link BigQuerySourceBase} for reading BigQuery tables. */
 @VisibleForTesting
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class BigQueryTableSource<T> extends BigQuerySourceBase<T> {
   private static final Logger LOG = LoggerFactory.getLogger(BigQueryTableSource.class);
 
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTableSourceDef.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTableSourceDef.java
index 01a5714..f28b8d2 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTableSourceDef.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTableSourceDef.java
@@ -33,6 +33,7 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class BigQueryTableSourceDef implements BigQuerySourceDef {
   private static final Logger LOG = LoggerFactory.getLogger(BigQueryTableSourceDef.class);
 
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 67f6f38..9f62f2b 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
@@ -65,6 +65,7 @@
 import org.joda.time.format.DateTimeFormatterBuilder;
 
 /** Utility methods for BigQuery related operations. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BigQueryUtils {
 
   /** Options for how to convert BigQuery data to Beam data. */
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/CreateTables.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/CreateTables.java
index 69722cc..ec33e80 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/CreateTables.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/CreateTables.java
@@ -45,6 +45,7 @@
  * Creates any tables needed before performing streaming writes to the tables. This is a side-effect
  * {@link DoFn}, and returns the original collection unchanged.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class CreateTables<DestinationT, ElementT>
     extends PTransform<
         PCollection<KV<DestinationT, ElementT>>, PCollection<KV<TableDestination, ElementT>>> {
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/DynamicDestinations.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/DynamicDestinations.java
index ba9acba..bf6f046 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/DynamicDestinations.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/DynamicDestinations.java
@@ -73,6 +73,7 @@
  * be a compact type with an efficient coder, as these objects may be used as a key in a {@link
  * org.apache.beam.sdk.transforms.GroupByKey}.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public abstract class DynamicDestinations<T, DestinationT> implements Serializable {
   interface SideInputAccessor {
     <SideInputT> SideInputT sideInput(PCollectionView<SideInputT> view);
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/DynamicDestinationsHelpers.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/DynamicDestinationsHelpers.java
index 830c7f8..aae19c2 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/DynamicDestinationsHelpers.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/DynamicDestinationsHelpers.java
@@ -49,6 +49,7 @@
 import org.slf4j.LoggerFactory;
 
 /** Contains some useful helper instances of {@link DynamicDestinations}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class DynamicDestinationsHelpers {
   private static final Logger LOG = LoggerFactory.getLogger(DynamicDestinationsHelpers.class);
 
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/ErrorContainer.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/ErrorContainer.java
index 8b05bd0..72f44ce 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/ErrorContainer.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/ErrorContainer.java
@@ -22,6 +22,7 @@
 import com.google.api.services.bigquery.model.TableRow;
 import java.io.Serializable;
 import java.util.List;
+import org.apache.beam.sdk.values.FailsafeValueInSingleWindow;
 import org.apache.beam.sdk.values.ValueInSingleWindow;
 
 /**
@@ -29,19 +30,24 @@
  *
  * @param <T>
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public interface ErrorContainer<T> extends Serializable {
   void add(
       List<ValueInSingleWindow<T>> failedInserts,
       TableDataInsertAllResponse.InsertErrors error,
       TableReference ref,
-      ValueInSingleWindow<TableRow> tableRow);
+      FailsafeValueInSingleWindow<TableRow, TableRow> tableRow);
 
   ErrorContainer<TableRow> TABLE_ROW_ERROR_CONTAINER =
-      (failedInserts, error, ref, tableRow) -> failedInserts.add(tableRow);
+      (failedInserts, error, ref, tableRow) ->
+          failedInserts.add(
+              ValueInSingleWindow.of(
+                  tableRow.getFailsafeValue(), tableRow.getTimestamp(),
+                  tableRow.getWindow(), tableRow.getPane()));
 
   ErrorContainer<BigQueryInsertError> BIG_QUERY_INSERT_ERROR_ERROR_CONTAINER =
       (failedInserts, error, ref, tableRow) -> {
-        BigQueryInsertError err = new BigQueryInsertError(tableRow.getValue(), error, ref);
+        BigQueryInsertError err = new BigQueryInsertError(tableRow.getFailsafeValue(), error, ref);
         failedInserts.add(
             ValueInSingleWindow.of(
                 err, tableRow.getTimestamp(), tableRow.getWindow(), tableRow.getPane()));
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/PrepareWrite.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/PrepareWrite.java
index 716ce68..6d7a1ef 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/PrepareWrite.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/PrepareWrite.java
@@ -37,6 +37,7 @@
  * which tables each element is written to, and format the element into a {@link TableRow} using the
  * user-supplied format function.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class PrepareWrite<InputT, DestinationT, OutputT>
     extends PTransform<PCollection<InputT>, PCollection<KV<DestinationT, OutputT>>> {
   private DynamicDestinations<InputT, DestinationT> dynamicDestinations;
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/ReifyAsIterable.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/ReifyAsIterable.java
index 0088d06..4b56eb6 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/ReifyAsIterable.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/ReifyAsIterable.java
@@ -31,6 +31,7 @@
  * This transforms turns a side input into a singleton PCollection that can be used as the main
  * input for another transform.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ReifyAsIterable<T> extends PTransform<PCollection<T>, PCollection<Iterable<T>>> {
   @Override
   public PCollection<Iterable<T>> expand(PCollection<T> input) {
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/RowWriterFactory.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/RowWriterFactory.java
index 7229957..1c70fd2 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/RowWriterFactory.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/RowWriterFactory.java
@@ -24,6 +24,7 @@
 import org.apache.avro.io.DatumWriter;
 import org.apache.beam.sdk.transforms.SerializableFunction;
 
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 abstract class RowWriterFactory<ElementT, DestinationT> implements Serializable {
   private RowWriterFactory() {}
 
@@ -40,23 +41,35 @@
       String tempFilePrefix, DestinationT destination) throws Exception;
 
   static <ElementT, DestinationT> RowWriterFactory<ElementT, DestinationT> tableRows(
-      SerializableFunction<ElementT, TableRow> toRow) {
-    return new TableRowWriterFactory<ElementT, DestinationT>(toRow);
+      SerializableFunction<ElementT, TableRow> toRow,
+      SerializableFunction<ElementT, TableRow> toFailsafeRow) {
+    return new TableRowWriterFactory<ElementT, DestinationT>(toRow, toFailsafeRow);
   }
 
   static final class TableRowWriterFactory<ElementT, DestinationT>
       extends RowWriterFactory<ElementT, DestinationT> {
 
     private final SerializableFunction<ElementT, TableRow> toRow;
+    private final SerializableFunction<ElementT, TableRow> toFailsafeRow;
 
-    private TableRowWriterFactory(SerializableFunction<ElementT, TableRow> toRow) {
+    private TableRowWriterFactory(
+        SerializableFunction<ElementT, TableRow> toRow,
+        SerializableFunction<ElementT, TableRow> toFailsafeRow) {
       this.toRow = toRow;
+      this.toFailsafeRow = toFailsafeRow;
     }
 
     public SerializableFunction<ElementT, TableRow> getToRowFn() {
       return toRow;
     }
 
+    public SerializableFunction<ElementT, TableRow> getToFailsafeRowFn() {
+      if (toFailsafeRow == null) {
+        return toRow;
+      }
+      return toFailsafeRow;
+    }
+
     @Override
     public OutputType getOutputType() {
       return OutputType.JsonTableRow;
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StreamingInserts.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StreamingInserts.java
index d00adbb..10bf39d 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StreamingInserts.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StreamingInserts.java
@@ -29,6 +29,7 @@
  * PTransform that performs streaming BigQuery write. To increase consistency, it leverages
  * BigQuery's best effort de-dup mechanism.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class StreamingInserts<DestinationT, ElementT>
     extends PTransform<PCollection<KV<DestinationT, ElementT>>, WriteResult> {
   private BigQueryServices bigQueryServices;
@@ -42,13 +43,15 @@
   private final String kmsKey;
   private final Coder<ElementT> elementCoder;
   private final SerializableFunction<ElementT, TableRow> toTableRow;
+  private final SerializableFunction<ElementT, TableRow> toFailsafeTableRow;
 
   /** Constructor. */
   public StreamingInserts(
       CreateDisposition createDisposition,
       DynamicDestinations<?, DestinationT> dynamicDestinations,
       Coder<ElementT> elementCoder,
-      SerializableFunction<ElementT, TableRow> toTableRow) {
+      SerializableFunction<ElementT, TableRow> toTableRow,
+      SerializableFunction<ElementT, TableRow> toFailsafeTableRow) {
     this(
         createDisposition,
         dynamicDestinations,
@@ -60,6 +63,7 @@
         false,
         elementCoder,
         toTableRow,
+        toFailsafeTableRow,
         null);
   }
 
@@ -75,6 +79,7 @@
       boolean ignoreInsertIds,
       Coder<ElementT> elementCoder,
       SerializableFunction<ElementT, TableRow> toTableRow,
+      SerializableFunction<ElementT, TableRow> toFailsafeTableRow,
       String kmsKey) {
     this.createDisposition = createDisposition;
     this.dynamicDestinations = dynamicDestinations;
@@ -86,6 +91,7 @@
     this.ignoreInsertIds = ignoreInsertIds;
     this.elementCoder = elementCoder;
     this.toTableRow = toTableRow;
+    this.toFailsafeTableRow = toFailsafeTableRow;
     this.kmsKey = kmsKey;
   }
 
@@ -103,6 +109,7 @@
         ignoreInsertIds,
         elementCoder,
         toTableRow,
+        toFailsafeTableRow,
         kmsKey);
   }
 
@@ -119,6 +126,7 @@
         ignoreInsertIds,
         elementCoder,
         toTableRow,
+        toFailsafeTableRow,
         kmsKey);
   }
 
@@ -134,6 +142,7 @@
         ignoreInsertIds,
         elementCoder,
         toTableRow,
+        toFailsafeTableRow,
         kmsKey);
   }
 
@@ -149,6 +158,7 @@
         ignoreInsertIds,
         elementCoder,
         toTableRow,
+        toFailsafeTableRow,
         kmsKey);
   }
 
@@ -164,6 +174,7 @@
         ignoreInsertIds,
         elementCoder,
         toTableRow,
+        toFailsafeTableRow,
         kmsKey);
   }
 
@@ -179,6 +190,7 @@
         ignoreInsertIds,
         elementCoder,
         toTableRow,
+        toFailsafeTableRow,
         kmsKey);
   }
 
@@ -194,6 +206,7 @@
         ignoreInsertIds,
         elementCoder,
         toTableRow,
+        toFailsafeTableRow,
         kmsKey);
   }
 
@@ -215,6 +228,7 @@
             .withIgnoreUnknownValues(ignoreUnknownValues)
             .withIgnoreInsertIds(ignoreInsertIds)
             .withElementCoder(elementCoder)
-            .withToTableRow(toTableRow));
+            .withToTableRow(toTableRow)
+            .withToFailsafeTableRow(toFailsafeTableRow));
   }
 }
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StreamingWriteFn.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StreamingWriteFn.java
index a956f0d..d295231 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StreamingWriteFn.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StreamingWriteFn.java
@@ -32,6 +32,7 @@
 import org.apache.beam.sdk.transforms.windowing.PaneInfo;
 import org.apache.beam.sdk.util.Histogram;
 import org.apache.beam.sdk.util.SystemDoFnInternal;
+import org.apache.beam.sdk.values.FailsafeValueInSingleWindow;
 import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.ShardedKey;
 import org.apache.beam.sdk.values.TupleTag;
@@ -46,6 +47,7 @@
 /** Implementation of DoFn to perform streaming BigQuery write. */
 @SystemDoFnInternal
 @VisibleForTesting
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class StreamingWriteFn<ErrorT, ElementT>
     extends DoFn<KV<ShardedKey<String>, TableRowInfo<ElementT>>, Void> {
   private static final Logger LOG = LoggerFactory.getLogger(StreamingWriteFn.class);
@@ -58,9 +60,10 @@
   private final boolean ignoreUnknownValues;
   private final boolean ignoreInsertIds;
   private final SerializableFunction<ElementT, TableRow> toTableRow;
+  private final SerializableFunction<ElementT, TableRow> toFailsafeTableRow;
 
   /** JsonTableRows to accumulate BigQuery rows in order to batch writes. */
-  private transient Map<String, List<ValueInSingleWindow<TableRow>>> tableRows;
+  private transient Map<String, List<FailsafeValueInSingleWindow<TableRow, TableRow>>> tableRows;
 
   /** The list of unique ids for each BigQuery table row. */
   private transient Map<String, List<String>> uniqueIdsForTableRows;
@@ -79,7 +82,8 @@
       boolean skipInvalidRows,
       boolean ignoreUnknownValues,
       boolean ignoreInsertIds,
-      SerializableFunction<ElementT, TableRow> toTableRow) {
+      SerializableFunction<ElementT, TableRow> toTableRow,
+      SerializableFunction<ElementT, TableRow> toFailsafeTableRow) {
     this.bqServices = bqServices;
     this.retryPolicy = retryPolicy;
     this.failedOutputTag = failedOutputTag;
@@ -88,6 +92,7 @@
     this.ignoreUnknownValues = ignoreUnknownValues;
     this.ignoreInsertIds = ignoreInsertIds;
     this.toTableRow = toTableRow;
+    this.toFailsafeTableRow = toFailsafeTableRow;
   }
 
   @Setup
@@ -120,13 +125,14 @@
       BoundedWindow window,
       PaneInfo pane) {
     String tableSpec = element.getKey().getKey();
-    List<ValueInSingleWindow<TableRow>> rows =
+    List<FailsafeValueInSingleWindow<TableRow, TableRow>> rows =
         BigQueryHelpers.getOrCreateMapListValue(tableRows, tableSpec);
     List<String> uniqueIds =
         BigQueryHelpers.getOrCreateMapListValue(uniqueIdsForTableRows, tableSpec);
 
     TableRow tableRow = toTableRow.apply(element.getValue().tableRow);
-    rows.add(ValueInSingleWindow.of(tableRow, timestamp, window, pane));
+    TableRow failsafeTableRow = toFailsafeTableRow.apply(element.getValue().tableRow);
+    rows.add(FailsafeValueInSingleWindow.of(tableRow, timestamp, window, pane, failsafeTableRow));
     uniqueIds.add(element.getValue().uniqueId);
   }
 
@@ -135,7 +141,8 @@
   public void finishBundle(FinishBundleContext context) throws Exception {
     List<ValueInSingleWindow<ErrorT>> failedInserts = Lists.newArrayList();
     BigQueryOptions options = context.getPipelineOptions().as(BigQueryOptions.class);
-    for (Map.Entry<String, List<ValueInSingleWindow<TableRow>>> entry : tableRows.entrySet()) {
+    for (Map.Entry<String, List<FailsafeValueInSingleWindow<TableRow, TableRow>>> entry :
+        tableRows.entrySet()) {
       TableReference tableReference = BigQueryHelpers.parseTableSpec(entry.getKey());
       flushRows(
           tableReference,
@@ -173,7 +180,7 @@
   /** Writes the accumulated rows into BigQuery with streaming API. */
   private void flushRows(
       TableReference tableReference,
-      List<ValueInSingleWindow<TableRow>> tableRows,
+      List<FailsafeValueInSingleWindow<TableRow, TableRow>> tableRows,
       List<String> uniqueIds,
       BigQueryOptions options,
       List<ValueInSingleWindow<ErrorT>> failedInserts)
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StreamingWriteTables.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StreamingWriteTables.java
index 81f097a..2170588 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StreamingWriteTables.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StreamingWriteTables.java
@@ -45,6 +45,7 @@
  * <p>This transform assumes that all destination tables already exist by the time it sees a write
  * for that table.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class StreamingWriteTables<ElementT>
     extends PTransform<PCollection<KV<TableDestination, ElementT>>, WriteResult> {
   private BigQueryServices bigQueryServices;
@@ -56,6 +57,7 @@
   private final boolean ignoreInsertIds;
   private final Coder<ElementT> elementCoder;
   private final SerializableFunction<ElementT, TableRow> toTableRow;
+  private final SerializableFunction<ElementT, TableRow> toFailsafeTableRow;
 
   public StreamingWriteTables() {
     this(
@@ -66,7 +68,8 @@
         false, // ignoreUnknownValues
         false, // ignoreInsertIds
         null, // elementCoder
-        null); // toTableRow
+        null, // toTableRow
+        null); // toFailsafeTableRow
   }
 
   private StreamingWriteTables(
@@ -77,7 +80,8 @@
       boolean ignoreUnknownValues,
       boolean ignoreInsertIds,
       Coder<ElementT> elementCoder,
-      SerializableFunction<ElementT, TableRow> toTableRow) {
+      SerializableFunction<ElementT, TableRow> toTableRow,
+      SerializableFunction<ElementT, TableRow> toFailsafeTableRow) {
     this.bigQueryServices = bigQueryServices;
     this.retryPolicy = retryPolicy;
     this.extendedErrorInfo = extendedErrorInfo;
@@ -86,6 +90,7 @@
     this.ignoreInsertIds = ignoreInsertIds;
     this.elementCoder = elementCoder;
     this.toTableRow = toTableRow;
+    this.toFailsafeTableRow = toFailsafeTableRow;
   }
 
   StreamingWriteTables<ElementT> withTestServices(BigQueryServices bigQueryServices) {
@@ -97,7 +102,8 @@
         ignoreUnknownValues,
         ignoreInsertIds,
         elementCoder,
-        toTableRow);
+        toTableRow,
+        toFailsafeTableRow);
   }
 
   StreamingWriteTables<ElementT> withInsertRetryPolicy(InsertRetryPolicy retryPolicy) {
@@ -109,7 +115,8 @@
         ignoreUnknownValues,
         ignoreInsertIds,
         elementCoder,
-        toTableRow);
+        toTableRow,
+        toFailsafeTableRow);
   }
 
   StreamingWriteTables<ElementT> withExtendedErrorInfo(boolean extendedErrorInfo) {
@@ -121,7 +128,8 @@
         ignoreUnknownValues,
         ignoreInsertIds,
         elementCoder,
-        toTableRow);
+        toTableRow,
+        toFailsafeTableRow);
   }
 
   StreamingWriteTables<ElementT> withSkipInvalidRows(boolean skipInvalidRows) {
@@ -133,7 +141,8 @@
         ignoreUnknownValues,
         ignoreInsertIds,
         elementCoder,
-        toTableRow);
+        toTableRow,
+        toFailsafeTableRow);
   }
 
   StreamingWriteTables<ElementT> withIgnoreUnknownValues(boolean ignoreUnknownValues) {
@@ -145,7 +154,8 @@
         ignoreUnknownValues,
         ignoreInsertIds,
         elementCoder,
-        toTableRow);
+        toTableRow,
+        toFailsafeTableRow);
   }
 
   StreamingWriteTables<ElementT> withIgnoreInsertIds(boolean ignoreInsertIds) {
@@ -157,7 +167,8 @@
         ignoreUnknownValues,
         ignoreInsertIds,
         elementCoder,
-        toTableRow);
+        toTableRow,
+        toFailsafeTableRow);
   }
 
   StreamingWriteTables<ElementT> withElementCoder(Coder<ElementT> elementCoder) {
@@ -169,7 +180,8 @@
         ignoreUnknownValues,
         ignoreInsertIds,
         elementCoder,
-        toTableRow);
+        toTableRow,
+        toFailsafeTableRow);
   }
 
   StreamingWriteTables<ElementT> withToTableRow(
@@ -182,7 +194,22 @@
         ignoreUnknownValues,
         ignoreInsertIds,
         elementCoder,
-        toTableRow);
+        toTableRow,
+        toFailsafeTableRow);
+  }
+
+  StreamingWriteTables<ElementT> withToFailsafeTableRow(
+      SerializableFunction<ElementT, TableRow> toFailsafeTableRow) {
+    return new StreamingWriteTables<>(
+        bigQueryServices,
+        retryPolicy,
+        extendedErrorInfo,
+        skipInvalidRows,
+        ignoreUnknownValues,
+        ignoreInsertIds,
+        elementCoder,
+        toTableRow,
+        toFailsafeTableRow);
   }
 
   @Override
@@ -264,7 +291,8 @@
                             skipInvalidRows,
                             ignoreUnknownValues,
                             ignoreInsertIds,
-                            toTableRow))
+                            toTableRow,
+                            toFailsafeTableRow))
                     .withOutputTags(mainOutputTag, TupleTagList.of(failedInsertsTag)));
     PCollection<T> failedInserts = tuple.get(failedInsertsTag);
     failedInserts.setCoder(coder);
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableDestination.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableDestination.java
index 927d8f8..549c92d 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableDestination.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableDestination.java
@@ -25,6 +25,7 @@
 import org.checkerframework.checker.nullness.qual.Nullable;
 
 /** Encapsulates a BigQuery table destination. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class TableDestination implements Serializable {
   private static final long serialVersionUID = 1L;
   private final String tableSpec;
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableDestinationCoder.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableDestinationCoder.java
index edb994c..6ffd031 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableDestinationCoder.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableDestinationCoder.java
@@ -27,6 +27,7 @@
 import org.apache.beam.sdk.coders.StringUtf8Coder;
 
 /** A coder for {@link TableDestination} objects. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class TableDestinationCoder extends AtomicCoder<TableDestination> {
   private static final TableDestinationCoder INSTANCE = new TableDestinationCoder();
   private static final Coder<String> tableSpecCoder = StringUtf8Coder.of();
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableDestinationCoderV2.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableDestinationCoderV2.java
index 2fee354..768d751 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableDestinationCoderV2.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableDestinationCoderV2.java
@@ -31,6 +31,7 @@
  * old {@link TableDestinationCoder}) for compatibility reasons. The old coder is kept around for
  * the same compatibility reasons.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class TableDestinationCoderV2 extends AtomicCoder<TableDestination> {
   private static final TableDestinationCoderV2 INSTANCE = new TableDestinationCoderV2();
   private static final Coder<String> timePartitioningCoder = NullableCoder.of(StringUtf8Coder.of());
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableDestinationCoderV3.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableDestinationCoderV3.java
index 36d16d2..b4be8c8 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableDestinationCoderV3.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableDestinationCoderV3.java
@@ -31,6 +31,7 @@
  * options on {@link BigQueryIO.Write}, otherwise {@link TableDestinationCoderV2} will be used and
  * clustering information will be discarded.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class TableDestinationCoderV3 extends AtomicCoder<TableDestination> {
   private static final TableDestinationCoderV3 INSTANCE = new TableDestinationCoderV3();
   private static final Coder<String> timePartitioningCoder = NullableCoder.of(StringUtf8Coder.of());
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TagWithUniqueIds.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TagWithUniqueIds.java
index fbc95c6..40ab0fa 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TagWithUniqueIds.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TagWithUniqueIds.java
@@ -32,6 +32,7 @@
  * sequential number.
  */
 @VisibleForTesting
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class TagWithUniqueIds<ElementT>
     extends DoFn<KV<ShardedKey<String>, ElementT>, KV<ShardedKey<String>, TableRowInfo<ElementT>>> {
   private transient String randomUUID;
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TestBigQuery.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TestBigQuery.java
index a5a5218..9983441 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TestBigQuery.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TestBigQuery.java
@@ -65,6 +65,7 @@
  *
  * <p>Deletes the table on test shutdown.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class TestBigQuery implements TestRule {
   private static final DateTimeFormatter DATETIME_FORMAT =
       DateTimeFormat.forPattern("YYYY_MM_dd_HH_mm_ss_SSS");
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteBundlesToFiles.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteBundlesToFiles.java
index d5f4b99..a1afcad 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteBundlesToFiles.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteBundlesToFiles.java
@@ -53,6 +53,7 @@
  * the element will be spilled into the output, and the {@link WriteGroupedRecordsToFiles} transform
  * will take care of writing it to a file.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class WriteBundlesToFiles<DestinationT, ElementT>
     extends DoFn<KV<DestinationT, ElementT>, Result<DestinationT>> {
 
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WritePartition.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WritePartition.java
index e651cce..79ff3d3 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WritePartition.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WritePartition.java
@@ -33,6 +33,7 @@
  * Partitions temporary files based on number of files and file sizes. Output key is a pair of
  * tablespec and the list of files corresponding to each partition of that table.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class WritePartition<DestinationT>
     extends DoFn<
         Iterable<WriteBundlesToFiles.Result<DestinationT>>,
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteRename.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteRename.java
index 5ceecef..3461b01 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteRename.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteRename.java
@@ -46,6 +46,7 @@
  * Copies temporary tables to destination table. The input element is an {@link Iterable} that
  * provides the list of all temporary tables created for a given {@link TableDestination}.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class WriteRename extends DoFn<Iterable<KV<TableDestination, String>>, Void> {
   private static final Logger LOG = LoggerFactory.getLogger(WriteRename.class);
 
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteResult.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteResult.java
index b625eb8..215c721 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteResult.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteResult.java
@@ -31,6 +31,7 @@
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
 
 /** The result of a {@link BigQueryIO.Write} transform. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public final class WriteResult implements POutput {
   private final Pipeline pipeline;
   private final TupleTag<TableRow> failedInsertsTag;
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteTables.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteTables.java
index 45f5ea4..eb7cd69 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteTables.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteTables.java
@@ -83,6 +83,7 @@
  * the data directly into the final table, skipping temporary tables. In this case, the output
  * {@link KV} maps the final table to itself.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class WriteTables<DestinationT>
     extends PTransform<
         PCollection<KV<ShardedKey<DestinationT>, List<String>>>,
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfig.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfig.java
index 6daf8f8..de32a86 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfig.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfig.java
@@ -34,6 +34,7 @@
 
 /** Configuration for a Cloud Bigtable client. */
 @AutoValue
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 abstract class BigtableConfig implements Serializable {
 
   /** Returns the project id being written to. */
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java
index 556b564..edd708c 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java
@@ -172,6 +172,7 @@
  * pipeline. Please refer to the documentation of corresponding {@link PipelineRunner
  * PipelineRunners} for more details.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BigtableIO {
   private static final Logger LOG = LoggerFactory.getLogger(BigtableIO.class);
 
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableReadOptions.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableReadOptions.java
index 676fada..722f6fd 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableReadOptions.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableReadOptions.java
@@ -31,6 +31,7 @@
 
 /** Configuration for which values to read from Bigtable. */
 @AutoValue
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 abstract class BigtableReadOptions implements Serializable {
 
   /** Returns the row filter to use. */
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java
index 478de53..95a90adc 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java
@@ -57,6 +57,7 @@
  * An implementation of {@link BigtableService} that actually communicates with the Cloud Bigtable
  * service.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class BigtableServiceImpl implements BigtableService {
   private static final Logger LOG = LoggerFactory.getLogger(BigtableServiceImpl.class);
 
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteResult.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteResult.java
index d1a6bfd..9198c97 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteResult.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteResult.java
@@ -27,6 +27,7 @@
  */
 @DefaultCoder(BigtableWriteResultCoder.class)
 @AutoValue
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public abstract class BigtableWriteResult {
   public static BigtableWriteResult create(long rowsWritten) {
     return new AutoValue_BigtableWriteResult(rowsWritten);
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DataStoreV1SchemaIOProvider.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DataStoreV1SchemaIOProvider.java
index 216ccf4..d516114 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DataStoreV1SchemaIOProvider.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DataStoreV1SchemaIOProvider.java
@@ -40,6 +40,7 @@
  */
 @Internal
 @AutoService(SchemaIOProvider.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DataStoreV1SchemaIOProvider implements SchemaIOProvider {
   public static final String KEY_FIELD_PROPERTY = "keyField";
   static final String DEFAULT_KEY_FIELD = "__key__";
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1.java
index bed9c9b..10a13f5 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1.java
@@ -195,6 +195,7 @@
  *
  * @see PipelineRunner
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DatastoreV1 {
 
   // A package-private constructor to prevent direct instantiation from outside of this package
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/EntityToRow.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/EntityToRow.java
index 11e320e..8a03d67 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/EntityToRow.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/EntityToRow.java
@@ -36,6 +36,7 @@
 import org.slf4j.LoggerFactory;
 
 /** A {@code PTransform} to perform a conversion of {@link Entity} to {@link Row}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class EntityToRow extends PTransform<PCollection<Entity>, PCollection<Row>> {
   private final Schema schema;
   private final String keyField;
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/RowToEntity.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/RowToEntity.java
index 39dc27f..ad0f7dd 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/RowToEntity.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/RowToEntity.java
@@ -41,6 +41,7 @@
 import org.slf4j.LoggerFactory;
 
 /** A {@code PTransform} to perform a conversion of {@link Row} to {@link Entity}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class RowToEntity extends PTransform<PCollection<Row>, PCollection<Entity>> {
   private final Supplier<String> keySupplier;
   private final String kind;
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIO.java
index 86dcc18..e8410d3 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIO.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIO.java
@@ -19,6 +19,7 @@
 
 import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkState;
 
+import com.fasterxml.jackson.core.JsonProcessingException;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.google.api.services.healthcare.v1beta1.model.DeidentifyConfig;
 import com.google.api.services.healthcare.v1beta1.model.HttpBody;
@@ -84,7 +85,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.ImmutableMap;
 import org.checkerframework.checker.nullness.qual.Nullable;
-import org.codehaus.jackson.JsonProcessingException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -199,6 +199,7 @@
  * }***
  * </pre>
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class FhirIO {
 
   /**
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HL7v2IO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HL7v2IO.java
index 35d61ea..c4ea414 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HL7v2IO.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HL7v2IO.java
@@ -153,6 +153,7 @@
  * }***
  * </pre>
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class HL7v2IO {
 
   /** Write HL7v2 Messages to a store. */
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HL7v2Message.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HL7v2Message.java
index c4cbe9e..0f705fe 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HL7v2Message.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HL7v2Message.java
@@ -25,6 +25,7 @@
 import org.checkerframework.checker.nullness.qual.Nullable;
 
 /** The type HL7v2 message to wrap the {@link Message} model. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class HL7v2Message {
   private final String name;
   private final String messageType;
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HealthcareIOError.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HealthcareIOError.java
index 10737c7..25cd754 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HealthcareIOError.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HealthcareIOError.java
@@ -26,6 +26,7 @@
 
 /** Class for capturing errors on IO operations on Google Cloud Healthcare APIs resources. */
 @DefaultCoder(HealthcareIOErrorCoder.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class HealthcareIOError<T> {
   private T dataResource;
   private String errorMessage;
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HealthcareIOErrorCoder.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HealthcareIOErrorCoder.java
index 6ad7ae0..7892f06 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HealthcareIOErrorCoder.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HealthcareIOErrorCoder.java
@@ -28,6 +28,7 @@
 import org.apache.beam.sdk.coders.TextualIntegerCoder;
 import org.joda.time.Instant;
 
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class HealthcareIOErrorCoder<T> extends CustomCoder<HealthcareIOError<T>> {
   private final Coder<T> originalCoder;
   private static final NullableCoder<String> STRING_CODER = NullableCoder.of(StringUtf8Coder.of());
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HealthcareIOErrorToTableRow.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HealthcareIOErrorToTableRow.java
index bb1dfa7..0831b6a 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HealthcareIOErrorToTableRow.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HealthcareIOErrorToTableRow.java
@@ -57,6 +57,7 @@
  *
  * @param <T> the type parameter for the {@link HealthcareIOError}
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class HealthcareIOErrorToTableRow<T>
     implements SerializableFunction<HealthcareIOError<T>, TableRow> {
 
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HttpHealthcareApiClient.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HttpHealthcareApiClient.java
index b6030a0..6baf85a 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HttpHealthcareApiClient.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/healthcare/HttpHealthcareApiClient.java
@@ -81,6 +81,7 @@
  * mainly to encapsulate the unserializable dependencies, since most generated classes are not
  * serializable in the HTTP client.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class HttpHealthcareApiClient implements HealthcareApiClient, Serializable {
   private static final String USER_AGENT =
       String.format(
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/ExternalWrite.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/ExternalWrite.java
index d4df6f6..4733e32 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/ExternalWrite.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/ExternalWrite.java
@@ -36,6 +36,7 @@
 /** Exposes {@link PubsubIO.Write} as an external transform for cross-language usage. */
 @Experimental
 @AutoService(ExternalTransformRegistrar.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public final class ExternalWrite implements ExternalTransformRegistrar {
   public ExternalWrite() {}
 
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubClient.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubClient.java
index 77cc209..cefa37d 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubClient.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubClient.java
@@ -36,6 +36,7 @@
 import org.checkerframework.checker.nullness.qual.Nullable;
 
 /** An (abstract) helper class for talking to Pubsub via an underlying transport. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public abstract class PubsubClient implements Closeable {
   /** Factory for creating clients. */
   public interface PubsubClientFactory extends Serializable {
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubGrpcClient.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubGrpcClient.java
index ecef734..cb2f03b 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubGrpcClient.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubGrpcClient.java
@@ -63,6 +63,7 @@
 import org.checkerframework.checker.nullness.qual.Nullable;
 
 /** A helper class for talking to Pubsub via grpc. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class PubsubGrpcClient extends PubsubClient {
   private static final int LIST_BATCH_SIZE = 1000;
 
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIO.java
index 2696dcd..315a290 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIO.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIO.java
@@ -88,6 +88,7 @@
  * pipeline. Please refer to the documentation of corresponding {@link PipelineRunner
  * PipelineRunners} for more details.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class PubsubIO {
 
   private static final Logger LOG = LoggerFactory.getLogger(PubsubIO.class);
@@ -603,9 +604,8 @@
     /** User function for parsing PubsubMessage object. */
     abstract @Nullable SerializableFunction<PubsubMessage, T> getParseFn();
 
-    @Nullable
     @Experimental(Kind.SCHEMAS)
-    abstract Schema getBeamSchema();
+    abstract @Nullable Schema getBeamSchema();
 
     abstract @Nullable TypeDescriptor<T> getTypeDescriptor();
 
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubJsonClient.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubJsonClient.java
index 3f449d2..70a2a45 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubJsonClient.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubJsonClient.java
@@ -54,6 +54,7 @@
 import org.checkerframework.checker.nullness.qual.Nullable;
 
 /** A Pubsub client using JSON transport. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class PubsubJsonClient extends PubsubClient {
 
   private static class PubsubJsonClientFactory implements PubsubClientFactory {
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubMessage.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubMessage.java
index b66a850..7ae6615 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubMessage.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubMessage.java
@@ -27,6 +27,7 @@
  * Class representing a Pub/Sub message. Each message contains a single message payload, a map of
  * attached attributes, and a message id.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class PubsubMessage {
 
   private byte[] message;
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubMessageToRow.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubMessageToRow.java
index c5bcd66..062c2a8 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubMessageToRow.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubMessageToRow.java
@@ -49,6 +49,7 @@
 @Internal
 @Experimental
 @AutoValue
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 abstract class PubsubMessageToRow extends PTransform<PCollection<PubsubMessage>, PCollectionTuple>
     implements Serializable {
   static final String TIMESTAMP_FIELD = "event_timestamp";
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubMessageWithAttributesAndMessageIdCoder.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubMessageWithAttributesAndMessageIdCoder.java
index 377bfc6..0bdf991 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubMessageWithAttributesAndMessageIdCoder.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubMessageWithAttributesAndMessageIdCoder.java
@@ -30,6 +30,7 @@
 import org.apache.beam.sdk.values.TypeDescriptor;
 
 /** A coder for PubsubMessage including attributes and the message id from the PubSub server. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class PubsubMessageWithAttributesAndMessageIdCoder extends CustomCoder<PubsubMessage> {
   // A message's payload cannot be null
   private static final Coder<byte[]> PAYLOAD_CODER = ByteArrayCoder.of();
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubMessageWithAttributesCoder.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubMessageWithAttributesCoder.java
index c111177..b540cb8 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubMessageWithAttributesCoder.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubMessageWithAttributesCoder.java
@@ -30,6 +30,7 @@
 import org.apache.beam.sdk.values.TypeDescriptor;
 
 /** A coder for PubsubMessage including attributes. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class PubsubMessageWithAttributesCoder extends CustomCoder<PubsubMessage> {
   // A message's payload can not be null
   private static final Coder<byte[]> PAYLOAD_CODER = ByteArrayCoder.of();
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubMessageWithMessageIdCoder.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubMessageWithMessageIdCoder.java
index f38e14d..9cf1fe7 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubMessageWithMessageIdCoder.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubMessageWithMessageIdCoder.java
@@ -30,6 +30,7 @@
  * A coder for PubsubMessage treating the raw bytes being decoded as the message's payload, with the
  * message id from the PubSub server.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class PubsubMessageWithMessageIdCoder extends CustomCoder<PubsubMessage> {
   private static final Coder<byte[]> PAYLOAD_CODER = ByteArrayCoder.of();
   // A message's messageId cannot be null
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubSchemaIOProvider.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubSchemaIOProvider.java
index dfacdf4..7b5483a 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubSchemaIOProvider.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubSchemaIOProvider.java
@@ -90,6 +90,7 @@
  */
 @Internal
 @AutoService(SchemaIOProvider.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class PubsubSchemaIOProvider implements SchemaIOProvider {
   public static final FieldType VARCHAR = FieldType.STRING;
   public static final FieldType TIMESTAMP = FieldType.DATETIME;
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubTestClient.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubTestClient.java
index 7746bcd..5742d17 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubTestClient.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubTestClient.java
@@ -40,6 +40,7 @@
  * methods. Relies on statics to mimic the Pubsub service, though we try to hide that.
  */
 @Experimental
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class PubsubTestClient extends PubsubClient implements Serializable {
   /**
    * Mimic the state of the simulated Pubsub 'service'.
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSink.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSink.java
index 1078623..ff41700 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSink.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSink.java
@@ -81,6 +81,7 @@
  *       dedup messages.
  * </ul>
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class PubsubUnboundedSink extends PTransform<PCollection<PubsubMessage>, PDone> {
   /** Default maximum number of messages per publish. */
   static final int DEFAULT_PUBLISH_BATCH_SIZE = 1000;
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSource.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSource.java
index eeb9216..f2ac634 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSource.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSource.java
@@ -109,6 +109,7 @@
  *       UnboundedSource.UnboundedReader} instances to execute concurrently and thus hide latency.
  * </ul>
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class PubsubUnboundedSource extends PTransform<PBegin, PCollection<PubsubMessage>> {
   private static final Logger LOG = LoggerFactory.getLogger(PubsubUnboundedSource.class);
 
@@ -1077,9 +1078,8 @@
       return reader;
     }
 
-    @Nullable
     @Override
-    public Coder<PubsubCheckpoint> getCheckpointMarkCoder() {
+    public @Nullable Coder<PubsubCheckpoint> getCheckpointMarkCoder() {
       return CHECKPOINT_CODER;
     }
 
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/RowToPubsubMessage.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/RowToPubsubMessage.java
index 1865b92..a86d9f7 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/RowToPubsubMessage.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/RowToPubsubMessage.java
@@ -38,6 +38,7 @@
  * event_timestamp}, which is either ignored or written to the message attributes, depending on
  * whether config.getValue("timestampAttributeKey") is set.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class RowToPubsubMessage extends PTransform<PCollection<Row>, PCollection<PubsubMessage>> {
   private final boolean useTimestampAttribute;
 
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/TestPubsub.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/TestPubsub.java
index ba7506f..bc95e04 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/TestPubsub.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/TestPubsub.java
@@ -53,6 +53,7 @@
  *
  * <p>Deletes topic and subscription on shutdown.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class TestPubsub implements TestRule {
   private static final DateTimeFormatter DATETIME_FORMAT =
       DateTimeFormat.forPattern("YYYY-MM-dd-HH-mm-ss-SSS");
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/TestPubsubSignal.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/TestPubsubSignal.java
index f9cfc01..2394fed 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/TestPubsubSignal.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/TestPubsubSignal.java
@@ -67,6 +67,7 @@
  *
  * <p>Uses a random temporary Pubsub topic for synchronization.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class TestPubsubSignal implements TestRule {
   private static final Logger LOG = LoggerFactory.getLogger(TestPubsubSignal.class);
   private static final String RESULT_TOPIC_NAME = "result";
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/BufferingPullSubscriber.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/BufferingPullSubscriber.java
deleted file mode 100644
index 975e125..0000000
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/BufferingPullSubscriber.java
+++ /dev/null
@@ -1,116 +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.io.gcp.pubsublite;
-
-import com.google.api.core.ApiService.Listener;
-import com.google.api.core.ApiService.State;
-import com.google.cloud.pubsublite.Offset;
-import com.google.cloud.pubsublite.cloudpubsub.FlowControlSettings;
-import com.google.cloud.pubsublite.internal.ExtractStatus;
-import com.google.cloud.pubsublite.internal.wire.Subscriber;
-import com.google.cloud.pubsublite.internal.wire.SubscriberFactory;
-import com.google.cloud.pubsublite.proto.Cursor;
-import com.google.cloud.pubsublite.proto.FlowControlRequest;
-import com.google.cloud.pubsublite.proto.SeekRequest;
-import com.google.cloud.pubsublite.proto.SequencedMessage;
-import io.grpc.StatusException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.atomic.AtomicReference;
-import java.util.stream.Collectors;
-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.util.concurrent.MoreExecutors;
-import org.checkerframework.checker.nullness.qual.Nullable;
-
-/**
- * A Pub/Sub Lite subscriber that transforms the streaming message delivery to a pull-based one
- * through buffering, enabling easier use from a beam source.
- */
-class BufferingPullSubscriber implements PullSubscriber {
-  private final Subscriber underlying;
-  private final AtomicReference<StatusException> error = new AtomicReference<>();
-  private final LinkedBlockingQueue<SequencedMessage> messages = new LinkedBlockingQueue<>();
-
-  BufferingPullSubscriber(SubscriberFactory factory, FlowControlSettings settings)
-      throws StatusException {
-    underlying =
-        factory.New(
-            newMessages -> {
-              messages.addAll(
-                  newMessages.stream()
-                      .map(message -> message.toProto())
-                      .collect(Collectors.toList()));
-            });
-    underlying.addListener(
-        new Listener() {
-          @Override
-          public void failed(State state, Throwable throwable) {
-            error.set(ExtractStatus.toCanonical(throwable));
-          }
-        },
-        MoreExecutors.directExecutor());
-    underlying.startAsync().awaitRunning();
-    underlying.allowFlow(
-        FlowControlRequest.newBuilder()
-            .setAllowedMessages(settings.messagesOutstanding())
-            .setAllowedBytes(settings.bytesOutstanding())
-            .build());
-  }
-
-  BufferingPullSubscriber(
-      SubscriberFactory factory, FlowControlSettings settings, Offset initialLocation)
-      throws StatusException {
-    this(factory, settings);
-    try {
-      underlying
-          .seek(
-              SeekRequest.newBuilder()
-                  .setCursor(Cursor.newBuilder().setOffset(initialLocation.value()))
-                  .build())
-          .get();
-    } catch (InterruptedException e) {
-      throw ExtractStatus.toCanonical(e);
-    } catch (ExecutionException e) {
-      throw ExtractStatus.toCanonical(e.getCause());
-    }
-  }
-
-  @Override
-  public List<SequencedMessage> pull() throws StatusException {
-    @Nullable StatusException maybeError = error.get();
-    if (maybeError != null) {
-      throw maybeError;
-    }
-    ArrayList<SequencedMessage> collection = new ArrayList<>();
-    messages.drainTo(collection);
-    long bytes = collection.stream().mapToLong(SequencedMessage::getSizeBytes).sum();
-    underlying.allowFlow(
-        FlowControlRequest.newBuilder()
-            .setAllowedBytes(bytes)
-            .setAllowedMessages(collection.size())
-            .build());
-    return ImmutableList.copyOf(collection);
-  }
-
-  @Override
-  public void close() {
-    underlying.stopAsync().awaitTerminated();
-  }
-}
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/PublisherOptions.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/PublisherOptions.java
index 15d4124..0c11f11 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/PublisherOptions.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/PublisherOptions.java
@@ -33,6 +33,7 @@
 
 /** Options needed for a Pub/Sub Lite Publisher. */
 @AutoValue
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public abstract class PublisherOptions implements Serializable {
   private static final long serialVersionUID = 275311613L;
 
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/PublisherOrError.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/PublisherOrError.java
index 1df5357..20dea86 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/PublisherOrError.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/PublisherOrError.java
@@ -24,6 +24,7 @@
 
 /** A helper representing either a Publisher or an error. */
 @AutoOneOf(PublisherOrError.Kind.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 abstract class PublisherOrError {
   enum Kind {
     PUBLISHER,
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/PubsubLiteIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/PubsubLiteIO.java
index b9178fe..b52a54d 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/PubsubLiteIO.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/PubsubLiteIO.java
@@ -45,10 +45,10 @@
    * Pipeline p = ...;
    *
    * SubscriptionPath subscriptionPath =
-   *         SubscriptionPaths.newBuilder()
-   *             .setZone(zone)
+   *         SubscriptionPath.newBuilder()
+   *             .setLocation(zone)
    *             .setProjectNumber(projectNum)
-   *             .setSubscriptionName(subscriptionName)
+   *             .setName(subscriptionName)
    *             .build();
    *
    * FlowControlSettings flowControlSettings =
@@ -102,10 +102,10 @@
    *
    * <pre>{@code
    * TopicPath topicPath =
-   *         TopicPaths.newBuilder()
+   *         TopicPath.newBuilder()
    *             .setProjectNumber(projectNum)
-   *             .setZone(zone)
-   *             .setTopicName(topicName)
+   *             .setLocation(zone)
+   *             .setName(topicName)
    *             .build();
    *
    * PCollection<Message> messages = ...;
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/PubsubLiteSink.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/PubsubLiteSink.java
index 01dffbc..22e181d 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/PubsubLiteSink.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/PubsubLiteSink.java
@@ -40,6 +40,7 @@
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.util.concurrent.MoreExecutors;
 
 /** A sink which publishes messages to Pub/Sub Lite. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class PubsubLiteSink extends DoFn<PubSubMessage, Void> {
   private final PublisherOptions options;
 
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/PubsubLiteUnboundedReader.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/PubsubLiteUnboundedReader.java
index 592798a..d357b86 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/PubsubLiteUnboundedReader.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/PubsubLiteUnboundedReader.java
@@ -24,6 +24,7 @@
 import com.google.cloud.pubsublite.internal.CloseableMonitor;
 import com.google.cloud.pubsublite.internal.ExtractStatus;
 import com.google.cloud.pubsublite.internal.ProxyService;
+import com.google.cloud.pubsublite.internal.PullSubscriber;
 import com.google.cloud.pubsublite.internal.wire.Committer;
 import com.google.cloud.pubsublite.proto.SequencedMessage;
 import com.google.errorprone.annotations.concurrent.GuardedBy;
@@ -50,6 +51,7 @@
 import org.joda.time.Instant;
 
 /** A reader for Pub/Sub Lite that generates a stream of SequencedMessages. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class PubsubLiteUnboundedReader extends UnboundedReader<SequencedMessage>
     implements OffsetFinalizer {
   private final UnboundedSource<SequencedMessage, ?> source;
@@ -139,7 +141,7 @@
   static class SubscriberState {
     Instant lastDeliveredPublishTimestamp = BoundedWindow.TIMESTAMP_MIN_VALUE;
     Optional<Offset> lastDelivered = Optional.empty();
-    PullSubscriber subscriber;
+    PullSubscriber<SequencedMessage> subscriber;
     Committer committer;
   }
 
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/PubsubLiteUnboundedSource.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/PubsubLiteUnboundedSource.java
index 53a594c..2c6453b 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/PubsubLiteUnboundedSource.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/PubsubLiteUnboundedSource.java
@@ -21,8 +21,11 @@
 
 import com.google.cloud.pubsublite.Offset;
 import com.google.cloud.pubsublite.Partition;
+import com.google.cloud.pubsublite.internal.BufferingPullSubscriber;
 import com.google.cloud.pubsublite.internal.wire.Committer;
 import com.google.cloud.pubsublite.internal.wire.SubscriberFactory;
+import com.google.cloud.pubsublite.proto.Cursor;
+import com.google.cloud.pubsublite.proto.SeekRequest;
 import com.google.cloud.pubsublite.proto.SequencedMessage;
 import io.grpc.StatusException;
 import java.io.IOException;
@@ -40,6 +43,7 @@
 import org.checkerframework.checker.nullness.qual.Nullable;
 
 /** An UnboundedSource of Pub/Sub Lite SequencedMessages. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class PubsubLiteUnboundedSource extends UnboundedSource<SequencedMessage, OffsetCheckpointMark> {
   private final SubscriberOptions subscriberOptions;
 
@@ -88,14 +92,18 @@
           Offset checkpointed = checkpointMark.partitionOffsetMap.get(partition);
           state.lastDelivered = Optional.of(checkpointed);
           state.subscriber =
-              new BufferingPullSubscriber(
-                  subscriberFactories.get(partition),
-                  subscriberOptions.flowControlSettings(),
-                  checkpointed);
+              new TranslatingPullSubscriber(
+                  new BufferingPullSubscriber(
+                      subscriberFactories.get(partition),
+                      subscriberOptions.flowControlSettings(),
+                      SeekRequest.newBuilder()
+                          .setCursor(Cursor.newBuilder().setOffset(checkpointed.value()))
+                          .build()));
         } else {
           state.subscriber =
-              new BufferingPullSubscriber(
-                  subscriberFactories.get(partition), subscriberOptions.flowControlSettings());
+              new TranslatingPullSubscriber(
+                  new BufferingPullSubscriber(
+                      subscriberFactories.get(partition), subscriberOptions.flowControlSettings()));
         }
         statesBuilder.put(partition, state);
       }
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/PullSubscriber.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/PullSubscriber.java
deleted file mode 100644
index 40d3fbc..0000000
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/PullSubscriber.java
+++ /dev/null
@@ -1,28 +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.io.gcp.pubsublite;
-
-import com.google.cloud.pubsublite.proto.SequencedMessage;
-import io.grpc.StatusException;
-import java.util.List;
-
-/** A PullSubscriber exposes a "pull" mechanism for retrieving messages. */
-interface PullSubscriber extends AutoCloseable {
-  /** Pull currently available messages from this subscriber. Does not block. */
-  List<SequencedMessage> pull() throws StatusException;
-}
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/SubscriberOptions.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/SubscriberOptions.java
index dbcb1c8..3c79c56 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/SubscriberOptions.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/SubscriberOptions.java
@@ -41,6 +41,7 @@
 
 /** Options needed for a Pub/Sub Lite Subscriber. */
 @AutoValue
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public abstract class SubscriberOptions implements Serializable {
   private static final long serialVersionUID = 269598118L;
 
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/TranslatingPullSubscriber.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/TranslatingPullSubscriber.java
new file mode 100644
index 0000000..6bf0702
--- /dev/null
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/TranslatingPullSubscriber.java
@@ -0,0 +1,48 @@
+/*
+ * 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.io.gcp.pubsublite;
+
+import com.google.cloud.pubsublite.internal.PullSubscriber;
+import com.google.cloud.pubsublite.proto.SequencedMessage;
+import io.grpc.StatusException;
+import java.util.List;
+import java.util.stream.Collectors;
+
+/**
+ * A PullSubscriber translating from {@link com.google.cloud.pubsublite.SequencedMessage}to {@link
+ * com.google.cloud.pubsublite.proto.SequencedMessage}.
+ */
+class TranslatingPullSubscriber implements PullSubscriber<SequencedMessage> {
+  private final PullSubscriber<com.google.cloud.pubsublite.SequencedMessage> underlying;
+
+  TranslatingPullSubscriber(
+      PullSubscriber<com.google.cloud.pubsublite.SequencedMessage> underlying) {
+    this.underlying = underlying;
+  }
+
+  @Override
+  public List<SequencedMessage> pull() throws StatusException {
+    List<com.google.cloud.pubsublite.SequencedMessage> messages = underlying.pull();
+    return messages.stream().map(m -> m.toProto()).collect(Collectors.toList());
+  }
+
+  @Override
+  public void close() throws Exception {
+    underlying.close();
+  }
+}
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/Uuid.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/Uuid.java
index c9d1455..d673ad0 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/Uuid.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/Uuid.java
@@ -27,6 +27,7 @@
 /** A Uuid storable in a Pub/Sub Lite attribute. */
 @DefaultCoder(UuidCoder.class)
 @AutoValue
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public abstract class Uuid {
   public static final String DEFAULT_ATTRIBUTE = "x-goog-pubsublite-dataflow-uuid";
 
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/UuidDeduplicationOptions.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/UuidDeduplicationOptions.java
index 58c2886..a5da233 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/UuidDeduplicationOptions.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/UuidDeduplicationOptions.java
@@ -30,6 +30,7 @@
 
 /** Options for deduplicating Pub/Sub Lite messages based on the UUID they were published with. */
 @AutoValue
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public abstract class UuidDeduplicationOptions implements Serializable {
   private static final long serialVersionUID = 9837489720893L;
 
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/BatchSpannerRead.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/BatchSpannerRead.java
index 2bf6766..747657d 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/BatchSpannerRead.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/BatchSpannerRead.java
@@ -39,6 +39,7 @@
  * Reads from multiple partitions are executed concurrently yet in the same read-only transaction.
  */
 @AutoValue
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 abstract class BatchSpannerRead
     extends PTransform<PCollection<ReadOperation>, PCollection<Struct>> {
 
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/CreateTransactionFn.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/CreateTransactionFn.java
index d888f9f..e52f87f 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/CreateTransactionFn.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/CreateTransactionFn.java
@@ -21,6 +21,7 @@
 import org.apache.beam.sdk.transforms.DoFn;
 
 /** Creates a batch transaction. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class CreateTransactionFn extends DoFn<Object, Transaction> {
 
   private final SpannerIO.CreateTransaction config;
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/NaiveSpannerRead.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/NaiveSpannerRead.java
index 31e6bcb..bc9686e 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/NaiveSpannerRead.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/NaiveSpannerRead.java
@@ -34,6 +34,7 @@
 /** A naive version of Spanner read that doesn't use the Batch API. */
 @VisibleForTesting
 @AutoValue
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 abstract class NaiveSpannerRead
     extends PTransform<PCollection<ReadOperation>, PCollection<Struct>> {
 
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/ReadOperation.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/ReadOperation.java
index 49f5925..e32c09c 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/ReadOperation.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/ReadOperation.java
@@ -28,6 +28,7 @@
 
 /** Encapsulates a spanner read operation. */
 @AutoValue
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public abstract class ReadOperation implements Serializable {
 
   public static ReadOperation create() {
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/ReadSpannerSchema.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/ReadSpannerSchema.java
index 548fc95..7bda7df 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/ReadSpannerSchema.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/ReadSpannerSchema.java
@@ -27,6 +27,7 @@
  * This {@link DoFn} reads Cloud Spanner 'information_schema.*' tables to build the {@link
  * SpannerSchema}.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class ReadSpannerSchema extends DoFn<Void, SpannerSchema> {
 
   private final SpannerConfig config;
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerAccessor.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerAccessor.java
index 7dcc44b..33ddf5b 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerAccessor.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerAccessor.java
@@ -44,6 +44,7 @@
 import org.slf4j.LoggerFactory;
 
 /** Manages lifecycle of {@link DatabaseClient} and {@link Spanner} instances. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class SpannerAccessor implements AutoCloseable {
   private static final Logger LOG = LoggerFactory.getLogger(SpannerAccessor.class);
 
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerConfig.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerConfig.java
index 0126700..5c07128 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerConfig.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerConfig.java
@@ -32,6 +32,7 @@
 
 /** Configuration for a Cloud Spanner client. */
 @AutoValue
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public abstract class SpannerConfig implements Serializable {
   // A default host name for batch traffic.
   private static final String DEFAULT_HOST = "https://batch-spanner.googleapis.com/";
@@ -52,9 +53,8 @@
 
   public abstract @Nullable ValueProvider<Duration> getMaxCumulativeBackoff();
 
-  @Nullable
   @VisibleForTesting
-  abstract ServiceFactory<Spanner, SpannerOptions> getServiceFactory();
+  abstract @Nullable ServiceFactory<Spanner, SpannerOptions> getServiceFactory();
 
   abstract Builder toBuilder();
 
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java
index 0554b74..4b9f515 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java
@@ -318,6 +318,7 @@
  * the write order of individual {@link Mutation}/{@link MutationGroup} objects is not guaranteed.
  */
 @Experimental(Kind.SOURCE_SINK)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SpannerIO {
   private static final Logger LOG = LoggerFactory.getLogger(SpannerIO.class);
 
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerSchema.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerSchema.java
index 4a5d814..274fb91 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerSchema.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerSchema.java
@@ -30,6 +30,7 @@
 
 /** Encapsulates Cloud Spanner Schema. */
 @AutoValue
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 abstract class SpannerSchema implements Serializable {
   abstract ImmutableList<String> tables();
 
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/Transaction.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/Transaction.java
index fc23ac2..e0b58b6 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/Transaction.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/Transaction.java
@@ -24,6 +24,7 @@
 
 /** A transaction object. */
 @AutoValue
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public abstract class Transaction implements Serializable {
 
   public abstract @Nullable BatchTransactionId transactionId();
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/testing/BigqueryClient.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/testing/BigqueryClient.java
index 5819ab5..7f5422d 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/testing/BigqueryClient.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/testing/BigqueryClient.java
@@ -112,6 +112,7 @@
  * ]}</pre>
  */
 @Internal
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BigqueryClient {
   private static final Logger LOG = LoggerFactory.getLogger(BigqueryClient.class);
   // The maximum number of retries to execute a BigQuery RPC
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/testing/BigqueryMatcher.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/testing/BigqueryMatcher.java
index 85d83fd..99698aa 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/testing/BigqueryMatcher.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/testing/BigqueryMatcher.java
@@ -55,6 +55,7 @@
  */
 @NotThreadSafe
 @Experimental
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BigqueryMatcher extends TypeSafeMatcher<TableAndQuery>
     implements SerializableMatcher<TableAndQuery> {
   private static final Logger LOG = LoggerFactory.getLogger(BigqueryMatcher.class);
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/testing/FakeBigQueryServices.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/testing/FakeBigQueryServices.java
index d85a1e9..4ec04f9 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/testing/FakeBigQueryServices.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/testing/FakeBigQueryServices.java
@@ -39,6 +39,7 @@
 
 /** A fake implementation of BigQuery's query service.. */
 @Internal
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class FakeBigQueryServices implements BigQueryServices {
   private JobService jobService;
   private DatasetService datasetService;
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/testing/FakeDatasetService.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/testing/FakeDatasetService.java
index 963edb4..0cea9b6 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/testing/FakeDatasetService.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/testing/FakeDatasetService.java
@@ -41,6 +41,7 @@
 import org.apache.beam.sdk.io.gcp.bigquery.InsertRetryPolicy.Context;
 import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
 import org.apache.beam.sdk.transforms.windowing.PaneInfo;
+import org.apache.beam.sdk.values.FailsafeValueInSingleWindow;
 import org.apache.beam.sdk.values.ValueInSingleWindow;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.HashBasedTable;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Lists;
@@ -49,6 +50,7 @@
 
 /** A fake dataset service that can be serialized, for use in testReadFromTable. */
 @Internal
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class FakeDatasetService implements DatasetService, Serializable {
   // Table information must be static, as each ParDo will get a separate instance of
   // FakeDatasetServices, and they must all modify the same storage.
@@ -213,14 +215,15 @@
   public long insertAll(
       TableReference ref, List<TableRow> rowList, @Nullable List<String> insertIdList)
       throws IOException, InterruptedException {
-    List<ValueInSingleWindow<TableRow>> windowedRows = Lists.newArrayList();
+    List<FailsafeValueInSingleWindow<TableRow, TableRow>> windowedRows = Lists.newArrayList();
     for (TableRow row : rowList) {
       windowedRows.add(
-          ValueInSingleWindow.of(
+          FailsafeValueInSingleWindow.of(
               row,
               GlobalWindow.TIMESTAMP_MAX_VALUE,
               GlobalWindow.INSTANCE,
-              PaneInfo.ON_TIME_AND_ONLY_FIRING));
+              PaneInfo.ON_TIME_AND_ONLY_FIRING,
+              row));
     }
     return insertAll(
         ref,
@@ -237,7 +240,7 @@
   @Override
   public <T> long insertAll(
       TableReference ref,
-      List<ValueInSingleWindow<TableRow>> rowList,
+      List<FailsafeValueInSingleWindow<TableRow, TableRow>> rowList,
       @Nullable List<String> insertIdList,
       InsertRetryPolicy retryPolicy,
       List<ValueInSingleWindow<T>> failedInserts,
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/testing/FakeJobService.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/testing/FakeJobService.java
index 48d25f2..c49d506 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/testing/FakeJobService.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/testing/FakeJobService.java
@@ -88,6 +88,7 @@
 
 /** A fake implementation of BigQuery's job service. */
 @Internal
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class FakeJobService implements JobService, Serializable {
   private static final JsonFactory JSON_FACTORY = Transport.getJsonFactory();
   // Whenever a job is started, the first 2 calls to GetJob will report the job as pending,
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/GcpApiSurfaceTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/GcpApiSurfaceTest.java
index d83ef0b..8178a70 100644
--- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/GcpApiSurfaceTest.java
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/GcpApiSurfaceTest.java
@@ -34,6 +34,7 @@
 
 /** API surface verification for {@link org.apache.beam.sdk.io.gcp}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class GcpApiSurfaceTest {
 
   @Test
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryAvroUtilsTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryAvroUtilsTest.java
index 506cc10..baaed76 100644
--- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryAvroUtilsTest.java
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryAvroUtilsTest.java
@@ -50,6 +50,7 @@
 
 /** Tests for {@link BigQueryAvroUtils}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BigQueryAvroUtilsTest {
   private List<TableFieldSchema> subFields =
       Lists.newArrayList(
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryHelpersTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryHelpersTest.java
index c7679ea..b610735 100644
--- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryHelpersTest.java
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryHelpersTest.java
@@ -54,6 +54,7 @@
 
 /** Tests for {@link BigQueryHelpers}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BigQueryHelpersTest {
   @Rule public transient ExpectedException thrown = ExpectedException.none();
 
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOReadIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOReadIT.java
index 40d6de2..81517d3 100644
--- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOReadIT.java
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOReadIT.java
@@ -39,6 +39,7 @@
  * number of records read equals the given expected number of records.
  */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BigQueryIOReadIT {
 
   private BigQueryIOReadOptions options;
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOReadTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOReadTest.java
index 51a476b..76328a3 100644
--- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOReadTest.java
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOReadTest.java
@@ -85,6 +85,7 @@
 
 /** Tests for {@link BigQueryIO#read}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BigQueryIOReadTest implements Serializable {
   private transient PipelineOptions options;
   private transient TemporaryFolder testFolder = new TemporaryFolder();
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageQueryIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageQueryIT.java
index fa29f7b..4f8f573 100644
--- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageQueryIT.java
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageQueryIT.java
@@ -43,6 +43,7 @@
  * pre-defined table and asserts that the number of records read is equal to the expected count.
  */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BigQueryIOStorageQueryIT {
 
   private static final Map<String, Long> EXPECTED_NUM_RECORDS =
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageQueryTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageQueryTest.java
index 9f4a1ed..32ef197 100644
--- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageQueryTest.java
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageQueryTest.java
@@ -95,6 +95,7 @@
 
 /** Tests for {@link BigQueryIO#readTableRows()} using {@link Method#DIRECT_READ}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BigQueryIOStorageQueryTest {
 
   private transient BigQueryOptions options;
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadIT.java
index 3b61b65..b6d46fb 100644
--- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadIT.java
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadIT.java
@@ -43,6 +43,7 @@
  * records read is equal to the expected count.
  */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BigQueryIOStorageReadIT {
 
   private static final Map<String, Long> EXPECTED_NUM_RECORDS =
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadTableRowIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadTableRowIT.java
index 734c3af..3875e30 100644
--- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadTableRowIT.java
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadTableRowIT.java
@@ -50,6 +50,7 @@
  * combination with {@link TableRowParser} to generate output in {@link TableRow} form.
  */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BigQueryIOStorageReadTableRowIT {
 
   private static final String DATASET_ID = "big_query_import_export";
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadTest.java
index d2401f8..ad1de6c 100644
--- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadTest.java
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOStorageReadTest.java
@@ -101,6 +101,7 @@
 
 /** Tests for {@link BigQueryIO#readTableRows() using {@link Method#DIRECT_READ}}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BigQueryIOStorageReadTest {
 
   private transient PipelineOptions options;
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOWriteTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOWriteTest.java
index 4489139..aff0214 100644
--- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOWriteTest.java
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOWriteTest.java
@@ -139,6 +139,7 @@
 
 /** Tests for {@link BigQueryIO#write}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BigQueryIOWriteTest implements Serializable {
   private transient PipelineOptions options;
   private transient TemporaryFolder testFolder = new TemporaryFolder();
@@ -1556,7 +1557,8 @@
             BatchLoads.DEFAULT_MAX_BYTES_PER_PARTITION,
             multiPartitionsTag,
             singlePartitionTag,
-            RowWriterFactory.tableRows(SerializableFunctions.identity()));
+            RowWriterFactory.tableRows(
+                SerializableFunctions.identity(), SerializableFunctions.identity()));
 
     DoFnTester<
             Iterable<WriteBundlesToFiles.Result<TableDestination>>,
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryKmsKeyIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryKmsKeyIT.java
index 50610f3..19ad5d5 100644
--- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryKmsKeyIT.java
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryKmsKeyIT.java
@@ -49,6 +49,7 @@
  */
 @RunWith(JUnit4.class)
 @Category(UsesKms.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BigQueryKmsKeyIT {
 
   private static final Logger LOG = LoggerFactory.getLogger(BigQueryKmsKeyIT.class);
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySchemaUpdateOptionsIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySchemaUpdateOptionsIT.java
index 8d030ca..ff4158c 100644
--- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySchemaUpdateOptionsIT.java
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySchemaUpdateOptionsIT.java
@@ -50,6 +50,7 @@
 
 /** Integration test for BigqueryIO with DataflowRunner and DirectRunner. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BigQuerySchemaUpdateOptionsIT {
   private static final Logger LOG = LoggerFactory.getLogger(BigQuerySchemaUpdateOptionsIT.class);
   private static String project;
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImplTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImplTest.java
index 15bfc0a..11e6822 100644
--- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImplTest.java
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImplTest.java
@@ -78,6 +78,7 @@
 import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
 import org.apache.beam.sdk.transforms.windowing.PaneInfo;
 import org.apache.beam.sdk.util.FluentBackoff;
+import org.apache.beam.sdk.values.FailsafeValueInSingleWindow;
 import org.apache.beam.sdk.values.ValueInSingleWindow;
 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.Lists;
@@ -93,6 +94,7 @@
 
 /** Tests for {@link BigQueryServicesImpl}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BigQueryServicesImplTest {
   @Rule public ExpectedException thrown = ExpectedException.none();
   @Rule public ExpectedLogs expectedLogs = ExpectedLogs.none(BigQueryServicesImpl.class);
@@ -483,7 +485,16 @@
     verify(response, times(1)).getContentType();
   }
 
-  private <T> ValueInSingleWindow<T> wrapValue(T value) {
+  private <T> FailsafeValueInSingleWindow<T, T> wrapValue(T value) {
+    return FailsafeValueInSingleWindow.of(
+        value,
+        GlobalWindow.TIMESTAMP_MAX_VALUE,
+        GlobalWindow.INSTANCE,
+        PaneInfo.ON_TIME_AND_ONLY_FIRING,
+        value);
+  }
+
+  private <T> ValueInSingleWindow<T> wrapErrorValue(T value) {
     return ValueInSingleWindow.of(
         value,
         GlobalWindow.TIMESTAMP_MAX_VALUE,
@@ -496,7 +507,7 @@
   public void testInsertRateLimitRetry() throws Exception {
     TableReference ref =
         new TableReference().setProjectId("project").setDatasetId("dataset").setTableId("table");
-    List<ValueInSingleWindow<TableRow>> rows = new ArrayList<>();
+    List<FailsafeValueInSingleWindow<TableRow, TableRow>> rows = new ArrayList<>();
     rows.add(wrapValue(new TableRow()));
 
     // First response is 403 rate limited, second response has valid payload.
@@ -532,7 +543,7 @@
   public void testInsertQuotaExceededRetry() throws Exception {
     TableReference ref =
         new TableReference().setProjectId("project").setDatasetId("dataset").setTableId("table");
-    List<ValueInSingleWindow<TableRow>> rows = new ArrayList<>();
+    List<FailsafeValueInSingleWindow<TableRow, TableRow>> rows = new ArrayList<>();
     rows.add(wrapValue(new TableRow()));
 
     // First response is 403 quota exceeded, second response has valid payload.
@@ -568,7 +579,7 @@
   public void testInsertStoppedRetry() throws Exception {
     TableReference ref =
         new TableReference().setProjectId("project").setDatasetId("dataset").setTableId("table");
-    List<ValueInSingleWindow<TableRow>> rows = new ArrayList<>();
+    List<FailsafeValueInSingleWindow<TableRow, TableRow>> rows = new ArrayList<>();
     rows.add(wrapValue(new TableRow()));
 
     // Respond 403 four times, then valid payload.
@@ -620,7 +631,7 @@
   public void testInsertRetrySelectRows() throws Exception {
     TableReference ref =
         new TableReference().setProjectId("project").setDatasetId("dataset").setTableId("table");
-    List<ValueInSingleWindow<TableRow>> rows =
+    List<FailsafeValueInSingleWindow<TableRow, TableRow>> rows =
         ImmutableList.of(
             wrapValue(new TableRow().set("row", "a")), wrapValue(new TableRow().set("row", "b")));
     List<String> insertIds = ImmutableList.of("a", "b");
@@ -664,7 +675,7 @@
   public void testInsertFailsGracefully() throws Exception {
     TableReference ref =
         new TableReference().setProjectId("project").setDatasetId("dataset").setTableId("table");
-    List<ValueInSingleWindow<TableRow>> rows =
+    List<FailsafeValueInSingleWindow<TableRow, TableRow>> rows =
         ImmutableList.of(wrapValue(new TableRow()), wrapValue(new TableRow()));
 
     final TableDataInsertAllResponse row1Failed =
@@ -723,7 +734,7 @@
   public void testFailInsertOtherRetry() throws Exception {
     TableReference ref =
         new TableReference().setProjectId("project").setDatasetId("dataset").setTableId("table");
-    List<ValueInSingleWindow<TableRow>> rows = new ArrayList<>();
+    List<FailsafeValueInSingleWindow<TableRow, TableRow>> rows = new ArrayList<>();
     rows.add(wrapValue(new TableRow()));
 
     // First response is 403 non-{rate-limited, quota-exceeded}, second response has valid payload
@@ -767,7 +778,7 @@
   public void testInsertRetryPolicy() throws InterruptedException, IOException {
     TableReference ref =
         new TableReference().setProjectId("project").setDatasetId("dataset").setTableId("table");
-    List<ValueInSingleWindow<TableRow>> rows =
+    List<FailsafeValueInSingleWindow<TableRow, TableRow>> rows =
         ImmutableList.of(wrapValue(new TableRow()), wrapValue(new TableRow()));
 
     // First time row0 fails with a retryable error, and row1 fails with a persistent error.
@@ -836,7 +847,7 @@
       throws InterruptedException, IOException {
     TableReference ref =
         new TableReference().setProjectId("project").setDatasetId("dataset").setTableId("table");
-    List<ValueInSingleWindow<TableRow>> rows =
+    List<FailsafeValueInSingleWindow<TableRow, TableRow>> rows =
         ImmutableList.of(wrapValue(new TableRow()), wrapValue(new TableRow()));
 
     final TableDataInsertAllResponse allRowsSucceeded = new TableDataInsertAllResponse();
@@ -1051,10 +1062,14 @@
   public void testSimpleErrorRetrieval() throws InterruptedException, IOException {
     TableReference ref =
         new TableReference().setProjectId("project").setDatasetId("dataset").setTableId("table");
-    List<ValueInSingleWindow<TableRow>> rows =
+    List<FailsafeValueInSingleWindow<TableRow, TableRow>> rows =
         ImmutableList.of(
             wrapValue(new TableRow().set("a", 1)), wrapValue(new TableRow().set("b", 2)));
 
+    final List<ValueInSingleWindow<TableRow>> expected =
+        ImmutableList.of(
+            wrapErrorValue(new TableRow().set("a", 1)), wrapErrorValue(new TableRow().set("b", 2)));
+
     final TableDataInsertAllResponse failures =
         new TableDataInsertAllResponse()
             .setInsertErrors(
@@ -1090,7 +1105,7 @@
         false,
         false);
 
-    assertThat(failedInserts, is(rows));
+    assertThat(failedInserts, is(expected));
   }
 
   /** Tests that {@link DatasetServiceImpl#insertAll} uses the supplied {@link ErrorContainer}. */
@@ -1098,7 +1113,7 @@
   public void testExtendedErrorRetrieval() throws InterruptedException, IOException {
     TableReference ref =
         new TableReference().setProjectId("project").setDatasetId("dataset").setTableId("table");
-    List<ValueInSingleWindow<TableRow>> rows =
+    List<FailsafeValueInSingleWindow<TableRow, TableRow>> rows =
         ImmutableList.of(
             wrapValue(new TableRow().set("a", 1)), wrapValue(new TableRow().set("b", 2)));
 
@@ -1115,10 +1130,10 @@
 
     final List<ValueInSingleWindow<BigQueryInsertError>> expected =
         ImmutableList.of(
-            wrapValue(
+            wrapErrorValue(
                 new BigQueryInsertError(
                     rows.get(0).getValue(), failures.getInsertErrors().get(0), ref)),
-            wrapValue(
+            wrapErrorValue(
                 new BigQueryInsertError(
                     rows.get(1).getValue(), failures.getInsertErrors().get(1), ref)));
 
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTimePartitioningClusteringIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTimePartitioningClusteringIT.java
index 63184d2..06f8bc9 100644
--- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTimePartitioningClusteringIT.java
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTimePartitioningClusteringIT.java
@@ -46,6 +46,7 @@
 
 /** Integration test that partitions and clusters sample data in BigQuery. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BigQueryTimePartitioningClusteringIT {
   private static final String WEATHER_SAMPLES_TABLE =
       "clouddataflow-readonly:samples.weather_stations";
@@ -103,9 +104,8 @@
       this.tableName = tableName;
     }
 
-    @Nullable
     @Override
-    public Coder<TableDestination> getDestinationCoder() {
+    public @Nullable Coder<TableDestination> getDestinationCoder() {
       return TableDestinationCoderV3.of();
     }
 
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryToTableIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryToTableIT.java
index 0cdd987..2b3feb9 100644
--- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryToTableIT.java
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryToTableIT.java
@@ -65,6 +65,7 @@
 
 /** Integration test for BigqueryIO with DataflowRunner and DirectRunner. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BigQueryToTableIT {
   private static final Logger LOG = LoggerFactory.getLogger(BigQueryToTableIT.class);
   private static String project;
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtilTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtilTest.java
index deb1952..08a7f28 100644
--- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtilTest.java
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryUtilTest.java
@@ -47,7 +47,7 @@
 import org.apache.beam.sdk.options.PipelineOptionsFactory;
 import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
 import org.apache.beam.sdk.transforms.windowing.PaneInfo;
-import org.apache.beam.sdk.values.ValueInSingleWindow;
+import org.apache.beam.sdk.values.FailsafeValueInSingleWindow;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Rule;
@@ -60,6 +60,7 @@
 
 /** Tests for util classes related to BigQuery. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BigQueryUtilTest {
 
   @Rule public ExpectedException thrown = ExpectedException.none();
@@ -196,15 +197,16 @@
     TableReference ref = BigQueryHelpers.parseTableSpec("project:dataset.table");
     DatasetServiceImpl datasetService = new DatasetServiceImpl(mockClient, options, 5);
 
-    List<ValueInSingleWindow<TableRow>> rows = new ArrayList<>();
+    List<FailsafeValueInSingleWindow<TableRow, TableRow>> rows = new ArrayList<>();
     List<String> ids = new ArrayList<>();
     for (int i = 0; i < 25; ++i) {
       rows.add(
-          ValueInSingleWindow.of(
+          FailsafeValueInSingleWindow.of(
               rawRow("foo", 1234),
               GlobalWindow.TIMESTAMP_MAX_VALUE,
               GlobalWindow.INSTANCE,
-              PaneInfo.ON_TIME_AND_ONLY_FIRING));
+              PaneInfo.ON_TIME_AND_ONLY_FIRING,
+              rawRow("foo", 1234)));
       ids.add("");
     }
 
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 a28f6a5..10cdbb9 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
@@ -59,6 +59,7 @@
 
 /** Tests for {@link BigQueryUtils}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BigQueryUtilsTest {
   private static final Schema FLAT_TYPE =
       Schema.builder()
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTest.java
index 0a3d4e4..c440480 100644
--- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTest.java
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableConfigTest.java
@@ -44,6 +44,7 @@
 
 /** Unit tests for {@link BigtableConfig}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BigtableConfigTest {
 
   static final ValueProvider<String> NOT_ACCESSIBLE_VALUE =
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java
index 8a7d4bf..46683f8 100644
--- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java
@@ -126,6 +126,7 @@
 
 /** Unit tests for {@link BigtableIO}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BigtableIOTest {
   @Rule public final transient TestPipeline p = TestPipeline.create();
   @Rule public ExpectedException thrown = ExpectedException.none();
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImplTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImplTest.java
index 69be079..d0da35f 100644
--- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImplTest.java
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImplTest.java
@@ -59,6 +59,7 @@
 
 /** Unit tests of BigtableServiceImpl. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BigtableServiceImplTest {
 
   private static final BigtableTableName TABLE_NAME =
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteIT.java
index 3ddc500..b7f4a08 100644
--- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteIT.java
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteIT.java
@@ -59,6 +59,7 @@
 
 /** End-to-end tests of BigtableWrite. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BigtableWriteIT implements Serializable {
   /**
    * These tests requires a static instances because the writers go through a serialization step
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/DataStoreV1SchemaIOProviderTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/DataStoreV1SchemaIOProviderTest.java
index 1474744..c1ae771 100644
--- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/DataStoreV1SchemaIOProviderTest.java
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/DataStoreV1SchemaIOProviderTest.java
@@ -31,6 +31,7 @@
 import org.junit.runners.JUnit4;
 
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DataStoreV1SchemaIOProviderTest {
   static final String DEFAULT_KEY_FIELD = "__key__";
   public static final String KEY_FIELD_PROPERTY = "keyField";
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1Test.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1Test.java
index de18790..0adebff 100644
--- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1Test.java
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1Test.java
@@ -108,6 +108,7 @@
 
 /** Tests for {@link DatastoreV1}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DatastoreV1Test {
   private static final String PROJECT_ID = "testProject";
   private static final String NAMESPACE = "testNamespace";
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/EntityToRowRowToEntityTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/EntityToRowRowToEntityTest.java
index 32090bc..3f3fb55 100644
--- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/EntityToRowRowToEntityTest.java
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/EntityToRowRowToEntityTest.java
@@ -51,6 +51,7 @@
 import org.junit.runners.JUnit4;
 
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class EntityToRowRowToEntityTest {
   private static final String KIND = "kind";
   private static final String UUID_VALUE = UUID.randomUUID().toString();
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/SplitQueryFnIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/SplitQueryFnIT.java
index 6d0bd52..dfec247 100644
--- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/SplitQueryFnIT.java
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/SplitQueryFnIT.java
@@ -49,6 +49,7 @@
  * </ul>
  */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SplitQueryFnIT {
   /** Tests {@link SplitQueryFn} to generate expected number of splits for a large dataset. */
   @Test
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1ReadIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1ReadIT.java
index 55b53b3..afbe6e5 100644
--- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1ReadIT.java
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1ReadIT.java
@@ -44,6 +44,7 @@
 
 /** End-to-end tests for Datastore DatastoreV1.Read. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class V1ReadIT {
   private V1TestOptions options;
   private String project;
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1TestUtil.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1TestUtil.java
index e3057c4..eec8605 100644
--- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1TestUtil.java
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1TestUtil.java
@@ -62,6 +62,7 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class V1TestUtil {
   private static final Logger LOG = LoggerFactory.getLogger(V1TestUtil.class);
 
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1WriteIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1WriteIT.java
index 78a5071..51ec41f 100644
--- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1WriteIT.java
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1WriteIT.java
@@ -37,6 +37,7 @@
 
 /** End-to-end tests for Datastore DatastoreV1.Write. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class V1WriteIT {
   private V1TestOptions options;
   private String project;
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOLROIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOLROIT.java
index e5d81b4..b66d042 100644
--- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOLROIT.java
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOLROIT.java
@@ -34,6 +34,7 @@
 import org.junit.runners.JUnit4;
 
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class FhirIOLROIT {
   @Rule public final transient TestPipeline pipeline = TestPipeline.create();
 
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOReadIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOReadIT.java
index 38c310e..061b939 100644
--- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOReadIT.java
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOReadIT.java
@@ -45,6 +45,7 @@
 import org.junit.runners.Parameterized.Parameters;
 
 @RunWith(Parameterized.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class FhirIOReadIT {
 
   @Parameters(name = "{0}")
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOTestUtil.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOTestUtil.java
index c893f59..c1f730c 100644
--- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOTestUtil.java
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOTestUtil.java
@@ -40,6 +40,7 @@
 import java.util.stream.Stream;
 import org.apache.beam.sdk.io.gcp.healthcare.HttpHealthcareApiClient.HealthcareHttpException;
 
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class FhirIOTestUtil {
   public static final String DEFAULT_TEMP_BUCKET = "temp-storage-for-healthcare-io-tests";
 
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOWriteIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOWriteIT.java
index 6e260ec..53a767d 100644
--- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOWriteIT.java
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOWriteIT.java
@@ -41,6 +41,7 @@
 import org.junit.runners.Parameterized.Parameters;
 
 @RunWith(Parameterized.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class FhirIOWriteIT {
 
   @Parameters(name = "{0}")
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/HL7V2MessagePagesTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/HL7V2MessagePagesTest.java
index dd909cd..189a2c5 100644
--- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/HL7V2MessagePagesTest.java
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/HL7V2MessagePagesTest.java
@@ -37,6 +37,7 @@
 
 /** The type HL7v2 message id pages test. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class HL7V2MessagePagesTest {
 
   /** The Healthcare API. */
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/HL7v2IOReadIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/HL7v2IOReadIT.java
index 610c830..97df274 100644
--- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/HL7v2IOReadIT.java
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/HL7v2IOReadIT.java
@@ -42,6 +42,7 @@
 import org.junit.runners.JUnit4;
 
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class HL7v2IOReadIT {
   private transient HealthcareApiClient client;
   private static String healthcareDataset;
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/HL7v2IOReadWriteIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/HL7v2IOReadWriteIT.java
index bb8079c..cb83e95 100644
--- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/HL7v2IOReadWriteIT.java
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/HL7v2IOReadWriteIT.java
@@ -47,6 +47,7 @@
  * with schematized data which should be output only.
  */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class HL7v2IOReadWriteIT {
 
   private transient HealthcareApiClient client;
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/HL7v2IOTestUtil.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/HL7v2IOTestUtil.java
index aa757ca..30a3c4f 100644
--- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/HL7v2IOTestUtil.java
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/HL7v2IOTestUtil.java
@@ -35,6 +35,7 @@
 import org.joda.time.Duration;
 import org.joda.time.Instant;
 
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class HL7v2IOTestUtil {
   public static final long HL7V2_INDEXING_TIMEOUT_MINUTES = 10L;
   /** Google Cloud Healthcare Dataset in Apache Beam integration test project. */
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/HL7v2IOWriteIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/HL7v2IOWriteIT.java
index da6b992..5531da9 100644
--- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/HL7v2IOWriteIT.java
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/HL7v2IOWriteIT.java
@@ -41,6 +41,7 @@
 import org.junit.runners.JUnit4;
 
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class HL7v2IOWriteIT {
 
   private transient HealthcareApiClient client;
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubGrpcClientTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubGrpcClientTest.java
index 4dd719b..73fcdf0 100644
--- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubGrpcClientTest.java
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubGrpcClientTest.java
@@ -56,6 +56,7 @@
 
 /** Tests for PubsubGrpcClient. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class PubsubGrpcClientTest {
   private ManagedChannel inProcessChannel;
 
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIOExternalTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIOExternalTest.java
index 891d468..eba3cd0 100644
--- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIOExternalTest.java
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIOExternalTest.java
@@ -52,6 +52,7 @@
 
 /** Tests for building {@link PubsubIO} externally via the ExpansionService. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class PubsubIOExternalTest {
   @Test
   public void testConstructPubsubRead() throws Exception {
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIOTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIOTest.java
index fe9439c..c6005de 100644
--- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIOTest.java
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIOTest.java
@@ -80,6 +80,7 @@
 
 /** Tests for PubsubIO Read and Write transforms. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class PubsubIOTest {
   @Rule public ExpectedException thrown = ExpectedException.none();
 
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubJsonClientTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubJsonClientTest.java
index 22c1cb1..9585e72 100644
--- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubJsonClientTest.java
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubJsonClientTest.java
@@ -57,6 +57,7 @@
 
 /** Tests for PubsubJsonClient. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class PubsubJsonClientTest {
   private Pubsub mockPubsub;
   private PubsubClient client;
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubMessageToRowTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubMessageToRowTest.java
index 68e141a..bd543d1 100644
--- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubMessageToRowTest.java
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubMessageToRowTest.java
@@ -48,6 +48,7 @@
 import org.junit.Test;
 
 /** Unit tests for {@link PubsubMessageToRow}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class PubsubMessageToRowTest implements Serializable {
 
   @Rule public transient TestPipeline pipeline = TestPipeline.create();
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubTestClientTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubTestClientTest.java
index 6b920e8..63e88e9 100644
--- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubTestClientTest.java
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubTestClientTest.java
@@ -39,6 +39,7 @@
 
 /** Tests for PubsubTestClient. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class PubsubTestClientTest {
   private static final TopicPath TOPIC = PubsubClient.topicPathFromName("testProject", "testTopic");
   private static final SubscriptionPath SUBSCRIPTION =
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSinkTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSinkTest.java
index f8cd86e..404cde1 100644
--- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSinkTest.java
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSinkTest.java
@@ -46,6 +46,7 @@
 
 /** Test PubsubUnboundedSink. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class PubsubUnboundedSinkTest implements Serializable {
   private static final TopicPath TOPIC = PubsubClient.topicPathFromName("testProject", "testTopic");
   private static final String DATA = "testData";
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSourceTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSourceTest.java
index 43ecbdc..c2a45c1 100644
--- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSourceTest.java
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSourceTest.java
@@ -62,6 +62,7 @@
 
 /** Test PubsubUnboundedSource. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class PubsubUnboundedSourceTest {
   private static final SubscriptionPath SUBSCRIPTION =
       PubsubClient.subscriptionPathFromName("testProject", "testSubscription");
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsublite/AddUuidsTransformTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsublite/AddUuidsTransformTest.java
index 5741a68..424d7ba 100644
--- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsublite/AddUuidsTransformTest.java
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsublite/AddUuidsTransformTest.java
@@ -36,6 +36,7 @@
 import org.junit.runners.JUnit4;
 
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public final class AddUuidsTransformTest {
   @Rule public final TestPipeline pipeline = TestPipeline.create();
 
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsublite/BufferingPullSubscriberTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsublite/BufferingPullSubscriberTest.java
deleted file mode 100644
index fd07d6a..0000000
--- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsublite/BufferingPullSubscriberTest.java
+++ /dev/null
@@ -1,183 +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.io.gcp.pubsublite;
-
-import static org.hamcrest.MatcherAssert.assertThat;
-import static org.hamcrest.Matchers.containsInAnyOrder;
-import static org.hamcrest.Matchers.empty;
-import static org.hamcrest.Matchers.equalTo;
-import static org.hamcrest.Matchers.notNullValue;
-import static org.junit.Assert.assertThrows;
-import static org.mockito.ArgumentMatchers.any;
-import static org.mockito.Mockito.doAnswer;
-import static org.mockito.Mockito.inOrder;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.when;
-
-import com.google.api.core.ApiFutures;
-import com.google.api.core.ApiService.Listener;
-import com.google.cloud.pubsublite.Offset;
-import com.google.cloud.pubsublite.cloudpubsub.FlowControlSettings;
-import com.google.cloud.pubsublite.internal.wire.Subscriber;
-import com.google.cloud.pubsublite.internal.wire.SubscriberFactory;
-import com.google.cloud.pubsublite.proto.Cursor;
-import com.google.cloud.pubsublite.proto.FlowControlRequest;
-import com.google.cloud.pubsublite.proto.PubSubMessage;
-import com.google.cloud.pubsublite.proto.SeekRequest;
-import com.google.cloud.pubsublite.proto.SequencedMessage;
-import com.google.common.collect.ImmutableList;
-import com.google.protobuf.Timestamp;
-import com.google.protobuf.util.Timestamps;
-import io.grpc.Status;
-import io.grpc.Status.Code;
-import io.grpc.StatusException;
-import java.util.function.Consumer;
-import java.util.function.Supplier;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.JUnit4;
-import org.mockito.InOrder;
-import org.mockito.stubbing.Answer;
-
-@RunWith(JUnit4.class)
-public class BufferingPullSubscriberTest {
-  private final SubscriberFactory underlyingFactory = mock(SubscriberFactory.class);
-  private final Subscriber underlying = mock(Subscriber.class);
-  private final Offset initialOffset = Offset.of(5);
-  private final FlowControlSettings flowControlSettings =
-      ((Supplier<FlowControlSettings>)
-              () -> {
-                try {
-                  return FlowControlSettings.builder()
-                      .setBytesOutstanding(10)
-                      .setMessagesOutstanding(20)
-                      .build();
-                } catch (StatusException e) {
-                  throw e.getStatus().asRuntimeException();
-                }
-              })
-          .get();
-  // Initialized in setUp.
-  private PullSubscriber subscriber;
-  private Consumer<ImmutableList<com.google.cloud.pubsublite.SequencedMessage>> messageConsumer;
-  private Listener errorListener;
-
-  private static SequencedMessage newMessage(Timestamp publishTime, Offset offset, long sizeBytes) {
-    return SequencedMessage.newBuilder()
-        .setMessage(PubSubMessage.getDefaultInstance())
-        .setPublishTime(publishTime)
-        .setCursor(Cursor.newBuilder().setOffset(offset.value()))
-        .setSizeBytes(sizeBytes)
-        .build();
-  }
-
-  private static com.google.cloud.pubsublite.SequencedMessage toWrapper(SequencedMessage proto) {
-    return com.google.cloud.pubsublite.SequencedMessage.fromProto(proto);
-  }
-
-  @Before
-  public void setUp() throws Exception {
-    when(underlying.startAsync()).thenReturn(underlying);
-    SeekRequest seek =
-        SeekRequest.newBuilder()
-            .setCursor(Cursor.newBuilder().setOffset(initialOffset.value()).build())
-            .build();
-    when(underlying.seek(seek)).thenReturn(ApiFutures.immediateFuture(initialOffset));
-    FlowControlRequest flow =
-        FlowControlRequest.newBuilder()
-            .setAllowedBytes(flowControlSettings.bytesOutstanding())
-            .setAllowedMessages(flowControlSettings.messagesOutstanding())
-            .build();
-    when(underlyingFactory.New(any()))
-        .thenAnswer(
-            args -> {
-              messageConsumer = args.getArgument(0);
-              return underlying;
-            });
-    doAnswer(
-            (Answer<Void>)
-                args -> {
-                  errorListener = args.getArgument(0);
-                  return null;
-                })
-        .when(underlying)
-        .addListener(any(), any());
-
-    subscriber = new BufferingPullSubscriber(underlyingFactory, flowControlSettings, initialOffset);
-
-    InOrder inOrder = inOrder(underlyingFactory, underlying);
-    inOrder.verify(underlyingFactory).New(any());
-    inOrder.verify(underlying).addListener(any(), any());
-    inOrder.verify(underlying).startAsync();
-    inOrder.verify(underlying).awaitRunning();
-    inOrder.verify(underlying).allowFlow(flow);
-    inOrder.verify(underlying).seek(seek);
-
-    assertThat(messageConsumer, notNullValue());
-    assertThat(errorListener, notNullValue());
-  }
-
-  @Test
-  public void createDestroy() {}
-
-  @Test
-  public void pullAfterErrorThrows() {
-    errorListener.failed(null, Status.INTERNAL.asException());
-    StatusException e = assertThrows(StatusException.class, subscriber::pull);
-    assertThat(e.getStatus().getCode(), equalTo(Code.INTERNAL));
-  }
-
-  @Test
-  public void emptyPull() throws StatusException {
-    assertThat(subscriber.pull(), empty());
-  }
-
-  @Test
-  public void pullEmptiesForNext() throws StatusException {
-    SequencedMessage message1 = newMessage(Timestamps.EPOCH, Offset.of(10), 10);
-    SequencedMessage message2 = newMessage(Timestamps.EPOCH, Offset.of(11), 10);
-    messageConsumer.accept(ImmutableList.of(toWrapper(message1), toWrapper(message2)));
-    assertThat(subscriber.pull(), containsInAnyOrder(message1, message2));
-    assertThat(subscriber.pull(), empty());
-  }
-
-  @Test
-  public void multipleBatchesAggregatedReturnsTokens() throws StatusException {
-    SequencedMessage message1 = newMessage(Timestamps.EPOCH, Offset.of(10), 10);
-    SequencedMessage message2 = newMessage(Timestamps.EPOCH, Offset.of(11), 20);
-    SequencedMessage message3 = newMessage(Timestamps.EPOCH, Offset.of(12), 30);
-    messageConsumer.accept(ImmutableList.of(toWrapper(message1), toWrapper(message2)));
-    messageConsumer.accept(ImmutableList.of(toWrapper(message3)));
-    assertThat(subscriber.pull(), containsInAnyOrder(message1, message2, message3));
-    assertThat(subscriber.pull(), empty());
-
-    FlowControlRequest flowControlRequest =
-        FlowControlRequest.newBuilder().setAllowedMessages(3).setAllowedBytes(60).build();
-    verify(underlying).allowFlow(flowControlRequest);
-  }
-
-  @Test
-  public void closeStops() throws Exception {
-    when(underlying.stopAsync()).thenReturn(underlying);
-    subscriber.close();
-    verify(underlying).stopAsync();
-    verify(underlying).awaitTerminated();
-  }
-}
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsublite/FakeSerializable.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsublite/FakeSerializable.java
index c802b9a..a565507 100644
--- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsublite/FakeSerializable.java
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsublite/FakeSerializable.java
@@ -26,6 +26,7 @@
  * static map. It is useful in the presence of in-process serialization, but not out of process
  * serialization.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 final class FakeSerializable {
   private static final AtomicInteger idCounter = new AtomicInteger(0);
   private static final ConcurrentHashMap<Integer, Object> map = new ConcurrentHashMap<>();
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsublite/OffsetCheckpointMarkTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsublite/OffsetCheckpointMarkTest.java
index 14f0310..e9f24ca 100644
--- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsublite/OffsetCheckpointMarkTest.java
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsublite/OffsetCheckpointMarkTest.java
@@ -39,6 +39,7 @@
 import org.mockito.MockitoAnnotations;
 
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class OffsetCheckpointMarkTest {
   @Captor private ArgumentCaptor<Map<Partition, Offset>> mapCaptor;
 
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsublite/PubsubLiteSinkTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsublite/PubsubLiteSinkTest.java
index f4524dd..7c12de8 100644
--- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsublite/PubsubLiteSinkTest.java
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsublite/PubsubLiteSinkTest.java
@@ -40,7 +40,7 @@
 import com.google.cloud.pubsublite.ProjectNumber;
 import com.google.cloud.pubsublite.PublishMetadata;
 import com.google.cloud.pubsublite.TopicName;
-import com.google.cloud.pubsublite.TopicPaths;
+import com.google.cloud.pubsublite.TopicPath;
 import com.google.cloud.pubsublite.internal.ExtractStatus;
 import com.google.cloud.pubsublite.internal.FakeApiService;
 import com.google.cloud.pubsublite.internal.Publisher;
@@ -70,6 +70,7 @@
 import org.mockito.stubbing.Answer;
 
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class PubsubLiteSinkTest {
   @Rule public final TestPipeline pipeline = TestPipeline.create();
 
@@ -82,10 +83,10 @@
     try {
       return PublisherOptions.newBuilder()
           .setTopicPath(
-              TopicPaths.newBuilder()
-                  .setProjectNumber(ProjectNumber.of(9))
-                  .setTopicName(TopicName.of("abc"))
-                  .setZone(CloudZone.of(CloudRegion.of("us-east1"), 'a'))
+              TopicPath.newBuilder()
+                  .setProject(ProjectNumber.of(9))
+                  .setName(TopicName.of("abc"))
+                  .setLocation(CloudZone.of(CloudRegion.of("us-east1"), 'a'))
                   .build())
           .build();
     } catch (StatusException e) {
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsublite/PubsubLiteUnboundedReaderTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsublite/PubsubLiteUnboundedReaderTest.java
index dbab965..2bff6b0 100644
--- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsublite/PubsubLiteUnboundedReaderTest.java
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsublite/PubsubLiteUnboundedReaderTest.java
@@ -33,6 +33,7 @@
 import com.google.cloud.pubsublite.Offset;
 import com.google.cloud.pubsublite.Partition;
 import com.google.cloud.pubsublite.internal.FakeApiService;
+import com.google.cloud.pubsublite.internal.PullSubscriber;
 import com.google.cloud.pubsublite.internal.wire.Committer;
 import com.google.cloud.pubsublite.proto.Cursor;
 import com.google.cloud.pubsublite.proto.SequencedMessage;
@@ -59,10 +60,11 @@
 import org.mockito.Spy;
 
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class PubsubLiteUnboundedReaderTest {
-  @Mock private PullSubscriber subscriber5;
+  @Mock private PullSubscriber<SequencedMessage> subscriber5;
 
-  @Mock private PullSubscriber subscriber8;
+  @Mock private PullSubscriber<SequencedMessage> subscriber8;
 
   abstract static class CommitterFakeService extends FakeApiService implements Committer {}
 
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/FakeServiceFactory.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/FakeServiceFactory.java
index 8c417bd..a816a6d 100644
--- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/FakeServiceFactory.java
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/FakeServiceFactory.java
@@ -37,6 +37,7 @@
  * A serialization friendly type service factory that maintains a mock {@link Spanner} and {@link
  * DatabaseClient}.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class FakeServiceFactory implements ServiceFactory<Spanner, SpannerOptions>, Serializable {
 
   // Marked as static so they could be returned by serviceFactory, which is serializable.
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/MutationKeyEncoderTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/MutationKeyEncoderTest.java
index 40b7ab8..69bb3f6 100644
--- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/MutationKeyEncoderTest.java
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/MutationKeyEncoderTest.java
@@ -38,6 +38,7 @@
 
 /** Tests for {@link MutationKeyEncoder}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class MutationKeyEncoderTest {
   @Rule public ExpectedException thrown = ExpectedException.none();
 
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/MutationSizeEstimatorTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/MutationSizeEstimatorTest.java
index 8d5a2e4..6e12ac1 100644
--- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/MutationSizeEstimatorTest.java
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/MutationSizeEstimatorTest.java
@@ -31,6 +31,7 @@
 
 /** A set of unit tests for {@link MutationSizeEstimator}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class MutationSizeEstimatorTest {
 
   @Test
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/OrderedCodeTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/OrderedCodeTest.java
index 522ff78..2493758 100644
--- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/OrderedCodeTest.java
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/OrderedCodeTest.java
@@ -36,6 +36,7 @@
 
 /** A set of unit tests to verify {@link OrderedCode}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class OrderedCodeTest {
   /** Data for a generic coding test case with known encoded outputs. */
   abstract static class CodingTestCase<T> {
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/ReadSpannerSchemaTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/ReadSpannerSchemaTest.java
index 6e8a91d..cb6c6ef 100644
--- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/ReadSpannerSchemaTest.java
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/ReadSpannerSchemaTest.java
@@ -42,6 +42,7 @@
 
 /** A test of {@link ReadSpannerSchemaTest}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ReadSpannerSchemaTest {
 
   @Rule public final transient ExpectedException thrown = ExpectedException.none();
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerAccessorTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerAccessorTest.java
index 8ce5d68..d4b3686 100644
--- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerAccessorTest.java
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerAccessorTest.java
@@ -29,6 +29,7 @@
 import org.junit.runners.JUnit4;
 
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SpannerAccessorTest {
 
   private FakeServiceFactory serviceFactory;
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIOReadTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIOReadTest.java
index 5977c2e..8bbbbe8 100644
--- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIOReadTest.java
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIOReadTest.java
@@ -54,6 +54,7 @@
 
 /** Unit tests for {@link SpannerIO}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SpannerIOReadTest implements Serializable {
 
   @Rule
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIOWriteTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIOWriteTest.java
index b0eb872..8d196eb 100644
--- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIOWriteTest.java
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIOWriteTest.java
@@ -93,6 +93,7 @@
  * pipeline.
  */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SpannerIOWriteTest implements Serializable {
   private static final long CELLS_PER_KEY = 7;
 
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerReadIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerReadIT.java
index cc20781..75ffe11 100644
--- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerReadIT.java
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerReadIT.java
@@ -54,6 +54,7 @@
 
 /** End-to-end test of Cloud Spanner Source. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SpannerReadIT {
 
   private static final int MAX_DB_NAME_LENGTH = 30;
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerWriteIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerWriteIT.java
index e007d3d..003e23d 100644
--- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerWriteIT.java
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/spanner/SpannerWriteIT.java
@@ -59,6 +59,7 @@
 
 /** End-to-end test of Cloud Spanner Sink. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SpannerWriteIT {
 
   private static final int MAX_DB_NAME_LENGTH = 30;
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/storage/GcsKmsKeyIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/storage/GcsKmsKeyIT.java
index 66fe8bf..7fea3c1 100644
--- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/storage/GcsKmsKeyIT.java
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/storage/GcsKmsKeyIT.java
@@ -54,6 +54,7 @@
 /** Integration test for GCS CMEK support. */
 @RunWith(JUnit4.class)
 @Category(UsesKms.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class GcsKmsKeyIT {
 
   private static final String INPUT_FILE = "gs://dataflow-samples/shakespeare/kinglear.txt";
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/testing/BigqueryClientTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/testing/BigqueryClientTest.java
index da8f944..aef19d0 100644
--- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/testing/BigqueryClientTest.java
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/testing/BigqueryClientTest.java
@@ -42,6 +42,7 @@
 /** Tests for {@link BigqueryClient}. */
 @RunWith(PowerMockRunner.class)
 @PrepareForTest(BigqueryClient.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BigqueryClientTest {
   private final String projectId = "test-project";
   private final String query = "test-query";
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/testing/BigqueryMatcherTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/testing/BigqueryMatcherTest.java
index 508dfec..4dd7de4 100644
--- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/testing/BigqueryMatcherTest.java
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/testing/BigqueryMatcherTest.java
@@ -41,6 +41,7 @@
 /** Tests for {@link BigqueryMatcher}. */
 @RunWith(PowerMockRunner.class)
 @PrepareForTest(BigqueryClient.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BigqueryMatcherTest {
   private final String appName = "test-app";
   private final String projectId = "test-project";
diff --git a/sdks/java/io/hadoop-common/src/main/java/org/apache/beam/sdk/io/hadoop/SerializableConfiguration.java b/sdks/java/io/hadoop-common/src/main/java/org/apache/beam/sdk/io/hadoop/SerializableConfiguration.java
index 46619df..4d8eb8b 100644
--- a/sdks/java/io/hadoop-common/src/main/java/org/apache/beam/sdk/io/hadoop/SerializableConfiguration.java
+++ b/sdks/java/io/hadoop-common/src/main/java/org/apache/beam/sdk/io/hadoop/SerializableConfiguration.java
@@ -31,6 +31,7 @@
  * A wrapper to allow Hadoop {@link Configuration}s to be serialized using Java's standard
  * serialization mechanisms.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SerializableConfiguration implements Externalizable {
   private static final long serialVersionUID = 0L;
 
diff --git a/sdks/java/io/hadoop-common/src/test/java/org/apache/beam/sdk/io/hadoop/SerializableConfigurationTest.java b/sdks/java/io/hadoop-common/src/test/java/org/apache/beam/sdk/io/hadoop/SerializableConfigurationTest.java
index 07b5b7f..9fa67d2 100644
--- a/sdks/java/io/hadoop-common/src/test/java/org/apache/beam/sdk/io/hadoop/SerializableConfigurationTest.java
+++ b/sdks/java/io/hadoop-common/src/test/java/org/apache/beam/sdk/io/hadoop/SerializableConfigurationTest.java
@@ -31,6 +31,7 @@
 
 /** Tests for SerializableConfiguration. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SerializableConfigurationTest {
   @Rule public final ExpectedException thrown = ExpectedException.none();
   private static final SerializableConfiguration DEFAULT_SERIALIZABLE_CONF =
diff --git a/sdks/java/io/hadoop-file-system/src/main/java/org/apache/beam/sdk/io/hdfs/HadoopFileSystemModule.java b/sdks/java/io/hadoop-file-system/src/main/java/org/apache/beam/sdk/io/hdfs/HadoopFileSystemModule.java
index 3a7e05d..47569eb 100644
--- a/sdks/java/io/hadoop-file-system/src/main/java/org/apache/beam/sdk/io/hdfs/HadoopFileSystemModule.java
+++ b/sdks/java/io/hadoop-file-system/src/main/java/org/apache/beam/sdk/io/hdfs/HadoopFileSystemModule.java
@@ -42,6 +42,7 @@
  * values dropping any configuration hierarchy and source information.
  */
 @AutoService(Module.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class HadoopFileSystemModule extends SimpleModule {
   public HadoopFileSystemModule() {
     super("HadoopFileSystemModule");
diff --git a/sdks/java/io/hadoop-file-system/src/main/java/org/apache/beam/sdk/io/hdfs/HadoopFileSystemOptions.java b/sdks/java/io/hadoop-file-system/src/main/java/org/apache/beam/sdk/io/hdfs/HadoopFileSystemOptions.java
index 2ab1a7b..43a6e7e 100644
--- a/sdks/java/io/hadoop-file-system/src/main/java/org/apache/beam/sdk/io/hdfs/HadoopFileSystemOptions.java
+++ b/sdks/java/io/hadoop-file-system/src/main/java/org/apache/beam/sdk/io/hdfs/HadoopFileSystemOptions.java
@@ -41,7 +41,10 @@
  * {@link PipelineOptions} which encapsulate {@link Configuration Hadoop Configuration} for the
  * {@link HadoopFileSystem}.
  */
-@SuppressWarnings("WeakerAccess")
+@SuppressWarnings({
+  "WeakerAccess",
+  "nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
+})
 @Experimental(Kind.FILESYSTEM)
 public interface HadoopFileSystemOptions extends PipelineOptions {
   @Description(
diff --git a/sdks/java/io/hadoop-file-system/src/main/java/org/apache/beam/sdk/io/hdfs/HadoopResourceId.java b/sdks/java/io/hadoop-file-system/src/main/java/org/apache/beam/sdk/io/hdfs/HadoopResourceId.java
index 9a6f4fe..3ddd80e 100644
--- a/sdks/java/io/hadoop-file-system/src/main/java/org/apache/beam/sdk/io/hdfs/HadoopResourceId.java
+++ b/sdks/java/io/hadoop-file-system/src/main/java/org/apache/beam/sdk/io/hdfs/HadoopResourceId.java
@@ -29,6 +29,7 @@
 import org.checkerframework.checker.nullness.qual.Nullable;
 
 /** {@link ResourceId} implementation for the {@link HadoopFileSystem}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class HadoopResourceId implements ResourceId {
   private final URI uri;
 
diff --git a/sdks/java/io/hadoop-file-system/src/test/java/org/apache/beam/sdk/io/hdfs/HadoopFileSystemRegistrarTest.java b/sdks/java/io/hadoop-file-system/src/test/java/org/apache/beam/sdk/io/hdfs/HadoopFileSystemRegistrarTest.java
index f8aaeb6..ba17b90 100644
--- a/sdks/java/io/hadoop-file-system/src/test/java/org/apache/beam/sdk/io/hdfs/HadoopFileSystemRegistrarTest.java
+++ b/sdks/java/io/hadoop-file-system/src/test/java/org/apache/beam/sdk/io/hdfs/HadoopFileSystemRegistrarTest.java
@@ -40,6 +40,7 @@
 
 /** Tests for {@link HadoopFileSystemRegistrar}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class HadoopFileSystemRegistrarTest {
 
   @Rule public TemporaryFolder tmpFolder = new TemporaryFolder();
diff --git a/sdks/java/io/hadoop-file-system/src/test/java/org/apache/beam/sdk/io/hdfs/HadoopFileSystemTest.java b/sdks/java/io/hadoop-file-system/src/test/java/org/apache/beam/sdk/io/hdfs/HadoopFileSystemTest.java
index f9464f9..3bede5b 100644
--- a/sdks/java/io/hadoop-file-system/src/test/java/org/apache/beam/sdk/io/hdfs/HadoopFileSystemTest.java
+++ b/sdks/java/io/hadoop-file-system/src/test/java/org/apache/beam/sdk/io/hdfs/HadoopFileSystemTest.java
@@ -66,6 +66,7 @@
 
 /** Tests for {@link HadoopFileSystem}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class HadoopFileSystemTest {
 
   @Rule public TestPipeline p = TestPipeline.create();
diff --git a/sdks/java/io/hadoop-file-system/src/test/java/org/apache/beam/sdk/io/hdfs/HadoopResourceIdTest.java b/sdks/java/io/hadoop-file-system/src/test/java/org/apache/beam/sdk/io/hdfs/HadoopResourceIdTest.java
index 1726a3e..27ecf9a 100644
--- a/sdks/java/io/hadoop-file-system/src/test/java/org/apache/beam/sdk/io/hdfs/HadoopResourceIdTest.java
+++ b/sdks/java/io/hadoop-file-system/src/test/java/org/apache/beam/sdk/io/hdfs/HadoopResourceIdTest.java
@@ -35,6 +35,7 @@
 import org.junit.rules.TemporaryFolder;
 
 /** Tests for {@link HadoopResourceId}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class HadoopResourceIdTest {
 
   private MiniDFSCluster hdfsCluster;
diff --git a/sdks/java/io/hadoop-format/build.gradle b/sdks/java/io/hadoop-format/build.gradle
index 498b90a..088b042 100644
--- a/sdks/java/io/hadoop-format/build.gradle
+++ b/sdks/java/io/hadoop-format/build.gradle
@@ -131,3 +131,12 @@
   systemProperty "beam.spark.test.reuseSparkContext", "true"
   systemProperty "beamTestPipelineOptions", JsonOutput.toJson(beamTestPipelineOptions)
 }
+
+task hadoopFormatIOElasticTest(type: Test) {
+  group = "Verification"
+  description = 'Runs HadoopFormatIO integration test with ElasticSearchContainer'
+  outputs.upToDateWhen { false }
+  testClassesDirs = sourceSets.test.output.classesDirs
+  include '**/HadoopFormatIOElasticIT.class'
+  useJUnit { }
+}
\ No newline at end of file
diff --git a/sdks/java/io/hadoop-format/src/main/java/org/apache/beam/sdk/io/hadoop/format/HadoopFormatIO.java b/sdks/java/io/hadoop-format/src/main/java/org/apache/beam/sdk/io/hadoop/format/HadoopFormatIO.java
index efb03e1..7248e5f 100644
--- a/sdks/java/io/hadoop-format/src/main/java/org/apache/beam/sdk/io/hadoop/format/HadoopFormatIO.java
+++ b/sdks/java/io/hadoop-format/src/main/java/org/apache/beam/sdk/io/hadoop/format/HadoopFormatIO.java
@@ -296,6 +296,7 @@
  * }</pre>
  */
 @Experimental(Kind.SOURCE_SINK)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class HadoopFormatIO {
   private static final Logger LOG = LoggerFactory.getLogger(HadoopFormatIO.class);
 
diff --git a/sdks/java/io/hadoop-format/src/main/java/org/apache/beam/sdk/io/hadoop/format/HadoopFormats.java b/sdks/java/io/hadoop-format/src/main/java/org/apache/beam/sdk/io/hadoop/format/HadoopFormats.java
index d417617..1db7c92 100644
--- a/sdks/java/io/hadoop-format/src/main/java/org/apache/beam/sdk/io/hadoop/format/HadoopFormats.java
+++ b/sdks/java/io/hadoop-format/src/main/java/org/apache/beam/sdk/io/hadoop/format/HadoopFormats.java
@@ -34,6 +34,7 @@
 import org.checkerframework.checker.nullness.qual.Nullable;
 
 /** Utility class for working with Hadoop related objects. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 final class HadoopFormats {
 
   private static final int DEFAULT_JOB_NUMBER = 0;
diff --git a/sdks/java/io/hadoop-format/src/test/java/org/apache/beam/sdk/io/hadoop/format/ConfigurableEmployeeInputFormat.java b/sdks/java/io/hadoop-format/src/test/java/org/apache/beam/sdk/io/hadoop/format/ConfigurableEmployeeInputFormat.java
index 5e5640b..028569d 100644
--- a/sdks/java/io/hadoop-format/src/test/java/org/apache/beam/sdk/io/hadoop/format/ConfigurableEmployeeInputFormat.java
+++ b/sdks/java/io/hadoop-format/src/test/java/org/apache/beam/sdk/io/hadoop/format/ConfigurableEmployeeInputFormat.java
@@ -37,6 +37,7 @@
  * Configurable. This validates if setConf() method is called before getSplits(). Known InputFormats
  * which implement Configurable are DBInputFormat, TableInputFormat etc.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class ConfigurableEmployeeInputFormat extends InputFormat<Text, Employee> implements Configurable {
   public boolean isConfSet = false;
 
diff --git a/sdks/java/io/hadoop-format/src/test/java/org/apache/beam/sdk/io/hadoop/format/Employee.java b/sdks/java/io/hadoop-format/src/test/java/org/apache/beam/sdk/io/hadoop/format/Employee.java
index fe2cda1..9703b58 100644
--- a/sdks/java/io/hadoop-format/src/test/java/org/apache/beam/sdk/io/hadoop/format/Employee.java
+++ b/sdks/java/io/hadoop-format/src/test/java/org/apache/beam/sdk/io/hadoop/format/Employee.java
@@ -27,6 +27,7 @@
  * HadoopFormatIO} for different unit tests.
  */
 @DefaultCoder(AvroCoder.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class Employee {
   private String empAddress;
   private String empName;
diff --git a/sdks/java/io/hadoop-format/src/test/java/org/apache/beam/sdk/io/hadoop/format/EmployeeInputFormat.java b/sdks/java/io/hadoop-format/src/test/java/org/apache/beam/sdk/io/hadoop/format/EmployeeInputFormat.java
index 93679f9..d2fb6ce 100644
--- a/sdks/java/io/hadoop-format/src/test/java/org/apache/beam/sdk/io/hadoop/format/EmployeeInputFormat.java
+++ b/sdks/java/io/hadoop-format/src/test/java/org/apache/beam/sdk/io/hadoop/format/EmployeeInputFormat.java
@@ -40,6 +40,7 @@
  * whether the {@linkplain HadoopFormatIO } source returns immutable records in the scenario when
  * RecordReader creates new key and value objects every time it reads data.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class EmployeeInputFormat extends InputFormat<Text, Employee> {
 
   public EmployeeInputFormat() {}
diff --git a/sdks/java/io/hadoop-format/src/test/java/org/apache/beam/sdk/io/hadoop/format/EmployeeOutputFormat.java b/sdks/java/io/hadoop-format/src/test/java/org/apache/beam/sdk/io/hadoop/format/EmployeeOutputFormat.java
index 83194de..dd9bf44 100644
--- a/sdks/java/io/hadoop-format/src/test/java/org/apache/beam/sdk/io/hadoop/format/EmployeeOutputFormat.java
+++ b/sdks/java/io/hadoop-format/src/test/java/org/apache/beam/sdk/io/hadoop/format/EmployeeOutputFormat.java
@@ -33,6 +33,7 @@
  * List<KV>}. {@linkplain EmployeeOutputFormat} is used to test the {@linkplain HadoopFormatIO }
  * sink.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class EmployeeOutputFormat extends OutputFormat<Text, Employee> {
   private static volatile List<KV<Text, Employee>> output;
   private static OutputCommitter outputCommitter;
diff --git a/sdks/java/io/hadoop-format/src/test/java/org/apache/beam/sdk/io/hadoop/format/HDFSSynchronizationTest.java b/sdks/java/io/hadoop-format/src/test/java/org/apache/beam/sdk/io/hadoop/format/HDFSSynchronizationTest.java
index 2a9236b..b23f7f0 100644
--- a/sdks/java/io/hadoop-format/src/test/java/org/apache/beam/sdk/io/hadoop/format/HDFSSynchronizationTest.java
+++ b/sdks/java/io/hadoop-format/src/test/java/org/apache/beam/sdk/io/hadoop/format/HDFSSynchronizationTest.java
@@ -39,6 +39,7 @@
 
 /** Tests functionality of {@link HDFSSynchronization} class. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class HDFSSynchronizationTest {
   private static final String DEFAULT_JOB_ID = String.valueOf(1);
   @Rule public TemporaryFolder tmpFolder = new TemporaryFolder();
diff --git a/sdks/java/io/hadoop-format/src/test/java/org/apache/beam/sdk/io/hadoop/format/HadoopFormatIOCassandraIT.java b/sdks/java/io/hadoop-format/src/test/java/org/apache/beam/sdk/io/hadoop/format/HadoopFormatIOCassandraIT.java
index cd85b97..7ed582c 100644
--- a/sdks/java/io/hadoop-format/src/test/java/org/apache/beam/sdk/io/hadoop/format/HadoopFormatIOCassandraIT.java
+++ b/sdks/java/io/hadoop-format/src/test/java/org/apache/beam/sdk/io/hadoop/format/HadoopFormatIOCassandraIT.java
@@ -63,6 +63,7 @@
  * invocation pipeline options.
  */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class HadoopFormatIOCassandraIT implements Serializable {
 
   private static final String CASSANDRA_KEYSPACE = "ycsb";
diff --git a/sdks/java/io/hadoop-format/src/test/java/org/apache/beam/sdk/io/hadoop/format/HadoopFormatIOCassandraTest.java b/sdks/java/io/hadoop-format/src/test/java/org/apache/beam/sdk/io/hadoop/format/HadoopFormatIOCassandraTest.java
index 22161c7..c1a415a 100644
--- a/sdks/java/io/hadoop-format/src/test/java/org/apache/beam/sdk/io/hadoop/format/HadoopFormatIOCassandraTest.java
+++ b/sdks/java/io/hadoop-format/src/test/java/org/apache/beam/sdk/io/hadoop/format/HadoopFormatIOCassandraTest.java
@@ -51,6 +51,7 @@
 
 /** Tests to validate HadoopFormatIO for embedded Cassandra instance. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class HadoopFormatIOCassandraTest implements Serializable {
   private static final long serialVersionUID = 1L;
   private static final String CASSANDRA_KEYSPACE = "beamdb";
diff --git a/sdks/java/io/hadoop-format/src/test/java/org/apache/beam/sdk/io/hadoop/format/HadoopFormatIOElasticIT.java b/sdks/java/io/hadoop-format/src/test/java/org/apache/beam/sdk/io/hadoop/format/HadoopFormatIOElasticIT.java
index 78c2cd2..464f999 100644
--- a/sdks/java/io/hadoop-format/src/test/java/org/apache/beam/sdk/io/hadoop/format/HadoopFormatIOElasticIT.java
+++ b/sdks/java/io/hadoop-format/src/test/java/org/apache/beam/sdk/io/hadoop/format/HadoopFormatIOElasticIT.java
@@ -17,8 +17,13 @@
  */
 package org.apache.beam.sdk.io.hadoop.format;
 
+import java.io.IOException;
 import java.io.Serializable;
+import java.util.HashMap;
+import java.util.Map;
 import org.apache.beam.sdk.io.common.HashingFn;
+import org.apache.beam.sdk.options.Default;
+import org.apache.beam.sdk.options.Description;
 import org.apache.beam.sdk.options.PipelineOptionsFactory;
 import org.apache.beam.sdk.testing.PAssert;
 import org.apache.beam.sdk.testing.TestPipeline;
@@ -33,13 +38,24 @@
 import org.apache.hadoop.io.MapWritable;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.http.HttpHost;
+import org.elasticsearch.action.index.IndexRequest;
+import org.elasticsearch.client.RequestOptions;
+import org.elasticsearch.client.RestClient;
+import org.elasticsearch.client.RestHighLevelClient;
 import org.elasticsearch.hadoop.cfg.ConfigurationOptions;
 import org.elasticsearch.hadoop.mr.LinkedMapWritable;
+import org.joda.time.DateTime;
+import org.joda.time.Instant;
+import org.joda.time.format.DateTimeFormat;
+import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
+import org.testcontainers.elasticsearch.ElasticsearchContainer;
+import org.testcontainers.utility.DockerImageName;
 
 /**
  * A test of {@link org.apache.beam.sdk.io.hadoop.format.HadoopFormatIO.Read} on an independent
@@ -57,7 +73,8 @@
  *  "--elasticServerIp=1.2.3.4",
  *  "--elasticServerPort=port",
  *  "--elasticUserName=user",
- *  "--elasticPassword=mypass" ]'
+ *  "--elasticPassword=mypass",
+ *  "--withESContainer=false" ]'
  *  --tests org.apache.beam.sdk.io.hadoop.format.HadoopFormatIOElasticIT
  *  -DintegrationTestRunner=direct
  * </pre>
@@ -67,17 +84,38 @@
  * invocation pipeline options.
  */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class HadoopFormatIOElasticIT implements Serializable {
 
   private static final String TRUE = "true";
   private static final String ELASTIC_INDEX_NAME = "test_data";
-  private static HadoopFormatIOTestOptions options;
+  private static HadoopFormatElasticIOTestOptions options;
   @Rule public final transient TestPipeline pipeline = TestPipeline.create();
 
+  private static ElasticsearchContainer elasticsearch;
+
+  public interface HadoopFormatElasticIOTestOptions extends HadoopFormatIOTestOptions {
+    @Description("Whether to use automatic ElasticSearch container")
+    @Default.Boolean(true)
+    Boolean isWithESContainer();
+
+    void setWithESContainer(Boolean withESContainer);
+  }
+
   @BeforeClass
-  public static void setUp() {
-    PipelineOptionsFactory.register(HadoopFormatIOTestOptions.class);
-    options = TestPipeline.testingPipelineOptions().as(HadoopFormatIOTestOptions.class);
+  public static void setUp() throws IOException {
+    PipelineOptionsFactory.register(HadoopFormatElasticIOTestOptions.class);
+    options = TestPipeline.testingPipelineOptions().as(HadoopFormatElasticIOTestOptions.class);
+    if (options.isWithESContainer()) {
+      setElasticsearchContainer();
+    }
+  }
+
+  @AfterClass
+  public static void afterClass() {
+    if (options.isWithESContainer()) {
+      elasticsearch.stop();
+    }
   }
 
   /**
@@ -210,4 +248,53 @@
     conf.set("es.batch.size.bytes", "8mb");
     return conf;
   }
+
+  private static void setElasticsearchContainer() throws IOException {
+    elasticsearch =
+        new ElasticsearchContainer(
+            DockerImageName.parse("docker.elastic.co/elasticsearch/elasticsearch")
+                .withTag("7.9.2"));
+    elasticsearch.start();
+    options.setElasticUserName("");
+    options.setElasticPassword("");
+    options.setElasticServerIp(elasticsearch.getContainerIpAddress());
+    options.setElasticServerPort(elasticsearch.getMappedPort(9200));
+    prepareElasticIndex();
+  }
+
+  private static Map<String, String> createElasticRow(Integer i) {
+    Map<String, String> data = new HashMap<>();
+    data.put("User_Name", "User_Name" + i);
+    data.put("Item_Code", "" + i);
+    data.put("Txn_ID", "" + i);
+    data.put("Item_ID", "" + i);
+    data.put("last_updated", "" + (i * 1000));
+    data.put("Price", "" + i);
+    data.put("Title", "Title" + i);
+    data.put("Description", "Description" + i);
+    data.put("Age", "" + i);
+    data.put("Item_Name", "Item_Name" + i);
+    data.put("Item_Price", "" + i);
+    data.put("Availability", "" + (i % 2 == 0));
+    data.put("Batch_Num", "" + i);
+    data.put(
+        "Last_Ordered",
+        new DateTime(Instant.ofEpochSecond(i))
+            .toString(DateTimeFormat.forPattern("yyyy-MM-dd'T'HH:mm:ss.000-0000")));
+    data.put("City", "City" + i);
+    return data;
+  }
+
+  private static void prepareElasticIndex() throws IOException {
+    RestHighLevelClient client =
+        new RestHighLevelClient(
+            RestClient.builder(
+                new HttpHost(
+                    options.getElasticServerIp(), options.getElasticServerPort(), "http")));
+
+    for (int i = 0; i < 1000; i++) {
+      IndexRequest request = new IndexRequest(ELASTIC_INDEX_NAME).source(createElasticRow(i));
+      client.index(request, RequestOptions.DEFAULT);
+    }
+  }
 }
diff --git a/sdks/java/io/hadoop-format/src/test/java/org/apache/beam/sdk/io/hadoop/format/HadoopFormatIOIT.java b/sdks/java/io/hadoop-format/src/test/java/org/apache/beam/sdk/io/hadoop/format/HadoopFormatIOIT.java
index cbfbb28..0a60565 100644
--- a/sdks/java/io/hadoop-format/src/test/java/org/apache/beam/sdk/io/hadoop/format/HadoopFormatIOIT.java
+++ b/sdks/java/io/hadoop-format/src/test/java/org/apache/beam/sdk/io/hadoop/format/HadoopFormatIOIT.java
@@ -90,6 +90,7 @@
  * performance testing framework.
  */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class HadoopFormatIOIT {
 
   private static final String NAMESPACE = HadoopFormatIOIT.class.getName();
diff --git a/sdks/java/io/hadoop-format/src/test/java/org/apache/beam/sdk/io/hadoop/format/HadoopFormatIOReadTest.java b/sdks/java/io/hadoop-format/src/test/java/org/apache/beam/sdk/io/hadoop/format/HadoopFormatIOReadTest.java
index aca7fe0..373a2e4 100644
--- a/sdks/java/io/hadoop-format/src/test/java/org/apache/beam/sdk/io/hadoop/format/HadoopFormatIOReadTest.java
+++ b/sdks/java/io/hadoop-format/src/test/java/org/apache/beam/sdk/io/hadoop/format/HadoopFormatIOReadTest.java
@@ -65,6 +65,7 @@
 
 /** Unit tests for {@link HadoopFormatIO.Read}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class HadoopFormatIOReadTest {
   private static SerializableConfiguration serConf;
   private static SimpleFunction<Text, String> myKeyTranslate;
diff --git a/sdks/java/io/hadoop-format/src/test/java/org/apache/beam/sdk/io/hadoop/format/HadoopFormatIOSequenceFileTest.java b/sdks/java/io/hadoop-format/src/test/java/org/apache/beam/sdk/io/hadoop/format/HadoopFormatIOSequenceFileTest.java
index 89a5c5b..f8c9ab2 100644
--- a/sdks/java/io/hadoop-format/src/test/java/org/apache/beam/sdk/io/hadoop/format/HadoopFormatIOSequenceFileTest.java
+++ b/sdks/java/io/hadoop-format/src/test/java/org/apache/beam/sdk/io/hadoop/format/HadoopFormatIOSequenceFileTest.java
@@ -69,6 +69,7 @@
 
 /** Tests {@link HadoopFormatIO} output with batch and stream pipeline. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class HadoopFormatIOSequenceFileTest {
 
   private static final Instant START_TIME = new Instant(0);
diff --git a/sdks/java/io/hadoop-format/src/test/java/org/apache/beam/sdk/io/hadoop/format/HadoopFormatIOWriteTest.java b/sdks/java/io/hadoop-format/src/test/java/org/apache/beam/sdk/io/hadoop/format/HadoopFormatIOWriteTest.java
index 9c41bcd..b1e74fb 100644
--- a/sdks/java/io/hadoop-format/src/test/java/org/apache/beam/sdk/io/hadoop/format/HadoopFormatIOWriteTest.java
+++ b/sdks/java/io/hadoop-format/src/test/java/org/apache/beam/sdk/io/hadoop/format/HadoopFormatIOWriteTest.java
@@ -51,6 +51,7 @@
 
 /** Unit tests for {@link HadoopFormatIO.Write}. */
 @RunWith(MockitoJUnitRunner.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class HadoopFormatIOWriteTest {
 
   private static final int REDUCERS_COUNT = 2;
diff --git a/sdks/java/io/hadoop-format/src/test/java/org/apache/beam/sdk/io/hadoop/format/IterableCombinerTest.java b/sdks/java/io/hadoop-format/src/test/java/org/apache/beam/sdk/io/hadoop/format/IterableCombinerTest.java
index 1f45f5f..7f7a252 100644
--- a/sdks/java/io/hadoop-format/src/test/java/org/apache/beam/sdk/io/hadoop/format/IterableCombinerTest.java
+++ b/sdks/java/io/hadoop-format/src/test/java/org/apache/beam/sdk/io/hadoop/format/IterableCombinerTest.java
@@ -35,6 +35,7 @@
 
 /** Tests Iterable combiner whether works correctly. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class IterableCombinerTest {
 
   private static final TypeDescriptor<String> STRING_TYPE_DESCRIPTOR = TypeDescriptors.strings();
diff --git a/sdks/java/io/hadoop-format/src/test/java/org/apache/beam/sdk/io/hadoop/format/ReuseObjectsEmployeeInputFormat.java b/sdks/java/io/hadoop-format/src/test/java/org/apache/beam/sdk/io/hadoop/format/ReuseObjectsEmployeeInputFormat.java
index 2e20102..7ae9e12 100644
--- a/sdks/java/io/hadoop-format/src/test/java/org/apache/beam/sdk/io/hadoop/format/ReuseObjectsEmployeeInputFormat.java
+++ b/sdks/java/io/hadoop-format/src/test/java/org/apache/beam/sdk/io/hadoop/format/ReuseObjectsEmployeeInputFormat.java
@@ -49,6 +49,7 @@
  * HadoopFormatIO } source returns immutable records for a scenario when RecordReader returns the
  * same key and value objects with updating values every time it reads data.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class ReuseObjectsEmployeeInputFormat extends InputFormat<Text, Employee> {
 
   public ReuseObjectsEmployeeInputFormat() {}
diff --git a/sdks/java/io/hadoop-format/src/test/java/org/apache/beam/sdk/io/hadoop/format/TestRowDBWritable.java b/sdks/java/io/hadoop-format/src/test/java/org/apache/beam/sdk/io/hadoop/format/TestRowDBWritable.java
index f4e3677..0ed033e 100644
--- a/sdks/java/io/hadoop-format/src/test/java/org/apache/beam/sdk/io/hadoop/format/TestRowDBWritable.java
+++ b/sdks/java/io/hadoop-format/src/test/java/org/apache/beam/sdk/io/hadoop/format/TestRowDBWritable.java
@@ -35,6 +35,7 @@
  * org.apache.hadoop.mapreduce.lib.db.DBInputFormat}.
  */
 @DefaultCoder(AvroCoder.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class TestRowDBWritable extends TestRow implements DBWritable, Writable {
 
   private Integer id;
diff --git a/sdks/java/io/hbase/src/main/java/org/apache/beam/sdk/io/hbase/HBaseIO.java b/sdks/java/io/hbase/src/main/java/org/apache/beam/sdk/io/hbase/HBaseIO.java
index e6f3c39..9b39476 100644
--- a/sdks/java/io/hbase/src/main/java/org/apache/beam/sdk/io/hbase/HBaseIO.java
+++ b/sdks/java/io/hbase/src/main/java/org/apache/beam/sdk/io/hbase/HBaseIO.java
@@ -157,6 +157,7 @@
  * .
  */
 @Experimental(Kind.SOURCE_SINK)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class HBaseIO {
   private static final Logger LOG = LoggerFactory.getLogger(HBaseIO.class);
 
diff --git a/sdks/java/io/hbase/src/test/java/org/apache/beam/sdk/io/hbase/HBaseIOIT.java b/sdks/java/io/hbase/src/test/java/org/apache/beam/sdk/io/hbase/HBaseIOIT.java
index 3b7787f..bfa3b17 100644
--- a/sdks/java/io/hbase/src/test/java/org/apache/beam/sdk/io/hbase/HBaseIOIT.java
+++ b/sdks/java/io/hbase/src/test/java/org/apache/beam/sdk/io/hbase/HBaseIOIT.java
@@ -67,6 +67,7 @@
  * </pre>
  */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class HBaseIOIT {
 
   /** HBaseIOIT options. */
diff --git a/sdks/java/io/hbase/src/test/java/org/apache/beam/sdk/io/hbase/HBaseIOTest.java b/sdks/java/io/hbase/src/test/java/org/apache/beam/sdk/io/hbase/HBaseIOTest.java
index 3e9a166..8aeb10b 100644
--- a/sdks/java/io/hbase/src/test/java/org/apache/beam/sdk/io/hbase/HBaseIOTest.java
+++ b/sdks/java/io/hbase/src/test/java/org/apache/beam/sdk/io/hbase/HBaseIOTest.java
@@ -79,6 +79,7 @@
 
 /** Test HBaseIO. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class HBaseIOTest {
   @Rule public final transient TestPipeline p = TestPipeline.create();
   @Rule public ExpectedException thrown = ExpectedException.none();
diff --git a/sdks/java/io/hcatalog/src/main/java/org/apache/beam/sdk/io/hcatalog/HCatToRow.java b/sdks/java/io/hcatalog/src/main/java/org/apache/beam/sdk/io/hcatalog/HCatToRow.java
index 373ccb1..fa211e8 100644
--- a/sdks/java/io/hcatalog/src/main/java/org/apache/beam/sdk/io/hcatalog/HCatToRow.java
+++ b/sdks/java/io/hcatalog/src/main/java/org/apache/beam/sdk/io/hcatalog/HCatToRow.java
@@ -30,6 +30,7 @@
 
 /** Utilities to convert {@link HCatRecord HCatRecords} to {@link Row Rows}. */
 @Experimental(Kind.SCHEMAS)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class HCatToRow {
 
   /**
diff --git a/sdks/java/io/hcatalog/src/main/java/org/apache/beam/sdk/io/hcatalog/HCatalogBeamSchema.java b/sdks/java/io/hcatalog/src/main/java/org/apache/beam/sdk/io/hcatalog/HCatalogBeamSchema.java
index bbed022..4d9ccbd 100644
--- a/sdks/java/io/hcatalog/src/main/java/org/apache/beam/sdk/io/hcatalog/HCatalogBeamSchema.java
+++ b/sdks/java/io/hcatalog/src/main/java/org/apache/beam/sdk/io/hcatalog/HCatalogBeamSchema.java
@@ -40,6 +40,7 @@
  * <p>One of the use cases is to perform the schema conversion without leaking any HCatalog types.
  */
 @Experimental(Kind.SCHEMAS)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class HCatalogBeamSchema {
 
   private final @Nullable IMetaStoreClient metastore;
diff --git a/sdks/java/io/hcatalog/src/main/java/org/apache/beam/sdk/io/hcatalog/HCatalogIO.java b/sdks/java/io/hcatalog/src/main/java/org/apache/beam/sdk/io/hcatalog/HCatalogIO.java
index 67c0af9..eb0f188 100644
--- a/sdks/java/io/hcatalog/src/main/java/org/apache/beam/sdk/io/hcatalog/HCatalogIO.java
+++ b/sdks/java/io/hcatalog/src/main/java/org/apache/beam/sdk/io/hcatalog/HCatalogIO.java
@@ -126,6 +126,7 @@
  * }</pre>
  */
 @Experimental(Kind.SOURCE_SINK)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class HCatalogIO {
 
   private static final Logger LOG = LoggerFactory.getLogger(HCatalogIO.class);
diff --git a/sdks/java/io/hcatalog/src/main/java/org/apache/beam/sdk/io/hcatalog/HCatalogUtils.java b/sdks/java/io/hcatalog/src/main/java/org/apache/beam/sdk/io/hcatalog/HCatalogUtils.java
index bf3638e..8fc30a7 100644
--- a/sdks/java/io/hcatalog/src/main/java/org/apache/beam/sdk/io/hcatalog/HCatalogUtils.java
+++ b/sdks/java/io/hcatalog/src/main/java/org/apache/beam/sdk/io/hcatalog/HCatalogUtils.java
@@ -32,6 +32,7 @@
 import org.apache.hive.hcatalog.common.HCatUtil;
 
 /** Utility classes to enable meta store conf/client creation. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class HCatalogUtils {
 
   private static final int DESIRED_BUNDLE_SIZE_BYTES = 134217728; // 128 MB
diff --git a/sdks/java/io/hcatalog/src/main/java/org/apache/beam/sdk/io/hcatalog/PartitionPollerFn.java b/sdks/java/io/hcatalog/src/main/java/org/apache/beam/sdk/io/hcatalog/PartitionPollerFn.java
index 2e40710..3d909a8 100644
--- a/sdks/java/io/hcatalog/src/main/java/org/apache/beam/sdk/io/hcatalog/PartitionPollerFn.java
+++ b/sdks/java/io/hcatalog/src/main/java/org/apache/beam/sdk/io/hcatalog/PartitionPollerFn.java
@@ -28,6 +28,7 @@
 import org.joda.time.Instant;
 
 /** Return the list of current partitions present. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class PartitionPollerFn extends PollFn<Read, Integer> {
   private transient IMetaStoreClient metaStoreClient;
 
diff --git a/sdks/java/io/hcatalog/src/main/java/org/apache/beam/sdk/io/hcatalog/PartitionReaderFn.java b/sdks/java/io/hcatalog/src/main/java/org/apache/beam/sdk/io/hcatalog/PartitionReaderFn.java
index fe69417..580d056 100644
--- a/sdks/java/io/hcatalog/src/main/java/org/apache/beam/sdk/io/hcatalog/PartitionReaderFn.java
+++ b/sdks/java/io/hcatalog/src/main/java/org/apache/beam/sdk/io/hcatalog/PartitionReaderFn.java
@@ -38,6 +38,7 @@
 import org.apache.hive.hcatalog.data.transfer.ReaderContext;
 
 /** Reads partition at a given index. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class PartitionReaderFn extends DoFn<KV<Read, Integer>, HCatRecord> {
   private transient IMetaStoreClient metaStoreClient;
   private Map<String, String> configProperties;
diff --git a/sdks/java/io/hcatalog/src/main/java/org/apache/beam/sdk/io/hcatalog/SchemaUtils.java b/sdks/java/io/hcatalog/src/main/java/org/apache/beam/sdk/io/hcatalog/SchemaUtils.java
index bc1562e..b7ffef5 100644
--- a/sdks/java/io/hcatalog/src/main/java/org/apache/beam/sdk/io/hcatalog/SchemaUtils.java
+++ b/sdks/java/io/hcatalog/src/main/java/org/apache/beam/sdk/io/hcatalog/SchemaUtils.java
@@ -33,6 +33,7 @@
 
 /** Utils to convert between HCatalog schema types and Beam schema types. */
 @Experimental(Kind.SCHEMAS)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class SchemaUtils {
 
   private static final Map<HCatFieldSchema.Type, FieldType> HCAT_TO_BEAM_TYPES_MAP =
diff --git a/sdks/java/io/hcatalog/src/test/java/org/apache/beam/sdk/io/hcatalog/HCatalogBeamSchemaTest.java b/sdks/java/io/hcatalog/src/test/java/org/apache/beam/sdk/io/hcatalog/HCatalogBeamSchemaTest.java
index 18d25ae..3b5b89f 100644
--- a/sdks/java/io/hcatalog/src/test/java/org/apache/beam/sdk/io/hcatalog/HCatalogBeamSchemaTest.java
+++ b/sdks/java/io/hcatalog/src/test/java/org/apache/beam/sdk/io/hcatalog/HCatalogBeamSchemaTest.java
@@ -39,6 +39,7 @@
 
 /** Unit tests for {@link HCatalogBeamSchema}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class HCatalogBeamSchemaTest implements Serializable {
 
   private static final String TEST_TABLE_PARTITIONED = TEST_TABLE + "_partitioned";
diff --git a/sdks/java/io/hcatalog/src/test/java/org/apache/beam/sdk/io/hcatalog/HCatalogIOIT.java b/sdks/java/io/hcatalog/src/test/java/org/apache/beam/sdk/io/hcatalog/HCatalogIOIT.java
index 60d9258..d7b2c65 100644
--- a/sdks/java/io/hcatalog/src/test/java/org/apache/beam/sdk/io/hcatalog/HCatalogIOIT.java
+++ b/sdks/java/io/hcatalog/src/test/java/org/apache/beam/sdk/io/hcatalog/HCatalogIOIT.java
@@ -65,6 +65,7 @@
  * runner invocation pipeline options.
  */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class HCatalogIOIT {
 
   /** PipelineOptions for testing {@link org.apache.beam.sdk.io.hcatalog.HCatalogIO}. */
diff --git a/sdks/java/io/hcatalog/src/test/java/org/apache/beam/sdk/io/hcatalog/HCatalogIOTest.java b/sdks/java/io/hcatalog/src/test/java/org/apache/beam/sdk/io/hcatalog/HCatalogIOTest.java
index c7f3a2e..8f7e2f3 100644
--- a/sdks/java/io/hcatalog/src/test/java/org/apache/beam/sdk/io/hcatalog/HCatalogIOTest.java
+++ b/sdks/java/io/hcatalog/src/test/java/org/apache/beam/sdk/io/hcatalog/HCatalogIOTest.java
@@ -83,6 +83,7 @@
 
 /** Test for HCatalogIO. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class HCatalogIOTest implements Serializable {
   private static final PipelineOptions OPTIONS = PipelineOptionsFactory.create();
 
diff --git a/sdks/java/io/hcatalog/src/test/java/org/apache/beam/sdk/io/hcatalog/HiveDatabaseTestHelper.java b/sdks/java/io/hcatalog/src/test/java/org/apache/beam/sdk/io/hcatalog/HiveDatabaseTestHelper.java
index fb83c00..2457340 100644
--- a/sdks/java/io/hcatalog/src/test/java/org/apache/beam/sdk/io/hcatalog/HiveDatabaseTestHelper.java
+++ b/sdks/java/io/hcatalog/src/test/java/org/apache/beam/sdk/io/hcatalog/HiveDatabaseTestHelper.java
@@ -24,6 +24,7 @@
 import org.apache.beam.sdk.io.common.DatabaseTestHelper;
 
 /** Helper for creating connection and test tables on hive database via JDBC driver. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class HiveDatabaseTestHelper {
   private static Connection con;
   private static Statement stmt;
diff --git a/sdks/java/io/hcatalog/src/test/java/org/apache/beam/sdk/io/hcatalog/SchemaUtilsTest.java b/sdks/java/io/hcatalog/src/test/java/org/apache/beam/sdk/io/hcatalog/SchemaUtilsTest.java
index 5b748da..56a1b5f 100644
--- a/sdks/java/io/hcatalog/src/test/java/org/apache/beam/sdk/io/hcatalog/SchemaUtilsTest.java
+++ b/sdks/java/io/hcatalog/src/test/java/org/apache/beam/sdk/io/hcatalog/SchemaUtilsTest.java
@@ -24,6 +24,7 @@
 import org.junit.Assert;
 import org.junit.Test;
 
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SchemaUtilsTest {
   @Test
   public void testParameterizedTypesToBeamTypes() {
diff --git a/sdks/java/io/influxdb/src/main/java/org/apache/beam/sdk/io/influxdb/InfluxDbIO.java b/sdks/java/io/influxdb/src/main/java/org/apache/beam/sdk/io/influxdb/InfluxDbIO.java
index 4155672..c937d09 100644
--- a/sdks/java/io/influxdb/src/main/java/org/apache/beam/sdk/io/influxdb/InfluxDbIO.java
+++ b/sdks/java/io/influxdb/src/main/java/org/apache/beam/sdk/io/influxdb/InfluxDbIO.java
@@ -99,6 +99,7 @@
  * <p>The source and sink also accepts optional configuration: {@code withRetentionPolicy()}
  */
 @Experimental(Kind.SOURCE_SINK)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class InfluxDbIO {
   private static final Logger LOG = LoggerFactory.getLogger(InfluxDbIO.class);
 
diff --git a/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcIO.java b/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcIO.java
index c2ef750..af304f9 100644
--- a/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcIO.java
+++ b/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcIO.java
@@ -206,6 +206,7 @@
  * statements</a> supported by your database instead.
  */
 @Experimental(Kind.SOURCE_SINK)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class JdbcIO {
 
   private static final Logger LOG = LoggerFactory.getLogger(JdbcIO.class);
diff --git a/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcSchemaIOProvider.java b/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcSchemaIOProvider.java
index 3304711..9d7af2d 100644
--- a/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcSchemaIOProvider.java
+++ b/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcSchemaIOProvider.java
@@ -40,6 +40,7 @@
  */
 @Internal
 @AutoService(SchemaIOProvider.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class JdbcSchemaIOProvider implements SchemaIOProvider {
 
   /** Returns an id that uniquely represents this IO. */
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 661d386..00d3896 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
@@ -36,6 +36,7 @@
 import org.joda.time.DateTime;
 
 /** Provides utility functions for working with {@link JdbcIO}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class JdbcUtil {
 
   /** Generates an insert statement based on {@link Schema.Field}. * */
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 c613d19..a00cd29 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
@@ -36,6 +36,7 @@
 
 /** Beam {@link org.apache.beam.sdk.schemas.Schema.LogicalType} implementations of JDBC types. */
 @Experimental(Kind.SCHEMAS)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class LogicalTypes {
   static final Schema.FieldType JDBC_BIT_TYPE =
       Schema.FieldType.logicalType(
diff --git a/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/SchemaUtil.java b/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/SchemaUtil.java
index a30bfdd..76c83d2 100644
--- a/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/SchemaUtil.java
+++ b/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/SchemaUtil.java
@@ -61,6 +61,7 @@
 
 /** Provides utility functions for working with Beam {@link Schema} types. */
 @Experimental(Kind.SCHEMAS)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class SchemaUtil {
   /**
    * Interface implemented by functions that extract values of different types from a JDBC
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 6c80002..1a52e3f 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
@@ -77,6 +77,7 @@
  * performance testing framework.
  */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class JdbcIOIT {
 
   private static final String NAMESPACE = JdbcIOIT.class.getName();
diff --git a/sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/JdbcIOTest.java b/sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/JdbcIOTest.java
index aaed524..24285f4 100644
--- a/sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/JdbcIOTest.java
+++ b/sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/JdbcIOTest.java
@@ -94,6 +94,7 @@
 
 /** Test on the JdbcIO. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class JdbcIOTest implements Serializable {
   private static final Logger LOG = LoggerFactory.getLogger(JdbcIOTest.class);
   private static final DataSourceConfiguration DATA_SOURCE_CONFIGURATION =
diff --git a/sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/JdbcTestHelper.java b/sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/JdbcTestHelper.java
index e929a5b..37ee5c9 100644
--- a/sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/JdbcTestHelper.java
+++ b/sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/JdbcTestHelper.java
@@ -26,6 +26,7 @@
  * Contains Test helper methods used by both Integration and Unit Tests in {@link
  * org.apache.beam.sdk.io.jdbc.JdbcIO}.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class JdbcTestHelper {
 
   static class CreateTestRowOfNameAndId implements JdbcIO.RowMapper<TestRow> {
diff --git a/sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/SchemaUtilTest.java b/sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/SchemaUtilTest.java
index 7ec9e7b..a1c7ddc 100644
--- a/sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/SchemaUtilTest.java
+++ b/sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/SchemaUtilTest.java
@@ -48,6 +48,7 @@
 
 /** Test SchemaUtils. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SchemaUtilTest {
   @Test
   public void testToBeamSchema() throws SQLException {
diff --git a/sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/JmsIO.java b/sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/JmsIO.java
index 815c392..2a25d4c 100644
--- a/sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/JmsIO.java
+++ b/sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/JmsIO.java
@@ -111,6 +111,7 @@
  * }</pre>
  */
 @Experimental(Kind.SOURCE_SINK)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class JmsIO {
 
   public static Read<JmsRecord> read() {
diff --git a/sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/JmsRecord.java b/sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/JmsRecord.java
index 11434c3..ba0c84d 100644
--- a/sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/JmsRecord.java
+++ b/sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/JmsRecord.java
@@ -28,6 +28,7 @@
  * JmsRecord contains message payload of the record as well as metadata (JMS headers and
  * properties).
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class JmsRecord implements Serializable {
 
   private final @Nullable String jmsMessageID;
diff --git a/sdks/java/io/jms/src/test/java/org/apache/beam/sdk/io/jms/JmsIOTest.java b/sdks/java/io/jms/src/test/java/org/apache/beam/sdk/io/jms/JmsIOTest.java
index a865882..5bb4d18 100644
--- a/sdks/java/io/jms/src/test/java/org/apache/beam/sdk/io/jms/JmsIOTest.java
+++ b/sdks/java/io/jms/src/test/java/org/apache/beam/sdk/io/jms/JmsIOTest.java
@@ -68,6 +68,7 @@
 
 /** Tests of {@link JmsIO}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class JmsIOTest {
 
   private static final String BROKER_URL = "vm://localhost";
diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/ConfluentSchemaRegistryDeserializerProvider.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/ConfluentSchemaRegistryDeserializerProvider.java
index ed1d99d..cd46151 100644
--- a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/ConfluentSchemaRegistryDeserializerProvider.java
+++ b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/ConfluentSchemaRegistryDeserializerProvider.java
@@ -45,6 +45,7 @@
  * {@link Deserializer}s and {@link Coder} given a subject.
  */
 @Experimental(Kind.SOURCE_SINK)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ConfluentSchemaRegistryDeserializerProvider<T> implements DeserializerProvider<T> {
   private final SerializableFunction<Void, SchemaRegistryClient> schemaRegistryClientProviderFn;
   private final String schemaRegistryUrl;
@@ -121,6 +122,7 @@
   }
 }
 
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class ConfluentSchemaRegistryDeserializer extends KafkaAvroDeserializer {
   Schema readerSchema;
 
diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaCheckpointMark.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaCheckpointMark.java
index e5f709b..a5ac8c2 100644
--- a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaCheckpointMark.java
+++ b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaCheckpointMark.java
@@ -32,6 +32,7 @@
  * the latest offset consumed so far.
  */
 @DefaultCoder(AvroCoder.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class KafkaCheckpointMark implements UnboundedSource.CheckpointMark {
 
   private List<PartitionMark> partitions;
diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaExactlyOnceSink.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaExactlyOnceSink.java
index 6ea6baf..721abed 100644
--- a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaExactlyOnceSink.java
+++ b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaExactlyOnceSink.java
@@ -88,6 +88,7 @@
  * Exactly-once sink transform for Kafka. See {@link KafkaIO} for user visible documentation and
  * example usage.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class KafkaExactlyOnceSink<K, V>
     extends PTransform<PCollection<ProducerRecord<K, V>>, PCollection<Void>> {
 
diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java
index d28f04d..1f6d5fa 100644
--- a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java
+++ b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java
@@ -392,6 +392,7 @@
  * incompatibility.
  */
 @Experimental(Kind.SOURCE_SINK)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class KafkaIO {
 
   /**
@@ -1231,35 +1232,28 @@
 
     abstract Map<String, Object> getConsumerConfig();
 
-    @Nullable
-    abstract Map<String, Object> getOffsetConsumerConfig();
+    abstract @Nullable Map<String, Object> getOffsetConsumerConfig();
 
-    @Nullable
-    abstract DeserializerProvider getKeyDeserializerProvider();
+    abstract @Nullable DeserializerProvider getKeyDeserializerProvider();
 
-    @Nullable
-    abstract DeserializerProvider getValueDeserializerProvider();
+    abstract @Nullable DeserializerProvider getValueDeserializerProvider();
 
-    @Nullable
-    abstract Coder<K> getKeyCoder();
+    abstract @Nullable Coder<K> getKeyCoder();
 
-    @Nullable
-    abstract Coder<V> getValueCoder();
+    abstract @Nullable Coder<V> getValueCoder();
 
     abstract SerializableFunction<Map<String, Object>, Consumer<byte[], byte[]>>
         getConsumerFactoryFn();
 
-    @Nullable
-    abstract SerializableFunction<KafkaRecord<K, V>, Instant> getExtractOutputTimestampFn();
+    abstract @Nullable SerializableFunction<KafkaRecord<K, V>, Instant>
+        getExtractOutputTimestampFn();
 
-    @Nullable
-    abstract SerializableFunction<Instant, WatermarkEstimator<Instant>>
+    abstract @Nullable SerializableFunction<Instant, WatermarkEstimator<Instant>>
         getCreateWatermarkEstimatorFn();
 
     abstract boolean isCommitOffsetEnabled();
 
-    @Nullable
-    abstract TimestampPolicyFactory<K, V> getTimestampPolicyFactory();
+    abstract @Nullable TimestampPolicyFactory<K, V> getTimestampPolicyFactory();
 
     abstract ReadSourceDescriptors.Builder<K, V> toBuilder();
 
diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaRecord.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaRecord.java
index 46835eb..d970bd9 100644
--- a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaRecord.java
+++ b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaRecord.java
@@ -27,6 +27,7 @@
  * KafkaRecord contains key and value of the record as well as metadata for the record (topic name,
  * partition id, and offset).
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class KafkaRecord<K, V> {
   // This is based on {@link ConsumerRecord} received from Kafka Consumer.
   // The primary difference is that this contains deserialized key and value, and runtime
diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaRecordCoder.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaRecordCoder.java
index c7f40ab..9b4b915 100644
--- a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaRecordCoder.java
+++ b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaRecordCoder.java
@@ -37,6 +37,7 @@
 import org.apache.kafka.common.header.Headers;
 
 /** {@link Coder} for {@link KafkaRecord}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class KafkaRecordCoder<K, V> extends StructuredCoder<KafkaRecord<K, V>> {
 
   private static final StringUtf8Coder stringCoder = StringUtf8Coder.of();
diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaSourceDescriptor.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaSourceDescriptor.java
index e509d91..95446e2 100644
--- a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaSourceDescriptor.java
+++ b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaSourceDescriptor.java
@@ -32,6 +32,7 @@
 /** Represents a Kafka source description. */
 @DefaultSchema(AutoValueSchema.class)
 @AutoValue
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public abstract class KafkaSourceDescriptor implements Serializable {
   @SchemaFieldName("topic")
   abstract String getTopic();
diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaUnboundedReader.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaUnboundedReader.java
index 72329b9..f7bc369 100644
--- a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaUnboundedReader.java
+++ b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaUnboundedReader.java
@@ -70,6 +70,7 @@
  * An unbounded reader to read from Kafka. Each reader consumes messages from one or more Kafka
  * partitions. See {@link KafkaIO} for user visible documentation and example usage.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class KafkaUnboundedReader<K, V> extends UnboundedReader<KafkaRecord<K, V>> {
 
   ///////////////////// Reader API ////////////////////////////////////////////////////////////
diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaUnboundedSource.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaUnboundedSource.java
index 7eb397f..af4af79 100644
--- a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaUnboundedSource.java
+++ b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaUnboundedSource.java
@@ -40,6 +40,7 @@
  * An {@link UnboundedSource} to read from Kafka, used by {@link Read} transform in KafkaIO. See
  * {@link KafkaIO} for user visible documentation and example usage.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class KafkaUnboundedSource<K, V> extends UnboundedSource<KafkaRecord<K, V>, KafkaCheckpointMark> {
 
   /**
diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaWriter.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaWriter.java
index 633a5b2..81806ee 100644
--- a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaWriter.java
+++ b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaWriter.java
@@ -37,6 +37,7 @@
  * A DoFn to write to Kafka, used in KafkaIO WriteRecords transform. See {@link KafkaIO} for user
  * visible documentation and example usage.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class KafkaWriter<K, V> extends DoFn<ProducerRecord<K, V>, Void> {
 
   @Setup
diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/LocalDeserializerProvider.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/LocalDeserializerProvider.java
index def2217..bb97ebe 100644
--- a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/LocalDeserializerProvider.java
+++ b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/LocalDeserializerProvider.java
@@ -34,6 +34,7 @@
  * CoderRegistry} configures a {@link Deserializer} instance and infers its corresponding {@link
  * Coder}.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class LocalDeserializerProvider<T> implements DeserializerProvider<T> {
   private Class<? extends Deserializer<T>> deserializer;
 
diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/ProducerRecordCoder.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/ProducerRecordCoder.java
index b2f8a69..8eed48a 100644
--- a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/ProducerRecordCoder.java
+++ b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/ProducerRecordCoder.java
@@ -38,6 +38,7 @@
 import org.apache.kafka.common.header.Headers;
 
 /** {@link Coder} for {@link ProducerRecord}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ProducerRecordCoder<K, V> extends StructuredCoder<ProducerRecord<K, V>> {
   private static final StringUtf8Coder stringCoder = StringUtf8Coder.of();
   private static final VarLongCoder longCoder = VarLongCoder.of();
diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/ProducerSpEL.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/ProducerSpEL.java
index 0f622ea..5d8ea2e 100644
--- a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/ProducerSpEL.java
+++ b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/ProducerSpEL.java
@@ -32,6 +32,7 @@
  * ProducerSpEL to handle newer versions Producer API. The API is updated in Kafka 0.11 to support
  * exactly-once semantics.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class ProducerSpEL {
 
   private static boolean supportsTransactions;
diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/ReadFromKafkaDoFn.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/ReadFromKafkaDoFn.java
index 6f26662..dc37f61 100644
--- a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/ReadFromKafkaDoFn.java
+++ b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/ReadFromKafkaDoFn.java
@@ -115,6 +115,7 @@
  * WatermarkEstimator}.
  */
 @UnboundedPerElement
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class ReadFromKafkaDoFn<K, V> extends DoFn<KafkaSourceDescriptor, KafkaRecord<K, V>> {
 
   ReadFromKafkaDoFn(ReadSourceDescriptors transform) {
diff --git a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/ConfluentSchemaRegistryDeserializerProviderTest.java b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/ConfluentSchemaRegistryDeserializerProviderTest.java
index dcbf2b3..853d946 100644
--- a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/ConfluentSchemaRegistryDeserializerProviderTest.java
+++ b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/ConfluentSchemaRegistryDeserializerProviderTest.java
@@ -38,6 +38,7 @@
 import org.junit.runners.JUnit4;
 
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ConfluentSchemaRegistryDeserializerProviderTest {
   @Test
   public void testGetCoder() {
diff --git a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/CustomTimestampPolicyWithLimitedDelayTest.java b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/CustomTimestampPolicyWithLimitedDelayTest.java
index affff6c..9c38e19 100644
--- a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/CustomTimestampPolicyWithLimitedDelayTest.java
+++ b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/CustomTimestampPolicyWithLimitedDelayTest.java
@@ -36,6 +36,7 @@
 
 /** Tests for {@link CustomTimestampPolicyWithLimitedDelay}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class CustomTimestampPolicyWithLimitedDelayTest {
 
   // Takes offsets of timestamps from now returns the results as offsets from 'now'.
diff --git a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOExternalTest.java b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOExternalTest.java
index 8b04ec7..255de41 100644
--- a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOExternalTest.java
+++ b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOExternalTest.java
@@ -61,6 +61,7 @@
 
 /** Tests for building {@link KafkaIO} externally via the ExpansionService. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class KafkaIOExternalTest {
   @Test
   public void testConstructKafkaRead() throws Exception {
diff --git a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOIT.java b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOIT.java
index feb62c1..64c117f 100644
--- a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOIT.java
+++ b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOIT.java
@@ -68,6 +68,7 @@
  * topic so that we could read them back after writing.
  */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class KafkaIOIT {
 
   private static final String READ_TIME_METRIC_NAME = "read_time";
diff --git a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOTest.java b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOTest.java
index 7c2b2ee..9bc973b 100644
--- a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOTest.java
+++ b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOTest.java
@@ -149,6 +149,7 @@
  * specific Kafka version.
  */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class KafkaIOTest {
 
   private static final Logger LOG = LoggerFactory.getLogger(KafkaIOTest.class);
diff --git a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOUtilsTest.java b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOUtilsTest.java
index c913fa5..efed8cd 100644
--- a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOUtilsTest.java
+++ b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/KafkaIOUtilsTest.java
@@ -29,6 +29,7 @@
 
 /** Tests of {@link KafkaIOUtils}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class KafkaIOUtilsTest {
 
   @Test
diff --git a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/ProducerRecordCoderTest.java b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/ProducerRecordCoderTest.java
index 5b489ea..6cfd6a5 100644
--- a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/ProducerRecordCoderTest.java
+++ b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/ProducerRecordCoderTest.java
@@ -42,6 +42,7 @@
 /** Tests for {@link ProducerRecordCoder}. */
 @RunWith(PowerMockRunner.class)
 @PrepareForTest(ConsumerSpEL.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ProducerRecordCoderTest {
   @Test
   public void testCoderIsSerializableWithWellKnownCoderType() {
diff --git a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisIO.java b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisIO.java
index 5b958d1..c7355cc 100644
--- a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisIO.java
+++ b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisIO.java
@@ -293,6 +293,7 @@
  * of configuration file</a>.
  */
 @Experimental(Kind.SOURCE_SINK)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public final class KinesisIO {
 
   private static final Logger LOG = LoggerFactory.getLogger(KinesisIO.class);
diff --git a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisReader.java b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisReader.java
index 887aad3..dfa4c76 100644
--- a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisReader.java
+++ b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisReader.java
@@ -32,6 +32,7 @@
  * Reads data from multiple kinesis shards in a single thread. It uses simple round robin algorithm
  * when fetching data from shards.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class KinesisReader extends UnboundedSource.UnboundedReader<KinesisRecord> {
 
   private static final Logger LOG = LoggerFactory.getLogger(KinesisReader.class);
diff --git a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisRecord.java b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisRecord.java
index f5b0ad8..6789374 100644
--- a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisRecord.java
+++ b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisRecord.java
@@ -28,6 +28,7 @@
 import org.joda.time.Instant;
 
 /** {@link UserRecord} enhanced with utility methods. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class KinesisRecord {
 
   private Instant readTime;
diff --git a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisSource.java b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisSource.java
index 98f7a88..442a8b6 100644
--- a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisSource.java
+++ b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisSource.java
@@ -30,6 +30,7 @@
 import org.slf4j.LoggerFactory;
 
 /** Represents source for single stream in Kinesis. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class KinesisSource extends UnboundedSource<KinesisRecord, KinesisReaderCheckpoint> {
 
   private static final Logger LOG = LoggerFactory.getLogger(KinesisSource.class);
diff --git a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisTransformRegistrar.java b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisTransformRegistrar.java
index fe7863e..79bf234 100644
--- a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisTransformRegistrar.java
+++ b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisTransformRegistrar.java
@@ -41,6 +41,7 @@
  */
 @Experimental(Kind.PORTABILITY)
 @AutoService(ExternalTransformRegistrar.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class KinesisTransformRegistrar implements ExternalTransformRegistrar {
   public static final String WRITE_URN = "beam:external:java:kinesis:write:v1";
   public static final String READ_DATA_URN = "beam:external:java:kinesis:read_data:v1";
@@ -129,16 +130,16 @@
           ReadDataBuilder.Configuration, PBegin, PCollection<byte[]>> {
 
     public static class Configuration extends CrossLanguageConfiguration {
-      @Nullable private Long maxNumRecords;
-      @Nullable private Duration maxReadTime;
-      @Nullable private InitialPositionInStream initialPositionInStream;
-      @Nullable private Instant initialTimestampInStream;
-      @Nullable private Integer requestRecordsLimit;
-      @Nullable private Duration upToDateThreshold;
-      @Nullable private Long maxCapacityPerShard;
-      @Nullable private WatermarkPolicy watermarkPolicy;
-      @Nullable private Duration watermarkIdleDurationThreshold;
-      @Nullable private Duration rateLimit;
+      private @Nullable Long maxNumRecords;
+      private @Nullable Duration maxReadTime;
+      private @Nullable InitialPositionInStream initialPositionInStream;
+      private @Nullable Instant initialTimestampInStream;
+      private @Nullable Integer requestRecordsLimit;
+      private @Nullable Duration upToDateThreshold;
+      private @Nullable Long maxCapacityPerShard;
+      private @Nullable WatermarkPolicy watermarkPolicy;
+      private @Nullable Duration watermarkIdleDurationThreshold;
+      private @Nullable Duration rateLimit;
 
       public void setMaxNumRecords(@Nullable Long maxNumRecords) {
         this.maxNumRecords = maxNumRecords;
diff --git a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/ShardCheckpoint.java b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/ShardCheckpoint.java
index 0f5acd7..4b2e9df 100644
--- a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/ShardCheckpoint.java
+++ b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/ShardCheckpoint.java
@@ -42,6 +42,7 @@
  *
  * This class is immutable.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class ShardCheckpoint implements Serializable {
 
   private final String streamName;
diff --git a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/ShardReadersPool.java b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/ShardReadersPool.java
index 07f078f..fa7f4aa 100644
--- a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/ShardReadersPool.java
+++ b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/ShardReadersPool.java
@@ -45,6 +45,7 @@
  * Internal shard iterators pool. It maintains the thread pool for reading Kinesis shards in
  * separate threads. Read records are stored in a blocking queue of limited capacity.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class ShardReadersPool {
 
   private static final Logger LOG = LoggerFactory.getLogger(ShardReadersPool.class);
diff --git a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/SimplifiedKinesisClient.java b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/SimplifiedKinesisClient.java
index c726382..8df2428 100644
--- a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/SimplifiedKinesisClient.java
+++ b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/SimplifiedKinesisClient.java
@@ -51,6 +51,7 @@
 import org.joda.time.Minutes;
 
 /** Wraps {@link AmazonKinesis} class providing much simpler interface and proper error handling. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class SimplifiedKinesisClient {
 
   private static final String KINESIS_NAMESPACE = "AWS/Kinesis";
diff --git a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/StartingPoint.java b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/StartingPoint.java
index 4c0b3ff..d8ffd69 100644
--- a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/StartingPoint.java
+++ b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/StartingPoint.java
@@ -31,6 +31,7 @@
  * expressed either as an {@link InitialPositionInStream} enum constant or a timestamp, in which
  * case the reader will start reading at the specified point in time.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class StartingPoint implements Serializable {
 
   private final InitialPositionInStream position;
diff --git a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/StartingPointShardsFinder.java b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/StartingPointShardsFinder.java
index 317a13a..8fce91f 100644
--- a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/StartingPointShardsFinder.java
+++ b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/StartingPointShardsFinder.java
@@ -32,6 +32,7 @@
  * This class is responsible for establishing the initial set of shards that existed at the given
  * starting point.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class StartingPointShardsFinder implements Serializable {
 
   private static final Logger LOG = LoggerFactory.getLogger(StartingPointShardsFinder.class);
diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/AmazonKinesisMock.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/AmazonKinesisMock.java
index 69b5c1a..5c1ab6f 100644
--- a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/AmazonKinesisMock.java
+++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/AmazonKinesisMock.java
@@ -104,6 +104,7 @@
 import org.mockito.Mockito;
 
 /** Mock implemenation of {@link AmazonKinesis} for testing. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class AmazonKinesisMock implements AmazonKinesis {
 
   static class TestData implements Serializable {
diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/DynamicCheckpointGeneratorTest.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/DynamicCheckpointGeneratorTest.java
index 352caa5..89e2e6b 100644
--- a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/DynamicCheckpointGeneratorTest.java
+++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/DynamicCheckpointGeneratorTest.java
@@ -31,6 +31,7 @@
 
 /** * */
 @RunWith(MockitoJUnitRunner.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DynamicCheckpointGeneratorTest {
 
   @Mock private SimplifiedKinesisClient kinesisClient;
diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisIOIT.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisIOIT.java
index 45c51cf..189ae80 100644
--- a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisIOIT.java
+++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisIOIT.java
@@ -57,6 +57,7 @@
  * when no options are provided an instance of localstack is used.
  */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class KinesisIOIT implements Serializable {
   private static final String LOCALSTACK_VERSION = "0.11.3";
 
diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisMockWriteTest.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisMockWriteTest.java
index f81065a..46f11ef 100644
--- a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisMockWriteTest.java
+++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisMockWriteTest.java
@@ -48,6 +48,7 @@
 
 /** Tests for {@link KinesisIO.Write}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class KinesisMockWriteTest {
   private static final String STREAM = "BEAM";
   private static final String PARTITION_KEY = "partitionKey";
diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisReaderCheckpointTest.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisReaderCheckpointTest.java
index 1653daf..6794c6e 100644
--- a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisReaderCheckpointTest.java
+++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisReaderCheckpointTest.java
@@ -31,6 +31,7 @@
 
 /** * */
 @RunWith(MockitoJUnitRunner.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class KinesisReaderCheckpointTest {
 
   @Mock private ShardCheckpoint a, b, c;
diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisReaderTest.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisReaderTest.java
index 64f0fe7..a5a6a61 100644
--- a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisReaderTest.java
+++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisReaderTest.java
@@ -38,6 +38,7 @@
 
 /** Tests {@link KinesisReader}. */
 @RunWith(MockitoJUnitRunner.Silent.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class KinesisReaderTest {
 
   @Mock private SimplifiedKinesisClient kinesis;
diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisServiceMock.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisServiceMock.java
index 0508b05..6e2888e 100644
--- a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisServiceMock.java
+++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisServiceMock.java
@@ -26,6 +26,7 @@
 import org.joda.time.DateTime;
 
 /** Simple mock implementation of Kinesis service for testing, singletone. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class KinesisServiceMock {
   private static KinesisServiceMock instance;
 
diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/RecordFilterTest.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/RecordFilterTest.java
index e17fa86..049b778 100644
--- a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/RecordFilterTest.java
+++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/RecordFilterTest.java
@@ -30,6 +30,7 @@
 
 /** * */
 @RunWith(MockitoJUnitRunner.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class RecordFilterTest {
 
   @Mock private ShardCheckpoint checkpoint;
diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/ShardCheckpointTest.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/ShardCheckpointTest.java
index 5abe605..c4b97d8 100644
--- a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/ShardCheckpointTest.java
+++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/ShardCheckpointTest.java
@@ -42,6 +42,7 @@
 
 /** */
 @RunWith(MockitoJUnitRunner.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ShardCheckpointTest {
 
   private static final String AT_SEQUENCE_SHARD_IT = "AT_SEQUENCE_SHARD_IT";
diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/ShardReadersPoolTest.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/ShardReadersPoolTest.java
index 3cd87bd..057ef26 100644
--- a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/ShardReadersPoolTest.java
+++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/ShardReadersPoolTest.java
@@ -47,6 +47,7 @@
 
 /** Tests {@link ShardReadersPool}. */
 @RunWith(MockitoJUnitRunner.Silent.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ShardReadersPoolTest {
 
   private static final int TIMEOUT_IN_MILLIS = (int) TimeUnit.SECONDS.toMillis(10);
diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/ShardRecordsIteratorTest.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/ShardRecordsIteratorTest.java
index 397dc98..db5a0fe 100644
--- a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/ShardRecordsIteratorTest.java
+++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/ShardRecordsIteratorTest.java
@@ -39,6 +39,7 @@
 
 /** Tests {@link ShardRecordsIterator}. */
 @RunWith(MockitoJUnitRunner.Silent.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ShardRecordsIteratorTest {
 
   private static final String INITIAL_ITERATOR = "INITIAL_ITERATOR";
diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/SimplifiedKinesisClientTest.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/SimplifiedKinesisClientTest.java
index 1c1d9e0..1bf4a9b 100644
--- a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/SimplifiedKinesisClientTest.java
+++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/SimplifiedKinesisClientTest.java
@@ -59,6 +59,7 @@
 
 /** * */
 @RunWith(MockitoJUnitRunner.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SimplifiedKinesisClientTest {
 
   private static final String STREAM = "stream";
diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/StartingPointShardsFinderTest.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/StartingPointShardsFinderTest.java
index a7eb9d8..4f55583 100644
--- a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/StartingPointShardsFinderTest.java
+++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/StartingPointShardsFinderTest.java
@@ -35,6 +35,7 @@
 
 /** Tests StartingPointShardsFinder. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class StartingPointShardsFinderTest {
 
   private static final String STREAM_NAME = "streamName";
diff --git a/sdks/java/io/kudu/src/main/java/org/apache/beam/sdk/io/kudu/KuduIO.java b/sdks/java/io/kudu/src/main/java/org/apache/beam/sdk/io/kudu/KuduIO.java
index 9b2054f..32480a3 100644
--- a/sdks/java/io/kudu/src/main/java/org/apache/beam/sdk/io/kudu/KuduIO.java
+++ b/sdks/java/io/kudu/src/main/java/org/apache/beam/sdk/io/kudu/KuduIO.java
@@ -121,6 +121,7 @@
  * {@code KuduIO} does not support authentication in this release.
  */
 @Experimental(Kind.SOURCE_SINK)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class KuduIO {
   private static final Logger LOG = LoggerFactory.getLogger(KuduIO.class);
 
diff --git a/sdks/java/io/kudu/src/main/java/org/apache/beam/sdk/io/kudu/KuduServiceImpl.java b/sdks/java/io/kudu/src/main/java/org/apache/beam/sdk/io/kudu/KuduServiceImpl.java
index 4c27a35..45eccdc 100644
--- a/sdks/java/io/kudu/src/main/java/org/apache/beam/sdk/io/kudu/KuduServiceImpl.java
+++ b/sdks/java/io/kudu/src/main/java/org/apache/beam/sdk/io/kudu/KuduServiceImpl.java
@@ -45,6 +45,7 @@
 import org.slf4j.LoggerFactory;
 
 /** An implementation of the {@link KuduService} that uses a Kudu instance. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class KuduServiceImpl<T> implements KuduService<T> {
   private static final Logger LOG = LoggerFactory.getLogger(KuduServiceImpl.class);
 
diff --git a/sdks/java/io/kudu/src/test/java/org/apache/beam/sdk/io/kudu/KuduIOIT.java b/sdks/java/io/kudu/src/test/java/org/apache/beam/sdk/io/kudu/KuduIOIT.java
index da1fd1d..7399d13 100644
--- a/sdks/java/io/kudu/src/test/java/org/apache/beam/sdk/io/kudu/KuduIOIT.java
+++ b/sdks/java/io/kudu/src/test/java/org/apache/beam/sdk/io/kudu/KuduIOIT.java
@@ -89,6 +89,7 @@
  * </pre>
  */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class KuduIOIT {
   private static final Logger LOG = LoggerFactory.getLogger(KuduIOIT.class);
 
diff --git a/sdks/java/io/kudu/src/test/java/org/apache/beam/sdk/io/kudu/KuduIOTest.java b/sdks/java/io/kudu/src/test/java/org/apache/beam/sdk/io/kudu/KuduIOTest.java
index 86d60e0..9278ac9 100644
--- a/sdks/java/io/kudu/src/test/java/org/apache/beam/sdk/io/kudu/KuduIOTest.java
+++ b/sdks/java/io/kudu/src/test/java/org/apache/beam/sdk/io/kudu/KuduIOTest.java
@@ -65,6 +65,7 @@
  * carried out in {@link KuduIOIT}.
  */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class KuduIOTest {
   private static final Logger LOG = LoggerFactory.getLogger(KuduIOTest.class);
 
diff --git a/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/FindQuery.java b/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/FindQuery.java
index 5474016..d068bda 100644
--- a/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/FindQuery.java
+++ b/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/FindQuery.java
@@ -38,6 +38,7 @@
 /** Builds a MongoDB FindQuery object. */
 @Experimental(Kind.SOURCE_SINK)
 @AutoValue
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public abstract class FindQuery
     implements SerializableFunction<MongoCollection<Document>, MongoCursor<Document>> {
 
diff --git a/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/MongoDbGridFSIO.java b/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/MongoDbGridFSIO.java
index 0da4211..d5ca2b9 100644
--- a/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/MongoDbGridFSIO.java
+++ b/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/MongoDbGridFSIO.java
@@ -114,6 +114,7 @@
  * separated with line feeds.
  */
 @Experimental(Kind.SOURCE_SINK)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class MongoDbGridFSIO {
 
   /** Callback for the parser to use to submit data. */
diff --git a/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/MongoDbIO.java b/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/MongoDbIO.java
index 72f957c..5049b68 100644
--- a/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/MongoDbIO.java
+++ b/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/MongoDbIO.java
@@ -109,6 +109,7 @@
  * }</pre>
  */
 @Experimental(Kind.SOURCE_SINK)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class MongoDbIO {
 
   private static final Logger LOG = LoggerFactory.getLogger(MongoDbIO.class);
diff --git a/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/SSLUtils.java b/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/SSLUtils.java
index 88a4718..98e9f9d 100644
--- a/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/SSLUtils.java
+++ b/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/SSLUtils.java
@@ -25,6 +25,7 @@
 import javax.net.ssl.X509TrustManager;
 
 /** Utility class for registration of ssl context, and to allow all certificate requests. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class SSLUtils {
 
   /** static class to allow all requests. */
diff --git a/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/MongoDBGridFSIOTest.java b/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/MongoDBGridFSIOTest.java
index 0ef57fb..b3adfaa 100644
--- a/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/MongoDBGridFSIOTest.java
+++ b/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/MongoDBGridFSIOTest.java
@@ -80,6 +80,7 @@
 
 /** Test on the MongoDbGridFSIO. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class MongoDBGridFSIOTest {
   private static final Logger LOG = LoggerFactory.getLogger(MongoDBGridFSIOTest.class);
 
diff --git a/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/MongoDBIOIT.java b/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/MongoDBIOIT.java
index 086d9a6..7b4c79d 100644
--- a/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/MongoDBIOIT.java
+++ b/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/MongoDBIOIT.java
@@ -78,6 +78,7 @@
  * performance testing framework.
  */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class MongoDBIOIT {
 
   private static final String NAMESPACE = MongoDBIOIT.class.getName();
diff --git a/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/MongoDbIOTest.java b/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/MongoDbIOTest.java
index 89af792..83d67f3 100644
--- a/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/MongoDbIOTest.java
+++ b/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/MongoDbIOTest.java
@@ -63,6 +63,7 @@
 
 /** Test on the MongoDbIO. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class MongoDbIOTest {
   private static final Logger LOG = LoggerFactory.getLogger(MongoDbIOTest.class);
 
diff --git a/sdks/java/io/mqtt/src/main/java/org/apache/beam/sdk/io/mqtt/MqttIO.java b/sdks/java/io/mqtt/src/main/java/org/apache/beam/sdk/io/mqtt/MqttIO.java
index 19d2fbf..6fa67f1 100644
--- a/sdks/java/io/mqtt/src/main/java/org/apache/beam/sdk/io/mqtt/MqttIO.java
+++ b/sdks/java/io/mqtt/src/main/java/org/apache/beam/sdk/io/mqtt/MqttIO.java
@@ -101,6 +101,7 @@
  * }</pre>
  */
 @Experimental(Kind.SOURCE_SINK)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class MqttIO {
 
   private static final Logger LOG = LoggerFactory.getLogger(MqttIO.class);
diff --git a/sdks/java/io/mqtt/src/test/java/org/apache/beam/sdk/io/mqtt/MqttIOTest.java b/sdks/java/io/mqtt/src/test/java/org/apache/beam/sdk/io/mqtt/MqttIOTest.java
index 2740493..b349221 100644
--- a/sdks/java/io/mqtt/src/test/java/org/apache/beam/sdk/io/mqtt/MqttIOTest.java
+++ b/sdks/java/io/mqtt/src/test/java/org/apache/beam/sdk/io/mqtt/MqttIOTest.java
@@ -56,6 +56,7 @@
 
 /** Tests of {@link MqttIO}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class MqttIOTest {
 
   private static final Logger LOG = LoggerFactory.getLogger(MqttIOTest.class);
diff --git a/sdks/java/io/parquet/src/main/java/org/apache/beam/sdk/io/parquet/ParquetIO.java b/sdks/java/io/parquet/src/main/java/org/apache/beam/sdk/io/parquet/ParquetIO.java
index ce90f5e..51ee5e6 100644
--- a/sdks/java/io/parquet/src/main/java/org/apache/beam/sdk/io/parquet/ParquetIO.java
+++ b/sdks/java/io/parquet/src/main/java/org/apache/beam/sdk/io/parquet/ParquetIO.java
@@ -182,6 +182,7 @@
  *     documentation</a>
  */
 @Experimental(Kind.SOURCE_SINK)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ParquetIO {
 
   /**
diff --git a/sdks/java/io/parquet/src/main/java/org/apache/beam/sdk/io/parquet/ParquetSchemaIOProvider.java b/sdks/java/io/parquet/src/main/java/org/apache/beam/sdk/io/parquet/ParquetSchemaIOProvider.java
index 4b90c53..f5bf2bd 100644
--- a/sdks/java/io/parquet/src/main/java/org/apache/beam/sdk/io/parquet/ParquetSchemaIOProvider.java
+++ b/sdks/java/io/parquet/src/main/java/org/apache/beam/sdk/io/parquet/ParquetSchemaIOProvider.java
@@ -37,6 +37,7 @@
  */
 @Internal
 @AutoService(SchemaIOProvider.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ParquetSchemaIOProvider implements SchemaIOProvider {
   /** Returns an id that uniquely represents this IO. */
   @Override
diff --git a/sdks/java/io/parquet/src/test/java/org/apache/beam/sdk/io/parquet/ParquetIOTest.java b/sdks/java/io/parquet/src/test/java/org/apache/beam/sdk/io/parquet/ParquetIOTest.java
index 04843fb..48b4e8d 100644
--- a/sdks/java/io/parquet/src/test/java/org/apache/beam/sdk/io/parquet/ParquetIOTest.java
+++ b/sdks/java/io/parquet/src/test/java/org/apache/beam/sdk/io/parquet/ParquetIOTest.java
@@ -50,6 +50,7 @@
 
 /** Test on the {@link ParquetIO}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ParquetIOTest implements Serializable {
   @Rule public transient TestPipeline mainPipeline = TestPipeline.create();
 
diff --git a/sdks/java/io/rabbitmq/src/main/java/org/apache/beam/sdk/io/rabbitmq/RabbitMqIO.java b/sdks/java/io/rabbitmq/src/main/java/org/apache/beam/sdk/io/rabbitmq/RabbitMqIO.java
index 8f080ce..8d22706 100644
--- a/sdks/java/io/rabbitmq/src/main/java/org/apache/beam/sdk/io/rabbitmq/RabbitMqIO.java
+++ b/sdks/java/io/rabbitmq/src/main/java/org/apache/beam/sdk/io/rabbitmq/RabbitMqIO.java
@@ -119,6 +119,7 @@
  * }</pre>
  */
 @Experimental(Kind.SOURCE_SINK)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class RabbitMqIO {
   public static Read read() {
     return new AutoValue_RabbitMqIO_Read.Builder()
diff --git a/sdks/java/io/rabbitmq/src/main/java/org/apache/beam/sdk/io/rabbitmq/RabbitMqMessage.java b/sdks/java/io/rabbitmq/src/main/java/org/apache/beam/sdk/io/rabbitmq/RabbitMqMessage.java
index 1c84fc5..19fff64 100644
--- a/sdks/java/io/rabbitmq/src/main/java/org/apache/beam/sdk/io/rabbitmq/RabbitMqMessage.java
+++ b/sdks/java/io/rabbitmq/src/main/java/org/apache/beam/sdk/io/rabbitmq/RabbitMqMessage.java
@@ -35,6 +35,7 @@
  * It contains the message payload, and additional metadata like routing key or attributes. The main
  * reason of this class is that AMQP.BasicProperties doesn't provide a serializable public API.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class RabbitMqMessage implements Serializable {
 
   /**
diff --git a/sdks/java/io/rabbitmq/src/test/java/org/apache/beam/sdk/io/rabbitmq/ExchangeTestPlan.java b/sdks/java/io/rabbitmq/src/test/java/org/apache/beam/sdk/io/rabbitmq/ExchangeTestPlan.java
index 669cc3c..a4c4406 100644
--- a/sdks/java/io/rabbitmq/src/test/java/org/apache/beam/sdk/io/rabbitmq/ExchangeTestPlan.java
+++ b/sdks/java/io/rabbitmq/src/test/java/org/apache/beam/sdk/io/rabbitmq/ExchangeTestPlan.java
@@ -32,6 +32,7 @@
  * that could be used, this class has been implemented to help represent the parameters of a test
  * oriented around reading messages published to an exchange.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class ExchangeTestPlan {
   static final String DEFAULT_ROUTING_KEY = "someRoutingKey";
 
diff --git a/sdks/java/io/rabbitmq/src/test/java/org/apache/beam/sdk/io/rabbitmq/RabbitMqIOTest.java b/sdks/java/io/rabbitmq/src/test/java/org/apache/beam/sdk/io/rabbitmq/RabbitMqIOTest.java
index c7b3c0e..e240be1 100644
--- a/sdks/java/io/rabbitmq/src/test/java/org/apache/beam/sdk/io/rabbitmq/RabbitMqIOTest.java
+++ b/sdks/java/io/rabbitmq/src/test/java/org/apache/beam/sdk/io/rabbitmq/RabbitMqIOTest.java
@@ -68,6 +68,7 @@
 
 /** Test of {@link RabbitMqIO}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class RabbitMqIOTest implements Serializable {
   private static final Logger LOG = LoggerFactory.getLogger(RabbitMqIOTest.class);
 
diff --git a/sdks/java/io/redis/build.gradle b/sdks/java/io/redis/build.gradle
index a3e2aa4..50c9ffd 100644
--- a/sdks/java/io/redis/build.gradle
+++ b/sdks/java/io/redis/build.gradle
@@ -25,7 +25,7 @@
 dependencies {
   compile library.java.vendored_guava_26_0_jre
   compile project(path: ":sdks:java:core", configuration: "shadow")
-  compile "redis.clients:jedis:3.0.1"
+  compile "redis.clients:jedis:3.3.0"
   testCompile project(path: ":sdks:java:io:common", configuration: "testRuntime")
   testCompile library.java.junit
   testCompile library.java.hamcrest_core
diff --git a/sdks/java/io/redis/src/main/java/org/apache/beam/sdk/io/redis/RedisConnectionConfiguration.java b/sdks/java/io/redis/src/main/java/org/apache/beam/sdk/io/redis/RedisConnectionConfiguration.java
index d725b4a..b356ca3 100644
--- a/sdks/java/io/redis/src/main/java/org/apache/beam/sdk/io/redis/RedisConnectionConfiguration.java
+++ b/sdks/java/io/redis/src/main/java/org/apache/beam/sdk/io/redis/RedisConnectionConfiguration.java
@@ -32,6 +32,7 @@
  * server or cluster.
  */
 @AutoValue
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public abstract class RedisConnectionConfiguration implements Serializable {
 
   abstract ValueProvider<String> host();
diff --git a/sdks/java/io/redis/src/main/java/org/apache/beam/sdk/io/redis/RedisIO.java b/sdks/java/io/redis/src/main/java/org/apache/beam/sdk/io/redis/RedisIO.java
index 736f267..9bad9f3 100644
--- a/sdks/java/io/redis/src/main/java/org/apache/beam/sdk/io/redis/RedisIO.java
+++ b/sdks/java/io/redis/src/main/java/org/apache/beam/sdk/io/redis/RedisIO.java
@@ -107,6 +107,7 @@
  * }</pre>
  */
 @Experimental(Kind.SOURCE_SINK)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class RedisIO {
 
   /** Read data from a Redis server. */
diff --git a/sdks/java/io/redis/src/test/java/org/apache/beam/sdk/io/redis/RedisIOTest.java b/sdks/java/io/redis/src/test/java/org/apache/beam/sdk/io/redis/RedisIOTest.java
index badf039..9eafe08 100644
--- a/sdks/java/io/redis/src/test/java/org/apache/beam/sdk/io/redis/RedisIOTest.java
+++ b/sdks/java/io/redis/src/test/java/org/apache/beam/sdk/io/redis/RedisIOTest.java
@@ -43,6 +43,7 @@
 
 /** Test on the Redis IO. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class RedisIOTest {
 
   private static final String REDIS_HOST = "localhost";
diff --git a/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/SnowflakeIO.java b/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/SnowflakeIO.java
index 250ccec..62688ab 100644
--- a/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/SnowflakeIO.java
+++ b/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/SnowflakeIO.java
@@ -175,6 +175,7 @@
  * specified stagingBucketName in directory named 'data' and then into Snowflake.
  */
 @Experimental
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SnowflakeIO {
   private static final Logger LOG = LoggerFactory.getLogger(SnowflakeIO.class);
 
diff --git a/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/crosslanguage/CrossLanguageConfiguration.java b/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/crosslanguage/CrossLanguageConfiguration.java
index 24c298b..227a3e6 100644
--- a/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/crosslanguage/CrossLanguageConfiguration.java
+++ b/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/crosslanguage/CrossLanguageConfiguration.java
@@ -20,6 +20,7 @@
 import org.apache.beam.sdk.io.snowflake.SnowflakeIO;
 
 /** Parameters abstract class to expose the transforms to an external SDK. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public abstract class CrossLanguageConfiguration {
   private String serverName;
   private String username;
diff --git a/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/crosslanguage/WriteBuilder.java b/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/crosslanguage/WriteBuilder.java
index 3f2374a..da32691 100644
--- a/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/crosslanguage/WriteBuilder.java
+++ b/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/crosslanguage/WriteBuilder.java
@@ -32,6 +32,7 @@
 import org.apache.beam.sdk.values.PDone;
 
 @Experimental(Kind.PORTABILITY)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class WriteBuilder
     implements ExternalTransformBuilder<WriteBuilder.Configuration, PCollection<byte[]>, PDone> {
 
diff --git a/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/data/SnowflakeColumn.java b/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/data/SnowflakeColumn.java
index 25fdda8..d4ec745 100644
--- a/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/data/SnowflakeColumn.java
+++ b/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/data/SnowflakeColumn.java
@@ -20,6 +20,7 @@
 import java.io.Serializable;
 
 /** POJO describing single Column within Snowflake Table. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SnowflakeColumn implements Serializable {
   private SnowflakeDataType dataType;
   private String name;
diff --git a/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/data/SnowflakeTableSchema.java b/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/data/SnowflakeTableSchema.java
index 66b5149..14953f8 100644
--- a/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/data/SnowflakeTableSchema.java
+++ b/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/data/SnowflakeTableSchema.java
@@ -30,6 +30,7 @@
  * @link org.apache.beam.sdk.io.snowflake.SnowflakeIO.Write.CreateDisposition#CREATE_IF_NEEDED}
  *     disposition is used.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SnowflakeTableSchema implements Serializable {
 
   @JsonProperty("schema")
diff --git a/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/data/text/SnowflakeBinary.java b/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/data/text/SnowflakeBinary.java
index 540f526..fb4f494 100644
--- a/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/data/text/SnowflakeBinary.java
+++ b/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/data/text/SnowflakeBinary.java
@@ -20,6 +20,7 @@
 import java.io.Serializable;
 import org.apache.beam.sdk.io.snowflake.data.SnowflakeDataType;
 
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SnowflakeBinary implements SnowflakeDataType, Serializable {
 
   public static final Long MAX_SIZE = 8388608L;
diff --git a/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/data/text/SnowflakeVarchar.java b/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/data/text/SnowflakeVarchar.java
index a17bd79..72e1575 100644
--- a/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/data/text/SnowflakeVarchar.java
+++ b/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/data/text/SnowflakeVarchar.java
@@ -20,6 +20,7 @@
 import java.io.Serializable;
 import org.apache.beam.sdk.io.snowflake.data.SnowflakeDataType;
 
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SnowflakeVarchar implements SnowflakeDataType, Serializable {
   public static final Long MAX_LENGTH = 16777216L;
   private Long length;
diff --git a/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/services/SnowflakeBatchServiceConfig.java b/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/services/SnowflakeBatchServiceConfig.java
index 28da85c..2a572b2 100644
--- a/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/services/SnowflakeBatchServiceConfig.java
+++ b/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/services/SnowflakeBatchServiceConfig.java
@@ -25,6 +25,7 @@
 import org.apache.beam.sdk.transforms.SerializableFunction;
 
 /** Class for preparing configuration for batch write and read. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SnowflakeBatchServiceConfig extends ServiceConfig {
   private final SerializableFunction<Void, DataSource> dataSourceProviderFn;
 
diff --git a/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/services/SnowflakeBatchServiceImpl.java b/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/services/SnowflakeBatchServiceImpl.java
index 2b73bd5..f542630 100644
--- a/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/services/SnowflakeBatchServiceImpl.java
+++ b/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/services/SnowflakeBatchServiceImpl.java
@@ -37,6 +37,7 @@
 import org.slf4j.LoggerFactory;
 
 /** Implemenation of {@link SnowflakeService} used in production. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SnowflakeBatchServiceImpl implements SnowflakeService<SnowflakeBatchServiceConfig> {
   private static final Logger LOG = LoggerFactory.getLogger(SnowflakeBatchServiceImpl.class);
   private static final String SNOWFLAKE_GCS_PREFIX = "gcs://";
diff --git a/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/services/SnowflakeStreamingServiceImpl.java b/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/services/SnowflakeStreamingServiceImpl.java
index 8d555ed..b31db68 100644
--- a/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/services/SnowflakeStreamingServiceImpl.java
+++ b/sdks/java/io/snowflake/src/main/java/org/apache/beam/sdk/io/snowflake/services/SnowflakeStreamingServiceImpl.java
@@ -28,6 +28,7 @@
 import org.slf4j.LoggerFactory;
 
 /** Implemenation of {@link SnowflakeService} used in production. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SnowflakeStreamingServiceImpl
     implements SnowflakeService<SnowflakeStreamingServiceConfig> {
   private static final Logger LOG = LoggerFactory.getLogger(SnowflakeStreamingServiceImpl.class);
diff --git a/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/BatchSnowflakeIOIT.java b/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/BatchSnowflakeIOIT.java
index 8503338..21ec278 100644
--- a/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/BatchSnowflakeIOIT.java
+++ b/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/BatchSnowflakeIOIT.java
@@ -78,6 +78,7 @@
  * -DintegrationTestRunner=dataflow
  * </pre>
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BatchSnowflakeIOIT {
   private static final String tableName = "IOIT";
 
diff --git a/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/FakeSnowflakeBasicDataSource.java b/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/FakeSnowflakeBasicDataSource.java
index 5fc694f..062d1a7 100644
--- a/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/FakeSnowflakeBasicDataSource.java
+++ b/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/FakeSnowflakeBasicDataSource.java
@@ -41,6 +41,7 @@
 /**
  * Fake implementation of {@link net.snowflake.client.jdbc.SnowflakeBasicDataSource} used in tests.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class FakeSnowflakeBasicDataSource extends SnowflakeBasicDataSource implements Serializable {
   @Override
   public FakeConnection getConnection() throws SQLException {
diff --git a/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/FakeSnowflakeBatchServiceImpl.java b/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/FakeSnowflakeBatchServiceImpl.java
index 90ee4b9..1f71e12 100644
--- a/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/FakeSnowflakeBatchServiceImpl.java
+++ b/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/FakeSnowflakeBatchServiceImpl.java
@@ -32,6 +32,7 @@
 import org.apache.beam.sdk.io.snowflake.services.SnowflakeService;
 
 /** Fake implementation of {@link SnowflakeService} used in tests. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class FakeSnowflakeBatchServiceImpl
     implements SnowflakeService<SnowflakeBatchServiceConfig> {
 
diff --git a/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/FakeSnowflakeDatabase.java b/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/FakeSnowflakeDatabase.java
index 32e4944..932e5c7 100644
--- a/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/FakeSnowflakeDatabase.java
+++ b/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/FakeSnowflakeDatabase.java
@@ -26,6 +26,7 @@
 import net.snowflake.client.jdbc.SnowflakeSQLException;
 
 /** Fake implementation of Snowflake warehouse used in test code. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class FakeSnowflakeDatabase implements Serializable {
   private static Map<String, List<String>> tables = new HashMap<>();
 
diff --git a/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/FakeSnowflakeStreamingServiceImpl.java b/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/FakeSnowflakeStreamingServiceImpl.java
index 362eb5e..5b0d210 100644
--- a/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/FakeSnowflakeStreamingServiceImpl.java
+++ b/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/FakeSnowflakeStreamingServiceImpl.java
@@ -29,6 +29,7 @@
 import org.apache.beam.sdk.io.snowflake.services.SnowflakeStreamingServiceConfig;
 
 /** Fake implementation of {@link SnowflakeService} used in tests. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class FakeSnowflakeStreamingServiceImpl
     implements SnowflakeService<SnowflakeStreamingServiceConfig> {
   private FakeSnowflakeIngestManager snowflakeIngestManager;
diff --git a/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/StreamingSnowflakeIOIT.java b/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/StreamingSnowflakeIOIT.java
index e2a4088..4cc2497 100644
--- a/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/StreamingSnowflakeIOIT.java
+++ b/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/StreamingSnowflakeIOIT.java
@@ -70,6 +70,7 @@
  * -DintegrationTestRunner=direct
  * </pre>
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class StreamingSnowflakeIOIT {
   private static final int TIMEOUT = 900000;
   private static final int INTERVAL = 30000;
diff --git a/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/TestUtils.java b/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/TestUtils.java
index 24f744d..02ba337 100644
--- a/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/TestUtils.java
+++ b/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/TestUtils.java
@@ -48,6 +48,7 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class TestUtils {
 
   private static final Logger LOG = LoggerFactory.getLogger(TestUtils.class);
diff --git a/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/unit/DataSourceConfigurationTest.java b/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/unit/DataSourceConfigurationTest.java
index 6bac967..02d496f 100644
--- a/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/unit/DataSourceConfigurationTest.java
+++ b/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/unit/DataSourceConfigurationTest.java
@@ -29,6 +29,7 @@
 import org.junit.Test;
 
 /** Unit tests for {@link org.apache.beam.sdk.io.snowflake.SnowflakeIO.DataSourceConfiguration}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class DataSourceConfigurationTest {
 
   private SnowflakeIO.DataSourceConfiguration configuration;
diff --git a/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/unit/read/SnowflakeIOReadTest.java b/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/unit/read/SnowflakeIOReadTest.java
index fdbd49f..92557ab 100644
--- a/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/unit/read/SnowflakeIOReadTest.java
+++ b/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/unit/read/SnowflakeIOReadTest.java
@@ -45,6 +45,7 @@
 import org.junit.runners.JUnit4;
 
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SnowflakeIOReadTest implements Serializable {
   public static final String FAKE_TABLE = "FAKE_TABLE";
   public static final String FAKE_QUERY = "SELECT * FROM FAKE_TABLE";
diff --git a/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/unit/write/CreateDispositionTest.java b/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/unit/write/CreateDispositionTest.java
index d9616d2..edb9ddd 100644
--- a/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/unit/write/CreateDispositionTest.java
+++ b/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/unit/write/CreateDispositionTest.java
@@ -48,6 +48,7 @@
 import org.junit.runners.JUnit4;
 
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class CreateDispositionTest {
   private static final String FAKE_TABLE = "FAKE_TABLE";
   private static final String BUCKET_NAME = "bucket/";
diff --git a/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/unit/write/QueryDispositionLocationTest.java b/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/unit/write/QueryDispositionLocationTest.java
index 423dcba..5af2940 100644
--- a/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/unit/write/QueryDispositionLocationTest.java
+++ b/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/unit/write/QueryDispositionLocationTest.java
@@ -45,6 +45,7 @@
 import org.junit.runners.JUnit4;
 
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class QueryDispositionLocationTest {
   private static final String FAKE_TABLE = "FAKE_TABLE";
   private static final String BUCKET_NAME = "BUCKET/";
diff --git a/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/unit/write/SchemaDispositionTest.java b/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/unit/write/SchemaDispositionTest.java
index 6ab3f20..5b0c9fe 100644
--- a/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/unit/write/SchemaDispositionTest.java
+++ b/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/unit/write/SchemaDispositionTest.java
@@ -54,6 +54,7 @@
 import org.junit.runners.JUnit4;
 
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SchemaDispositionTest {
   private static final String FAKE_TABLE = "FAKE_TABLE";
   private static final String BUCKET_NAME = "BUCKET/";
diff --git a/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/unit/write/SnowflakeIOWriteTest.java b/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/unit/write/SnowflakeIOWriteTest.java
index 70bb7e0..1224ddb 100644
--- a/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/unit/write/SnowflakeIOWriteTest.java
+++ b/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/unit/write/SnowflakeIOWriteTest.java
@@ -47,6 +47,7 @@
 import org.junit.runners.JUnit4;
 
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SnowflakeIOWriteTest {
   private static final String FAKE_TABLE = "FAKE_TABLE";
   private static final String BUCKET_NAME = "BUCKET/";
diff --git a/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/unit/write/StreamingWriteTest.java b/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/unit/write/StreamingWriteTest.java
index e376a3a..64aab07 100644
--- a/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/unit/write/StreamingWriteTest.java
+++ b/sdks/java/io/snowflake/src/test/java/org/apache/beam/sdk/io/snowflake/test/unit/write/StreamingWriteTest.java
@@ -60,6 +60,7 @@
 import org.slf4j.LoggerFactory;
 
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class StreamingWriteTest {
   private static final Logger LOG = LoggerFactory.getLogger(StreamingWriteTest.class);
   private static final String FAKE_TABLE = "TEST_TABLE";
diff --git a/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/AuthorizedSolrClient.java b/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/AuthorizedSolrClient.java
index 70056f3..cc0adfa 100644
--- a/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/AuthorizedSolrClient.java
+++ b/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/AuthorizedSolrClient.java
@@ -40,6 +40,7 @@
  *
  * @param <ClientT> type of SolrClient
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class AuthorizedSolrClient<ClientT extends SolrClient> implements Closeable {
   private final ClientT solrClient;
   private final String username;
diff --git a/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/SolrIO.java b/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/SolrIO.java
index f738509..c7c9e41 100644
--- a/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/SolrIO.java
+++ b/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/SolrIO.java
@@ -111,6 +111,7 @@
  * default this is disabled and only one attempt will be made.
  */
 @Experimental(Kind.SOURCE_SINK)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SolrIO {
 
   private static final Logger LOG = LoggerFactory.getLogger(SolrIO.class);
diff --git a/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/JavaBinCodecCoderTest.java b/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/JavaBinCodecCoderTest.java
index 2605c4a..81d05d0 100644
--- a/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/JavaBinCodecCoderTest.java
+++ b/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/JavaBinCodecCoderTest.java
@@ -36,6 +36,7 @@
 
 /** Test case for {@link JavaBinCodecCoder}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class JavaBinCodecCoderTest {
   private static final Coder<SolrDocument> TEST_CODER = JavaBinCodecCoder.of(SolrDocument.class);
   private static final List<SolrDocument> TEST_VALUES = new ArrayList<>();
diff --git a/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOTest.java b/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOTest.java
index 6f666b8..8bc4f66 100644
--- a/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOTest.java
+++ b/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOTest.java
@@ -66,6 +66,7 @@
 @ThreadLeakScope(value = ThreadLeakScope.Scope.NONE)
 @SolrTestCaseJ4.SuppressSSL
 @RunWith(RandomizedRunner.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SolrIOTest extends SolrCloudTestCase {
   private static final Logger LOG = LoggerFactory.getLogger(SolrIOTest.class);
 
diff --git a/sdks/java/io/splunk/src/main/java/org/apache/beam/sdk/io/splunk/HttpEventPublisher.java b/sdks/java/io/splunk/src/main/java/org/apache/beam/sdk/io/splunk/HttpEventPublisher.java
index f9d2c1f..5cc8895 100644
--- a/sdks/java/io/splunk/src/main/java/org/apache/beam/sdk/io/splunk/HttpEventPublisher.java
+++ b/sdks/java/io/splunk/src/main/java/org/apache/beam/sdk/io/splunk/HttpEventPublisher.java
@@ -61,6 +61,7 @@
  * (HEC) endpoint.
  */
 @AutoValue
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 abstract class HttpEventPublisher {
 
   private static final Logger LOG = LoggerFactory.getLogger(HttpEventPublisher.class);
diff --git a/sdks/java/io/splunk/src/main/java/org/apache/beam/sdk/io/splunk/SplunkEvent.java b/sdks/java/io/splunk/src/main/java/org/apache/beam/sdk/io/splunk/SplunkEvent.java
index 6781c55..90b2199 100644
--- a/sdks/java/io/splunk/src/main/java/org/apache/beam/sdk/io/splunk/SplunkEvent.java
+++ b/sdks/java/io/splunk/src/main/java/org/apache/beam/sdk/io/splunk/SplunkEvent.java
@@ -54,9 +54,8 @@
 
   public abstract @Nullable String source();
 
-  @Nullable
   @SerializedName("sourcetype")
-  public abstract String sourceType();
+  public abstract @Nullable String sourceType();
 
   public abstract @Nullable String index();
 
diff --git a/sdks/java/io/splunk/src/main/java/org/apache/beam/sdk/io/splunk/SplunkEventWriter.java b/sdks/java/io/splunk/src/main/java/org/apache/beam/sdk/io/splunk/SplunkEventWriter.java
index 812a982..ca321f1 100644
--- a/sdks/java/io/splunk/src/main/java/org/apache/beam/sdk/io/splunk/SplunkEventWriter.java
+++ b/sdks/java/io/splunk/src/main/java/org/apache/beam/sdk/io/splunk/SplunkEventWriter.java
@@ -54,6 +54,7 @@
 
 /** A {@link DoFn} to write {@link SplunkEvent}s to Splunk's HEC endpoint. */
 @AutoValue
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 abstract class SplunkEventWriter extends DoFn<KV<Integer, SplunkEvent>, SplunkWriteError> {
 
   private static final Integer DEFAULT_BATCH_COUNT = 1;
diff --git a/sdks/java/io/splunk/src/main/java/org/apache/beam/sdk/io/splunk/SplunkIO.java b/sdks/java/io/splunk/src/main/java/org/apache/beam/sdk/io/splunk/SplunkIO.java
index 0e11661..4ab7244 100644
--- a/sdks/java/io/splunk/src/main/java/org/apache/beam/sdk/io/splunk/SplunkIO.java
+++ b/sdks/java/io/splunk/src/main/java/org/apache/beam/sdk/io/splunk/SplunkIO.java
@@ -85,6 +85,7 @@
  * }</pre>
  */
 @Experimental(Kind.SOURCE_SINK)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SplunkIO {
 
   /**
diff --git a/sdks/java/io/splunk/src/test/java/org/apache/beam/sdk/io/splunk/SplunkEventWriterTest.java b/sdks/java/io/splunk/src/test/java/org/apache/beam/sdk/io/splunk/SplunkEventWriterTest.java
index e31ec0c..cd67a8a 100644
--- a/sdks/java/io/splunk/src/test/java/org/apache/beam/sdk/io/splunk/SplunkEventWriterTest.java
+++ b/sdks/java/io/splunk/src/test/java/org/apache/beam/sdk/io/splunk/SplunkEventWriterTest.java
@@ -45,6 +45,7 @@
 import org.mockserver.verify.VerificationTimes;
 
 /** Unit tests for {@link SplunkEventWriter} class. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SplunkEventWriterTest {
 
   private static final String EXPECTED_PATH = "/" + HttpEventPublisher.HEC_URL_PATH;
diff --git a/sdks/java/io/splunk/src/test/java/org/apache/beam/sdk/io/splunk/SplunkIOTest.java b/sdks/java/io/splunk/src/test/java/org/apache/beam/sdk/io/splunk/SplunkIOTest.java
index f34047f..3a7a91d 100644
--- a/sdks/java/io/splunk/src/test/java/org/apache/beam/sdk/io/splunk/SplunkIOTest.java
+++ b/sdks/java/io/splunk/src/test/java/org/apache/beam/sdk/io/splunk/SplunkIOTest.java
@@ -37,6 +37,7 @@
 import org.mockserver.verify.VerificationTimes;
 
 /** Unit tests for {@link SplunkIO} class. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SplunkIOTest {
 
   private static final String EXPECTED_PATH = "/" + HttpEventPublisher.HEC_URL_PATH;
diff --git a/sdks/java/io/synthetic/src/main/java/org/apache/beam/sdk/io/synthetic/SyntheticBoundedSource.java b/sdks/java/io/synthetic/src/main/java/org/apache/beam/sdk/io/synthetic/SyntheticBoundedSource.java
index 85562cd..ac58663 100644
--- a/sdks/java/io/synthetic/src/main/java/org/apache/beam/sdk/io/synthetic/SyntheticBoundedSource.java
+++ b/sdks/java/io/synthetic/src/main/java/org/apache/beam/sdk/io/synthetic/SyntheticBoundedSource.java
@@ -52,6 +52,7 @@
  * defined in {@link SyntheticSourceOptions}.*
  */
 @Experimental(Kind.SOURCE_SINK)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SyntheticBoundedSource extends OffsetBasedSource<KV<byte[], byte[]>> {
 
   private static final long serialVersionUID = 0;
diff --git a/sdks/java/io/synthetic/src/main/java/org/apache/beam/sdk/io/synthetic/SyntheticOptions.java b/sdks/java/io/synthetic/src/main/java/org/apache/beam/sdk/io/synthetic/SyntheticOptions.java
index 3179c1c..d4e02f6 100644
--- a/sdks/java/io/synthetic/src/main/java/org/apache/beam/sdk/io/synthetic/SyntheticOptions.java
+++ b/sdks/java/io/synthetic/src/main/java/org/apache/beam/sdk/io/synthetic/SyntheticOptions.java
@@ -46,6 +46,7 @@
  * This {@link SyntheticOptions} class provides common parameterizable synthetic options that are
  * used by {@link SyntheticBoundedSource} and {@link SyntheticUnboundedSource}.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SyntheticOptions implements Serializable {
   private static final long serialVersionUID = 0;
 
diff --git a/sdks/java/io/synthetic/src/main/java/org/apache/beam/sdk/io/synthetic/SyntheticSourceOptions.java b/sdks/java/io/synthetic/src/main/java/org/apache/beam/sdk/io/synthetic/SyntheticSourceOptions.java
index 3ae8f76..0c68b73 100644
--- a/sdks/java/io/synthetic/src/main/java/org/apache/beam/sdk/io/synthetic/SyntheticSourceOptions.java
+++ b/sdks/java/io/synthetic/src/main/java/org/apache/beam/sdk/io/synthetic/SyntheticSourceOptions.java
@@ -31,6 +31,7 @@
  * fields for details. {@code SyntheticSourceOptions} uses jackson annotations which
  * PipelineOptionsFactory can use to parse and construct an instance.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SyntheticSourceOptions extends SyntheticOptions {
   private static final long serialVersionUID = 0;
 
diff --git a/sdks/java/io/synthetic/src/main/java/org/apache/beam/sdk/io/synthetic/SyntheticUnboundedSource.java b/sdks/java/io/synthetic/src/main/java/org/apache/beam/sdk/io/synthetic/SyntheticUnboundedSource.java
index 7d8c670..9662f68 100644
--- a/sdks/java/io/synthetic/src/main/java/org/apache/beam/sdk/io/synthetic/SyntheticUnboundedSource.java
+++ b/sdks/java/io/synthetic/src/main/java/org/apache/beam/sdk/io/synthetic/SyntheticUnboundedSource.java
@@ -36,6 +36,7 @@
 
 /** A {@link SyntheticUnboundedSource} that reads {@code KV<byte[], byte[]>}. */
 @Experimental(Kind.SOURCE_SINK)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SyntheticUnboundedSource
     extends UnboundedSource<KV<byte[], byte[]>, SyntheticRecordsCheckpoint> {
 
diff --git a/sdks/java/io/synthetic/src/test/java/org/apache/beam/sdk/io/synthetic/BundleSplitterTest.java b/sdks/java/io/synthetic/src/test/java/org/apache/beam/sdk/io/synthetic/BundleSplitterTest.java
index 1a30ee4..f329596 100644
--- a/sdks/java/io/synthetic/src/test/java/org/apache/beam/sdk/io/synthetic/BundleSplitterTest.java
+++ b/sdks/java/io/synthetic/src/test/java/org/apache/beam/sdk/io/synthetic/BundleSplitterTest.java
@@ -30,6 +30,7 @@
 
 /** Unit tests for {@link BundleSplitter}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BundleSplitterTest {
 
   private SyntheticSourceOptions options;
diff --git a/sdks/java/io/synthetic/src/test/java/org/apache/beam/sdk/io/synthetic/SyntheticBoundedSourceTest.java b/sdks/java/io/synthetic/src/test/java/org/apache/beam/sdk/io/synthetic/SyntheticBoundedSourceTest.java
index c6c8df4..ab0619d 100644
--- a/sdks/java/io/synthetic/src/test/java/org/apache/beam/sdk/io/synthetic/SyntheticBoundedSourceTest.java
+++ b/sdks/java/io/synthetic/src/test/java/org/apache/beam/sdk/io/synthetic/SyntheticBoundedSourceTest.java
@@ -43,6 +43,7 @@
 
 /** Unit tests for {@link SyntheticBoundedSource}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SyntheticBoundedSourceTest {
   @Rule public final ExpectedException thrown = ExpectedException.none();
 
diff --git a/sdks/java/io/synthetic/src/test/java/org/apache/beam/sdk/io/synthetic/SyntheticUnboundedSourceTest.java b/sdks/java/io/synthetic/src/test/java/org/apache/beam/sdk/io/synthetic/SyntheticUnboundedSourceTest.java
index 8dfaefa..b74e67c 100644
--- a/sdks/java/io/synthetic/src/test/java/org/apache/beam/sdk/io/synthetic/SyntheticUnboundedSourceTest.java
+++ b/sdks/java/io/synthetic/src/test/java/org/apache/beam/sdk/io/synthetic/SyntheticUnboundedSourceTest.java
@@ -33,6 +33,7 @@
 
 /** Unit tests for {@link SyntheticUnboundedSource}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SyntheticUnboundedSourceTest {
 
   private SyntheticSourceOptions sourceOptions;
diff --git a/sdks/java/io/thrift/src/main/java/org/apache/beam/sdk/io/thrift/ThriftCoder.java b/sdks/java/io/thrift/src/main/java/org/apache/beam/sdk/io/thrift/ThriftCoder.java
index 498e4ae..7cadb8f 100644
--- a/sdks/java/io/thrift/src/main/java/org/apache/beam/sdk/io/thrift/ThriftCoder.java
+++ b/sdks/java/io/thrift/src/main/java/org/apache/beam/sdk/io/thrift/ThriftCoder.java
@@ -34,6 +34,7 @@
  *
  * @param <T> type of element handled by coder.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 class ThriftCoder<T> extends CustomCoder<T> {
 
   private final Class<T> type;
diff --git a/sdks/java/io/thrift/src/main/java/org/apache/beam/sdk/io/thrift/ThriftIO.java b/sdks/java/io/thrift/src/main/java/org/apache/beam/sdk/io/thrift/ThriftIO.java
index c408fc5..d7b248c 100644
--- a/sdks/java/io/thrift/src/main/java/org/apache/beam/sdk/io/thrift/ThriftIO.java
+++ b/sdks/java/io/thrift/src/main/java/org/apache/beam/sdk/io/thrift/ThriftIO.java
@@ -86,6 +86,7 @@
  * in future versions of the Apache Beam SDK.
  */
 @Experimental(Kind.SOURCE_SINK)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ThriftIO {
 
   private static final Logger LOG = LoggerFactory.getLogger(ThriftIO.class);
diff --git a/sdks/java/io/thrift/src/test/java/org/apache/beam/sdk/io/thrift/TestThriftStruct.java b/sdks/java/io/thrift/src/test/java/org/apache/beam/sdk/io/thrift/TestThriftStruct.java
index dd06f97..0e4850d 100644
--- a/sdks/java/io/thrift/src/test/java/org/apache/beam/sdk/io/thrift/TestThriftStruct.java
+++ b/sdks/java/io/thrift/src/test/java/org/apache/beam/sdk/io/thrift/TestThriftStruct.java
@@ -17,7 +17,14 @@
  */
 package org.apache.beam.sdk.io.thrift;
 
-@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked", "unused"})
+@SuppressWarnings({
+  "cast",
+  "rawtypes",
+  "serial",
+  "unchecked",
+  "unused",
+  "nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
+})
 @javax.annotation.Generated(
     value = "Autogenerated by Thrift Compiler (0.13.0)",
     date = "2019-12-21")
diff --git a/sdks/java/io/thrift/src/test/java/org/apache/beam/sdk/io/thrift/ThriftIOTest.java b/sdks/java/io/thrift/src/test/java/org/apache/beam/sdk/io/thrift/ThriftIOTest.java
index 60c5fa1..0d34d2d 100644
--- a/sdks/java/io/thrift/src/test/java/org/apache/beam/sdk/io/thrift/ThriftIOTest.java
+++ b/sdks/java/io/thrift/src/test/java/org/apache/beam/sdk/io/thrift/ThriftIOTest.java
@@ -46,6 +46,7 @@
 
 /** Tests for {@link ThriftIO}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ThriftIOTest implements Serializable {
 
   private static final String RESOURCE_DIR = "ThriftIOTest/";
diff --git a/sdks/java/io/tika/src/main/java/org/apache/beam/sdk/io/tika/ParseResult.java b/sdks/java/io/tika/src/main/java/org/apache/beam/sdk/io/tika/ParseResult.java
index 4894414..4fcb108 100644
--- a/sdks/java/io/tika/src/main/java/org/apache/beam/sdk/io/tika/ParseResult.java
+++ b/sdks/java/io/tika/src/main/java/org/apache/beam/sdk/io/tika/ParseResult.java
@@ -34,6 +34,7 @@
  * text, and optionally an error. If there is an error, the metadata and extracted text may be
  * partial (i.e. not represent the entire file).
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class ParseResult implements Serializable {
   private final String fileLocation;
   private final String content;
diff --git a/sdks/java/io/tika/src/main/java/org/apache/beam/sdk/io/tika/TikaIO.java b/sdks/java/io/tika/src/main/java/org/apache/beam/sdk/io/tika/TikaIO.java
index 032c1a8..4bf0568 100644
--- a/sdks/java/io/tika/src/main/java/org/apache/beam/sdk/io/tika/TikaIO.java
+++ b/sdks/java/io/tika/src/main/java/org/apache/beam/sdk/io/tika/TikaIO.java
@@ -90,6 +90,7 @@
  * }</pre>
  */
 @Experimental(Kind.SOURCE_SINK)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class TikaIO {
   /** Parses files matching a given filepattern. */
   public static Parse parse() {
diff --git a/sdks/java/io/tika/src/test/java/org/apache/beam/sdk/io/tika/TikaIOTest.java b/sdks/java/io/tika/src/test/java/org/apache/beam/sdk/io/tika/TikaIOTest.java
index 91b0890..839a320 100644
--- a/sdks/java/io/tika/src/test/java/org/apache/beam/sdk/io/tika/TikaIOTest.java
+++ b/sdks/java/io/tika/src/test/java/org/apache/beam/sdk/io/tika/TikaIOTest.java
@@ -46,6 +46,7 @@
 
 /** Tests for {@link TikaIO}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class TikaIOTest implements Serializable {
   private static final String PDF_ZIP_FILE =
       "\n\n\n\n\n\n\n\napache-beam-tika.pdf\n\n\nCombining\n\n\nApache Beam\n\n\n"
diff --git a/sdks/java/io/xml/src/main/java/org/apache/beam/sdk/io/xml/JAXBCoder.java b/sdks/java/io/xml/src/main/java/org/apache/beam/sdk/io/xml/JAXBCoder.java
index 90ae1a4..956d14b 100644
--- a/sdks/java/io/xml/src/main/java/org/apache/beam/sdk/io/xml/JAXBCoder.java
+++ b/sdks/java/io/xml/src/main/java/org/apache/beam/sdk/io/xml/JAXBCoder.java
@@ -42,6 +42,7 @@
  *
  * @param <T> type of JAXB annotated objects that will be serialized.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class JAXBCoder<T> extends CustomCoder<T> {
 
   private final Class<T> jaxbClass;
diff --git a/sdks/java/io/xml/src/main/java/org/apache/beam/sdk/io/xml/XmlIO.java b/sdks/java/io/xml/src/main/java/org/apache/beam/sdk/io/xml/XmlIO.java
index b7a7ad5..65e99a3 100644
--- a/sdks/java/io/xml/src/main/java/org/apache/beam/sdk/io/xml/XmlIO.java
+++ b/sdks/java/io/xml/src/main/java/org/apache/beam/sdk/io/xml/XmlIO.java
@@ -53,6 +53,7 @@
 import org.checkerframework.checker.nullness.qual.Nullable;
 
 /** Transforms for reading and writing XML files using JAXB mappers. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class XmlIO {
   // CHECKSTYLE.OFF: JavadocStyle
   /**
diff --git a/sdks/java/io/xml/src/main/java/org/apache/beam/sdk/io/xml/XmlSource.java b/sdks/java/io/xml/src/main/java/org/apache/beam/sdk/io/xml/XmlSource.java
index 6548511..536092b 100644
--- a/sdks/java/io/xml/src/main/java/org/apache/beam/sdk/io/xml/XmlSource.java
+++ b/sdks/java/io/xml/src/main/java/org/apache/beam/sdk/io/xml/XmlSource.java
@@ -48,6 +48,7 @@
 import org.codehaus.stax2.XMLInputFactory2;
 
 /** Implementation of {@link XmlIO#read}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class XmlSource<T> extends FileBasedSource<T> {
 
   private static final String XML_VERSION = "1.1";
diff --git a/sdks/java/io/xml/src/test/java/org/apache/beam/sdk/io/xml/JAXBCoderTest.java b/sdks/java/io/xml/src/test/java/org/apache/beam/sdk/io/xml/JAXBCoderTest.java
index 00cafb5..59504b3 100644
--- a/sdks/java/io/xml/src/test/java/org/apache/beam/sdk/io/xml/JAXBCoderTest.java
+++ b/sdks/java/io/xml/src/test/java/org/apache/beam/sdk/io/xml/JAXBCoderTest.java
@@ -46,6 +46,7 @@
 
 /** Unit tests for {@link JAXBCoder}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class JAXBCoderTest {
 
   @XmlRootElement
diff --git a/sdks/java/io/xml/src/test/java/org/apache/beam/sdk/io/xml/XmlIOTest.java b/sdks/java/io/xml/src/test/java/org/apache/beam/sdk/io/xml/XmlIOTest.java
index 43b7547..9c8f175 100644
--- a/sdks/java/io/xml/src/test/java/org/apache/beam/sdk/io/xml/XmlIOTest.java
+++ b/sdks/java/io/xml/src/test/java/org/apache/beam/sdk/io/xml/XmlIOTest.java
@@ -47,6 +47,7 @@
 
 /** Tests for {@link XmlIO}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class XmlIOTest {
   @Rule public TemporaryFolder tmpFolder = new TemporaryFolder();
 
diff --git a/sdks/java/io/xml/src/test/java/org/apache/beam/sdk/io/xml/XmlSourceTest.java b/sdks/java/io/xml/src/test/java/org/apache/beam/sdk/io/xml/XmlSourceTest.java
index 9f75dea..5008427 100644
--- a/sdks/java/io/xml/src/test/java/org/apache/beam/sdk/io/xml/XmlSourceTest.java
+++ b/sdks/java/io/xml/src/test/java/org/apache/beam/sdk/io/xml/XmlSourceTest.java
@@ -58,6 +58,7 @@
  * XmlIOTest}.
  */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class XmlSourceTest {
 
   @Rule public TestPipeline p = TestPipeline.create();
diff --git a/sdks/java/testing/expansion-service/src/test/java/org/apache/beam/sdk/testing/expansion/TestExpansionService.java b/sdks/java/testing/expansion-service/src/test/java/org/apache/beam/sdk/testing/expansion/TestExpansionService.java
index 75fb73b..e8451d0 100644
--- a/sdks/java/testing/expansion-service/src/test/java/org/apache/beam/sdk/testing/expansion/TestExpansionService.java
+++ b/sdks/java/testing/expansion-service/src/test/java/org/apache/beam/sdk/testing/expansion/TestExpansionService.java
@@ -61,6 +61,7 @@
 /**
  * An {@link org.apache.beam.runners.core.construction.expansion.ExpansionService} useful for tests.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class TestExpansionService {
 
   private static final String TEST_PREFIX_URN = "beam:transforms:xlang:test:prefix";
diff --git a/sdks/java/testing/load-tests/src/main/java/org/apache/beam/sdk/loadtests/LoadTest.java b/sdks/java/testing/load-tests/src/main/java/org/apache/beam/sdk/loadtests/LoadTest.java
index 9787e17..c663b2d 100644
--- a/sdks/java/testing/load-tests/src/main/java/org/apache/beam/sdk/loadtests/LoadTest.java
+++ b/sdks/java/testing/load-tests/src/main/java/org/apache/beam/sdk/loadtests/LoadTest.java
@@ -60,6 +60,7 @@
  * Base class for all load tests. Provides common operations such as initializing source/step
  * options, creating a pipeline, etc.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 abstract class LoadTest<OptionsT extends LoadTestOptions> {
 
   private static final Logger LOG = LoggerFactory.getLogger(LoadTest.class);
diff --git a/sdks/java/testing/load-tests/src/main/java/org/apache/beam/sdk/loadtests/SyntheticDataPublisher.java b/sdks/java/testing/load-tests/src/main/java/org/apache/beam/sdk/loadtests/SyntheticDataPublisher.java
index 4161add..a2748f4 100644
--- a/sdks/java/testing/load-tests/src/main/java/org/apache/beam/sdk/loadtests/SyntheticDataPublisher.java
+++ b/sdks/java/testing/load-tests/src/main/java/org/apache/beam/sdk/loadtests/SyntheticDataPublisher.java
@@ -71,6 +71,7 @@
  * <p>If parameters related to a specific sink are provided (Kafka, PubSub or Kinesis), the pipeline
  * writes to the sink. Writing to more than one sink is also acceptable.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SyntheticDataPublisher {
 
   private static final Coder RECORD_CODER = StringUtf8Coder.of();
diff --git a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/Main.java b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/Main.java
index 0d36a6c..bd83fa7 100644
--- a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/Main.java
+++ b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/Main.java
@@ -70,6 +70,7 @@
  * href="https://web.archive.org/web/20100620010601/http://datalab.cs.pdx.edu/niagaraST/NEXMark/">
  * Nexmark website</a>
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class Main {
 
   private static class Result {
diff --git a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/Monitor.java b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/Monitor.java
index 4de5db1..b44070d 100644
--- a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/Monitor.java
+++ b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/Monitor.java
@@ -32,6 +32,7 @@
  *
  * @param <T> Type of element we are monitoring.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class Monitor<T extends KnownSize> implements Serializable {
   private class MonitorDoFn extends DoFn<T, T> {
     final Counter elementCounter = Metrics.counter(name, prefix + ".elements");
diff --git a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/NexmarkConfiguration.java b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/NexmarkConfiguration.java
index ee29b60..ad17547 100644
--- a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/NexmarkConfiguration.java
+++ b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/NexmarkConfiguration.java
@@ -30,6 +30,7 @@
  * programmatically. We only capture properties which may influence the resulting pipeline
  * performance, as captured by {@link NexmarkPerf}.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class NexmarkConfiguration implements Serializable {
   public static final NexmarkConfiguration DEFAULT = new NexmarkConfiguration();
 
diff --git a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/NexmarkLauncher.java b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/NexmarkLauncher.java
index f935fd3..fc9fa20 100644
--- a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/NexmarkLauncher.java
+++ b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/NexmarkLauncher.java
@@ -114,6 +114,7 @@
 import org.slf4j.LoggerFactory;
 
 /** Run a single Nexmark query using a given configuration. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class NexmarkLauncher<OptionT extends NexmarkOptions> {
 
   private static final org.slf4j.Logger LOG = LoggerFactory.getLogger(NexmarkLauncher.class);
diff --git a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/NexmarkOptions.java b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/NexmarkOptions.java
index 3a7d7e3..508b589 100644
--- a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/NexmarkOptions.java
+++ b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/NexmarkOptions.java
@@ -93,22 +93,22 @@
   void setOutputPath(String outputPath);
 
   @Description("Base name of pubsub topic to publish to in streaming mode.")
-  @Nullable
   @Default.String("nexmark")
+  @Nullable
   String getPubsubTopic();
 
   void setPubsubTopic(String pubsubTopic);
 
   @Description("Base name of pubsub subscription to read from in streaming mode.")
-  @Nullable
   @Default.String("nexmark")
+  @Nullable
   String getPubsubSubscription();
 
   void setPubsubSubscription(String pubsubSubscription);
 
   @Description("Base name of BigQuery table name if using BigQuery output.")
-  @Nullable
   @Default.String("nexmark")
+  @Nullable
   String getBigQueryTable();
 
   void setBigQueryTable(String bigQueryTable);
@@ -445,15 +445,15 @@
   void setQueryLanguage(String value);
 
   @Description("Base name of Kafka events topic in streaming mode.")
-  @Nullable
   @Default.String("nexmark")
+  @Nullable
   String getKafkaTopic();
 
   void setKafkaTopic(String value);
 
   @Description("Base name of Kafka results topic in streaming mode.")
-  @Nullable
   @Default.String("nexmark-results")
+  @Nullable
   String getKafkaResultsTopic();
 
   void setKafkaResultsTopic(String value);
@@ -505,8 +505,8 @@
   void setExportSummaryToInfluxDB(boolean exportSummaryToInfluxDB);
 
   @Description("Base name of measurement name if using InfluxDB output.")
-  @Nullable
   @Default.String("nexmark")
+  @Nullable
   String getBaseInfluxMeasurement();
 
   void setBaseInfluxMeasurement(String influxDBMeasurement);
diff --git a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/NexmarkQueryName.java b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/NexmarkQueryName.java
index acc1a75..24c7e3d 100644
--- a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/NexmarkQueryName.java
+++ b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/NexmarkQueryName.java
@@ -20,7 +20,10 @@
 import org.checkerframework.checker.nullness.qual.Nullable;
 
 /** Known "Nexmark" queries, some of which are of our own devising but use the same data set. */
-@SuppressWarnings("ImmutableEnumChecker")
+@SuppressWarnings({
+  "ImmutableEnumChecker",
+  "nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
+})
 public enum NexmarkQueryName {
   // A baseline
   PASSTHROUGH(0),
diff --git a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/NexmarkUtils.java b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/NexmarkUtils.java
index f6b56fd..e4ad039 100644
--- a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/NexmarkUtils.java
+++ b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/NexmarkUtils.java
@@ -87,6 +87,7 @@
 import org.slf4j.LoggerFactory;
 
 /** Odd's 'n Ends used throughout queries and driver. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class NexmarkUtils {
   private static final Logger LOG = LoggerFactory.getLogger(NexmarkUtils.class);
 
diff --git a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/Auction.java b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/Auction.java
index 59f125a..e5bc5fb 100644
--- a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/Auction.java
+++ b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/Auction.java
@@ -38,6 +38,7 @@
 
 /** An auction submitted by a person. */
 @DefaultSchema(JavaFieldSchema.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class Auction implements KnownSize, Serializable {
   private static final Coder<Instant> INSTANT_CODER = InstantCoder.of();
   private static final Coder<Long> LONG_CODER = VarLongCoder.of();
diff --git a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/AuctionBid.java b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/AuctionBid.java
index cc7c73c..482651c 100644
--- a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/AuctionBid.java
+++ b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/AuctionBid.java
@@ -32,6 +32,7 @@
 import org.checkerframework.checker.nullness.qual.Nullable;
 
 /** Result of {@link WinningBids} transform. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class AuctionBid implements KnownSize, Serializable {
   public static final Coder<AuctionBid> CODER =
       new CustomCoder<AuctionBid>() {
diff --git a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/Bid.java b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/Bid.java
index 75b2682..16001ad 100644
--- a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/Bid.java
+++ b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/Bid.java
@@ -39,6 +39,7 @@
 
 /** A bid for an item on auction. */
 @DefaultSchema(JavaFieldSchema.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class Bid implements KnownSize, Serializable {
   private static final Coder<Instant> INSTANT_CODER = InstantCoder.of();
   private static final Coder<Long> LONG_CODER = VarLongCoder.of();
diff --git a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/Done.java b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/Done.java
index ee59aff..29cb512 100644
--- a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/Done.java
+++ b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/Done.java
@@ -32,6 +32,7 @@
 import org.checkerframework.checker.nullness.qual.Nullable;
 
 /** Result of query 10. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class Done implements KnownSize, Serializable {
   private static final Coder<String> STRING_CODER = StringUtf8Coder.of();
 
diff --git a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/Event.java b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/Event.java
index 083b50f..d99f15b 100644
--- a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/Event.java
+++ b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/Event.java
@@ -34,6 +34,7 @@
  * {@link Bid}.
  */
 @DefaultSchema(JavaFieldSchema.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class Event implements KnownSize, Serializable {
 
   @Override
@@ -114,11 +115,11 @@
         }
       };
 
-  public @Nullable @org.apache.avro.reflect.Nullable Person newPerson;
+  public @org.apache.avro.reflect.Nullable @Nullable Person newPerson;
 
-  public @Nullable @org.apache.avro.reflect.Nullable Auction newAuction;
+  public @org.apache.avro.reflect.Nullable @Nullable Auction newAuction;
 
-  public @Nullable @org.apache.avro.reflect.Nullable Bid bid;
+  public @org.apache.avro.reflect.Nullable @Nullable Bid bid;
 
   @SuppressWarnings("unused")
   public Event() {
diff --git a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/IdNameReserve.java b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/IdNameReserve.java
index 909d973..f3b5412 100644
--- a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/IdNameReserve.java
+++ b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/IdNameReserve.java
@@ -33,6 +33,7 @@
 import org.checkerframework.checker.nullness.qual.Nullable;
 
 /** Result type of Query8. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class IdNameReserve implements KnownSize, Serializable {
   private static final Coder<Long> LONG_CODER = VarLongCoder.of();
   private static final Coder<String> STRING_CODER = StringUtf8Coder.of();
diff --git a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/NameCityStateId.java b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/NameCityStateId.java
index 2d02796..9edb66a 100644
--- a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/NameCityStateId.java
+++ b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/NameCityStateId.java
@@ -36,6 +36,7 @@
 
 /** Result of Query3. */
 @DefaultSchema(JavaFieldSchema.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class NameCityStateId implements KnownSize, Serializable {
   private static final Coder<Long> LONG_CODER = VarLongCoder.of();
   private static final Coder<String> STRING_CODER = StringUtf8Coder.of();
diff --git a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/Person.java b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/Person.java
index 86a4110..e100f36 100644
--- a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/Person.java
+++ b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/Person.java
@@ -38,6 +38,7 @@
 
 /** A person either creating an auction or making a bid. */
 @DefaultSchema(JavaFieldSchema.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class Person implements KnownSize, Serializable {
   private static final Coder<Instant> INSTANT_CODER = InstantCoder.of();
   private static final Coder<Long> LONG_CODER = VarLongCoder.of();
diff --git a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/sql/SelectEvent.java b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/sql/SelectEvent.java
index f7515b0..cf7f80c 100644
--- a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/sql/SelectEvent.java
+++ b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/sql/SelectEvent.java
@@ -26,6 +26,7 @@
 import org.apache.beam.sdk.values.Row;
 
 /** Select a person bid or auction out of Java Event and return it as a row. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SelectEvent extends PTransform<PCollection<Event>, PCollection<Row>> {
   private Event.Type eventType;
   private int fieldIndex = -1;
diff --git a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/BoundedSideInputJoin.java b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/BoundedSideInputJoin.java
index 6f96a40..0315f9c 100644
--- a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/BoundedSideInputJoin.java
+++ b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/BoundedSideInputJoin.java
@@ -38,6 +38,7 @@
  * WHERE bid.id = sideInput.id
  * </pre>
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BoundedSideInputJoin extends NexmarkQueryTransform<Bid> {
   private final NexmarkConfiguration configuration;
 
diff --git a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/NexmarkQuery.java b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/NexmarkQuery.java
index 669c435..abd2036 100644
--- a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/NexmarkQuery.java
+++ b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/NexmarkQuery.java
@@ -30,6 +30,7 @@
 import org.apache.beam.sdk.values.TimestampedValue;
 
 /** Wrapper for 'NEXmark' query transforms that adds monitoring and snooping. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public final class NexmarkQuery<T extends KnownSize>
     extends PTransform<PCollection<Event>, PCollection<? extends TimestampedValue<T>>> {
 
diff --git a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/NexmarkQueryModel.java b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/NexmarkQueryModel.java
index ea0b2f9..06c46e8 100644
--- a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/NexmarkQueryModel.java
+++ b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/NexmarkQueryModel.java
@@ -36,6 +36,7 @@
  * Base class for models of the eight NEXMark queries. Provides an assertion function which can be
  * applied against the actual query results to check their consistency with the model.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public abstract class NexmarkQueryModel<T extends KnownSize> implements Serializable {
   public final NexmarkConfiguration configuration;
 
diff --git a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/NexmarkQueryTransform.java b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/NexmarkQueryTransform.java
index d6c5123..2333fe3 100644
--- a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/NexmarkQueryTransform.java
+++ b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/NexmarkQueryTransform.java
@@ -31,6 +31,7 @@
  *
  * <p>A query transform may request a faked side input to join with ids.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public abstract class NexmarkQueryTransform<T extends KnownSize>
     extends PTransform<PCollection<Event>, PCollection<T>> {
 
diff --git a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/NexmarkQueryUtil.java b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/NexmarkQueryUtil.java
index d700de4..31f401d 100644
--- a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/NexmarkQueryUtil.java
+++ b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/NexmarkQueryUtil.java
@@ -32,6 +32,7 @@
 import org.joda.time.Instant;
 
 /** Utilities for working with NEXmark data stream. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class NexmarkQueryUtil {
   // Do not instantiate
   private NexmarkQueryUtil() {}
diff --git a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query0.java b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query0.java
index b4d3db2..cbd5ebf 100644
--- a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query0.java
+++ b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query0.java
@@ -33,6 +33,7 @@
  * Query 0: Pass events through unchanged. However, force them to do a round trip through
  * serialization so that we measure the impact of the choice of coders.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class Query0 extends NexmarkQueryTransform<Event> {
   public Query0() {
     super("Query0");
diff --git a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query10.java b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query10.java
index c48e95d..84cd5f1 100644
--- a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query10.java
+++ b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query10.java
@@ -59,6 +59,7 @@
  *
  * <p>Every windowSizeSec, save all events from the last period into 2*maxWorkers log files.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class Query10 extends NexmarkQueryTransform<Done> {
   private static final Logger LOG = LoggerFactory.getLogger(Query10.class);
   private static final int NUM_SHARDS_PER_WORKER = 5;
diff --git a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query3.java b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query3.java
index 05d7bf3..c70666e 100644
--- a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query3.java
+++ b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query3.java
@@ -68,6 +68,7 @@
  *
  * <p>A real system would use an external system to maintain the id-to-person association.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class Query3 extends NexmarkQueryTransform<NameCityStateId> {
 
   private static final Logger LOG = LoggerFactory.getLogger(Query3.class);
diff --git a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query3Model.java b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query3Model.java
index 54a19d8..623e4ca 100644
--- a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query3Model.java
+++ b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query3Model.java
@@ -34,6 +34,7 @@
 import org.joda.time.Instant;
 
 /** A direct implementation of {@link Query3}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class Query3Model extends NexmarkQueryModel<NameCityStateId> implements Serializable {
   /** Simulator for query 3. */
   private static class Simulator extends AbstractSimulator<Event, NameCityStateId> {
diff --git a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query4Model.java b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query4Model.java
index b8f5ff3..083110d 100644
--- a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query4Model.java
+++ b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query4Model.java
@@ -37,6 +37,7 @@
 import org.junit.Assert;
 
 /** A direct implementation of {@link Query4}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class Query4Model extends NexmarkQueryModel<CategoryPrice> implements Serializable {
   /** Simulator for query 4. */
   private class Simulator extends AbstractSimulator<AuctionBid, CategoryPrice> {
diff --git a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query5Model.java b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query5Model.java
index 2ae08af..e616298 100644
--- a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query5Model.java
+++ b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query5Model.java
@@ -34,6 +34,7 @@
 import org.joda.time.Instant;
 
 /** A direct implementation of {@link Query5}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class Query5Model extends NexmarkQueryModel<AuctionCount> implements Serializable {
   /** Simulator for query 5. */
   private class Simulator extends AbstractSimulator<Event, AuctionCount> {
diff --git a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query6Model.java b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query6Model.java
index 55b9142..da5f0fb 100644
--- a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query6Model.java
+++ b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query6Model.java
@@ -36,6 +36,7 @@
 import org.junit.Assert;
 
 /** A direct implementation of {@link Query6}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class Query6Model extends NexmarkQueryModel<SellerPrice> implements Serializable {
   /** Simulator for query 6. */
   private static class Simulator extends AbstractSimulator<AuctionBid, SellerPrice> {
diff --git a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query7Model.java b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query7Model.java
index 4560033..8a79aff 100644
--- a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query7Model.java
+++ b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query7Model.java
@@ -32,6 +32,7 @@
 import org.joda.time.Instant;
 
 /** A direct implementation of {@link Query7}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class Query7Model extends NexmarkQueryModel<Bid> implements Serializable {
   /** Simulator for query 7. */
   private class Simulator extends AbstractSimulator<Event, Bid> {
diff --git a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query8Model.java b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query8Model.java
index 789cc08..c6917de 100644
--- a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query8Model.java
+++ b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query8Model.java
@@ -35,6 +35,7 @@
 import org.joda.time.Instant;
 
 /** A direct implementation of {@link Query8}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class Query8Model extends NexmarkQueryModel<IdNameReserve> implements Serializable {
   /** Simulator for query 8. */
   private class Simulator extends AbstractSimulator<Event, IdNameReserve> {
diff --git a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query9.java b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query9.java
index 96fd858..2b6cd22 100644
--- a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query9.java
+++ b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query9.java
@@ -27,6 +27,7 @@
  * Query "9", 'Winning bids'. Select just the winning bids. Not in original NEXMark suite, but handy
  * for testing. See {@link WinningBids} for the details.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class Query9 extends NexmarkQueryTransform<AuctionBid> {
   private final NexmarkConfiguration configuration;
 
diff --git a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/SessionSideInputJoin.java b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/SessionSideInputJoin.java
index 5d46530..88075fd 100644
--- a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/SessionSideInputJoin.java
+++ b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/SessionSideInputJoin.java
@@ -47,6 +47,7 @@
  * WHERE bid.id = sideInput.id
  * </pre>
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SessionSideInputJoin extends NexmarkQueryTransform<Bid> {
   private final NexmarkConfiguration configuration;
 
diff --git a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/SessionSideInputJoinModel.java b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/SessionSideInputJoinModel.java
index c81f352..5827964 100644
--- a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/SessionSideInputJoinModel.java
+++ b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/SessionSideInputJoinModel.java
@@ -36,6 +36,7 @@
 import org.joda.time.Instant;
 
 /** A direct implementation of {@link SessionSideInputJoin}. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SessionSideInputJoinModel extends NexmarkQueryModel<Bid> {
 
   /** Simulator for SESSION_SIDE_INPUT_JOIN query. */
diff --git a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/WinningBids.java b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/WinningBids.java
index bc0cd75..c69a0c7 100644
--- a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/WinningBids.java
+++ b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/WinningBids.java
@@ -75,6 +75,7 @@
  * <p>Our implementation will use a custom windowing function in order to bring bids and auctions
  * together without requiring global state.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class WinningBids extends PTransform<PCollection<Event>, PCollection<AuctionBid>> {
   /** Windows for open auctions and bids. */
   private static class AuctionOrBidWindow extends IntervalWindow {
diff --git a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/WinningBidsSimulator.java b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/WinningBidsSimulator.java
index 07bb851..d4f65ad 100644
--- a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/WinningBidsSimulator.java
+++ b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/WinningBidsSimulator.java
@@ -36,6 +36,7 @@
 import org.joda.time.Instant;
 
 /** A simulator of the {@code WinningBids} query. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class WinningBidsSimulator extends AbstractSimulator<Event, AuctionBid> {
   /** Auctions currently still open, indexed by auction id. */
   private final Map<Long, Auction> openAuctions;
diff --git a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/sql/SqlBoundedSideInputJoin.java b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/sql/SqlBoundedSideInputJoin.java
index 3bb6f66..7062041 100644
--- a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/sql/SqlBoundedSideInputJoin.java
+++ b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/sql/SqlBoundedSideInputJoin.java
@@ -40,6 +40,7 @@
 import org.apache.beam.sdk.values.TypeDescriptors;
 
 /** Basic stream enrichment: join a stream to a bounded side input. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SqlBoundedSideInputJoin extends NexmarkQueryTransform<Bid> {
   private final String query;
   private final NexmarkConfiguration configuration;
diff --git a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/sql/SqlQuery0.java b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/sql/SqlQuery0.java
index e77131b..103e1a9 100644
--- a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/sql/SqlQuery0.java
+++ b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/sql/SqlQuery0.java
@@ -50,6 +50,7 @@
  * <p>{@link Bid} events are used here at the moment, ås they are most numerous with default
  * configuration.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class SqlQuery0 extends NexmarkQueryTransform<Bid> {
 
   private final Class<? extends QueryPlanner> plannerClass;
diff --git a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/sources/UnboundedEventSource.java b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/sources/UnboundedEventSource.java
index fb38165..9e9d418 100644
--- a/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/sources/UnboundedEventSource.java
+++ b/sdks/java/testing/nexmark/src/main/java/org/apache/beam/sdk/nexmark/sources/UnboundedEventSource.java
@@ -46,6 +46,7 @@
  * the overall rate respect the {@code interEventDelayUs} period if possible. Otherwise, events are
  * returned every time the system asks for one.
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class UnboundedEventSource extends UnboundedSource<Event, GeneratorCheckpoint> {
   private static final Duration BACKLOG_PERIOD = Duration.standardSeconds(30);
   private static final Logger LOG = LoggerFactory.getLogger(UnboundedEventSource.class);
diff --git a/sdks/java/testing/nexmark/src/test/java/org/apache/beam/sdk/nexmark/NexmarkUtilsTest.java b/sdks/java/testing/nexmark/src/test/java/org/apache/beam/sdk/nexmark/NexmarkUtilsTest.java
index 3efba39..539bcc6 100644
--- a/sdks/java/testing/nexmark/src/test/java/org/apache/beam/sdk/nexmark/NexmarkUtilsTest.java
+++ b/sdks/java/testing/nexmark/src/test/java/org/apache/beam/sdk/nexmark/NexmarkUtilsTest.java
@@ -43,6 +43,7 @@
 
 /** Tests the {@link NexmarkUtils}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class NexmarkUtilsTest {
 
   @Rule public TestPipeline pipeline = TestPipeline.create();
diff --git a/sdks/java/testing/nexmark/src/test/java/org/apache/beam/sdk/nexmark/PerfsToBigQueryTest.java b/sdks/java/testing/nexmark/src/test/java/org/apache/beam/sdk/nexmark/PerfsToBigQueryTest.java
index 4b86c29..2668664 100644
--- a/sdks/java/testing/nexmark/src/test/java/org/apache/beam/sdk/nexmark/PerfsToBigQueryTest.java
+++ b/sdks/java/testing/nexmark/src/test/java/org/apache/beam/sdk/nexmark/PerfsToBigQueryTest.java
@@ -31,6 +31,7 @@
 import org.junit.Test;
 
 /** Test class for BigQuery sinks. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class PerfsToBigQueryTest {
 
   private static final NexmarkQueryName QUERY = NexmarkQueryName.CURRENCY_CONVERSION;
diff --git a/sdks/java/testing/test-utils/src/main/java/org/apache/beam/sdk/testutils/publishing/InfluxDBPublisher.java b/sdks/java/testing/test-utils/src/main/java/org/apache/beam/sdk/testutils/publishing/InfluxDBPublisher.java
index 8f16959..3b49a27 100644
--- a/sdks/java/testing/test-utils/src/main/java/org/apache/beam/sdk/testutils/publishing/InfluxDBPublisher.java
+++ b/sdks/java/testing/test-utils/src/main/java/org/apache/beam/sdk/testutils/publishing/InfluxDBPublisher.java
@@ -49,6 +49,7 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public final class InfluxDBPublisher {
   private static final Logger LOG = LoggerFactory.getLogger(InfluxDBPublisher.class);
 
diff --git a/sdks/java/testing/test-utils/src/main/java/org/apache/beam/sdk/testutils/publishing/InfluxDBSettings.java b/sdks/java/testing/test-utils/src/main/java/org/apache/beam/sdk/testutils/publishing/InfluxDBSettings.java
index 8e69ee5..b94fc62 100644
--- a/sdks/java/testing/test-utils/src/main/java/org/apache/beam/sdk/testutils/publishing/InfluxDBSettings.java
+++ b/sdks/java/testing/test-utils/src/main/java/org/apache/beam/sdk/testutils/publishing/InfluxDBSettings.java
@@ -19,6 +19,7 @@
 
 import static org.apache.beam.repackaged.core.org.apache.commons.lang3.StringUtils.isBlank;
 
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public final class InfluxDBSettings {
 
   public final String host;
diff --git a/sdks/java/testing/test-utils/src/test/java/org/apache/beam/sdk/testutils/fakes/FakeBigQueryClient.java b/sdks/java/testing/test-utils/src/test/java/org/apache/beam/sdk/testutils/fakes/FakeBigQueryClient.java
index 2bcbe4e..179fba1 100644
--- a/sdks/java/testing/test-utils/src/test/java/org/apache/beam/sdk/testutils/fakes/FakeBigQueryClient.java
+++ b/sdks/java/testing/test-utils/src/test/java/org/apache/beam/sdk/testutils/fakes/FakeBigQueryClient.java
@@ -29,6 +29,7 @@
  *
  * @see BigQueryClient
  */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class FakeBigQueryClient extends BigQueryClient {
 
   private Map<String, List<Map<String, ?>>> rowsPerTable;
diff --git a/sdks/java/testing/test-utils/src/test/java/org/apache/beam/sdk/testutils/fakes/FakeBigQueryResultsPublisher.java b/sdks/java/testing/test-utils/src/test/java/org/apache/beam/sdk/testutils/fakes/FakeBigQueryResultsPublisher.java
index e8adfb9..48938bc 100644
--- a/sdks/java/testing/test-utils/src/test/java/org/apache/beam/sdk/testutils/fakes/FakeBigQueryResultsPublisher.java
+++ b/sdks/java/testing/test-utils/src/test/java/org/apache/beam/sdk/testutils/fakes/FakeBigQueryResultsPublisher.java
@@ -25,6 +25,7 @@
 import org.apache.beam.sdk.testutils.publishing.BigQueryResultsPublisher;
 
 /** A fake implementation of {@link BigQueryResultsPublisher} for testing purposes only. */
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class FakeBigQueryResultsPublisher extends BigQueryResultsPublisher {
 
   private Map<String, List<TestResult>> recordsPerTable;
diff --git a/sdks/java/testing/test-utils/src/test/java/org/apache/beam/sdk/testutils/jvmverification/JvmVerification.java b/sdks/java/testing/test-utils/src/test/java/org/apache/beam/sdk/testutils/jvmverification/JvmVerification.java
index 1cc0ed0..696e770 100644
--- a/sdks/java/testing/test-utils/src/test/java/org/apache/beam/sdk/testutils/jvmverification/JvmVerification.java
+++ b/sdks/java/testing/test-utils/src/test/java/org/apache/beam/sdk/testutils/jvmverification/JvmVerification.java
@@ -30,6 +30,7 @@
 import org.apache.commons.codec.binary.Hex;
 import org.junit.Test;
 
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class JvmVerification {
 
   private static final Map<String, Java> versionMapping = new HashMap<>();
diff --git a/sdks/java/testing/test-utils/src/test/java/org/apache/beam/sdk/testutils/publishing/BigQueryResultsPublisherTest.java b/sdks/java/testing/test-utils/src/test/java/org/apache/beam/sdk/testutils/publishing/BigQueryResultsPublisherTest.java
index 07a4035..aebaca8 100644
--- a/sdks/java/testing/test-utils/src/test/java/org/apache/beam/sdk/testutils/publishing/BigQueryResultsPublisherTest.java
+++ b/sdks/java/testing/test-utils/src/test/java/org/apache/beam/sdk/testutils/publishing/BigQueryResultsPublisherTest.java
@@ -33,6 +33,7 @@
 
 /** Tests for {@link BigQueryResultsPublisher}. */
 @RunWith(JUnit4.class)
+@SuppressWarnings("nullness") // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 public class BigQueryResultsPublisherTest {
 
   private static final String TABLE_NAME = "table";
diff --git a/sdks/python/apache_beam/coders/coder_impl.py b/sdks/python/apache_beam/coders/coder_impl.py
index 075f58c..6abebd0 100644
--- a/sdks/python/apache_beam/coders/coder_impl.py
+++ b/sdks/python/apache_beam/coders/coder_impl.py
@@ -61,6 +61,7 @@
 from apache_beam.coders import observable
 from apache_beam.coders.avro_record import AvroRecord
 from apache_beam.utils import windowed_value
+from apache_beam.utils.sharded_key import ShardedKey
 from apache_beam.utils.timestamp import MAX_TIMESTAMP
 from apache_beam.utils.timestamp import MIN_TIMESTAMP
 from apache_beam.utils.timestamp import Timestamp
@@ -975,12 +976,13 @@
   # Default buffer size of 64kB of handling iterables of unknown length.
   _DEFAULT_BUFFER_SIZE = 64 * 1024
 
-  def __init__(self,
-               elem_coder,  # type: CoderImpl
-               read_state=None,  # type: Optional[IterableStateReader]
-               write_state=None,  # type: Optional[IterableStateWriter]
-               write_state_threshold=0  # type: int
-              ):
+  def __init__(
+      self,
+      elem_coder,  # type: CoderImpl
+      read_state=None,  # type: Optional[IterableStateReader]
+      write_state=None,  # type: Optional[IterableStateWriter]
+      write_state_threshold=0  # type: int
+  ):
     self._elem_coder = elem_coder
     self._read_state = read_state
     self._write_state = write_state
@@ -1268,8 +1270,7 @@
     value = self._value_coder.decode_from_stream(in_stream, nested)
     return windowed_value.create(
         value,
-        # Avoid creation of Timestamp object.
-        timestamp,
+        timestamp,  # Avoid creation of Timestamp object.
         windows,
         pane_info)
 
@@ -1365,3 +1366,37 @@
     # type: (Any, bool) -> int
     value_size = self._value_coder.estimate_size(value)
     return get_varint_size(value_size) + value_size
+
+
+class ShardedKeyCoderImpl(StreamCoderImpl):
+  """For internal use only; no backwards-compatibility guarantees.
+
+  A coder for sharded user keys.
+
+  The encoding and decoding should follow the order:
+      shard id byte string
+      encoded user key
+  """
+  def __init__(self, key_coder_impl):
+    self._shard_id_coder_impl = BytesCoderImpl()
+    self._key_coder_impl = key_coder_impl
+
+  def encode_to_stream(self, value, out, nested):
+    # type: (ShardedKey, create_OutputStream, bool) -> None
+    self._shard_id_coder_impl.encode_to_stream(value._shard_id, out, True)
+    self._key_coder_impl.encode_to_stream(value.key, out, True)
+
+  def decode_from_stream(self, in_stream, nested):
+    # type: (create_InputStream, bool) -> ShardedKey
+    shard_id = self._shard_id_coder_impl.decode_from_stream(in_stream, True)
+    key = self._key_coder_impl.decode_from_stream(in_stream, True)
+    return ShardedKey(key=key, shard_id=shard_id)
+
+  def estimate_size(self, value, nested=False):
+    # type: (Any, bool) -> int
+    estimated_size = 0
+    estimated_size += (
+        self._shard_id_coder_impl.estimate_size(value._shard_id, nested=True))
+    estimated_size += (
+        self._key_coder_impl.estimate_size(value.key, nested=True))
+    return estimated_size
diff --git a/sdks/python/apache_beam/coders/coders.py b/sdks/python/apache_beam/coders/coders.py
index bc35779..3b660c3 100644
--- a/sdks/python/apache_beam/coders/coders.py
+++ b/sdks/python/apache_beam/coders/coders.py
@@ -107,12 +107,13 @@
 def serialize_coder(coder):
   from apache_beam.internal import pickler
   return b'%s$%s' % (
-      coder.__class__.__name__.encode('utf-8'), pickler.dumps(coder))
+      coder.__class__.__name__.encode('utf-8'),
+      pickler.dumps(coder, use_zlib=True))
 
 
 def deserialize_coder(serialized):
   from apache_beam.internal import pickler
-  return pickler.loads(serialized.split(b'$', 1)[1])
+  return pickler.loads(serialized.split(b'$', 1)[1], use_zlib=True)
 
 
 # pylint: enable=wrong-import-order, wrong-import-position
@@ -312,11 +313,12 @@
 
   @classmethod
   @overload
-  def register_urn(cls,
-                   urn,  # type: str
-                   parameter_type,  # type: Optional[Type[T]]
-                   fn  # type: Callable[[T, List[Coder], PipelineContext], Any]
-                  ):
+  def register_urn(
+      cls,
+      urn,  # type: str
+      parameter_type,  # type: Optional[Type[T]]
+      fn  # type: Callable[[T, List[Coder], PipelineContext], Any]
+  ):
     # type: (...) -> None
     pass
 
@@ -1455,6 +1457,42 @@
         if payload else StateBackedIterableCoder.DEFAULT_WRITE_THRESHOLD)
 
 
+class ShardedKeyCoder(FastCoder):
+  """A coder for sharded key."""
+  def __init__(self, key_coder):
+    # type: (Coder) -> None
+    self._key_coder = key_coder
+
+  def _get_component_coders(self):
+    # type: () -> List[Coder]
+    return [self._key_coder]
+
+  def _create_impl(self):
+    return coder_impl.ShardedKeyCoderImpl(self._key_coder.get_impl())
+
+  def is_deterministic(self):
+    # type: () -> bool
+    return self._key_coder.is_deterministic()
+
+  def as_cloud_object(self, coders_context=None):
+    return {
+        '@type': 'kind:sharded_key',
+        'component_encodings': [
+            self._key_coder.as_cloud_object(coders_context)
+        ],
+    }
+
+  def __eq__(self, other):
+    return type(self) == type(other) and self._key_coder == other._key_coder
+
+  def __hash__(self):
+    return hash(type(self)) + hash(self._key_coder)
+
+
+Coder.register_structured_urn(
+    common_urns.coders.SHARDED_KEY.urn, ShardedKeyCoder)
+
+
 class CoderElementType(typehints.TypeConstraint):
   """An element type that just holds a coder."""
   def __init__(self, coder):
diff --git a/sdks/python/apache_beam/coders/coders_test_common.py b/sdks/python/apache_beam/coders/coders_test_common.py
index e1ce23b..63ed0ba 100644
--- a/sdks/python/apache_beam/coders/coders_test_common.py
+++ b/sdks/python/apache_beam/coders/coders_test_common.py
@@ -37,6 +37,7 @@
 from apache_beam.transforms.window import GlobalWindow
 from apache_beam.utils import timestamp
 from apache_beam.utils import windowed_value
+from apache_beam.utils.sharded_key import ShardedKey
 from apache_beam.utils.timestamp import MIN_TIMESTAMP
 
 from . import observable
@@ -570,6 +571,41 @@
         }, {}, {i: str(i)
                 for i in range(5000)})
 
+  def test_sharded_key_coder(self):
+    key_and_coders = [(b'', b'\x00', coders.BytesCoder()),
+                      (b'key', b'\x03key', coders.BytesCoder()),
+                      ('key', b'\03\x6b\x65\x79', coders.StrUtf8Coder()),
+                      (('k', 1),
+                       b'\x01\x6b\x01',
+                       coders.TupleCoder(
+                           (coders.StrUtf8Coder(), coders.VarIntCoder())))]
+
+    for key, bytes_repr, key_coder in key_and_coders:
+      coder = coders.ShardedKeyCoder(key_coder)
+      # Verify cloud object representation
+      self.assertEqual({
+          '@type': 'kind:sharded_key',
+          'component_encodings': [key_coder.as_cloud_object()]
+      },
+                       coder.as_cloud_object())
+      self.assertEqual(b'\x00' + bytes_repr, coder.encode(ShardedKey(key, b'')))
+      self.assertEqual(
+          b'\x03123' + bytes_repr, coder.encode(ShardedKey(key, b'123')))
+
+      # Test unnested
+      self.check_coder(coder, ShardedKey(key, b''))
+      self.check_coder(coder, ShardedKey(key, b'123'))
+
+      for other_key, _, other_key_coder in key_and_coders:
+        other_coder = coders.ShardedKeyCoder(other_key_coder)
+        # Test nested
+        self.check_coder(
+            coders.TupleCoder((coder, other_coder)),
+            (ShardedKey(key, b''), ShardedKey(other_key, b'')))
+        self.check_coder(
+            coders.TupleCoder((coder, other_coder)),
+            (ShardedKey(key, b'123'), ShardedKey(other_key, b'')))
+
 
 if __name__ == '__main__':
   logging.getLogger().setLevel(logging.INFO)
diff --git a/sdks/python/apache_beam/coders/standard_coders_test.py b/sdks/python/apache_beam/coders/standard_coders_test.py
index 1ec594d..87320c8 100644
--- a/sdks/python/apache_beam/coders/standard_coders_test.py
+++ b/sdks/python/apache_beam/coders/standard_coders_test.py
@@ -44,6 +44,7 @@
 from apache_beam.transforms.window import IntervalWindow
 from apache_beam.typehints import schemas
 from apache_beam.utils import windowed_value
+from apache_beam.utils.sharded_key import ShardedKey
 from apache_beam.utils.timestamp import Timestamp
 from apache_beam.utils.windowed_value import PaneInfo
 from apache_beam.utils.windowed_value import PaneInfoTiming
@@ -190,6 +191,9 @@
                   x['pane']['index'],
                   x['pane']['on_time_index'])),
       'beam:coder:double:v1': parse_float,
+      'beam:coder:sharded_key:v1': lambda x,
+      value_parser: ShardedKey(
+          key=value_parser(x['key']), shard_id=x['shardId'].encode('utf-8'))
   }
 
   def test_standard_coders(self):
diff --git a/sdks/python/apache_beam/dataframe/frames.py b/sdks/python/apache_beam/dataframe/frames.py
index 1eee5f2..6c748cd 100644
--- a/sdks/python/apache_beam/dataframe/frames.py
+++ b/sdks/python/apache_beam/dataframe/frames.py
@@ -19,6 +19,7 @@
 import collections
 import math
 
+import numpy as np
 import pandas as pd
 
 from apache_beam.dataframe import expressions
@@ -32,9 +33,192 @@
     raise frame_base.WontImplementError(
         'Conversion to a non-deferred a numpy array.')
 
+  @frame_base.args_to_kwargs(pd.DataFrame)
+  @frame_base.populate_defaults(pd.DataFrame)
+  def droplevel(self, level, axis):
+    return frame_base.DeferredFrame.wrap(
+        expressions.ComputedExpression(
+            'droplevel',
+            lambda df: df.droplevel(level, axis=axis), [self._expr],
+            requires_partition_by=partitionings.Nothing(),
+            preserves_partition_by=partitionings.Index()
+            if axis in (1, 'column') else partitionings.Nothing()))
+
+  @frame_base.args_to_kwargs(pd.DataFrame)
+  @frame_base.populate_defaults(pd.DataFrame)
+  @frame_base.maybe_inplace
+  def fillna(self, value, method, axis, **kwargs):
+    if method is not None and axis in (0, 'index'):
+      raise frame_base.WontImplementError('order-sensitive')
+    if isinstance(value, frame_base.DeferredBase):
+      value_expr = value._expr
+    else:
+      value_expr = expressions.ConstantExpression(value)
+    return frame_base.DeferredFrame.wrap(
+        # yapf: disable
+        expressions.ComputedExpression(
+            'fillna',
+            lambda df,
+            value: df.fillna(value, method=method, axis=axis, **kwargs),
+            [self._expr, value_expr],
+            preserves_partition_by=partitionings.Singleton(),
+            requires_partition_by=partitionings.Nothing()))
+
+  @frame_base.args_to_kwargs(pd.DataFrame)
+  @frame_base.populate_defaults(pd.DataFrame)
+  def ffill(self, **kwargs):
+    return self.fillna(method='ffill', **kwargs)
+
+  pad = ffill
+
+  @frame_base.args_to_kwargs(pd.DataFrame)
+  @frame_base.populate_defaults(pd.DataFrame)
+  def groupby(self, by, level, axis, as_index, group_keys, **kwargs):
+    if not as_index:
+      raise NotImplementedError('groupby(as_index=False)')
+    if not group_keys:
+      raise NotImplementedError('groupby(group_keys=False)')
+
+    if axis in (1, 'columns'):
+      return _DeferredGroupByCols(
+          expressions.ComputedExpression(
+              'groupbycols',
+              lambda df: df.groupby(by, axis=axis, **kwargs), [self._expr],
+              requires_partition_by=partitionings.Nothing(),
+              preserves_partition_by=partitionings.Index()))
+
+    if level is None and by is None:
+      raise TypeError("You have to supply one of 'by' and 'level'")
+
+    elif level is not None:
+      if isinstance(level, (list, tuple)):
+        levels = level
+      else:
+        levels = [level]
+      all_levels = self._expr.proxy().index.names
+      levels = [all_levels[i] if isinstance(i, int) else i for i in levels]
+      levels_to_drop = self._expr.proxy().index.names.difference(levels)
+      if levels_to_drop:
+        to_group = self.droplevel(levels_to_drop)._expr
+      else:
+        to_group = self._expr
+
+    elif callable(by):
+
+      def map_index(df):
+        df = df.copy()
+        df.index = df.index.map(by)
+        return df
+
+      to_group = expressions.ComputedExpression(
+          'map_index',
+          map_index, [self._expr],
+          requires_partition_by=partitionings.Nothing(),
+          preserves_partition_by=partitionings.Singleton())
+
+    elif isinstance(by, DeferredSeries):
+
+      if isinstance(self, DeferredSeries):
+
+        def set_index(s, by):
+          df = pd.DataFrame(s)
+          df, by = df.align(by, axis=0)
+          return df.set_index(by).iloc[:, 0]
+
+      else:
+
+        def set_index(df, by):  # type: ignore
+          df, by = df.align(by, axis=0)
+          return df.set_index(by)
+
+      to_group = expressions.ComputedExpression(
+          'set_index',
+          set_index,  #
+          [self._expr, by._expr],
+          requires_partition_by=partitionings.Index(),
+          preserves_partition_by=partitionings.Singleton())
+
+    elif isinstance(by, np.ndarray):
+      raise frame_base.WontImplementError('order sensitive')
+
+    elif isinstance(self, DeferredDataFrame):
+      if not isinstance(by, list):
+        by = [by]
+      index_names = self._expr.proxy().index.names
+      index_names_in_by = list(set(by).intersection(index_names))
+      if index_names_in_by:
+        if set(by) == set(index_names):
+          to_group = self._expr
+        elif set(by).issubset(index_names):
+          to_group = self.droplevel(index_names.difference(by))._expr
+        else:
+          to_group = self.reset_index(index_names_in_by).set_index(by)._expr
+      else:
+        to_group = self.set_index(by)._expr
+
+    else:
+      raise NotImplementedError(by)
+
+    return DeferredGroupBy(
+        expressions.ComputedExpression(
+            'groupbyindex',
+            lambda df: df.groupby(
+                level=list(range(df.index.nlevels)), **kwargs), [to_group],
+            requires_partition_by=partitionings.Index(),
+            preserves_partition_by=partitionings.Singleton()),
+        kwargs)
+
+  abs = frame_base._elementwise_method('abs')
+  astype = frame_base._elementwise_method('astype')
+  copy = frame_base._elementwise_method('copy')
+  get = frame_base.not_implemented_method('get')
+
+  @property
+  def dtype(self):
+    return self._expr.proxy().dtype
+
+  dtypes = dtype
+
 
 @frame_base.DeferredFrame._register_for(pd.Series)
 class DeferredSeries(DeferredDataFrameOrSeries):
+  def __getitem__(self, key):
+    if _is_null_slice(key) or key is Ellipsis:
+      return self
+
+    elif (isinstance(key, int) or _is_integer_slice(key)
+          ) and self._expr.proxy().index._should_fallback_to_positional():
+      raise frame_base.WontImplementError('order sensitive')
+
+    elif isinstance(key, slice) or callable(key):
+      return frame_base.DeferredFrame.wrap(
+          expressions.ComputedExpression(
+              # yapf: disable
+              'getitem',
+              lambda df: df[key],
+              [self._expr],
+              requires_partition_by=partitionings.Nothing(),
+              preserves_partition_by=partitionings.Singleton()))
+
+    elif isinstance(key, DeferredSeries) and key._expr.proxy().dtype == bool:
+      return frame_base.DeferredFrame.wrap(
+          expressions.ComputedExpression(
+              # yapf: disable
+              'getitem',
+              lambda df,
+              indexer: df[indexer],
+              [self._expr, key._expr],
+              requires_partition_by=partitionings.Index(),
+              preserves_partition_by=partitionings.Singleton()))
+
+    elif pd.core.series.is_iterator(key) or pd.core.common.is_bool_indexer(key):
+      raise frame_base.WontImplementError('order sensitive')
+
+    else:
+      # We could consider returning a deferred scalar, but that might
+      # be more surprising than a clear error.
+      raise frame_base.WontImplementError('non-deferred')
+
   @frame_base.args_to_kwargs(pd.Series)
   @frame_base.populate_defaults(pd.Series)
   def align(self, other, join, axis, level, method, **kwargs):
@@ -54,7 +238,7 @@
             preserves_partition_by=partitionings.Index()))
     return aligned.iloc[:, 0], aligned.iloc[:, 1]
 
-  astype = frame_base._elementwise_method('astype')
+  array = property(frame_base.wont_implement_method('non-deferred value'))
 
   between = frame_base._elementwise_method('between')
 
@@ -237,25 +421,8 @@
   isna = frame_base._elementwise_method('isna')
   notnull = notna = frame_base._elementwise_method('notna')
 
-  @frame_base.args_to_kwargs(pd.Series)
-  @frame_base.populate_defaults(pd.Series)
-  @frame_base.maybe_inplace
-  def fillna(self, value, method):
-    if method is not None:
-      raise frame_base.WontImplementError('order-sensitive')
-    if isinstance(value, frame_base.DeferredBase):
-      value_expr = value._expr
-    else:
-      value_expr = expressions.ConstantExpression(value)
-    return frame_base.DeferredFrame.wrap(
-        expressions.ComputedExpression(
-            'fillna',
-            lambda df,
-            value: df.fillna(value, method=method), [self._expr, value_expr],
-            preserves_partition_by=partitionings.Singleton(),
-            requires_partition_by=partitionings.Nothing()))
-
   reindex = frame_base.not_implemented_method('reindex')
+  rolling = frame_base.not_implemented_method('rolling')
 
   to_numpy = to_string = frame_base.wont_implement_method('non-deferred value')
 
@@ -291,6 +458,12 @@
 
   agg = aggregate
 
+  @property
+  def axes(self):
+    return [self.index]
+
+  clip = frame_base._elementwise_method('clip')
+
   all = frame_base._agg_method('all')
   any = frame_base._agg_method('any')
   min = frame_base._agg_method('min')
@@ -304,6 +477,8 @@
 
   head = tail = frame_base.wont_implement_method('order-sensitive')
 
+  filter = frame_base._elementwise_method('filter')
+
   memory_usage = frame_base.wont_implement_method('non-deferred value')
 
   # In Series __contains__ checks the index
@@ -354,6 +529,9 @@
               preserves_partition_by=partitionings.Singleton(),
               requires_partition_by=partitionings.Singleton()))
 
+  plot = frame_base.wont_implement_method('plot')
+  pop = frame_base.wont_implement_method('non-lazy')
+
   rename_axis = frame_base._elementwise_method('rename_axis')
 
   @frame_base.args_to_kwargs(pd.Series)
@@ -427,18 +605,6 @@
   def columns(self):
     return self._expr.proxy().columns
 
-  def groupby(self, by):
-    # TODO: what happens to the existing index?
-    # We set the columns to index as we have a notion of being partitioned by
-    # index, but not partitioned by an arbitrary subset of columns.
-    return DeferredGroupBy(
-        expressions.ComputedExpression(
-            'groupbyindex',
-            lambda df: df.groupby(level=list(range(df.index.nlevels))),
-            [self.set_index(by)._expr],
-            requires_partition_by=partitionings.Index(),
-            preserves_partition_by=partitionings.Singleton()))
-
   def __getattr__(self, name):
     # Column attribute access.
     if name in self._expr.proxy().columns:
@@ -448,17 +614,31 @@
 
   def __getitem__(self, key):
     # TODO: Replicate pd.DataFrame.__getitem__ logic
-    if isinstance(key, frame_base.DeferredBase):
+    if isinstance(key, DeferredSeries) and key._expr.proxy().dtype == bool:
+      return self.loc[key]
+
+    elif isinstance(key, frame_base.DeferredBase):
       # Fail early if key is a DeferredBase as it interacts surprisingly with
       # key in self._expr.proxy().columns
       raise NotImplementedError(
-          "Indexing with a deferred frame is not yet supported. Consider "
-          "using df.loc[...]")
+          "Indexing with a non-bool deferred frame is not yet supported. "
+          "Consider using df.loc[...]")
 
-    if (isinstance(key, list) and
-        all(key_column in self._expr.proxy().columns
-            for key_column in key)) or key in self._expr.proxy().columns:
+    elif isinstance(key, slice):
+      if _is_null_slice(key):
+        return self
+      elif _is_integer_slice(key):
+        # This depends on the contents of the index.
+        raise frame_base.WontImplementError(
+            'Use iloc or loc with integer slices.')
+      else:
+        return self.loc[key]
+
+    elif (isinstance(key, list) and
+          all(key_column in self._expr.proxy().columns
+              for key_column in key)) or key in self._expr.proxy().columns:
       return self._elementwise(lambda df: df[key], 'get_column')
+
     else:
       raise NotImplementedError(key)
 
@@ -467,7 +647,11 @@
     return self._expr.proxy().__contains__(key)
 
   def __setitem__(self, key, value):
-    if isinstance(key, str):
+    if isinstance(
+        key, str) or (isinstance(key, list) and
+                      all(isinstance(c, str)
+                          for c in key)) or (isinstance(key, DeferredSeries) and
+                                             key._expr.proxy().dtype == bool):
       # yapf: disable
       return self._elementwise(
           lambda df, key, value: df.__setitem__(key, value),
@@ -503,13 +687,19 @@
   def iloc(self):
     return _DeferredILoc(self)
 
-  _get_index = _set_index = frame_base.not_implemented_method('index')
-  index = property(_get_index, _set_index)
+  def _get_index(self):
+    return _DeferredIndex(self)
+
+  index = property(_get_index, frame_base.not_implemented_method('index'))
 
   @property
   def axes(self):
     return (self.index, self.columns)
 
+  @property
+  def dtypes(self):
+    return self._expr.proxy().dtypes
+
   def assign(self, **kwargs):
     for name, value in kwargs.items():
       if not callable(value) and not isinstance(value, DeferredSeries):
@@ -601,6 +791,9 @@
   all = frame_base._agg_method('all')
   any = frame_base._agg_method('any')
 
+  clip = frame_base._elementwise_method(
+      'clip', restrictions={'axis': lambda axis: axis in (0, 'index')})
+
   @frame_base.args_to_kwargs(pd.DataFrame)
   @frame_base.populate_defaults(pd.DataFrame)
   def corr(self, method, min_periods):
@@ -742,25 +935,6 @@
             preserves_partition_by=partitionings.Singleton(),
             requires_partition_by=requires_partition_by))
 
-  @frame_base.args_to_kwargs(pd.DataFrame)
-  @frame_base.populate_defaults(pd.DataFrame)
-  @frame_base.maybe_inplace
-  def fillna(self, value, method, axis, **kwargs):
-    if method is not None and axis in (0, 'index'):
-      raise frame_base.WontImplementError('order-sensitive')
-    if isinstance(value, frame_base.DeferredBase):
-      value_expr = value._expr
-    else:
-      value_expr = expressions.ConstantExpression(value)
-    return frame_base.DeferredFrame.wrap(
-        expressions.ComputedExpression(
-            'fillna',
-            lambda df, value: df.fillna(
-                value, method=method, axis=axis, **kwargs),
-            [self._expr, value_expr],
-            preserves_partition_by=partitionings.Singleton(),
-            requires_partition_by=partitionings.Nothing()))
-
   isna = frame_base._elementwise_method('isna')
   notnull = notna = frame_base._elementwise_method('notna')
 
@@ -945,6 +1119,18 @@
             preserves_partition_by=partitionings.Singleton(),
             requires_partition_by=requires_partition_by))
 
+  plot = frame_base.wont_implement_method('plot')
+
+  def pop(self, item):
+    result = self[item]
+    self._expr = expressions.ComputedExpression(
+            'popped',
+            lambda df: (df.pop(item), df)[-1],
+            [self._expr],
+            preserves_partition_by=partitionings.Singleton(),
+            requires_partition_by=partitionings.Nothing())
+    return result
+
   prod = product = frame_base._agg_method('prod')
 
   @frame_base.args_to_kwargs(pd.DataFrame)
@@ -1016,7 +1202,7 @@
   def reset_index(self, level=None, **kwargs):
     if level is not None and not isinstance(level, (tuple, list)):
       level = [level]
-    if level is None or len(level) == len(self._expr.proxy().index.levels):
+    if level is None or len(level) == self._expr.proxy().index.nlevels:
       # TODO: Could do distributed re-index with offsets.
       requires_partition_by = partitionings.Singleton()
     else:
@@ -1102,6 +1288,8 @@
       requires_partition_by=partitionings.Index(),
       preserves_partition_by=partitionings.Index())
 
+  values = property(frame_base.wont_implement_method('non-deferred value'))
+
 
 for io_func in dir(io):
   if io_func.startswith('to_'):
@@ -1113,6 +1301,10 @@
 
 
 class DeferredGroupBy(frame_base.DeferredFrame):
+  def __init__(self, expr, kwargs):
+    super(DeferredGroupBy, self).__init__(expr)
+    self._kwargs = kwargs
+
   def agg(self, fn):
     if not callable(fn):
       # TODO: Add support for strings in (UN)LIFTABLE_AGGREGATIONS. Test by
@@ -1126,22 +1318,37 @@
             requires_partition_by=partitionings.Index(),
             preserves_partition_by=partitionings.Singleton()))
 
+  aggregate = agg
+
+  first = last = head = tail = frame_base.not_implemented_method(
+      'order sensitive')
+
+  # TODO(robertwb): Consider allowing this for categorical keys.
+  __len__ = frame_base.wont_implement_method('non-deferred')
+  get_group = __getitem__ = frame_base.not_implemented_method('get_group')
+  groups = property(frame_base.wont_implement_method('non-deferred'))
+
 
 def _liftable_agg(meth):
   name, func = frame_base.name_and_func(meth)
 
-  def wrapper(self, *args, **kargs):
+  def wrapper(self, *args, **kwargs):
     assert isinstance(self, DeferredGroupBy)
     ungrouped = self._expr.args()[0]
+    groupby_kwargs = self._kwargs
     pre_agg = expressions.ComputedExpression(
         'pre_combine_' + name,
-        lambda df: func(df.groupby(level=list(range(df.index.nlevels)))),
+        lambda df: func(
+            df.groupby(level=list(range(df.index.nlevels)), **groupby_kwargs),
+            **kwargs),
         [ungrouped],
         requires_partition_by=partitionings.Nothing(),
         preserves_partition_by=partitionings.Singleton())
     post_agg = expressions.ComputedExpression(
         'post_combine_' + name,
-        lambda df: func(df.groupby(level=list(range(df.index.nlevels)))),
+        lambda df: func(
+            df.groupby(level=list(range(df.index.nlevels)), **groupby_kwargs),
+            **kwargs),
         [pre_agg],
         requires_partition_by=partitionings.Index(),
         preserves_partition_by=partitionings.Singleton())
@@ -1153,12 +1360,15 @@
 def _unliftable_agg(meth):
   name, func = frame_base.name_and_func(meth)
 
-  def wrapper(self, *args, **kargs):
+  def wrapper(self, *args, **kwargs):
     assert isinstance(self, DeferredGroupBy)
     ungrouped = self._expr.args()[0]
+    groupby_kwargs = self._kwargs
     post_agg = expressions.ComputedExpression(
         name,
-        lambda df: func(df.groupby(level=list(range(df.index.nlevels)))),
+        lambda df: func(
+            df.groupby(level=list(range(df.index.nlevels)), **groupby_kwargs),
+            **kwargs),
         [ungrouped],
         requires_partition_by=partitionings.Index(),
         preserves_partition_by=partitionings.Singleton())
@@ -1181,6 +1391,99 @@
       and func.__module__ in ('numpy', 'builtins'))
 
 
+class _DeferredGroupByCols(frame_base.DeferredFrame):
+  # It's not clear that all of these make sense in Pandas either...
+  agg = aggregate = frame_base._elementwise_method('agg')
+  any = frame_base._elementwise_method('any')
+  all = frame_base._elementwise_method('all')
+  apply = frame_base.not_implemented_method('apply')
+  backfill = bfill = frame_base.not_implemented_method('backfill')
+  boxplot = frame_base.wont_implement_method('plot')
+  corr = frame_base.not_implemented_method('corr')
+  corrwith = frame_base.not_implemented_method('corrwith')
+  cov = frame_base.not_implemented_method('cov')
+  cumcount = cummax = cummin = cumprod = cumsum = (
+      frame_base.not_implemented_method('cum*'))
+  describe = frame_base.wont_implement_method('describe')
+  diff = frame_base._elementwise_method('diff')
+  dtypes = frame_base.not_implemented_method('dtypes')
+  expanding = frame_base.not_implemented_method('expanding')
+  ffill = frame_base.not_implemented_method('ffill')
+  fillna = frame_base._elementwise_method('fillna')
+  filter = frame_base._elementwise_method('filter')
+  first = frame_base.wont_implement_method('order sensitive')
+  get_group = frame_base._elementwise_method('group')
+  head = frame_base.wont_implement_method('order sensitive')
+  hist = frame_base.wont_implement_method('plot')
+  idxmax = frame_base._elementwise_method('idxmax')
+  idxmin = frame_base._elementwise_method('idxmin')
+  last = frame_base.wont_implement_method('order sensitive')
+  mad = frame_base._elementwise_method('mad')
+  max = frame_base._elementwise_method('max')
+  mean = frame_base._elementwise_method('mean')
+  median = frame_base._elementwise_method('median')
+  min = frame_base._elementwise_method('min')
+  nth = frame_base.not_implemented_method('nth')
+  nunique = frame_base._elementwise_method('nunique')
+  ohlc = frame_base.not_implemented_method('ohlc')
+  pad = frame_base.not_implemented_method('pad')
+  pct_change = frame_base.not_implemented_method('pct_change')
+  pipe = frame_base.not_implemented_method('pipe')
+  plot = frame_base.wont_implement_method('plot')
+  prod = frame_base._elementwise_method('prod')
+  quantile = frame_base._elementwise_method('quantile')
+  rank = frame_base.not_implemented_method('rank')
+  resample = frame_base.not_implemented_method('resample')
+  rolling = frame_base.not_implemented_method('rolling')
+  sample = frame_base.not_implemented_method('sample')
+  shift = frame_base._elementwise_method('shift')
+  size = frame_base._elementwise_method('size')
+  skew = frame_base._elementwise_method('skew')
+  std = frame_base._elementwise_method('std')
+  sum = frame_base._elementwise_method('sum')
+  tail = frame_base.wont_implement_method('order sensitive')
+  take = frame_base.wont_implement_method('deprectated')
+  transform = frame_base.not_implemented_method('transform')
+  tshift = frame_base._elementwise_method('tshift')
+  var = frame_base._elementwise_method('var')
+
+  @property
+  def groups(self):
+    return self._expr.proxy().groups
+
+  @property
+  def indices(self):
+    return self._expr.proxy().indices
+
+  @property
+  def ndim(self):
+    return self._expr.proxy().ndim
+
+  @property
+  def ngroups(self):
+    return self._expr.proxy().ngroups
+
+
+class _DeferredIndex(object):
+  def __init__(self, frame):
+    self._frame = frame
+
+  @property
+  def names(self):
+    return self._frame._expr.proxy().index.names
+
+  @property
+  def ndim(self):
+    return self._frame._expr.proxy().index.ndim
+
+  @property
+  def nlevels(self):
+    return self._frame._expr.proxy().index.nlevels
+
+  def __getattr__(self, name):
+    raise NotImplementedError('index.%s' % name)
+
+
 class _DeferredLoc(object):
   def __init__(self, frame):
     self._frame = frame
@@ -1232,6 +1535,8 @@
                 else partitionings.Nothing()),
             preserves_partition_by=partitionings.Singleton()))
 
+  __setitem__ = frame_base.not_implemented_method('loc.setitem')
+
 class _DeferredILoc(object):
   def __init__(self, frame):
     self._frame = frame
@@ -1251,6 +1556,8 @@
     else:
       raise frame_base.WontImplementError('order-sensitive')
 
+  __setitem__ = frame_base.wont_implement_method('iloc.setitem')
+
 
 class _DeferredStringMethods(frame_base.DeferredBase):
   @frame_base.args_to_kwargs(pd.core.strings.StringMethods)
@@ -1401,3 +1708,24 @@
 for name in ['__lt__', '__le__', '__gt__', '__ge__', '__eq__', '__ne__']:
   setattr(DeferredSeries, name, frame_base._elementwise_method(name))
   setattr(DeferredDataFrame, name, frame_base._elementwise_method(name))
+
+for name in ['__neg__', '__pos__', '__invert__']:
+  setattr(DeferredSeries, name, frame_base._elementwise_method(name))
+  setattr(DeferredDataFrame, name, frame_base._elementwise_method(name))
+
+DeferredSeries.multiply = DeferredSeries.mul  # type: ignore
+DeferredDataFrame.multiply = DeferredDataFrame.mul  # type: ignore
+
+
+def _slice_parts(s):
+  yield s.start
+  yield s.stop
+  yield s.step
+
+def _is_null_slice(s):
+  return isinstance(s, slice) and all(x is None for x in _slice_parts(s))
+
+def _is_integer_slice(s):
+  return isinstance(s, slice) and all(
+      x is None or isinstance(x, int)
+      for x in _slice_parts(s)) and not _is_null_slice(s)
diff --git a/sdks/python/apache_beam/dataframe/frames_test.py b/sdks/python/apache_beam/dataframe/frames_test.py
index 3cd77ab..ebd1051 100644
--- a/sdks/python/apache_beam/dataframe/frames_test.py
+++ b/sdks/python/apache_beam/dataframe/frames_test.py
@@ -111,6 +111,27 @@
           df1,
           df2)
 
+  def test_series_getitem(self):
+    s = pd.Series([x**2 for x in range(10)])
+    self._run_test(lambda s: s[...], s, distributed=True)
+    self._run_test(lambda s: s[:], s, distributed=True)
+    self._run_test(lambda s: s[s < 10], s, distributed=True)
+    self._run_test(lambda s: s[lambda s: s < 10], s, distributed=True)
+
+    s.index = s.index.map(float)
+    self._run_test(lambda s: s[1.5:6], s, distributed=True)
+
+  def test_dataframe_getitem(self):
+    df = pd.DataFrame({'A': [x**2 for x in range(6)], 'B': list('abcdef')})
+    self._run_test(lambda df: df['A'], df, distributed=True)
+    self._run_test(lambda df: df[['A', 'B']], df, distributed=True)
+
+    self._run_test(lambda df: df[:], df, distributed=True)
+    self._run_test(lambda df: df[df.A < 10], df, distributed=True)
+
+    df.index = df.index.map(float)
+    self._run_test(lambda df: df[1.5:4], df, distributed=True)
+
   def test_loc(self):
     dates = pd.date_range('1/1/2000', periods=8)
     df = pd.DataFrame(
diff --git a/sdks/python/apache_beam/dataframe/pandas_doctests_test.py b/sdks/python/apache_beam/dataframe/pandas_doctests_test.py
index 2d412c9..40d0f75 100644
--- a/sdks/python/apache_beam/dataframe/pandas_doctests_test.py
+++ b/sdks/python/apache_beam/dataframe/pandas_doctests_test.py
@@ -79,7 +79,6 @@
         not_implemented_ok={
             'pandas.core.frame.DataFrame.isin': ['*'],
             'pandas.core.frame.DataFrame.melt': ['*'],
-            'pandas.core.frame.DataFrame.axes': ['*'],
             'pandas.core.frame.DataFrame.count': ['*'],
             'pandas.core.frame.DataFrame.reindex': ['*'],
             'pandas.core.frame.DataFrame.reindex_axis': ['*'],
@@ -88,11 +87,6 @@
             # columns
             'pandas.core.frame.DataFrame.pivot': ['*'],
 
-            # DataFrame.__getitem__ cannot be used as loc
-            'pandas.core.frame.DataFrame.query': [
-                'df[df.A > df.B]', "df[df.B == df['C C']]"
-            ],
-
             # We can implement this as a zipping operator, but it won't have the
             # same capability. The doctest includes an example that branches on
             # a deferred result.
@@ -122,6 +116,10 @@
             'pandas.core.frame.DataFrame.append': ['*'],
         },
         skip={
+            'pandas.core.frame.DataFrame.axes': [
+                # Returns deferred index.
+                'df.axes',
+            ],
             'pandas.core.frame.DataFrame.compare': ['*'],
             'pandas.core.frame.DataFrame.cov': [
                 # Relies on setting entries ahead of time.
@@ -131,15 +129,13 @@
             ],
             'pandas.core.frame.DataFrame.drop_duplicates': ['*'],
             'pandas.core.frame.DataFrame.duplicated': ['*'],
-            'pandas.core.frame.DataFrame.groupby': [
-                'df.groupby(level=0).mean()',
-                'df.groupby(level="Type").mean()',
-                'df.groupby(by=["b"], dropna=False).sum()',
-                'df.groupby(by="a", dropna=False).sum()'
-            ],
             'pandas.core.frame.DataFrame.idxmax': ['*'],
             'pandas.core.frame.DataFrame.idxmin': ['*'],
-            'pandas.core.frame.DataFrame.pop': ['*'],
+            'pandas.core.frame.DataFrame.rename': [
+                # Returns deferred index.
+                'df.index',
+                'df.rename(index=str).index',
+            ],
             'pandas.core.frame.DataFrame.set_axis': ['*'],
             'pandas.core.frame.DataFrame.sort_index': ['*'],
             'pandas.core.frame.DataFrame.to_markdown': ['*'],
@@ -186,10 +182,6 @@
             ],
             'pandas.core.frame.DataFrame.to_sparse': ['type(df)'],
 
-            # DeferredSeries has no attribute dtype. Should we allow this and
-            # defer to proxy?
-            'pandas.core.frame.DataFrame.iterrows': ["print(df['int'].dtype)"],
-
             # Skipped because "seen_wont_implement" is reset before getting to
             # these calls, so the NameError they raise is not ignored.
             'pandas.core.frame.DataFrame.T': [
@@ -208,6 +200,7 @@
         report=True,
         wont_implement_ok={
             'pandas.core.series.Series.__array__': ['*'],
+            'pandas.core.series.Series.array': ['*'],
             'pandas.core.series.Series.cummax': ['*'],
             'pandas.core.series.Series.cummin': ['*'],
             'pandas.core.series.Series.cumsum': ['*'],
@@ -230,6 +223,7 @@
                 "s.nsmallest(3)",
                 "s.nsmallest(3, keep='last')",
             ],
+            'pandas.core.series.Series.pop': ['*'],
             'pandas.core.series.Series.searchsorted': ['*'],
             'pandas.core.series.Series.shift': ['*'],
             'pandas.core.series.Series.take': ['*'],
@@ -240,10 +234,14 @@
             'pandas.core.series.Series.view': ['*'],
         },
         not_implemented_ok={
+            'pandas.core.series.Series.groupby': [
+                'ser.groupby(["a", "b", "a", "b"]).mean()',
+                'ser.groupby(["a", "b", "a", np.nan]).mean()',
+                'ser.groupby(["a", "b", "a", np.nan], dropna=False).mean()',
+            ],
             'pandas.core.series.Series.reindex': ['*'],
         },
         skip={
-            'pandas.core.series.Series.array': ['*'],
             'pandas.core.series.Series.append': ['*'],
             'pandas.core.series.Series.argmax': ['*'],
             'pandas.core.series.Series.argmin': ['*'],
@@ -260,13 +258,12 @@
             'pandas.core.series.Series.drop_duplicates': ['*'],
             'pandas.core.series.Series.duplicated': ['*'],
             'pandas.core.series.Series.explode': ['*'],
-            'pandas.core.series.Series.groupby': ['*'],
             'pandas.core.series.Series.idxmax': ['*'],
             'pandas.core.series.Series.idxmin': ['*'],
             'pandas.core.series.Series.name': ['*'],
             'pandas.core.series.Series.nonzero': ['*'],
-            'pandas.core.series.Series.pop': ['*'],
             'pandas.core.series.Series.quantile': ['*'],
+            'pandas.core.series.Series.pop': ['ser'],  # testing side effect
             'pandas.core.series.Series.rename': ['*'],
             'pandas.core.series.Series.repeat': ['*'],
             'pandas.core.series.Series.replace': ['*'],
diff --git a/sdks/python/apache_beam/examples/snippets/snippets.py b/sdks/python/apache_beam/examples/snippets/snippets.py
index 96e4820..c4a569d 100644
--- a/sdks/python/apache_beam/examples/snippets/snippets.py
+++ b/sdks/python/apache_beam/examples/snippets/snippets.py
@@ -1180,6 +1180,36 @@
       create_disposition=beam.io.BigQueryDisposition.CREATE_IF_NEEDED)
   # [END model_bigqueryio_write]
 
+  # [START model_bigqueryio_write_dynamic_destinations]
+  fictional_characters_view = beam.pvalue.AsDict(
+      p | 'CreateCharacters' >> beam.Create([('Yoda', True),
+                                             ('Obi Wan Kenobi', True)]))
+
+  def table_fn(element, fictional_characters):
+    if element in fictional_characters:
+      return 'my_dataset.fictional_quotes'
+    else:
+      return 'my_dataset.real_quotes'
+
+  quotes | 'WriteWithDynamicDestination' >> beam.io.WriteToBigQuery(
+      table_fn,
+      schema=table_schema,
+      table_side_inputs=(fictional_characters_view, ),
+      write_disposition=beam.io.BigQueryDisposition.WRITE_TRUNCATE,
+      create_disposition=beam.io.BigQueryDisposition.CREATE_IF_NEEDED)
+  # [END model_bigqueryio_write_dynamic_destinations]
+
+  # [START model_bigqueryio_time_partitioning]
+  quotes | 'WriteWithTimePartitioning' >> beam.io.WriteToBigQuery(
+      table_spec,
+      schema=table_schema,
+      write_disposition=beam.io.BigQueryDisposition.WRITE_TRUNCATE,
+      create_disposition=beam.io.BigQueryDisposition.CREATE_IF_NEEDED,
+      additional_bq_parameters={'timePartitioning': {
+          'type': 'HOUR'
+      }})
+  # [END model_bigqueryio_time_partitioning]
+
 
 def model_composite_transform_example(contents, output_path):
   """Example of a composite transform.
@@ -1609,3 +1639,210 @@
         | 'Parse adjacency list to JSON' >> beam.Map(json.dumps)
         | 'Write adjacency list' >> beam.io.WriteToText('adjancency_list.txt'))
   # [END nlp_analyze_text]
+
+
+def sdf_basic_example():
+  import os
+  from apache_beam.io.restriction_trackers import OffsetRange
+  read_next_record = None
+
+  # [START SDF_BasicExample]
+  class FileToWordsRestrictionProvider(beam.io.RestrictionProvider):
+    def initial_restriction(self, file_name):
+      return OffsetRange(0, os.stat(file_name).st_size)
+
+    def create_tracker(self, restriction):
+      return beam.io.restriction_trackers.OffsetRestrictionTracker()
+
+  class FileToWordsFn(beam.DoFn):
+    def process(
+        self,
+        file_name,
+        tracker=beam.DoFn.RestrictionParam(FileToWordsRestrictionProvider())):
+      with open(file_name) as file_handle:
+        file_handle.seek(tracker.current_restriction.start())
+        while tracker.try_claim(file_handle.tell()):
+          yield read_next_record(file_handle)
+
+    # Providing the coder is only necessary if it can not be inferred at
+    # runtime.
+    def restriction_coder(self):
+      return ...
+
+  # [END SDF_BasicExample]
+
+
+def sdf_basic_example_with_splitting():
+  from apache_beam.io.restriction_trackers import OffsetRange
+
+  # [START SDF_BasicExampleWithSplitting]
+  class FileToWordsRestrictionProvider(beam.io.RestrictionProvider):
+    def split(self, file_name, restriction):
+      # Compute and output 64 MiB size ranges to process in parallel
+      split_size = 64 * (1 << 20)
+      i = restriction.start
+      while i < restriction.end - split_size:
+        yield OffsetRange(i, i + split_size)
+        i += split_size
+      yield OffsetRange(i, restriction.end)
+
+  # [END SDF_BasicExampleWithSplitting]
+
+
+def sdf_sdk_initiated_checkpointing():
+  timestamp = None
+  external_service = None
+
+  class MyRestrictionProvider(object):
+    pass
+
+  # [START SDF_UserInitiatedCheckpoint]
+  class MySplittableDoFn(beam.DoFn):
+    def process(
+        self,
+        element,
+        restriction_tracker=beam.DoFn.RestrictionParam(
+            MyRestrictionProvider())):
+      current_position = restriction_tracker.current_restriction.start()
+      while True:
+        # Pull records from an external service.
+        try:
+          records = external_service.fetch(current_position)
+          if records.empty():
+            # Set a shorter delay in case we are being throttled.
+            restriction_tracker.defer_remainder(timestamp.Duration(second=10))
+            return
+          for record in records:
+            if restriction_tracker.try_claim(record.position):
+              current_position = record.position
+              yield record
+            else:
+              return
+        except TimeoutError:
+          # Set a longer delay in case we are being throttled.
+          restriction_tracker.defer_remainder(timestamp.Duration(seconds=60))
+          return
+
+  # [END SDF_UserInitiatedCheckpoint]
+
+
+def sdf_get_size():
+  # [START SDF_GetSize]
+  # The RestrictionProvider is responsible for calculating the size of given
+  # restriction.
+  class MyRestrictionProvider(beam.transforms.core.RestrictionProvider):
+    def restriction_size(self, file_name, restriction):
+      weight = 2 if "expensiveRecords" in file_name else 1
+      return restriction.size() * weight
+
+  # [END SDF_GetSize]
+
+
+def sdf_bad_try_claim_loop():
+  class FileToWordsRestrictionProvider(object):
+    pass
+
+  read_next_record = None
+
+  # [START SDF_BadTryClaimLoop]
+  class BadTryClaimLoop(beam.DoFn):
+    def process(
+        self,
+        file_name,
+        tracker=beam.DoFn.RestrictionParam(FileToWordsRestrictionProvider())):
+      with open(file_name) as file_handle:
+        file_handle.seek(tracker.current_restriction.start())
+        # The restriction tracker can be modified by another thread in parallel
+        # so storing state locally is ill advised.
+        end = tracker.current_restriction.end()
+        while file_handle.tell() < end:
+          # Only after successfully claiming should we produce any output and/or
+          # perform side effects.
+          tracker.try_claim(file_handle.tell())
+          yield read_next_record(file_handle)
+
+  # [END SDF_BadTryClaimLoop]
+
+
+def sdf_custom_watermark_estimator():
+  from apache_beam.io.iobase import WatermarkEstimator
+  from apache_beam.transforms.core import WatermarkEstimatorProvider
+  current_watermark = None
+
+  class MyRestrictionProvider(object):
+    pass
+
+  # [START SDF_CustomWatermarkEstimator]
+  # (Optional) Define a custom watermark state type to save information between
+  # bundle processing rounds.
+  class MyCustomerWatermarkEstimatorState(object):
+    def __init__(self, element, restriction):
+      # Store data necessary for future watermark computations
+      pass
+
+  # Define a WatermarkEstimator
+  class MyCustomWatermarkEstimator(WatermarkEstimator):
+    def __init__(self, estimator_state):
+      self.state = estimator_state
+
+    def observe_timestamp(self, timestamp):
+      # Will be invoked on each output from the SDF
+      pass
+
+    def current_watermark(self):
+      # Return a monotonically increasing value
+      return current_watermark
+
+    def get_estimator_state(self):
+      # Return state to resume future watermark estimation after a
+      # checkpoint/split
+      return self.state
+
+  # Then, a WatermarkEstimatorProvider needs to be created for this
+  # WatermarkEstimator
+  class MyWatermarkEstimatorProvider(WatermarkEstimatorProvider):
+    def initial_estimator_state(self, element, restriction):
+      return MyCustomerWatermarkEstimatorState(element, restriction)
+
+    def create_watermark_estimator(self, estimator_state):
+      return MyCustomWatermarkEstimator(estimator_state)
+
+  # Finally, define the SDF using your estimator.
+  class MySplittableDoFn(beam.DoFn):
+    def process(
+        self,
+        element,
+        restriction_tracker=beam.DoFn.RestrictionParam(MyRestrictionProvider()),
+        watermark_estimator=beam.DoFn.WatermarkEstimatorParam(
+            MyWatermarkEstimatorProvider())):
+      # The current watermark can be inspected.
+      watermark_estimator.current_watermark()
+
+  # [END SDF_CustomWatermarkEstimator]
+
+
+def sdf_truncate():
+  # [START SDF_Truncate]
+  class MyRestrictionProvider(beam.transforms.core.RestrictionProvider):
+    def truncate(self, file_name, restriction):
+      if "optional" in file_name:
+        # Skip optional files
+        return None
+      return restriction
+
+  # [END SDF_Truncate]
+
+
+def bundle_finalize():
+  my_callback_func = None
+
+  # [START BundleFinalize]
+  class MySplittableDoFn(beam.DoFn):
+    def process(self, element, bundle_finalizer=beam.DoFn.BundleFinalizerParam):
+      # ... produce output ...
+
+      # Register callback function for this bundle that performs the side
+      # effect.
+      bundle_finalizer.register(my_callback_func)
+
+  # [END BundleFinalize]
diff --git a/sdks/python/apache_beam/internal/metrics/__init__.py b/sdks/python/apache_beam/internal/metrics/__init__.py
new file mode 100644
index 0000000..84381ed
--- /dev/null
+++ b/sdks/python/apache_beam/internal/metrics/__init__.py
@@ -0,0 +1,20 @@
+#
+# 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.
+#
+
+"""For internal use only; no backwards-compatibility guarantees."""
+
+from __future__ import absolute_import
diff --git a/sdks/python/apache_beam/internal/metrics/cells.py b/sdks/python/apache_beam/internal/metrics/cells.py
new file mode 100644
index 0000000..44ee395
--- /dev/null
+++ b/sdks/python/apache_beam/internal/metrics/cells.py
@@ -0,0 +1,190 @@
+#
+# 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.
+#
+
+"""
+This file contains internal metric cell classes. A metric cell is used to
+accumulate in-memory changes to a metric. It represents a specific metric
+in a single context.
+
+For internal use only. No backwards compatibility guarantees.
+"""
+
+# pytype: skip-file
+
+from __future__ import absolute_import
+from __future__ import division
+
+from builtins import object
+from typing import TYPE_CHECKING
+from typing import Optional
+
+from apache_beam.metrics.cells import MetricAggregator
+from apache_beam.metrics.cells import MetricCell
+from apache_beam.metrics.cells import MetricCellFactory
+from apache_beam.utils.histogram import Histogram
+
+if TYPE_CHECKING:
+  from apache_beam.utils.histogram import BucketType
+
+
+class HistogramCell(MetricCell):
+  """For internal use only; no backwards-compatibility guarantees.
+
+  Tracks the current value and delta for a histogram metric.
+
+  Each cell tracks the state of a metric independently per context per bundle.
+  Therefore, each metric has a different cell in each bundle, that is later
+  aggregated.
+
+  This class is thread safe since underlying histogram object is thread safe.
+  """
+  def __init__(self, bucket_type):
+    self._bucket_type = bucket_type
+    self.data = HistogramAggregator(bucket_type).identity_element()
+
+  def reset(self):
+    self.data = HistogramAggregator(self._bucket_type).identity_element()
+
+  def combine(self, other):
+    # type: (HistogramCell) -> HistogramCell
+    result = HistogramCell(self._bucket_type)
+    result.data = self.data.combine(other.data)
+    return result
+
+  def update(self, value):
+    self.data.histogram.record(value)
+
+  def get_cumulative(self):
+    # type: () -> HistogramData
+    return self.data.get_cumulative()
+
+  def to_runner_api_monitoring_info(self, name, transform_id):
+    # Histogram metric is currently worker-local and internal
+    # use only. This method should be implemented when runners
+    # support Histogram metric reporting.
+    return None
+
+
+class HistogramCellFactory(MetricCellFactory):
+  def __init__(self, bucket_type):
+    self._bucket_type = bucket_type
+
+  def __call__(self):
+    return HistogramCell(self._bucket_type)
+
+  def __eq__(self, other):
+    if not isinstance(other, HistogramCellFactory):
+      return False
+    return self._bucket_type == other._bucket_type
+
+  def __hash__(self):
+    return hash(self._bucket_type)
+
+
+class HistogramResult(object):
+  def __init__(self, data):
+    # type: (HistogramData) -> None
+    self.data = data
+
+  def __eq__(self, other):
+    if isinstance(other, HistogramResult):
+      return self.data == other.data
+    else:
+      return False
+
+  def __hash__(self):
+    return hash(self.data)
+
+  def __ne__(self, other):
+    # TODO(BEAM-5949): Needed for Python 2 compatibility.
+    return not self == other
+
+  def __repr__(self):
+    return '<HistogramResult({})>'.format(
+        self.data.histogram.get_percentile_info())
+
+  @property
+  def p99(self):
+    return self.data.histogram.p99()
+
+  @property
+  def p95(self):
+    return self.data.histogram.p95()
+
+  @property
+  def p90(self):
+    return self.data.histogram.p90()
+
+
+class HistogramData(object):
+  """For internal use only; no backwards-compatibility guarantees.
+
+  The data structure that holds data about a histogram metric.
+
+  This object is not thread safe, so it's not supposed to be modified
+  outside the HistogramCell.
+  """
+  def __init__(self, histogram):
+    self.histogram = histogram
+
+  def __eq__(self, other):
+    return self.histogram == other.histogram
+
+  def __hash__(self):
+    return hash(self.histogram)
+
+  def __ne__(self, other):
+    # TODO(BEAM-5949): Needed for Python 2 compatibility.
+    return not self == other
+
+  def __repr__(self):
+    return 'HistogramData({})'.format(self.histogram.get_percentile_info())
+
+  def get_cumulative(self):
+    # type: () -> HistogramData
+    return HistogramData(self.histogram)
+
+  def combine(self, other):
+    # type: (Optional[HistogramData]) -> HistogramData
+    if other is None:
+      return self
+
+    return HistogramData(self.histogram.combine(other.histogram))
+
+
+class HistogramAggregator(MetricAggregator):
+  """For internal use only; no backwards-compatibility guarantees.
+
+  Aggregator for Histogram metric data during pipeline execution.
+
+  Values aggregated should be ``HistogramData`` objects.
+  """
+  def __init__(self, bucket_type):
+    # type: (BucketType) -> None
+    self._bucket_type = bucket_type
+
+  def identity_element(self):
+    # type: () -> HistogramData
+    return HistogramData(Histogram(self._bucket_type))
+
+  def combine(self, x, y):
+    # type: (HistogramData, HistogramData) -> HistogramData
+    return x.combine(y)
+
+  def result(self, x):
+    # type: (HistogramData) -> HistogramResult
+    return HistogramResult(x.get_cumulative())
diff --git a/sdks/python/apache_beam/internal/metrics/cells_test.py b/sdks/python/apache_beam/internal/metrics/cells_test.py
new file mode 100644
index 0000000..89fe23d
--- /dev/null
+++ b/sdks/python/apache_beam/internal/metrics/cells_test.py
@@ -0,0 +1,80 @@
+#
+# 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.
+#
+
+# pytype: skip-file
+
+from __future__ import absolute_import
+
+import threading
+import unittest
+from builtins import range
+
+from apache_beam.internal.metrics.cells import HistogramCell
+from apache_beam.internal.metrics.cells import HistogramCellFactory
+from apache_beam.internal.metrics.cells import HistogramData
+from apache_beam.utils.histogram import Histogram
+from apache_beam.utils.histogram import LinearBucket
+
+
+class TestHistogramCell(unittest.TestCase):
+  @classmethod
+  def _modify_histogram(cls, d):
+    for i in range(cls.NUM_ITERATIONS):
+      d.update(i)
+
+  NUM_THREADS = 5
+  NUM_ITERATIONS = 100
+
+  def test_parallel_access(self):
+    # We create NUM_THREADS threads that concurrently modify the distribution.
+    threads = []
+    bucket_type = LinearBucket(0, 1, 100)
+    d = HistogramCell(bucket_type)
+    for _ in range(TestHistogramCell.NUM_THREADS):
+      t = threading.Thread(
+          target=TestHistogramCell._modify_histogram, args=(d, ))
+      threads.append(t)
+      t.start()
+
+    for t in threads:
+      t.join()
+
+    histogram = Histogram(bucket_type)
+    for _ in range(self.NUM_THREADS):
+      for i in range(self.NUM_ITERATIONS):
+        histogram.record(i)
+
+    self.assertEqual(d.get_cumulative(), HistogramData(histogram))
+
+  def test_basic_operations(self):
+    d = HistogramCellFactory(LinearBucket(0, 1, 10))()
+    d.update(10)
+    self.assertEqual(
+        str(d.get_cumulative()),
+        'HistogramData(Total count: 1, P99: >=10, P90: >=10, P50: >=10)')
+    d.update(0)
+    self.assertEqual(
+        str(d.get_cumulative()),
+        'HistogramData(Total count: 2, P99: >=10, P90: >=10, P50: 1)')
+    d.update(5)
+    self.assertEqual(
+        str(d.get_cumulative()),
+        'HistogramData(Total count: 3, P99: >=10, P90: >=10, P50: 6)')
+
+
+if __name__ == '__main__':
+  unittest.main()
diff --git a/sdks/python/apache_beam/internal/metrics/metric.py b/sdks/python/apache_beam/internal/metrics/metric.py
new file mode 100644
index 0000000..2fbb963
--- /dev/null
+++ b/sdks/python/apache_beam/internal/metrics/metric.py
@@ -0,0 +1,138 @@
+#
+# 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.
+#
+
+"""
+Metrics API classes for internal use only.
+
+Users should use apache_beam.metrics.metric package instead.
+
+For internal use only. No backwards compatibility guarantees.
+"""
+# pytype: skip-file
+# mypy: disallow-untyped-defs
+
+from __future__ import absolute_import
+
+import datetime
+import logging
+import threading
+import time
+from builtins import object
+from typing import TYPE_CHECKING
+from typing import Dict
+from typing import Optional
+from typing import Type
+from typing import Union
+
+from apache_beam.internal.metrics.cells import HistogramCellFactory
+from apache_beam.metrics.execution import MetricUpdater
+from apache_beam.metrics.metric import Metrics as UserMetrics
+from apache_beam.metrics.metricbase import Histogram
+from apache_beam.metrics.metricbase import MetricName
+
+if TYPE_CHECKING:
+  from apache_beam.metrics.cells import MetricCell
+  from apache_beam.metrics.cells import MetricCellFactory
+  from apache_beam.utils.histogram import BucketType
+
+__all__ = ['Metrics']
+
+_LOGGER = logging.getLogger(__name__)
+
+
+class Metrics(object):
+  @staticmethod
+  def histogram(namespace, name, bucket_type, logger=None):
+    # type: (Union[Type, str], str, BucketType, Optional[MetricLogger]) -> Metrics.DelegatingHistogram
+
+    """Obtains or creates a Histogram metric.
+
+    Args:
+      namespace: A class or string that gives the namespace to a metric
+      name: A string that gives a unique name to a metric
+      bucket_type: A type of bucket used in a histogram. A subclass of
+        apache_beam.utils.histogram.BucketType
+      logger: MetricLogger for logging locally aggregated metric
+
+    Returns:
+      A Histogram object.
+    """
+    namespace = UserMetrics.get_namespace(namespace)
+    return Metrics.DelegatingHistogram(
+        MetricName(namespace, name), bucket_type, logger)
+
+  class DelegatingHistogram(Histogram):
+    """Metrics Histogram that Delegates functionality to MetricsEnvironment."""
+    def __init__(self, metric_name, bucket_type, logger):
+      # type: (MetricName, BucketType, Optional[MetricLogger]) -> None
+      super(Metrics.DelegatingHistogram, self).__init__(metric_name)
+      self.metric_name = metric_name
+      self.cell_type = HistogramCellFactory(bucket_type)
+      self.logger = logger
+      self.updater = MetricUpdater(self.cell_type, self.metric_name)
+
+    def update(self, value):
+      # type: (object) -> None
+      self.updater(value)
+      if self.logger:
+        self.logger.update(self.cell_type, self.metric_name, value)
+
+
+class MetricLogger(object):
+  """Simple object to locally aggregate and log metrics.
+
+  This class is experimental. No backwards-compatibility guarantees.
+  """
+  def __init__(self):
+    # type: () -> None
+    self._metric = dict()  # type: Dict[MetricName, MetricCell]
+    self._lock = threading.Lock()
+    self._last_logging_millis = int(time.time() * 1000)
+    self.minimum_logging_frequency_msec = 180000
+
+  def update(self, cell_type, metric_name, value):
+    # type: (Union[Type[MetricCell], MetricCellFactory], MetricName, object) -> None
+    cell = self._get_metric_cell(cell_type, metric_name)
+    cell.update(value)
+
+  def _get_metric_cell(self, cell_type, metric_name):
+    # type: (Union[Type[MetricCell], MetricCellFactory], MetricName) -> MetricCell
+    with self._lock:
+      if metric_name not in self._metric:
+        self._metric[metric_name] = cell_type()
+    return self._metric[metric_name]
+
+  def log_metrics(self, reset_after_logging=False):
+    # type: (bool) -> None
+    if self._lock.acquire(False):
+      try:
+        current_millis = int(time.time() * 1000)
+        if ((current_millis - self._last_logging_millis) >
+            self.minimum_logging_frequency_msec):
+          logging_metric_info = [
+              '[Locally aggregated metrics since %s]' %
+              datetime.datetime.fromtimestamp(
+                  self._last_logging_millis / 1000.0)
+          ]
+          for name, cell in self._metric.items():
+            logging_metric_info.append('%s: %s' % (name, cell.get_cumulative()))
+          _LOGGER.info('\n'.join(logging_metric_info))
+          if reset_after_logging:
+            self._metric = dict()
+          self._last_logging_millis = current_millis
+      finally:
+        self._lock.release()
diff --git a/sdks/python/apache_beam/internal/metrics/metric_test.py b/sdks/python/apache_beam/internal/metrics/metric_test.py
new file mode 100644
index 0000000..88c2903
--- /dev/null
+++ b/sdks/python/apache_beam/internal/metrics/metric_test.py
@@ -0,0 +1,52 @@
+#
+# 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.
+#
+
+# pytype: skip-file
+
+from __future__ import absolute_import
+
+import unittest
+
+from mock import patch
+
+from apache_beam.internal.metrics.cells import HistogramCellFactory
+from apache_beam.internal.metrics.metric import MetricLogger
+from apache_beam.metrics.metric import Metrics
+from apache_beam.metrics.metricbase import MetricName
+from apache_beam.utils.histogram import LinearBucket
+
+
+class MetricLoggerTest(unittest.TestCase):
+  @patch('apache_beam.internal.metrics.metric._LOGGER')
+  def test_log_metrics(self, mock_logger):
+    logger = MetricLogger()
+    logger.minimum_logging_frequency_msec = -1
+    namespace = Metrics.get_namespace(self.__class__)
+    metric_name = MetricName(namespace, 'metric_logger_test')
+    logger.update(HistogramCellFactory(LinearBucket(0, 1, 10)), metric_name, 1)
+    logger.log_metrics()
+
+    class Contains(str):
+      def __eq__(self, other):
+        return self in other
+
+    mock_logger.info.assert_called_once_with(
+        Contains('HistogramData(Total count: 1, P99: 2, P90: 2, P50: 2)'))
+
+
+if __name__ == '__main__':
+  unittest.main()
diff --git a/sdks/python/apache_beam/internal/pickler.py b/sdks/python/apache_beam/internal/pickler.py
index c4bfb44..395d511 100644
--- a/sdks/python/apache_beam/internal/pickler.py
+++ b/sdks/python/apache_beam/internal/pickler.py
@@ -39,6 +39,7 @@
 import threading
 import traceback
 import types
+import zlib
 from typing import Any
 from typing import Dict
 from typing import Tuple
@@ -241,7 +242,7 @@
 logging.getLogger('dill').setLevel(logging.WARN)
 
 
-def dumps(o, enable_trace=True):
+def dumps(o, enable_trace=True, use_zlib=False):
   # type: (...) -> bytes
 
   """For internal use only; no backwards-compatibility guarantees."""
@@ -260,18 +261,28 @@
   # Compress as compactly as possible (compresslevel=9) to decrease peak memory
   # usage (of multiple in-memory copies) and to avoid hitting protocol buffer
   # limits.
-  c = bz2.compress(s, compresslevel=9)
+  # WARNING: Be cautious about compressor change since it can lead to pipeline
+  # representation change, and can break streaming job update compatibility on
+  # runners such as Dataflow.
+  if use_zlib:
+    c = zlib.compress(s, 9)
+  else:
+    c = bz2.compress(s, compresslevel=9)
   del s  # Free up some possibly large and no-longer-needed memory.
 
   return base64.b64encode(c)
 
 
-def loads(encoded, enable_trace=True):
+def loads(encoded, enable_trace=True, use_zlib=False):
   """For internal use only; no backwards-compatibility guarantees."""
 
   c = base64.b64decode(encoded)
 
-  s = bz2.decompress(c)
+  if use_zlib:
+    s = zlib.decompress(c)
+  else:
+    s = bz2.decompress(c)
+
   del c  # Free up some possibly large and no-longer-needed memory.
 
   with _pickle_lock_unless_py2:
diff --git a/sdks/python/apache_beam/io/aws/clients/s3/boto3_client.py b/sdks/python/apache_beam/io/aws/clients/s3/boto3_client.py
index cea22a8..e441d6f 100644
--- a/sdks/python/apache_beam/io/aws/clients/s3/boto3_client.py
+++ b/sdks/python/apache_beam/io/aws/clients/s3/boto3_client.py
@@ -20,6 +20,7 @@
 from __future__ import absolute_import
 
 from apache_beam.io.aws.clients.s3 import messages
+from apache_beam.options import pipeline_options
 
 try:
   # pylint: disable=wrong-import-order, wrong-import-position
@@ -34,9 +35,38 @@
   """
   Wrapper for boto3 library
   """
-  def __init__(self):
+  def __init__(self, options):
     assert boto3 is not None, 'Missing boto3 requirement'
-    self.client = boto3.client('s3')
+    if isinstance(options, pipeline_options.PipelineOptions):
+      s3_options = options.view_as(pipeline_options.S3Options)
+      access_key_id = s3_options.s3_access_key_id
+      secret_access_key = s3_options.s3_secret_access_key
+      session_token = s3_options.s3_session_token
+      endpoint_url = s3_options.s3_endpoint_url
+      use_ssl = not s3_options.s3_disable_ssl
+      region_name = s3_options.s3_region_name
+      api_version = s3_options.s3_api_version
+      verify = s3_options.s3_verify
+    else:
+      access_key_id = options.get('s3_access_key_id')
+      secret_access_key = options.get('s3_secret_access_key')
+      session_token = options.get('s3_session_token')
+      endpoint_url = options.get('s3_endpoint_url')
+      use_ssl = not options.get('s3_disable_ssl', False)
+      region_name = options.get('s3_region_name')
+      api_version = options.get('s3_api_version')
+      verify = options.get('s3_verify')
+
+    self.client = boto3.client(
+        service_name='s3',
+        region_name=region_name,
+        api_version=api_version,
+        use_ssl=use_ssl,
+        verify=verify,
+        endpoint_url=endpoint_url,
+        aws_access_key_id=access_key_id,
+        aws_secret_access_key=secret_access_key,
+        aws_session_token=session_token)
 
   def get_object_metadata(self, request):
     r"""Retrieves an object's metadata.
diff --git a/sdks/python/apache_beam/io/aws/clients/s3/client_test.py b/sdks/python/apache_beam/io/aws/clients/s3/client_test.py
index 2100f25..67797b1 100644
--- a/sdks/python/apache_beam/io/aws/clients/s3/client_test.py
+++ b/sdks/python/apache_beam/io/aws/clients/s3/client_test.py
@@ -23,6 +23,7 @@
 from apache_beam.io.aws import s3io
 from apache_beam.io.aws.clients.s3 import fake_client
 from apache_beam.io.aws.clients.s3 import messages
+from apache_beam.options import pipeline_options
 
 
 class ClientErrorTest(unittest.TestCase):
@@ -46,7 +47,7 @@
       self.client.known_buckets.add(test_data_bucket)
       self.aws = s3io.S3IO(self.client)
     else:
-      self.aws = s3io.S3IO()
+      self.aws = s3io.S3IO(options=pipeline_options.S3Options())
 
   def test_get_object_metadata(self):
 
diff --git a/sdks/python/apache_beam/io/aws/s3filesystem.py b/sdks/python/apache_beam/io/aws/s3filesystem.py
index 4bb0b6b..dce8cde 100644
--- a/sdks/python/apache_beam/io/aws/s3filesystem.py
+++ b/sdks/python/apache_beam/io/aws/s3filesystem.py
@@ -40,6 +40,15 @@
   CHUNK_SIZE = s3io.MAX_BATCH_OPERATION_SIZE
   S3_PREFIX = 's3://'
 
+  def __init__(self, pipeline_options):
+    """Initializes a connection to S3.
+
+    Connection configuration is done by passing pipeline options.
+    See :class:`~apache_beam.options.pipeline_options.S3Options`.
+    """
+    super(S3FileSystem, self).__init__(pipeline_options)
+    self._options = pipeline_options
+
   @classmethod
   def scheme(cls):
     """URI scheme for the FileSystem
@@ -123,7 +132,8 @@
       ``BeamIOError``: if listing fails, but not if no files were found.
     """
     try:
-      for path, size in iteritems(s3io.S3IO().list_prefix(dir_or_prefix)):
+      for path, size in iteritems(
+          s3io.S3IO(options=self._options).list_prefix(dir_or_prefix)):
         yield FileMetadata(path, size)
     except Exception as e:  # pylint: disable=broad-except
       raise BeamIOError("List operation failed", {dir_or_prefix: e})
@@ -138,7 +148,8 @@
     """
     compression_type = FileSystem._get_compression_type(path, compression_type)
     mime_type = CompressionTypes.mime_type(compression_type, mime_type)
-    raw_file = s3io.S3IO().open(path, mode, mime_type=mime_type)
+    raw_file = s3io.S3IO(options=self._options).open(
+        path, mode, mime_type=mime_type)
     if compression_type == CompressionTypes.UNCOMPRESSED:
       return raw_file
     return CompressedFile(raw_file, compression_type=compression_type)
@@ -189,7 +200,7 @@
       message = 'Unable to copy unequal number of sources and destinations'
       raise BeamIOError(message)
     src_dest_pairs = list(zip(source_file_names, destination_file_names))
-    return s3io.S3IO().copy_paths(src_dest_pairs)
+    return s3io.S3IO(options=self._options).copy_paths(src_dest_pairs)
 
   def rename(self, source_file_names, destination_file_names):
     """Rename the files at the source list to the destination list.
@@ -206,7 +217,7 @@
       message = 'Unable to rename unequal number of sources and destinations'
       raise BeamIOError(message)
     src_dest_pairs = list(zip(source_file_names, destination_file_names))
-    results = s3io.S3IO().rename_files(src_dest_pairs)
+    results = s3io.S3IO(options=self._options).rename_files(src_dest_pairs)
     exceptions = {(src, dest): error
                   for (src, dest, error) in results if error is not None}
     if exceptions:
@@ -221,7 +232,7 @@
     Returns: boolean flag indicating if path exists
     """
     try:
-      return s3io.S3IO().exists(path)
+      return s3io.S3IO(options=self._options).exists(path)
     except Exception as e:  # pylint: disable=broad-except
       raise BeamIOError("exists() operation failed", {path: e})
 
@@ -237,7 +248,7 @@
       ``BeamIOError``: if path doesn't exist.
     """
     try:
-      return s3io.S3IO().size(path)
+      return s3io.S3IO(options=self._options).size(path)
     except Exception as e:  # pylint: disable=broad-except
       raise BeamIOError("size() operation failed", {path: e})
 
@@ -253,7 +264,7 @@
       ``BeamIOError``: if path doesn't exist.
     """
     try:
-      return s3io.S3IO().last_updated(path)
+      return s3io.S3IO(options=self._options).last_updated(path)
     except Exception as e:  # pylint: disable=broad-except
       raise BeamIOError("last_updated operation failed", {path: e})
 
@@ -270,7 +281,7 @@
       ``BeamIOError``: if path isn't a file or doesn't exist.
     """
     try:
-      return s3io.S3IO().checksum(path)
+      return s3io.S3IO(options=self._options).checksum(path)
     except Exception as e:  # pylint: disable=broad-except
       raise BeamIOError("Checksum operation failed", {path: e})
 
@@ -281,7 +292,7 @@
     Args:
       paths: list of paths that give the file objects to be deleted
     """
-    results = s3io.S3IO().delete_paths(paths)
+    results = s3io.S3IO(options=self._options).delete_paths(paths)
     exceptions = {
         path: error
         for (path, error) in results.items() if error is not None
diff --git a/sdks/python/apache_beam/io/aws/s3filesystem_test.py b/sdks/python/apache_beam/io/aws/s3filesystem_test.py
index 3252fd7..92d2a52 100644
--- a/sdks/python/apache_beam/io/aws/s3filesystem_test.py
+++ b/sdks/python/apache_beam/io/aws/s3filesystem_test.py
@@ -84,7 +84,7 @@
   def test_match_multiples(self, unused_mock_arg):
     # Prepare mocks.
     s3io_mock = mock.MagicMock()
-    s3filesystem.s3io.S3IO = lambda: s3io_mock  # type: ignore[misc]
+    s3filesystem.s3io.S3IO = lambda options: s3io_mock  # type: ignore[misc]
     s3io_mock.list_prefix.return_value = {
         's3://bucket/file1': 1, 's3://bucket/file2': 2
     }
@@ -102,7 +102,7 @@
     # Prepare mocks.
     s3io_mock = mock.MagicMock()
     limit = 1
-    s3filesystem.s3io.S3IO = lambda: s3io_mock  # type: ignore[misc]
+    s3filesystem.s3io.S3IO = lambda options: s3io_mock  # type: ignore[misc]
     s3io_mock.list_prefix.return_value = {'s3://bucket/file1': 1}
     expected_results = set([FileMetadata('s3://bucket/file1', 1)])
     match_result = self.fs.match(['s3://bucket/'], [limit])[0]
@@ -114,7 +114,7 @@
   def test_match_multiples_error(self, unused_mock_arg):
     # Prepare mocks.
     s3io_mock = mock.MagicMock()
-    s3filesystem.s3io.S3IO = lambda: s3io_mock  # type: ignore[misc]
+    s3filesystem.s3io.S3IO = lambda options: s3io_mock  # type: ignore[misc]
     exception = IOError('Failed')
     s3io_mock.list_prefix.side_effect = exception
 
@@ -128,7 +128,7 @@
   def test_match_multiple_patterns(self, unused_mock_arg):
     # Prepare mocks.
     s3io_mock = mock.MagicMock()
-    s3filesystem.s3io.S3IO = lambda: s3io_mock  # type: ignore[misc]
+    s3filesystem.s3io.S3IO = lambda options: s3io_mock  # type: ignore[misc]
     s3io_mock.list_prefix.side_effect = [
         {
             's3://bucket/file1': 1
@@ -146,7 +146,7 @@
   def test_create(self, unused_mock_arg):
     # Prepare mocks.
     s3io_mock = mock.MagicMock()
-    s3filesystem.s3io.S3IO = lambda: s3io_mock  # type: ignore[misc]
+    s3filesystem.s3io.S3IO = lambda options: s3io_mock  # type: ignore[misc]
     # Issue file copy
     _ = self.fs.create('s3://bucket/from1', 'application/octet-stream')
 
@@ -157,7 +157,7 @@
   def test_open(self, unused_mock_arg):
     # Prepare mocks.
     s3io_mock = mock.MagicMock()
-    s3filesystem.s3io.S3IO = lambda: s3io_mock  # type: ignore[misc]
+    s3filesystem.s3io.S3IO = lambda options: s3io_mock  # type: ignore[misc]
     # Issue file copy
     _ = self.fs.open('s3://bucket/from1', 'application/octet-stream')
 
@@ -168,7 +168,7 @@
   def test_copy_file(self, unused_mock_arg):
     # Prepare mocks.
     s3io_mock = mock.MagicMock()
-    s3filesystem.s3io.S3IO = lambda: s3io_mock  # type: ignore[misc]
+    s3filesystem.s3io.S3IO = lambda options: s3io_mock  # type: ignore[misc]
 
     sources = ['s3://bucket/from1', 's3://bucket/from2']
     destinations = ['s3://bucket/to1', 's3://bucket/to2']
@@ -183,7 +183,7 @@
   def test_copy_file_error(self, unused_mock_arg):
     # Prepare mocks.
     s3io_mock = mock.MagicMock()
-    s3filesystem.s3io.S3IO = lambda: s3io_mock  # type: ignore[misc]
+    s3filesystem.s3io.S3IO = lambda options: s3io_mock  # type: ignore[misc]
 
     sources = ['s3://bucket/from1', 's3://bucket/from2', 's3://bucket/from3']
     destinations = ['s3://bucket/to1', 's3://bucket/to2']
@@ -196,7 +196,7 @@
   def test_delete(self, unused_mock_arg):
     # Prepare mocks.
     s3io_mock = mock.MagicMock()
-    s3filesystem.s3io.S3IO = lambda: s3io_mock  # type: ignore[misc]
+    s3filesystem.s3io.S3IO = lambda options: s3io_mock  # type: ignore[misc]
     s3io_mock.size.return_value = 0
     files = [
         's3://bucket/from1',
@@ -212,7 +212,7 @@
   def test_delete_error(self, unused_mock_arg):
     # Prepare mocks.
     s3io_mock = mock.MagicMock()
-    s3filesystem.s3io.S3IO = lambda: s3io_mock  # type: ignore[misc]
+    s3filesystem.s3io.S3IO = lambda options: s3io_mock  # type: ignore[misc]
 
     problematic_directory = 's3://nonexistent-bucket/tree/'
     exception = messages.S3ClientError('Not found', 404)
@@ -242,7 +242,7 @@
   def test_rename(self, unused_mock_arg):
     # Prepare mocks.
     s3io_mock = mock.MagicMock()
-    s3filesystem.s3io.S3IO = lambda: s3io_mock  # type: ignore[misc]
+    s3filesystem.s3io.S3IO = lambda options: s3io_mock  # type: ignore[misc]
 
     sources = ['s3://bucket/from1', 's3://bucket/from2']
     destinations = ['s3://bucket/to1', 's3://bucket/to2']
diff --git a/sdks/python/apache_beam/io/aws/s3io.py b/sdks/python/apache_beam/io/aws/s3io.py
index 16a4de2..21b1130 100644
--- a/sdks/python/apache_beam/io/aws/s3io.py
+++ b/sdks/python/apache_beam/io/aws/s3io.py
@@ -58,11 +58,13 @@
 
 class S3IO(object):
   """S3 I/O client."""
-  def __init__(self, client=None):
+  def __init__(self, client=None, options=None):
+    if client is None and options is None:
+      raise ValueError('Must provide one of client or options')
     if client is not None:
       self.client = client
     elif BOTO3_INSTALLED:
-      self.client = boto3_client.Client()
+      self.client = boto3_client.Client(options=options)
     else:
       message = 'AWS dependencies are not installed, and no alternative ' \
       'client was provided to S3IO.'
diff --git a/sdks/python/apache_beam/io/aws/s3io_test.py b/sdks/python/apache_beam/io/aws/s3io_test.py
index 711f93a..ad308f3 100644
--- a/sdks/python/apache_beam/io/aws/s3io_test.py
+++ b/sdks/python/apache_beam/io/aws/s3io_test.py
@@ -29,6 +29,7 @@
 from apache_beam.io.aws import s3io
 from apache_beam.io.aws.clients.s3 import fake_client
 from apache_beam.io.aws.clients.s3 import messages
+from apache_beam.options import pipeline_options
 
 
 class TestS3PathParser(unittest.TestCase):
@@ -99,7 +100,7 @@
       self.aws = s3io.S3IO(self.client)
 
     else:
-      self.aws = s3io.S3IO()
+      self.aws = s3io.S3IO(options=pipeline_options.S3Options())
       self.client = self.aws.client
 
   def test_size(self):
diff --git a/sdks/python/apache_beam/io/gcp/bigquery.py b/sdks/python/apache_beam/io/gcp/bigquery.py
index 9ef80cd..f682789 100644
--- a/sdks/python/apache_beam/io/gcp/bigquery.py
+++ b/sdks/python/apache_beam/io/gcp/bigquery.py
@@ -239,7 +239,6 @@
 import json
 import logging
 import random
-import threading
 import time
 import uuid
 from builtins import object
@@ -267,7 +266,6 @@
 from apache_beam.io.textio import _TextSource as TextSource
 from apache_beam.metrics import Metrics
 from apache_beam.options import value_provider as vp
-from apache_beam.options.pipeline_options import BigQueryOptions
 from apache_beam.options.pipeline_options import DebugOptions
 from apache_beam.options.pipeline_options import GoogleCloudOptions
 from apache_beam.options.pipeline_options import StandardOptions
@@ -285,8 +283,6 @@
 from apache_beam.transforms.window import GlobalWindows
 from apache_beam.utils import retry
 from apache_beam.utils.annotations import deprecated
-from apache_beam.utils.histogram import Histogram
-from apache_beam.utils.histogram import LinearBucket
 
 __all__ = [
     'TableRowJsonCoder',
@@ -1086,9 +1082,7 @@
   DEFAULT_MAX_BATCH_SIZE = 500
 
   FAILED_ROWS = 'FailedRows'
-  LATENCY_LOGGING_HISTOGRAM = Histogram(LinearBucket(0, 20, 3000))
-  LATENCY_LOGGING_LAST_REPORTED_MILLIS = int(time.time() * 1000)
-  LATENCY_LOGGING_LOCK = threading.Lock()
+  STREAMING_API_LOGGING_FREQUENCY_SEC = 300
 
   def __init__(
       self,
@@ -1101,8 +1095,7 @@
       max_buffered_rows=None,
       retry_strategy=None,
       additional_bq_parameters=None,
-      ignore_insert_ids=False,
-      latency_logging_frequency_sec=None):
+      ignore_insert_ids=False):
     """Initialize a WriteToBigQuery transform.
 
     Args:
@@ -1143,8 +1136,6 @@
         duplication of data inserted to BigQuery, set `ignore_insert_ids`
         to True to increase the throughput for BQ writing. See:
         https://cloud.google.com/bigquery/streaming-data-into-bigquery#disabling_best_effort_de-duplication
-      latency_logging_frequency_sec: The frequency in seconds that the logger
-        prints out streaming insert API latency percentile information.
     """
     self.schema = schema
     self.test_client = test_client
@@ -1172,8 +1163,8 @@
     self.failed_rows_metric = Metrics.distribution(
         self.__class__, "rows_failed_per_batch")
     self.bigquery_wrapper = None
-    self._latency_logging_frequency_msec = (
-        latency_logging_frequency_sec or 180) * 1000
+    self.streaming_api_logging_frequency_sec = (
+        BigQueryWriteFn.STREAMING_API_LOGGING_FREQUENCY_SEC)
 
   def display_data(self):
     return {
@@ -1217,6 +1208,11 @@
       self.bigquery_wrapper = bigquery_tools.BigQueryWrapper(
           client=self.test_client)
 
+    (
+        bigquery_tools.BigQueryWrapper.HISTOGRAM_METRIC_LOGGER.
+        minimum_logging_frequency_msec
+    ) = self.streaming_api_logging_frequency_sec * 1000
+
     self._backoff_calculator = iter(
         retry.FuzzedExponentialIntervals(
             initial_delay_secs=0.2, num_retries=10000, max_delay_secs=1500))
@@ -1276,20 +1272,8 @@
       return self._flush_all_batches()
 
   def finish_bundle(self):
-    if BigQueryWriteFn.LATENCY_LOGGING_LOCK.acquire(False):
-      try:
-        current_millis = int(time.time() * 1000)
-        if (BigQueryWriteFn.LATENCY_LOGGING_HISTOGRAM.total_count() > 0 and
-            (current_millis -
-             BigQueryWriteFn.LATENCY_LOGGING_LAST_REPORTED_MILLIS) >
-            self._latency_logging_frequency_msec):
-          _LOGGER.info(
-              BigQueryWriteFn.LATENCY_LOGGING_HISTOGRAM.get_percentile_info(
-                  'streaming insert requests', 'ms'))
-          BigQueryWriteFn.LATENCY_LOGGING_HISTOGRAM.clear()
-          BigQueryWriteFn.LATENCY_LOGGING_LAST_REPORTED_MILLIS = current_millis
-      finally:
-        BigQueryWriteFn.LATENCY_LOGGING_LOCK.release()
+    bigquery_tools.BigQueryWrapper.HISTOGRAM_METRIC_LOGGER.log_metrics(
+        reset_after_logging=True)
     return self._flush_all_batches()
 
   def _flush_all_batches(self):
@@ -1334,8 +1318,7 @@
           table_id=table_reference.tableId,
           rows=rows,
           insert_ids=insert_ids,
-          skip_invalid_rows=True,
-          latency_recoder=BigQueryWriteFn.LATENCY_LOGGING_HISTOGRAM)
+          skip_invalid_rows=True)
       self.batch_latency_metric.update((time.time() - start) * 1000)
 
       failed_rows = [rows[entry.index] for entry in errors]
@@ -1388,7 +1371,6 @@
       retry_strategy,
       additional_bq_parameters,
       ignore_insert_ids,
-      latency_logging_frequency_sec,
       test_client=None):
     self.table_reference = table_reference
     self.table_side_inputs = table_side_inputs
@@ -1402,7 +1384,6 @@
     self.test_client = test_client
     self.additional_bq_parameters = additional_bq_parameters
     self.ignore_insert_ids = ignore_insert_ids
-    self.latency_logging_frequency_sec = latency_logging_frequency_sec
 
   class InsertIdPrefixFn(DoFn):
     def __init__(self, shards=DEFAULT_SHARDS_PER_DESTINATION):
@@ -1431,8 +1412,7 @@
         retry_strategy=self.retry_strategy,
         test_client=self.test_client,
         additional_bq_parameters=self.additional_bq_parameters,
-        ignore_insert_ids=self.ignore_insert_ids,
-        latency_logging_frequency_sec=self.latency_logging_frequency_sec)
+        ignore_insert_ids=self.ignore_insert_ids)
 
     def drop_shard(elms):
       key_and_shard = elms[0]
@@ -1682,8 +1662,6 @@
     experiments = p.options.view_as(DebugOptions).experiments or []
     # TODO(pabloem): Use a different method to determine if streaming or batch.
     is_streaming_pipeline = p.options.view_as(StandardOptions).streaming
-    latency_logging_frequency_sec = p.options.view_as(
-        BigQueryOptions).latency_logging_frequency_sec
 
     method_to_use = self._compute_method(experiments, is_streaming_pipeline)
 
@@ -1710,7 +1688,6 @@
           self.insert_retry_strategy,
           self.additional_bq_parameters,
           self._ignore_insert_ids,
-          latency_logging_frequency_sec,
           test_client=self.test_client)
 
       return {BigQueryWriteFn.FAILED_ROWS: outputs[BigQueryWriteFn.FAILED_ROWS]}
@@ -1938,6 +1915,7 @@
       gcs_location_vp,  # type: Optional[ValueProvider]
       temp_location,  # type: Optional[str]
       unique_id,  # type: str
+      directory_only=False,  # type: bool
   ):
     """Returns the fully qualified Google Cloud Storage URI where the
     extracted table should be written.
@@ -1959,7 +1937,10 @@
           'gcs_location in the constructor nor the fallback option '
           '--temp_location is set.')
 
-    return FileSystems.join(gcs_base, unique_id, file_pattern)
+    if directory_only:
+      return FileSystems.join(gcs_base, unique_id)
+    else:
+      return FileSystems.join(gcs_base, unique_id, file_pattern)
 
   def expand(self, pcoll):
     class RemoveExportedFiles(beam.DoFn):
@@ -1970,12 +1951,8 @@
 
       def process(self, unused_element, signal):
         gcs_location = ReadFromBigQuery.get_destination_uri(
-            self._gcs_location_vp, self._temp_location, self._unique_id)
-        match_result = FileSystems.match([gcs_location])[0].metadata_list
-        _LOGGER.debug(
-            "%s: matched %s files", self.__class__.__name__, len(match_result))
-        paths = [x.path for x in match_result]
-        FileSystems.delete(paths)
+            self._gcs_location_vp, self._temp_location, self._unique_id, True)
+        FileSystems.delete([gcs_location + '/'])
 
     unique_id = str(uuid.uuid4())[0:10]
     temp_location = pcoll.pipeline.options.view_as(
diff --git a/sdks/python/apache_beam/io/gcp/bigquery_test.py b/sdks/python/apache_beam/io/gcp/bigquery_test.py
index b399d83..5ec879d 100644
--- a/sdks/python/apache_beam/io/gcp/bigquery_test.py
+++ b/sdks/python/apache_beam/io/gcp/bigquery_test.py
@@ -889,7 +889,6 @@
               None,
               None, [],
               ignore_insert_ids=False,
-              latency_logging_frequency_sec=None,
               test_client=client))
 
     with open(file_name_1) as f1, open(file_name_2) as f2:
diff --git a/sdks/python/apache_beam/io/gcp/bigquery_tools.py b/sdks/python/apache_beam/io/gcp/bigquery_tools.py
index 133f017..0495453 100644
--- a/sdks/python/apache_beam/io/gcp/bigquery_tools.py
+++ b/sdks/python/apache_beam/io/gcp/bigquery_tools.py
@@ -50,6 +50,8 @@
 from apache_beam.internal.gcp.json_value import from_json_value
 from apache_beam.internal.gcp.json_value import to_json_value
 from apache_beam.internal.http_client import get_new_http
+from apache_beam.internal.metrics.metric import MetricLogger
+from apache_beam.internal.metrics.metric import Metrics
 from apache_beam.io.gcp import bigquery_avro_tools
 from apache_beam.io.gcp.bigquery_io_metadata import create_bigquery_io_metadata
 from apache_beam.io.gcp.internal.clients import bigquery
@@ -59,6 +61,7 @@
 from apache_beam.transforms import DoFn
 from apache_beam.typehints.typehints import Any
 from apache_beam.utils import retry
+from apache_beam.utils.histogram import LinearBucket
 
 # Protect against environments where bigquery library is not available.
 # pylint: disable=wrong-import-order, wrong-import-position
@@ -261,6 +264,8 @@
   TEMP_TABLE = 'temp_table_'
   TEMP_DATASET = 'temp_dataset_'
 
+  HISTOGRAM_METRIC_LOGGER = MetricLogger()
+
   def __init__(self, client=None):
     self.client = client or bigquery.BigqueryV2(
         http=get_new_http(),
@@ -271,6 +276,11 @@
     # randomized prefix for row IDs.
     self._row_id_prefix = '' if client else uuid.uuid4()
     self._temporary_table_suffix = uuid.uuid4().hex
+    self._latency_histogram_metric = Metrics.histogram(
+        self.__class__,
+        'latency_histogram_ms',
+        LinearBucket(0, 20, 3000),
+        BigQueryWrapper.HISTOGRAM_METRIC_LOGGER)
 
   @property
   def unique_row_id(self):
@@ -536,13 +546,7 @@
       num_retries=MAX_RETRIES,
       retry_filter=retry.retry_on_server_errors_timeout_or_quota_issues_filter)
   def _insert_all_rows(
-      self,
-      project_id,
-      dataset_id,
-      table_id,
-      rows,
-      skip_invalid_rows=False,
-      latency_recoder=None):
+      self, project_id, dataset_id, table_id, rows, skip_invalid_rows=False):
     """Calls the insertAll BigQuery API endpoint.
 
     Docs for this BQ call: https://cloud.google.com/bigquery/docs/reference\
@@ -558,13 +562,13 @@
             skipInvalidRows=skip_invalid_rows,
             # TODO(silviuc): Should have an option for ignoreUnknownValues?
             rows=rows))
-    started_millis = int(time.time() * 1000) if latency_recoder else None
+    started_millis = int(time.time() * 1000)
     try:
       response = self.client.tabledata.InsertAll(request)
       # response.insertErrors is not [] if errors encountered.
     finally:
-      if latency_recoder:
-        latency_recoder.record(int(time.time() * 1000) - started_millis)
+      self._latency_histogram_metric.update(
+          int(time.time() * 1000) - started_millis)
     return not response.insertErrors, response.insertErrors
 
   @retry.with_exponential_backoff(
@@ -980,8 +984,7 @@
       table_id,
       rows,
       insert_ids=None,
-      skip_invalid_rows=False,
-      latency_recoder=None):
+      skip_invalid_rows=False):
     """Inserts rows into the specified table.
 
     Args:
@@ -992,9 +995,6 @@
         each key in it is the name of a field.
       skip_invalid_rows: If there are rows with insertion errors, whether they
         should be skipped, and all others should be inserted successfully.
-      latency_recoder: The object that records request-to-response latencies.
-        The object should provide `record(int)` method to be invoked with
-        milliseconds latency values.
 
     Returns:
       A tuple (bool, errors). If first element is False then the second element
@@ -1015,8 +1015,7 @@
           bigquery.TableDataInsertAllRequest.RowsValueListEntry(
               insertId=insert_id, json=json_row))
     result, errors = self._insert_all_rows(
-        project_id, dataset_id, table_id, final_rows, skip_invalid_rows,
-        latency_recoder)
+        project_id, dataset_id, table_id, final_rows, skip_invalid_rows)
     return result, errors
 
   def _convert_to_json_row(self, row):
diff --git a/sdks/python/apache_beam/io/gcp/bigquery_tools_test.py b/sdks/python/apache_beam/io/gcp/bigquery_tools_test.py
index 6e6c01e..777b5b2 100644
--- a/sdks/python/apache_beam/io/gcp/bigquery_tools_test.py
+++ b/sdks/python/apache_beam/io/gcp/bigquery_tools_test.py
@@ -216,26 +216,6 @@
     wrapper._delete_table('', '', '')
     self.assertTrue(client.tables.Delete.called)
 
-  def test_insert_latency_recorded(self):
-    client = mock.Mock()
-    insert_response = mock.Mock()
-    insert_response.insertErrors = []
-    client.tabledata.InsertAll.return_value = insert_response
-    wrapper = beam.io.gcp.bigquery_tools.BigQueryWrapper(client)
-    mock_recoder = mock.Mock()
-    wrapper._insert_all_rows('', '', '', [], latency_recoder=mock_recoder)
-    self.assertTrue(mock_recoder.record.called)
-
-  def test_insert_error_latency_recorded(self):
-    client = mock.Mock()
-    client.tabledata.InsertAll.side_effect = HttpError(
-        response={'status': '404'}, url='', content='')
-    wrapper = beam.io.gcp.bigquery_tools.BigQueryWrapper(client)
-    mock_recoder = mock.Mock()
-    with self.assertRaises(HttpError):
-      wrapper._insert_all_rows('', '', '', [], latency_recoder=mock_recoder)
-    self.assertTrue(mock_recoder.record.called)
-
   @mock.patch('time.sleep', return_value=None)
   def test_temporary_dataset_is_unique(self, patched_time_sleep):
     client = mock.Mock()
diff --git a/sdks/python/apache_beam/io/gcp/gcsio.py b/sdks/python/apache_beam/io/gcp/gcsio.py
index 2e5b059..a83364b 100644
--- a/sdks/python/apache_beam/io/gcp/gcsio.py
+++ b/sdks/python/apache_beam/io/gcp/gcsio.py
@@ -35,6 +35,7 @@
 import time
 import traceback
 from builtins import object
+from itertools import islice
 
 from apache_beam.internal.http_client import get_new_http
 from apache_beam.io.filesystemio import Downloader
@@ -257,26 +258,32 @@
     """
     if not paths:
       return []
-    batch_request = BatchApiRequest(
-        batch_url=GCS_BATCH_ENDPOINT,
-        retryable_codes=retry.SERVER_ERROR_OR_TIMEOUT_CODES,
-        response_encoding='utf-8')
-    for path in paths:
-      bucket, object_path = parse_gcs_path(path)
-      request = storage.StorageObjectsDeleteRequest(
-          bucket=bucket, object=object_path)
-      batch_request.Add(self.client.objects, 'Delete', request)
-    api_calls = batch_request.Execute(self.client._http)  # pylint: disable=protected-access
+
+    paths = iter(paths)
     result_statuses = []
-    for i, api_call in enumerate(api_calls):
-      path = paths[i]
-      exception = None
-      if api_call.is_error:
-        exception = api_call.exception
-        # Return success when the file doesn't exist anymore for idempotency.
-        if isinstance(exception, HttpError) and exception.status_code == 404:
-          exception = None
-      result_statuses.append((path, exception))
+    while True:
+      paths_chunk = list(islice(paths, MAX_BATCH_OPERATION_SIZE))
+      if not paths_chunk:
+        return result_statuses
+      batch_request = BatchApiRequest(
+          batch_url=GCS_BATCH_ENDPOINT,
+          retryable_codes=retry.SERVER_ERROR_OR_TIMEOUT_CODES,
+          response_encoding='utf-8')
+      for path in paths_chunk:
+        bucket, object_path = parse_gcs_path(path)
+        request = storage.StorageObjectsDeleteRequest(
+            bucket=bucket, object=object_path)
+        batch_request.Add(self.client.objects, 'Delete', request)
+      api_calls = batch_request.Execute(self.client._http)  # pylint: disable=protected-access
+      for i, api_call in enumerate(api_calls):
+        path = paths_chunk[i]
+        exception = None
+        if api_call.is_error:
+          exception = api_call.exception
+          # Return success when the file doesn't exist anymore for idempotency.
+          if isinstance(exception, HttpError) and exception.status_code == 404:
+            exception = None
+        result_statuses.append((path, exception))
     return result_statuses
 
   @retry.with_exponential_backoff(
diff --git a/sdks/python/apache_beam/io/gcp/pubsub_test.py b/sdks/python/apache_beam/io/gcp/pubsub_test.py
index d32afae..77f1bc9 100644
--- a/sdks/python/apache_beam/io/gcp/pubsub_test.py
+++ b/sdks/python/apache_beam/io/gcp/pubsub_test.py
@@ -622,6 +622,31 @@
         transform_from_proto.source.full_topic)
     self.assertTrue(transform_from_proto.source.with_attributes)
 
+  def test_runner_api_transformation_properties_none(self, unused_mock_pubsub):
+    # Confirming that properties stay None after a runner API transformation.
+    source = _PubSubSource(
+        topic='projects/fakeprj/topics/a_topic', with_attributes=True)
+    transform = Read(source)
+
+    context = pipeline_context.PipelineContext()
+    proto_transform_spec = transform.to_runner_api(context)
+    self.assertEqual(
+        common_urns.composites.PUBSUB_READ.urn, proto_transform_spec.urn)
+
+    pubsub_read_payload = (
+        proto_utils.parse_Bytes(
+            proto_transform_spec.payload,
+            beam_runner_api_pb2.PubSubReadPayload))
+
+    proto_transform = beam_runner_api_pb2.PTransform(
+        unique_name="dummy_label", spec=proto_transform_spec)
+
+    transform_from_proto = Read.from_runner_api_parameter(
+        proto_transform, pubsub_read_payload, None)
+    self.assertIsNone(transform_from_proto.source.full_subscription)
+    self.assertIsNone(transform_from_proto.source.id_label)
+    self.assertIsNone(transform_from_proto.source.timestamp_attribute)
+
   def test_runner_api_transformation_with_subscription(
       self, unused_mock_pubsub):
     source = _PubSubSource(
@@ -787,6 +812,36 @@
     self.assertEqual(
         'projects/fakeprj/topics/a_topic', transform_from_proto.sink.full_topic)
 
+  def test_runner_api_transformation_properties_none(self, unused_mock_pubsub):
+    # Confirming that properties stay None after a runner API transformation.
+    sink = _PubSubSink(
+        topic='projects/fakeprj/topics/a_topic',
+        id_label=None,
+        with_attributes=True,
+        # We expect encoded PubSub write transform to always return attributes.
+        timestamp_attribute=None)
+    transform = Write(sink)
+
+    context = pipeline_context.PipelineContext()
+    proto_transform_spec = transform.to_runner_api(context)
+    self.assertEqual(
+        common_urns.composites.PUBSUB_WRITE.urn, proto_transform_spec.urn)
+
+    pubsub_write_payload = (
+        proto_utils.parse_Bytes(
+            proto_transform_spec.payload,
+            beam_runner_api_pb2.PubSubWritePayload))
+    proto_transform = beam_runner_api_pb2.PTransform(
+        unique_name="dummy_label", spec=proto_transform_spec)
+    transform_from_proto = Write.from_runner_api_parameter(
+        proto_transform, pubsub_write_payload, None)
+
+    self.assertTrue(isinstance(transform_from_proto, Write))
+    self.assertTrue(isinstance(transform_from_proto.sink, _PubSubSink))
+    self.assertTrue(transform_from_proto.sink.with_attributes)
+    self.assertIsNone(transform_from_proto.sink.id_label)
+    self.assertIsNone(transform_from_proto.sink.timestamp_attribute)
+
 
 if __name__ == '__main__':
   logging.getLogger().setLevel(logging.INFO)
diff --git a/sdks/python/apache_beam/io/iobase.py b/sdks/python/apache_beam/io/iobase.py
index 05052eb..665227d 100644
--- a/sdks/python/apache_beam/io/iobase.py
+++ b/sdks/python/apache_beam/io/iobase.py
@@ -957,11 +957,11 @@
       # Importing locally to prevent circular dependencies.
       from apache_beam.io.gcp.pubsub import _PubSubSource
       source = _PubSubSource(
-          topic=payload.topic,
-          subscription=payload.subscription,
-          id_label=payload.id_attribute,
+          topic=payload.topic or None,
+          subscription=payload.subscription or None,
+          id_label=payload.id_attribute or None,
           with_attributes=payload.with_attributes,
-          timestamp_attribute=payload.timestamp_attribute)
+          timestamp_attribute=payload.timestamp_attribute or None)
       return Read(source)
     else:
       return Read(SourceBase.from_runner_api(payload.source, context))
@@ -1050,10 +1050,10 @@
     # Importing locally to prevent circular dependencies.
     from apache_beam.io.gcp.pubsub import _PubSubSink
     sink = _PubSubSink(
-        topic=payload.topic,
-        id_label=payload.id_attribute,
+        topic=payload.topic or None,
+        id_label=payload.id_attribute or None,
         with_attributes=payload.with_attributes,
-        timestamp_attribute=payload.timestamp_attribute)
+        timestamp_attribute=payload.timestamp_attribute or None)
     return Write(sink)
 
 
diff --git a/sdks/python/apache_beam/metrics/cells.py b/sdks/python/apache_beam/metrics/cells.py
index fa791c7..f638698 100644
--- a/sdks/python/apache_beam/metrics/cells.py
+++ b/sdks/python/apache_beam/metrics/cells.py
@@ -42,7 +42,13 @@
 
   globals()['cython'] = fake_cython
 
-__all__ = ['DistributionResult', 'GaugeResult']
+__all__ = [
+    'MetricAggregator',
+    'MetricCell',
+    'MetricCellFactory',
+    'DistributionResult',
+    'GaugeResult'
+]
 
 
 class MetricCell(object):
@@ -75,6 +81,12 @@
     raise NotImplementedError
 
 
+class MetricCellFactory(object):
+  def __call__(self):
+    # type: () -> MetricCell
+    raise NotImplementedError
+
+
 class CounterCell(MetricCell):
   """For internal use only; no backwards-compatibility guarantees.
 
diff --git a/sdks/python/apache_beam/metrics/execution.py b/sdks/python/apache_beam/metrics/execution.py
index 1ed1dc0..4afd815 100644
--- a/sdks/python/apache_beam/metrics/execution.py
+++ b/sdks/python/apache_beam/metrics/execution.py
@@ -55,6 +55,7 @@
   from apache_beam.metrics.cells import GaugeData
   from apache_beam.metrics.cells import DistributionData
   from apache_beam.metrics.cells import MetricCell
+  from apache_beam.metrics.cells import MetricCellFactory
   from apache_beam.metrics.metricbase import MetricName
   from apache_beam.portability.api import metrics_pb2
 
@@ -169,7 +170,7 @@
 class _TypedMetricName(object):
   """Like MetricName, but also stores the cell type of the metric."""
   def __init__(self,
-               cell_type,  # type: Type[MetricCell]
+               cell_type,  # type: Union[Type[MetricCell], MetricCellFactory]
                metric_name  # type: Union[str, MetricName]
               ):
     # type: (...) -> None
@@ -203,7 +204,7 @@
 class MetricUpdater(object):
   """A callable that updates the metric as quickly as possible."""
   def __init__(self,
-               cell_type,  # type: Type[MetricCell]
+               cell_type,  # type: Union[Type[MetricCell], MetricCellFactory]
                metric_name,  # type: Union[str, MetricName]
                default=None):
     self.typed_metric_name = _TypedMetricName(cell_type, metric_name)
@@ -300,7 +301,10 @@
         for key,
         cell in self.metrics.items()
     ]
-    return {monitoring_infos.to_key(mi): mi for mi in all_user_metrics}
+    return {
+        monitoring_infos.to_key(mi): mi
+        for mi in all_user_metrics if mi is not None
+    }
 
   def reset(self):
     # type: () -> None
diff --git a/sdks/python/apache_beam/metrics/metric.py b/sdks/python/apache_beam/metrics/metric.py
index 39a7347..335c568 100644
--- a/sdks/python/apache_beam/metrics/metric.py
+++ b/sdks/python/apache_beam/metrics/metric.py
@@ -29,6 +29,7 @@
 
 from __future__ import absolute_import
 
+import logging
 from builtins import object
 from typing import TYPE_CHECKING
 from typing import Dict
@@ -53,6 +54,8 @@
 
 __all__ = ['Metrics', 'MetricsFilter']
 
+_LOGGER = logging.getLogger(__name__)
+
 
 class Metrics(object):
   """Lets users create/access metric objects during pipeline execution."""
diff --git a/sdks/python/apache_beam/metrics/metricbase.py b/sdks/python/apache_beam/metrics/metricbase.py
index 26ecd87..2c9e6a8 100644
--- a/sdks/python/apache_beam/metrics/metricbase.py
+++ b/sdks/python/apache_beam/metrics/metricbase.py
@@ -38,7 +38,9 @@
 
 from builtins import object
 
-__all__ = ['Metric', 'Counter', 'Distribution', 'Gauge', 'MetricName']
+__all__ = [
+    'Metric', 'Counter', 'Distribution', 'Gauge', 'Histogram', 'MetricName'
+]
 
 
 class MetricName(object):
@@ -111,3 +113,12 @@
   execution."""
   def set(self, value):
     raise NotImplementedError
+
+
+class Histogram(Metric):
+  """Histogram Metric interface.
+
+  Allows statistics about the percentile of a variable to be collected during
+  pipeline execution."""
+  def update(self, value):
+    raise NotImplementedError
diff --git a/sdks/python/apache_beam/options/pipeline_options.py b/sdks/python/apache_beam/options/pipeline_options.py
index 1047091..5006016 100644
--- a/sdks/python/apache_beam/options/pipeline_options.py
+++ b/sdks/python/apache_beam/options/pipeline_options.py
@@ -52,6 +52,7 @@
     'ProfilingOptions',
     'SetupOptions',
     'TestOptions',
+    'S3Options'
 ]
 
 PipelineOptionsT = TypeVar('PipelineOptionsT', bound='PipelineOptions')
@@ -718,27 +719,6 @@
     return errors
 
 
-class BigQueryOptions(PipelineOptions):
-  """BigQueryIO configuration options."""
-  @classmethod
-  def _add_argparse_args(cls, parser):
-    parser.add_argument(
-        '--latency_logging_frequency_sec',
-        type=int,
-        default=180,
-        help=(
-            'The minimum duration in seconds between percentile latencies '
-            'logging. The interval might be longer than the specified value '
-            'due to each bundle processing time.'))
-
-  def validate(self, validator):
-    errors = []
-    errors.extend(
-        validator.validate_optional_argument_positive(
-            self, 'latency_logging_frequency_sec'))
-    return errors
-
-
 # Command line options controlling the worker pool configuration.
 # TODO(silviuc): Update description when autoscaling options are in.
 class WorkerOptions(PipelineOptions):
@@ -1342,3 +1322,44 @@
       for name, value in override.items():
         setattr(options, name, value)
     return options
+
+
+class S3Options(PipelineOptions):
+  @classmethod
+  def _add_argparse_args(cls, parser):
+    # These options are passed to the S3 IO Client
+    parser.add_argument(
+        '--s3_access_key_id',
+        default=None,
+        help='The secret key to use when creating the s3 client.')
+    parser.add_argument(
+        '--s3_secret_access_key',
+        default=None,
+        help='The secret key to use when creating the s3 client.')
+    parser.add_argument(
+        '--s3_session_token',
+        default=None,
+        help='The session token to use when creating the s3 client.')
+    parser.add_argument(
+        '--s3_endpoint_url',
+        default=None,
+        help='The complete URL to use for the constructed s3 client.')
+    parser.add_argument(
+        '--s3_region_name',
+        default=None,
+        help='The name of the region associated with the s3 client.')
+    parser.add_argument(
+        '--s3_api_version',
+        default=None,
+        help='The API version to use with the s3 client.')
+    parser.add_argument(
+        '--s3_verify',
+        default=None,
+        help='Whether or not to verify SSL certificates with the s3 client.')
+    parser.add_argument(
+        '--s3_disable_ssl',
+        default=False,
+        action='store_true',
+        help=(
+            'Whether or not to use SSL with the s3 client. '
+            'By default, SSL is used.'))
diff --git a/sdks/python/apache_beam/runners/interactive/testing/integration/goldens/Linux/7a35f487b2a5f3a9b9852a8659eeb4bd.png b/sdks/python/apache_beam/runners/interactive/testing/integration/goldens/Linux/7a35f487b2a5f3a9b9852a8659eeb4bd.png
index 3e9f47c..412d3cd 100644
--- a/sdks/python/apache_beam/runners/interactive/testing/integration/goldens/Linux/7a35f487b2a5f3a9b9852a8659eeb4bd.png
+++ b/sdks/python/apache_beam/runners/interactive/testing/integration/goldens/Linux/7a35f487b2a5f3a9b9852a8659eeb4bd.png
Binary files differ
diff --git a/sdks/python/apache_beam/runners/portability/artifact_service.py b/sdks/python/apache_beam/runners/portability/artifact_service.py
index 1f3ec1c..18537f4 100644
--- a/sdks/python/apache_beam/runners/portability/artifact_service.py
+++ b/sdks/python/apache_beam/runners/portability/artifact_service.py
@@ -33,9 +33,15 @@
 import sys
 import tempfile
 import threading
-import typing
 from io import BytesIO
+from typing import Any
+from typing import BinaryIO  # pylint: disable=unused-import
 from typing import Callable
+from typing import Dict
+from typing import List
+from typing import MutableMapping
+from typing import Optional
+from typing import Tuple
 
 import grpc
 from future.moves.urllib.request import urlopen
@@ -48,11 +54,6 @@
 from apache_beam.portability.api import beam_runner_api_pb2
 from apache_beam.utils import proto_utils
 
-if typing.TYPE_CHECKING:
-  from typing import BinaryIO  # pylint: disable=ungrouped-imports
-  from typing import Iterable
-  from typing import MutableMapping
-
 
 class ArtifactRetrievalService(
     beam_artifact_api_pb2_grpc.ArtifactRetrievalServiceServicer):
@@ -61,7 +62,7 @@
 
   def __init__(
       self,
-      file_reader,  # type: Callable[[str], BinaryIO],
+      file_reader,  # type: Callable[[str], BinaryIO]
       chunk_size=None,
   ):
     self._file_reader = file_reader
@@ -105,7 +106,8 @@
       file_writer,  # type: Callable[[str, Optional[str]], Tuple[BinaryIO, str]]
     ):
     self._lock = threading.Lock()
-    self._jobs_to_stage = {}
+    self._jobs_to_stage = {
+    }  # type: Dict[str, Tuple[Dict[Any, List[beam_runner_api_pb2.ArtifactInformation]], threading.Event]]
     self._file_writer = file_writer
 
   def register_job(
diff --git a/sdks/python/apache_beam/runners/portability/fn_api_runner/execution.py b/sdks/python/apache_beam/runners/portability/fn_api_runner/execution.py
index 5b8e91c..bc69123 100644
--- a/sdks/python/apache_beam/runners/portability/fn_api_runner/execution.py
+++ b/sdks/python/apache_beam/runners/portability/fn_api_runner/execution.py
@@ -17,6 +17,8 @@
 
 """Set of utilities for execution of a pipeline by the FnApiRunner."""
 
+# mypy: disallow-untyped-defs
+
 from __future__ import absolute_import
 
 import collections
@@ -26,10 +28,12 @@
 from typing import Any
 from typing import DefaultDict
 from typing import Dict
+from typing import Iterable
 from typing import Iterator
 from typing import List
 from typing import MutableMapping
 from typing import Optional
+from typing import Set
 from typing import Tuple
 
 from typing_extensions import Protocol
@@ -59,9 +63,13 @@
 from apache_beam.utils import windowed_value
 
 if TYPE_CHECKING:
-  from apache_beam.coders.coder_impl import CoderImpl
+  from apache_beam.coders.coder_impl import CoderImpl, WindowedValueCoderImpl
+  from apache_beam.portability.api import endpoints_pb2
   from apache_beam.runners.portability.fn_api_runner import worker_handlers
+  from apache_beam.runners.portability.fn_api_runner.fn_runner import DataOutput
+  from apache_beam.runners.portability.fn_api_runner.fn_runner import OutputTimers
   from apache_beam.runners.portability.fn_api_runner.translations import DataSideInput
+  from apache_beam.transforms import core
   from apache_beam.transforms.window import BoundedWindow
 
 ENCODED_IMPULSE_VALUE = WindowedValueCoder(
@@ -87,13 +95,27 @@
     # type: (int) -> List[List[bytes]]
     pass
 
+  @property
+  def cleared(self):
+    # type: () -> bool
+    pass
+
+  def clear(self):
+    # type: () -> None
+    pass
+
+  def reset(self):
+    # type: () -> None
+    pass
+
 
 class ListBuffer(object):
   """Used to support parititioning of a list."""
   def __init__(self, coder_impl):
+    # type: (CoderImpl) -> None
     self._coder_impl = coder_impl
     self._inputs = []  # type: List[bytes]
-    self._grouped_output = None
+    self._grouped_output = None  # type: Optional[List[List[bytes]]]
     self.cleared = False
 
   def append(self, element):
@@ -139,6 +161,8 @@
     self._grouped_output = None
 
   def reset(self):
+    # type: () -> None
+
     """Resets a cleared buffer for reuse."""
     if not self.cleared:
       raise RuntimeError('Trying to reset a non-cleared ListBuffer.')
@@ -150,7 +174,7 @@
   def __init__(self,
                pre_grouped_coder,  # type: coders.Coder
                post_grouped_coder,  # type: coders.Coder
-               windowing
+               windowing  # type: core.Windowing
               ):
     # type: (...) -> None
     self._key_coder = pre_grouped_coder.key_coder()
@@ -227,12 +251,24 @@
     """
     return itertools.chain(*self.partition(1))
 
+  # these should never be accessed, but they allow this class to meet the
+  # PartionableBuffer protocol
+  cleared = False
+
+  def clear(self):
+    # type: () -> None
+    pass
+
+  def reset(self):
+    # type: () -> None
+    pass
+
 
 class WindowGroupingBuffer(object):
   """Used to partition windowed side inputs."""
   def __init__(
       self,
-      access_pattern,
+      access_pattern,  # type: beam_runner_api_pb2.FunctionSpec
       coder  # type: WindowedValueCoder
   ):
     # type: (...) -> None
@@ -283,17 +319,21 @@
   URN = 'internal-generic-non-merging'
 
   def __init__(self, coder):
+    # type: (coders.Coder) -> None
     self._coder = coder
 
   def assign(self, assign_context):
+    # type: (window.WindowFn.AssignContext) -> Iterable[BoundedWindow]
     raise NotImplementedError()
 
   def get_window_coder(self):
+    # type: () -> coders.Coder
     return self._coder
 
   @staticmethod
   @window.urns.RunnerApiFn.register_urn(URN, bytes)
   def from_runner_api_parameter(window_coder_id, context):
+    # type: (bytes, Any) -> GenericNonMergingWindowFn
     return GenericNonMergingWindowFn(
         context.coders[window_coder_id.decode('utf-8')])
 
@@ -308,9 +348,11 @@
       stages,  # type: List[translations.Stage]
       worker_handler_manager,  # type: worker_handlers.WorkerHandlerManager
       pipeline_components,  # type: beam_runner_api_pb2.Components
-      safe_coders,
-      data_channel_coders,
+      safe_coders,  # type: Dict[str, str]
+      data_channel_coders,  # type: Dict[str, str]
                ):
+    # type: (...) -> None
+
     """
     :param worker_handler_manager: This class manages the set of worker
         handlers, and the communication with state / control APIs.
@@ -365,7 +407,7 @@
       return all_side_inputs
 
     all_side_inputs = frozenset(get_all_side_inputs())
-    data_side_inputs_by_producing_stage = {}
+    data_side_inputs_by_producing_stage = {}  # type: Dict[str, DataSideInput]
 
     producing_stages_by_pcoll = {}
 
@@ -397,6 +439,7 @@
     return data_side_inputs_by_producing_stage
 
   def _make_safe_windowing_strategy(self, id):
+    # type: (str) -> str
     windowing_strategy_proto = self.pipeline_components.windowing_strategies[id]
     if windowing_strategy_proto.window_fn.urn in SAFE_WINDOW_FNS:
       return id
@@ -420,15 +463,17 @@
 
   @property
   def state_servicer(self):
+    # type: () -> worker_handlers.StateServicer
     # TODO(BEAM-9625): Ensure FnApiRunnerExecutionContext owns StateServicer
     return self.worker_handler_manager.state_servicer
 
   def next_uid(self):
+    # type: () -> str
     self._last_uid += 1
     return str(self._last_uid)
 
   def _iterable_state_write(self, values, element_coder_impl):
-    # type: (...) -> bytes
+    # type: (Iterable, CoderImpl) -> bytes
     token = unique_name(None, 'iter').encode('ascii')
     out = create_OutputStream()
     for element in values:
@@ -484,21 +529,23 @@
                stage,  # type: translations.Stage
                num_workers,  # type: int
               ):
+    # type: (...) -> None
     self.execution_context = execution_context
     self.stage = stage
     self.bundle_uid = self.execution_context.next_uid()
     self.num_workers = num_workers
 
     # Properties that are lazily initialized
-    self._process_bundle_descriptor = None
-    self._worker_handlers = None
+    self._process_bundle_descriptor = None  # type: Optional[beam_fn_api_pb2.ProcessBundleDescriptor]
+    self._worker_handlers = None  # type: Optional[List[worker_handlers.WorkerHandler]]
     # a mapping of {(transform_id, timer_family_id): timer_coder_id}. The map
     # is built after self._process_bundle_descriptor is initialized.
     # This field can be used to tell whether current bundle has timers.
-    self._timer_coder_ids = None
+    self._timer_coder_ids = None  # type: Optional[Dict[Tuple[str, str], str]]
 
   @property
   def worker_handlers(self):
+    # type: () -> List[worker_handlers.WorkerHandler]
     if self._worker_handlers is None:
       self._worker_handlers = (
           self.execution_context.worker_handler_manager.get_worker_handlers(
@@ -506,23 +553,27 @@
     return self._worker_handlers
 
   def data_api_service_descriptor(self):
+    # type: () -> Optional[endpoints_pb2.ApiServiceDescriptor]
     # All worker_handlers share the same grpc server, so we can read grpc server
     # info from any worker_handler and read from the first worker_handler.
     return self.worker_handlers[0].data_api_service_descriptor()
 
   def state_api_service_descriptor(self):
+    # type: () -> Optional[endpoints_pb2.ApiServiceDescriptor]
     # All worker_handlers share the same grpc server, so we can read grpc server
     # info from any worker_handler and read from the first worker_handler.
     return self.worker_handlers[0].state_api_service_descriptor()
 
   @property
   def process_bundle_descriptor(self):
+    # type: () -> beam_fn_api_pb2.ProcessBundleDescriptor
     if self._process_bundle_descriptor is None:
       self._process_bundle_descriptor = self._build_process_bundle_descriptor()
       self._timer_coder_ids = self._build_timer_coders_id_map()
     return self._process_bundle_descriptor
 
   def _build_process_bundle_descriptor(self):
+    # type: () -> beam_fn_api_pb2.ProcessBundleDescriptor
     # Cannot be invoked until *after* _extract_endpoints is called.
     # Always populate the timer_api_service_descriptor.
     return beam_fn_api_pb2.ProcessBundleDescriptor(
@@ -543,7 +594,7 @@
         timer_api_service_descriptor=self.data_api_service_descriptor())
 
   def extract_bundle_inputs_and_outputs(self):
-    # type: (...) -> Tuple[Dict[str, PartitionableBuffer], DataOutput, Dict[Tuple[str, str], str]]
+    # type: () -> Tuple[Dict[str, PartitionableBuffer], DataOutput, Dict[Tuple[str, str], bytes]]
 
     """Returns maps of transform names to PCollection identifiers.
 
@@ -560,7 +611,7 @@
     data_input = {}  # type: Dict[str, PartitionableBuffer]
     data_output = {}  # type: DataOutput
     # A mapping of {(transform_id, timer_family_id) : buffer_id}
-    expected_timer_output = {}  # type: Dict[Tuple[str, str], str]
+    expected_timer_output = {}  # type: OutputTimers
     for transform in self.stage.transforms:
       if transform.spec.urn in (bundle_processor.DATA_INPUT_URN,
                                 bundle_processor.DATA_OUTPUT_URN):
@@ -609,6 +660,8 @@
     return self.get_coder_impl(coder_id)
 
   def _build_timer_coders_id_map(self):
+    # type: () -> Dict[Tuple[str, str], str]
+    assert self._process_bundle_descriptor is not None
     timer_coder_ids = {}
     for transform_id, transform_proto in (self._process_bundle_descriptor
         .transforms.items()):
@@ -621,6 +674,7 @@
     return timer_coder_ids
 
   def get_coder_impl(self, coder_id):
+    # type: (str) -> CoderImpl
     if coder_id in self.execution_context.safe_coders:
       return self.execution_context.pipeline_context.coders[
           self.execution_context.safe_coders[coder_id]].get_impl()
@@ -628,6 +682,8 @@
       return self.execution_context.pipeline_context.coders[coder_id].get_impl()
 
   def get_timer_coder_impl(self, transform_id, timer_family_id):
+    # type: (str, str) -> CoderImpl
+    assert self._timer_coder_ids is not None
     return self.get_coder_impl(
         self._timer_coder_ids[(transform_id, timer_family_id)])
 
diff --git a/sdks/python/apache_beam/runners/portability/fn_api_runner/fn_runner.py b/sdks/python/apache_beam/runners/portability/fn_api_runner/fn_runner.py
index 9c43f33..404261f 100644
--- a/sdks/python/apache_beam/runners/portability/fn_api_runner/fn_runner.py
+++ b/sdks/python/apache_beam/runners/portability/fn_api_runner/fn_runner.py
@@ -18,6 +18,7 @@
 """A PipelineRunner using the SDK harness.
 """
 # pytype: skip-file
+# mypy: check-untyped-defs
 
 from __future__ import absolute_import
 from __future__ import print_function
@@ -36,12 +37,14 @@
 from typing import TYPE_CHECKING
 from typing import Callable
 from typing import Dict
+from typing import Iterator
 from typing import List
 from typing import Mapping
 from typing import MutableMapping
 from typing import Optional
 from typing import Tuple
 from typing import TypeVar
+from typing import Union
 
 from apache_beam.coders.coder_impl import create_OutputStream
 from apache_beam.metrics import metric
@@ -62,13 +65,14 @@
 from apache_beam.runners.portability.fn_api_runner.translations import only_element
 from apache_beam.runners.portability.fn_api_runner.worker_handlers import WorkerHandlerManager
 from apache_beam.transforms import environments
-from apache_beam.utils import profiler
 from apache_beam.utils import proto_utils
 from apache_beam.utils import thread_pool_executor
+from apache_beam.utils.profiler import Profile
 
 if TYPE_CHECKING:
   from apache_beam.pipeline import Pipeline
   from apache_beam.portability.api import metrics_pb2
+  from apache_beam.runners.portability.fn_api_runner.worker_handlers import WorkerHandler
 
 _LOGGER = logging.getLogger(__name__)
 
@@ -77,6 +81,7 @@
 DataSideInput = Dict[Tuple[str, str],
                      Tuple[bytes, beam_runner_api_pb2.FunctionSpec]]
 DataOutput = Dict[str, bytes]
+OutputTimers = Dict[Tuple[str, str], bytes]
 BundleProcessResult = Tuple[beam_fn_api_pb2.InstructionResponse,
                             List[beam_fn_api_pb2.ProcessBundleSplitResponse]]
 
@@ -88,11 +93,12 @@
   def __init__(
       self,
       default_environment=None,  # type: Optional[environments.Environment]
-      bundle_repeat=0,
-      use_state_iterables=False,
+      bundle_repeat=0,  # type: int
+      use_state_iterables=False,  # type: bool
       provision_info=None,  # type: Optional[ExtendedProvisionInfo]
-      progress_request_frequency=None,
-      is_drain=False):
+      progress_request_frequency=None,  # type: Optional[float]
+      is_drain=False  # type: bool
+  ):
     # type: (...) -> None
 
     """Creates a new Fn API Runner.
@@ -114,7 +120,7 @@
     self._bundle_repeat = bundle_repeat
     self._num_workers = 1
     self._progress_frequency = progress_request_frequency
-    self._profiler_factory = None  # type: Optional[Callable[..., profiler.Profile]]
+    self._profiler_factory = None  # type: Optional[Callable[..., Profile]]
     self._use_state_iterables = use_state_iterables
     self._is_drain = is_drain
     self._provision_info = provision_info or ExtendedProvisionInfo(
@@ -123,6 +129,7 @@
 
   @staticmethod
   def supported_requirements():
+    # type: () -> Tuple[str, ...]
     return (
         common_urns.requirements.REQUIRES_STATEFUL_PROCESSING.urn,
         common_urns.requirements.REQUIRES_BUNDLE_FINALIZATION.urn,
@@ -169,7 +176,7 @@
       self._default_environment = environments.SubprocessSDKEnvironment(
           command_string=command_string)
 
-    self._profiler_factory = profiler.Profile.factory_from_options(
+    self._profiler_factory = Profile.factory_from_options(
         options.view_as(pipeline_options.ProfilingOptions))
 
     self._latest_run_result = self.run_via_runner_api(
@@ -187,6 +194,7 @@
 
   @contextlib.contextmanager
   def maybe_profile(self):
+    # type: () -> Iterator[None]
     if self._profiler_factory:
       try:
         profile_id = 'direct-' + subprocess.check_output([
@@ -194,7 +202,8 @@
         ]).decode(errors='ignore').strip()
       except subprocess.CalledProcessError:
         profile_id = 'direct-unknown'
-      profiler = self._profiler_factory(profile_id, time_prefix='')
+      profiler = self._profiler_factory(
+          profile_id, time_prefix='')  # type: Optional[Profile]
     else:
       profiler = None
 
@@ -231,10 +240,13 @@
       yield
 
   def _validate_requirements(self, pipeline_proto):
+    # type: (beam_runner_api_pb2.Pipeline) -> None
+
     """As a test runner, validate requirements were set correctly."""
     expected_requirements = set()
 
     def add_requirements(transform_id):
+      # type: (str) -> None
       transform = pipeline_proto.components.transforms[transform_id]
       if transform.spec.urn in translations.PAR_DO_URNS:
         payload = proto_utils.parse_Bytes(
@@ -266,6 +278,8 @@
           (expected_requirements - set(pipeline_proto.requirements)))
 
   def _check_requirements(self, pipeline_proto):
+    # type: (beam_runner_api_pb2.Pipeline) -> None
+
     """Check that this runner can satisfy all pipeline requirements."""
     supported_requirements = set(self.supported_requirements())
     for requirement in pipeline_proto.requirements:
@@ -355,10 +369,10 @@
       self,
       runner_execution_context,  # type: execution.FnApiRunnerExecutionContext
       bundle_manager,  # type: BundleManager
-      data_input,
+      data_input,  # type: Dict[str, execution.PartitionableBuffer]
       data_output,  # type: DataOutput
-      fired_timers,
-      expected_output_timers,
+      fired_timers,  # type: Mapping[Tuple[str, str], execution.PartitionableBuffer]
+      expected_output_timers,  # type: Dict[Tuple[str, str], bytes]
   ):
     # type: (...) -> None
 
@@ -407,7 +421,12 @@
         written_timers.clear()
 
   def _add_sdk_delayed_applications_to_deferred_inputs(
-      self, bundle_context_manager, bundle_result, deferred_inputs):
+      self,
+      bundle_context_manager,  # type: execution.BundleContextManager
+      bundle_result,  # type: beam_fn_api_pb2.InstructionResponse
+      deferred_inputs  # type: MutableMapping[str, execution.PartitionableBuffer]
+  ):
+    # type: (...) -> None
     for delayed_application in bundle_result.process_bundle.residual_roots:
       name = bundle_context_manager.input_for(
           delayed_application.application.transform_id,
@@ -421,8 +440,8 @@
       self,
       splits,  # type: List[beam_fn_api_pb2.ProcessBundleSplitResponse]
       bundle_context_manager,  # type: execution.BundleContextManager
-      last_sent,
-      deferred_inputs  # type: MutableMapping[str, PartitionableBuffer]
+      last_sent,  # type: Dict[str, execution.PartitionableBuffer]
+      deferred_inputs  # type: MutableMapping[str, execution.PartitionableBuffer]
   ):
     # type: (...) -> None
 
@@ -485,7 +504,8 @@
     """
     data_input, data_output, expected_timer_output = (
         bundle_context_manager.extract_bundle_inputs_and_outputs())
-    input_timers = {}
+    input_timers = {
+    }  # type: Mapping[Tuple[str, str], execution.PartitionableBuffer]
 
     worker_handler_manager = runner_execution_context.worker_handler_manager
     _LOGGER.info('Running %s', bundle_context_manager.stage.name)
@@ -509,9 +529,11 @@
         self._progress_frequency,
         cache_token_generator=cache_token_generator)
 
-    final_result = None
+    final_result = None  # type: Optional[beam_fn_api_pb2.InstructionResponse]
 
     def merge_results(last_result):
+      # type: (beam_fn_api_pb2.InstructionResponse) -> beam_fn_api_pb2.InstructionResponse
+
       """ Merge the latest result with other accumulated results. """
       return (
           last_result
@@ -539,7 +561,6 @@
       else:
         data_input = deferred_inputs
         input_timers = fired_timers
-        bundle_manager._registered = True
 
     # Store the required downstream side inputs into state so it is accessible
     # for the worker when it runs bundles that consume this stage's output.
@@ -552,13 +573,16 @@
 
   def _run_bundle(
       self,
-      runner_execution_context,
-      bundle_context_manager,
-      data_input,
-      data_output,
-      input_timers,
-      expected_timer_output,
-      bundle_manager):
+      runner_execution_context,  # type: execution.FnApiRunnerExecutionContext
+      bundle_context_manager,  # type: execution.BundleContextManager
+      data_input,  # type: Dict[str, execution.PartitionableBuffer]
+      data_output,  # type: DataOutput
+      input_timers,  # type: Mapping[Tuple[str, str], execution.PartitionableBuffer]
+      expected_timer_output,  # type: Dict[Tuple[str, str], bytes]
+      bundle_manager  # type: BundleManager
+  ):
+    # type: (...) -> Tuple[beam_fn_api_pb2.InstructionResponse, Dict[str, execution.PartitionableBuffer], Dict[Tuple[str, str], ListBuffer]]
+
     """Execute a bundle, and return a result object, and deferred inputs."""
     self._run_bundle_multiple_times_for_testing(
         runner_execution_context,
@@ -576,7 +600,7 @@
     # - SDK-initiated deferred applications of root elements
     # - Runner-initiated deferred applications of root elements
     deferred_inputs = {}  # type: Dict[str, execution.PartitionableBuffer]
-    fired_timers = {}
+    fired_timers = {}  # type: Dict[Tuple[str, str], ListBuffer]
 
     self._collect_written_timers_and_add_to_fired_timers(
         bundle_context_manager, fired_timers)
@@ -601,12 +625,15 @@
 
   @staticmethod
   def get_cache_token_generator(static=True):
+    # type: (bool) -> Iterator[beam_fn_api_pb2.ProcessBundleRequest.CacheToken]
+
     """A generator for cache tokens.
          :arg static If True, generator always returns the same cache token
                      If False, generator returns a new cache token each time
          :return A generator which returns a cache token on next(generator)
      """
     def generate_token(identifier):
+      # type: (int) -> beam_fn_api_pb2.ProcessBundleRequest.CacheToken
       return beam_fn_api_pb2.ProcessBundleRequest.CacheToken(
           user_state=beam_fn_api_pb2.ProcessBundleRequest.CacheToken.UserState(
           ),
@@ -614,44 +641,55 @@
 
     class StaticGenerator(object):
       def __init__(self):
+        # type: () -> None
         self._token = generate_token(1)
 
       def __iter__(self):
+        # type: () -> StaticGenerator
         # pylint: disable=non-iterator-returned
         return self
 
       def __next__(self):
+        # type: () -> beam_fn_api_pb2.ProcessBundleRequest.CacheToken
         return self._token
 
     class DynamicGenerator(object):
       def __init__(self):
+        # type: () -> None
         self._counter = 0
         self._lock = threading.Lock()
 
       def __iter__(self):
+        # type: () -> DynamicGenerator
         # pylint: disable=non-iterator-returned
         return self
 
       def __next__(self):
+        # type: () -> beam_fn_api_pb2.ProcessBundleRequest.CacheToken
         with self._lock:
           self._counter += 1
           return generate_token(self._counter)
 
-    return StaticGenerator() if static else DynamicGenerator()
+    if static:
+      return StaticGenerator()
+    else:
+      return DynamicGenerator()
 
 
 class ExtendedProvisionInfo(object):
   def __init__(self,
                provision_info=None,  # type: Optional[beam_provision_api_pb2.ProvisionInfo]
-               artifact_staging_dir=None,
+               artifact_staging_dir=None,  # type: Optional[str]
                job_name=None,  # type: Optional[str]
               ):
+    # type: (...) -> None
     self.provision_info = (
         provision_info or beam_provision_api_pb2.ProvisionInfo())
     self.artifact_staging_dir = artifact_staging_dir
     self.job_name = job_name
 
   def for_environment(self, env):
+    # type: (...) -> ExtendedProvisionInfo
     if env.dependencies:
       provision_info_with_deps = copy.deepcopy(self.provision_info)
       provision_info_with_deps.dependencies.extend(env.dependencies)
@@ -699,9 +737,11 @@
 
   def __init__(self,
                bundle_context_manager,  # type: execution.BundleContextManager
-               progress_frequency=None,
+               progress_frequency=None,  # type: Optional[float]
                cache_token_generator=FnApiRunner.get_cache_token_generator()
               ):
+    # type: (...) -> None
+
     """Set up a bundle manager.
 
     Args:
@@ -709,7 +749,7 @@
     """
     self.bundle_context_manager = bundle_context_manager  # type: execution.BundleContextManager
     self._progress_frequency = progress_frequency
-    self._worker_handler = None  # type: Optional[execution.WorkerHandler]
+    self._worker_handler = None  # type: Optional[WorkerHandler]
     self._cache_token_generator = cache_token_generator
 
   def _send_input_to_worker(self,
@@ -727,6 +767,7 @@
 
   def _send_timers_to_worker(
       self, process_bundle_id, transform_id, timer_family_id, timers):
+    # type: (...) -> None
     assert self._worker_handler is not None
     timer_out = self._worker_handler.data_conn.output_timer_stream(
         process_bundle_id, transform_id, timer_family_id)
@@ -751,8 +792,9 @@
 
   def _generate_splits_for_testing(self,
                                    split_manager,
-                                   inputs,  # type: Mapping[str, PartitionableBuffer]
-                                   process_bundle_id):
+                                   inputs,  # type: Mapping[str, execution.PartitionableBuffer]
+                                   process_bundle_id
+                                  ):
     # type: (...) -> List[beam_fn_api_pb2.ProcessBundleSplitResponse]
     split_results = []  # type: List[beam_fn_api_pb2.ProcessBundleSplitResponse]
     read_transform_id, buffer_data = only_element(inputs.items())
@@ -819,8 +861,8 @@
                      inputs,  # type: Mapping[str, execution.PartitionableBuffer]
                      expected_outputs,  # type: DataOutput
                      fired_timers,  # type: Mapping[Tuple[str, str], execution.PartitionableBuffer]
-                     expected_output_timers,  # type: Dict[Tuple[str, str], str]
-                     dry_run=False,
+                     expected_output_timers,  # type: OutputTimers
+                     dry_run=False,  # type: bool
                     ):
     # type: (...) -> BundleProcessResult
     # Unique id for the instruction processing this bundle.
@@ -863,7 +905,8 @@
         split_results = self._generate_splits_for_testing(
             split_manager, inputs, process_bundle_id)
 
-      expect_reads = list(expected_outputs.keys())
+      expect_reads = list(
+          expected_outputs.keys())  # type: List[Union[str, Tuple[str, str]]]
       expect_reads.extend(list(expected_output_timers.keys()))
 
       # Gather all output data.
@@ -871,7 +914,7 @@
           process_bundle_id,
           expect_reads,
           abort_callback=lambda:
-          (result_future.is_done() and result_future.get().error)):
+          (result_future.is_done() and bool(result_future.get().error))):
         if isinstance(output, beam_fn_api_pb2.Elements.Timers) and not dry_run:
           with BundleManager._lock:
             timer_buffer = self.bundle_context_manager.get_buffer(
@@ -910,7 +953,7 @@
   def __init__(
       self,
       bundle_context_manager,  # type: execution.BundleContextManager
-      progress_frequency=None,
+      progress_frequency=None,  # type: Optional[float]
       cache_token_generator=None,
       **kwargs):
     # type: (...) -> None
@@ -924,8 +967,8 @@
                      inputs,  # type: Mapping[str, execution.PartitionableBuffer]
                      expected_outputs,  # type: DataOutput
                      fired_timers,  # type: Mapping[Tuple[str, str], execution.PartitionableBuffer]
-                     expected_output_timers,  # type: Dict[Tuple[str, str], str]
-                     dry_run=False,
+                     expected_output_timers,  # type: OutputTimers
+                     dry_run=False,  # type: bool
                     ):
     # type: (...) -> BundleProcessResult
     part_inputs = [{} for _ in range(self._num_workers)
@@ -993,7 +1036,7 @@
     self._instruction_id = instruction_id
     self._frequency = frequency
     self._done = False
-    self._latest_progress = None
+    self._latest_progress = None  # type: Optional[beam_fn_api_pb2.ProcessBundleProgressResponse]
     self._callback = callback
     self.daemon = True
 
diff --git a/sdks/python/apache_beam/runners/portability/fn_api_runner/translations.py b/sdks/python/apache_beam/runners/portability/fn_api_runner/translations.py
index e4d03c8..d9d722c 100644
--- a/sdks/python/apache_beam/runners/portability/fn_api_runner/translations.py
+++ b/sdks/python/apache_beam/runners/portability/fn_api_runner/translations.py
@@ -18,6 +18,7 @@
 """Pipeline transformations for the FnApiRunner.
 """
 # pytype: skip-file
+# mypy: check-untyped-defs
 
 from __future__ import absolute_import
 from __future__ import print_function
@@ -261,12 +262,11 @@
       stage_components = beam_runner_api_pb2.Components()
       stage_components.CopyFrom(components)
 
-      # Only keep the referenced PCollections.
-      # Make pcollectionKey snapshot to avoid "Map modified during iteration"
-      # in py3
-      for pcoll_id in list(stage_components.pcollections.keys()):
-        if pcoll_id not in all_inputs and pcoll_id not in all_outputs:
-          del stage_components.pcollections[pcoll_id]
+      # Only keep the PCollections referenced in this stage.
+      stage_components.pcollections.clear()
+      for pcoll_id in all_inputs.union(all_outputs):
+        stage_components.pcollections[pcoll_id].CopyFrom(
+            components.pcollections[pcoll_id])
 
       # Only keep the transforms in this stage.
       # Also gather up payload data as we iterate over the transforms.
@@ -307,6 +307,9 @@
           for side in side_inputs
       },
                           main_input=main_input_id)
+      # at this point we should have resolved an environment, as the key of
+      # components.environments cannot be None.
+      assert self.environment is not None
       exec_payload = beam_runner_api_pb2.ExecutableStagePayload(
           environment=components.environments[self.environment],
           input=main_input_id,
@@ -373,7 +376,7 @@
         None)  # type: ignore[arg-type]
     self.bytes_coder_id = self.add_or_get_coder_id(coder_proto, 'bytes_coder')
     self.safe_coders = {self.bytes_coder_id: self.bytes_coder_id}
-    self.data_channel_coders = {}
+    self.data_channel_coders = {}  # type: Dict[str, str]
 
   def add_or_get_coder_id(
       self,
@@ -517,6 +520,7 @@
   new_proto.CopyFrom(pipeline_proto)
   components = new_proto.components
   components.transforms.clear()
+  components.pcollections.clear()
 
   roots = set()
   parents = {
@@ -526,6 +530,11 @@
       for child in proto.subtransforms
   }
 
+  def copy_output_pcollections(transform):
+    for pcoll_id in transform.outputs.values():
+      components.pcollections[pcoll_id].CopyFrom(
+          pipeline_proto.components.pcollections[pcoll_id])
+
   def add_parent(child, parent):
     if parent is None:
       roots.add(child)
@@ -536,6 +545,7 @@
           parent in pipeline_proto.components.transforms):
         components.transforms[parent].CopyFrom(
             pipeline_proto.components.transforms[parent])
+        copy_output_pcollections(components.transforms[parent])
         del components.transforms[parent].subtransforms[:]
         add_parent(parent, parents.get(parent))
       components.transforms[parent].subtransforms.append(child)
@@ -547,6 +557,7 @@
             'Could not find subtransform to copy: ' + subtransform_id)
       subtransform = pipeline_proto.components.transforms[subtransform_id]
       components.transforms[subtransform_id].CopyFrom(subtransform)
+      copy_output_pcollections(components.transforms[subtransform_id])
       copy_subtransforms(subtransform)
 
   all_consumers = collections.defaultdict(
@@ -562,9 +573,10 @@
       copy_subtransforms(transform)
     else:
       transform = stage.executable_stage_transform(
-          known_runner_urns, all_consumers, components)
+          known_runner_urns, all_consumers, pipeline_proto.components)
     transform_id = unique_name(components.transforms, stage.name)
     components.transforms[transform_id].CopyFrom(transform)
+    copy_output_pcollections(transform)
     add_parent(transform_id, stage.parent)
 
   del new_proto.root_transform_ids[:]
@@ -812,7 +824,8 @@
 
   def _get_fallback_coder_id():
     return context.add_or_get_coder_id(
-        coders.registry.get_coder(object).to_runner_api(None))
+        # passing None works here because there are no component coders
+        coders.registry.get_coder(object).to_runner_api(None))  # type: ignore[arg-type]
 
   def _get_component_coder_id_from_kv_coder(coder, index):
     assert index < 2
@@ -922,11 +935,14 @@
             is_bounded=input_pcoll.is_bounded))
 
     # Set up Pack stage.
+    # TODO(BEAM-7746): classes that inherit from RunnerApiFn are expected to
+    #  accept a PipelineContext for from_runner_api/to_runner_api.  Determine
+    #  how to accomodate this.
     pack_combine_fn = combiners.SingleInputTupleCombineFn(
         *[
-            core.CombineFn.from_runner_api(combine_payload.combine_fn, context)
+            core.CombineFn.from_runner_api(combine_payload.combine_fn, context)  # type: ignore[arg-type]
             for combine_payload in combine_payloads
-        ]).to_runner_api(context)
+        ]).to_runner_api(context)  # type: ignore[arg-type]
     pack_transform = beam_runner_api_pb2.PTransform(
         unique_name=pack_combine_name + '/Pack',
         spec=beam_runner_api_pb2.FunctionSpec(
@@ -1480,14 +1496,17 @@
 
 
 def greedily_fuse(stages, pipeline_context):
+  # type: (Iterable[Stage], TransformContext) -> FrozenSet[Stage]
+
   """Places transforms sharing an edge in the same stage, whenever possible.
   """
-  producers_by_pcoll = {}
-  consumers_by_pcoll = collections.defaultdict(list)
+  producers_by_pcoll = {}  # type: Dict[str, Stage]
+  consumers_by_pcoll = collections.defaultdict(
+      list)  # type: DefaultDict[str, List[Stage]]
 
   # Used to always reference the correct stage as the producer and
   # consumer maps are not updated when stages are fused away.
-  replacements = {}
+  replacements = {}  # type: Dict[Stage, Stage]
 
   def replacement(s):
     old_ss = []
diff --git a/sdks/python/apache_beam/runners/portability/fn_api_runner/worker_handlers.py b/sdks/python/apache_beam/runners/portability/fn_api_runner/worker_handlers.py
index 1f1d483..756e4ba 100644
--- a/sdks/python/apache_beam/runners/portability/fn_api_runner/worker_handlers.py
+++ b/sdks/python/apache_beam/runners/portability/fn_api_runner/worker_handlers.py
@@ -17,6 +17,8 @@
 
 """Code for communicating with the Workers."""
 
+# mypy: disallow-untyped-defs
+
 from __future__ import absolute_import
 
 import collections
@@ -29,6 +31,9 @@
 import threading
 import time
 from builtins import object
+from typing import TYPE_CHECKING
+from typing import Any
+from typing import BinaryIO  # pylint: disable=unused-import
 from typing import Callable
 from typing import DefaultDict
 from typing import Dict
@@ -38,6 +43,8 @@
 from typing import Mapping
 from typing import Optional
 from typing import Tuple
+from typing import Type
+from typing import TypeVar
 from typing import Union
 from typing import cast
 from typing import overload
@@ -64,6 +71,12 @@
 from apache_beam.runners.worker.statecache import StateCache
 from apache_beam.utils import proto_utils
 from apache_beam.utils import thread_pool_executor
+from apache_beam.utils.sentinel import Sentinel
+
+if TYPE_CHECKING:
+  from grpc import ServicerContext
+  from google.protobuf import message
+  from apache_beam.runners.portability.fn_api_runner.fn_runner import ExtendedProvisionInfo  # pylint: disable=ungrouped-imports
 
 # State caching is enabled in the fn_api_runner for testing, except for one
 # test which runs without state caching (FnApiRunnerTestWithDisabledCaching).
@@ -75,10 +88,11 @@
 
 _LOGGER = logging.getLogger(__name__)
 
+T = TypeVar('T')
 ConstructorFn = Callable[[
     Union['message.Message', bytes],
-    'StateServicer',
-    Optional['ExtendedProvisionInfo'],
+    'sdk_worker.StateHandler',
+    'ExtendedProvisionInfo',
     'GrpcServer'
 ],
                          'WorkerHandler']
@@ -90,8 +104,9 @@
   _lock = threading.Lock()
 
   def __init__(self):
+    # type: () -> None
     self._push_queue = queue.Queue(
-    )  # type: queue.Queue[beam_fn_api_pb2.InstructionRequest]
+    )  # type: queue.Queue[Union[beam_fn_api_pb2.InstructionRequest, Sentinel]]
     self._input = None  # type: Optional[Iterable[beam_fn_api_pb2.InstructionResponse]]
     self._futures_by_id = dict()  # type: Dict[str, ControlFuture]
     self._read_thread = threading.Thread(
@@ -99,12 +114,14 @@
     self._state = BeamFnControlServicer.UNSTARTED_STATE
 
   def _read(self):
+    # type: () -> None
+    assert self._input is not None
     for data in self._input:
       self._futures_by_id.pop(data.instruction_id).set(data)
 
   @overload
   def push(self, req):
-    # type: (BeamFnControlServicer.DoneMarker) -> None
+    # type: (Sentinel) -> None
     pass
 
   @overload
@@ -113,7 +130,8 @@
     pass
 
   def push(self, req):
-    if req == BeamFnControlServicer._DONE_MARKER:
+    # type: (Union[Sentinel, beam_fn_api_pb2.InstructionRequest]) -> Optional[ControlFuture]
+    if req is BeamFnControlServicer._DONE_MARKER:
       self._push_queue.put(req)
       return None
     if not req.instruction_id:
@@ -126,7 +144,7 @@
     return future
 
   def get_req(self):
-    # type: () -> beam_fn_api_pb2.InstructionRequest
+    # type: () -> Union[Sentinel, beam_fn_api_pb2.InstructionRequest]
     return self._push_queue.get()
 
   def set_input(self, input):
@@ -147,6 +165,7 @@
       self._state = BeamFnControlServicer.DONE_STATE
 
   def abort(self, exn):
+    # type: (Exception) -> None
     for future in self._futures_by_id.values():
       future.abort(exn)
 
@@ -158,23 +177,20 @@
   STARTED_STATE = 'started'
   DONE_STATE = 'done'
 
-  class DoneMarker(object):
-    pass
-
-  _DONE_MARKER = DoneMarker()
+  _DONE_MARKER = Sentinel.sentinel
 
   def __init__(
       self,
       worker_manager,  # type: WorkerHandlerManager
   ):
+    # type: (...) -> None
     self._worker_manager = worker_manager
     self._lock = threading.Lock()
     self._uid_counter = 0
     self._state = self.UNSTARTED_STATE
     # following self._req_* variables are used for debugging purpose, data is
     # added only when self._log_req is True.
-    self._req_sent = collections.defaultdict(int)
-    self._req_worker_mapping = {}
+    self._req_sent = collections.defaultdict(int)  # type: DefaultDict[str, int]
     self._log_req = logging.getLogger().getEffectiveLevel() <= logging.DEBUG
     self._connections_by_worker_id = collections.defaultdict(
         ControlConnection)  # type: DefaultDict[str, ControlConnection]
@@ -186,7 +202,7 @@
 
   def Control(self,
               iterator,  # type: Iterable[beam_fn_api_pb2.InstructionResponse]
-              context
+              context  # type: ServicerContext
              ):
     # type: (...) -> Iterator[beam_fn_api_pb2.InstructionRequest]
     with self._lock:
@@ -213,16 +229,14 @@
         self._req_sent[to_push.instruction_id] += 1
 
   def done(self):
+    # type: () -> None
     self._state = self.DONE_STATE
     _LOGGER.debug(
         'Runner: Requests sent by runner: %s',
         [(str(req), cnt) for req, cnt in self._req_sent.items()])
-    _LOGGER.debug(
-        'Runner: Requests multiplexing info: %s',
-        [(str(req), worker)
-         for req, worker in self._req_worker_mapping.items()])
 
   def GetProcessBundleDescriptor(self, id, context=None):
+    # type: (beam_fn_api_pb2.GetProcessBundleDescriptorRequest, Any) -> beam_fn_api_pb2.ProcessBundleDescriptor
     return self._worker_manager.get_process_bundle_descriptor(id)
 
 
@@ -242,10 +256,10 @@
   data_conn = None  # type: data_plane._GrpcDataChannel
 
   def __init__(self,
-               control_handler,
-               data_plane_handler,
-               state,  # type: StateServicer
-               provision_info  # type: Optional[ExtendedProvisionInfo]
+               control_handler,  # type: Any
+               data_plane_handler,  # type: Any
+               state,  # type: sdk_worker.StateHandler
+               provision_info  # type: ExtendedProvisionInfo
               ):
     # type: (...) -> None
 
@@ -278,6 +292,14 @@
     # type: () -> None
     raise NotImplementedError
 
+  def control_api_service_descriptor(self):
+    # type: () -> endpoints_pb2.ApiServiceDescriptor
+    raise NotImplementedError
+
+  def artifact_api_service_descriptor(self):
+    # type: () -> endpoints_pb2.ApiServiceDescriptor
+    raise NotImplementedError
+
   def data_api_service_descriptor(self):
     # type: () -> Optional[endpoints_pb2.ApiServiceDescriptor]
     raise NotImplementedError
@@ -296,9 +318,10 @@
       urn,  # type: str
       payload_type  # type: Optional[Type[T]]
   ):
-    # type: (...) -> Callable[[Callable[[T, StateServicer, Optional[ExtendedProvisionInfo], GrpcServer], WorkerHandler]], Callable[[T, StateServicer, Optional[ExtendedProvisionInfo], GrpcServer], WorkerHandler]]
+    # type: (...) -> Callable[[Callable[[T, sdk_worker.StateHandler, ExtendedProvisionInfo, GrpcServer], WorkerHandler]], Callable[[T, sdk_worker.StateHandler, ExtendedProvisionInfo, GrpcServer], WorkerHandler]]
     def wrapper(constructor):
-      cls._registered_environments[urn] = constructor, payload_type
+      # type: (Callable) -> Callable
+      cls._registered_environments[urn] = constructor, payload_type  # type: ignore[assignment]
       return constructor
 
     return wrapper
@@ -306,8 +329,8 @@
   @classmethod
   def create(cls,
              environment,  # type: beam_runner_api_pb2.Environment
-             state,  # type: StateServicer
-             provision_info,  # type: Optional[ExtendedProvisionInfo]
+             state,  # type: sdk_worker.StateHandler
+             provision_info,  # type: ExtendedProvisionInfo
              grpc_server  # type: GrpcServer
             ):
     # type: (...) -> WorkerHandler
@@ -319,14 +342,17 @@
         grpc_server)
 
 
-@WorkerHandler.register_environment(python_urns.EMBEDDED_PYTHON, None)
+# This takes a WorkerHandlerManager instead of GrpcServer, so it is not
+# compatible with WorkerHandler.register_environment.  There is a special case
+# in WorkerHandlerManager.get_worker_handlers() that allows it to work.
+@WorkerHandler.register_environment(python_urns.EMBEDDED_PYTHON, None)  # type: ignore[arg-type]
 class EmbeddedWorkerHandler(WorkerHandler):
   """An in-memory worker_handler for fn API control, state and data planes."""
 
   def __init__(self,
                unused_payload,  # type: None
                state,  # type: sdk_worker.StateHandler
-               provision_info,  # type: Optional[ExtendedProvisionInfo]
+               provision_info,  # type: ExtendedProvisionInfo
                worker_manager,  # type: WorkerHandlerManager
               ):
     # type: (...) -> None
@@ -347,6 +373,7 @@
     self._uid_counter = 0
 
   def push(self, request):
+    # type: (beam_fn_api_pb2.InstructionRequest) -> ControlFuture
     if not request.instruction_id:
       self._uid_counter += 1
       request.instruction_id = 'control_%s' % self._uid_counter
@@ -394,6 +421,7 @@
   }
 
   def Logging(self, log_messages, context=None):
+    # type: (Iterable[beam_fn_api_pb2.LogEntry.List], Any) -> Iterator[beam_fn_api_pb2.LogControl]
     yield beam_fn_api_pb2.LogControl()
     for log_message in log_messages:
       for log in log_message.log_entries:
@@ -403,12 +431,12 @@
 class BasicProvisionService(beam_provision_api_pb2_grpc.ProvisionServiceServicer
                             ):
   def __init__(self, base_info, worker_manager):
-    # type: (Optional[beam_provision_api_pb2.ProvisionInfo], WorkerHandlerManager) -> None
+    # type: (beam_provision_api_pb2.ProvisionInfo, WorkerHandlerManager) -> None
     self._base_info = base_info
     self._worker_manager = worker_manager
 
   def GetProvisionInfo(self, request, context=None):
-    # type: (...) -> beam_provision_api_pb2.GetProvisionInfoResponse
+    # type: (Any, Optional[ServicerContext]) -> beam_provision_api_pb2.GetProvisionInfoResponse
     if context:
       worker_id = dict(context.invocation_metadata())['worker_id']
       worker = self._worker_manager.get_worker(worker_id)
@@ -466,6 +494,7 @@
             self.control_server)
 
       def open_uncompressed(f):
+        # type: (str) -> BinaryIO
         return filesystems.FileSystems.open(
             f, compression_type=CompressionTypes.UNCOMPRESSED)
 
@@ -499,6 +528,7 @@
     self.control_server.start()
 
   def close(self):
+    # type: () -> None
     self.control_handler.done()
     to_wait = [
         self.control_server.stop(self._DEFAULT_SHUTDOWN_TIMEOUT_SECS),
@@ -515,7 +545,7 @@
 
   def __init__(self,
                state,  # type: StateServicer
-               provision_info,  # type: Optional[ExtendedProvisionInfo]
+               provision_info,  # type: ExtendedProvisionInfo
                grpc_server  # type: GrpcServer
               ):
     # type: (...) -> None
@@ -566,9 +596,11 @@
     super(GrpcWorkerHandler, self).close()
 
   def port_from_worker(self, port):
+    # type: (int) -> str
     return '%s:%s' % (self.host_from_worker(), port)
 
   def host_from_worker(self):
+    # type: () -> str
     return 'localhost'
 
 
@@ -578,7 +610,7 @@
   def __init__(self,
                external_payload,  # type: beam_runner_api_pb2.ExternalPayload
                state,  # type: StateServicer
-               provision_info,  # type: Optional[ExtendedProvisionInfo]
+               provision_info,  # type: ExtendedProvisionInfo
                grpc_server  # type: GrpcServer
               ):
     # type: (...) -> None
@@ -610,6 +642,7 @@
     pass
 
   def host_from_worker(self):
+    # type: () -> str
     # TODO(BEAM-8646): Reconcile across platforms.
     if sys.platform in ['win32', 'darwin']:
       return 'localhost'
@@ -622,7 +655,7 @@
   def __init__(self,
                payload,  # type: bytes
                state,  # type: StateServicer
-               provision_info,  # type: Optional[ExtendedProvisionInfo]
+               provision_info,  # type: ExtendedProvisionInfo
                grpc_server  # type: GrpcServer
               ):
     # type: (...) -> None
@@ -662,7 +695,7 @@
   def __init__(self,
                worker_command_line,  # type: bytes
                state,  # type: StateServicer
-               provision_info,  # type: Optional[ExtendedProvisionInfo]
+               provision_info,  # type: ExtendedProvisionInfo
                grpc_server  # type: GrpcServer
               ):
     # type: (...) -> None
@@ -690,7 +723,7 @@
   def __init__(self,
                payload,  # type: beam_runner_api_pb2.DockerPayload
                state,  # type: StateServicer
-               provision_info,  # type: Optional[ExtendedProvisionInfo]
+               provision_info,  # type: ExtendedProvisionInfo
                grpc_server  # type: GrpcServer
               ):
     # type: (...) -> None
@@ -700,6 +733,7 @@
     self._container_id = None  # type: Optional[bytes]
 
   def host_from_worker(self):
+    # type: () -> str
     if sys.platform == "darwin":
       # See https://docs.docker.com/docker-for-mac/networking/
       return 'host.docker.internal'
@@ -753,7 +787,9 @@
     t.start()
 
   def watch_container(self):
+    # type: () -> None
     while not self._done:
+      assert self._container_id is not None
       status = subprocess.check_output(
           ['docker', 'inspect', '-f', '{{.State.Status}}',
            self._container_id]).strip()
@@ -788,7 +824,7 @@
   """
   def __init__(self,
                environments,  # type: Mapping[str, beam_runner_api_pb2.Environment]
-               job_provision_info  # type: Optional[ExtendedProvisionInfo]
+               job_provision_info  # type: ExtendedProvisionInfo
               ):
     # type: (...) -> None
     self._environments = environments
@@ -798,13 +834,16 @@
     self._workers_by_id = {}  # type: Dict[str, WorkerHandler]
     self.state_servicer = StateServicer()
     self._grpc_server = None  # type: Optional[GrpcServer]
-    self._process_bundle_descriptors = {}
+    self._process_bundle_descriptors = {
+    }  # type: Dict[str, beam_fn_api_pb2.ProcessBundleDescriptor]
 
   def register_process_bundle_descriptor(self, process_bundle_descriptor):
+    # type: (beam_fn_api_pb2.ProcessBundleDescriptor) -> None
     self._process_bundle_descriptors[
         process_bundle_descriptor.id] = process_bundle_descriptor
 
   def get_process_bundle_descriptor(self, request):
+    # type: (beam_fn_api_pb2.GetProcessBundleDescriptorRequest) -> beam_fn_api_pb2.ProcessBundleDescriptor
     return self._process_bundle_descriptors[
         request.process_bundle_descriptor_id]
 
@@ -852,6 +891,7 @@
     return self._cached_handlers[environment_id][:num_workers]
 
   def close_all(self):
+    # type: () -> None
     for worker_handler_list in self._cached_handlers.values():
       for worker_handler in set(worker_handler_list):
         try:
@@ -859,13 +899,14 @@
         except Exception:
           _LOGGER.error(
               "Error closing worker_handler %s" % worker_handler, exc_info=True)
-    self._cached_handlers = {}
+    self._cached_handlers = {}  # type: ignore[assignment]
     self._workers_by_id = {}
     if self._grpc_server is not None:
       self._grpc_server.close()
       self._grpc_server = None
 
   def get_worker(self, worker_id):
+    # type: (str) -> WorkerHandler
     return self._workers_by_id[worker_id]
 
 
@@ -953,6 +994,7 @@
 
   @contextlib.contextmanager
   def process_instruction_id(self, unused_instruction_id):
+    # type: (Any) -> Iterator
     yield
 
   def get_raw(self,
@@ -1016,7 +1058,7 @@
 
   def State(self,
       request_stream,  # type: Iterable[beam_fn_api_pb2.StateRequest]
-      context=None
+      context=None  # type: Any
             ):
     # type: (...) -> Iterator[beam_fn_api_pb2.StateResponse]
     # Note that this eagerly mutates state, assuming any failures are fatal.
@@ -1062,24 +1104,29 @@
 
 
 class ControlFuture(object):
-  def __init__(self, instruction_id, response=None):
+  def __init__(self,
+               instruction_id,  # type: str
+               response=None  # type: Optional[beam_fn_api_pb2.InstructionResponse]
+              ):
+    # type: (...) -> None
     self.instruction_id = instruction_id
-    if response:
-      self._response = response
-    else:
-      self._response = None
+    self._response = response
+    if response is None:
       self._condition = threading.Condition()
-    self._exception = None
+    self._exception = None  # type: Optional[Exception]
 
   def is_done(self):
+    # type: () -> bool
     return self._response is not None
 
   def set(self, response):
+    # type: (beam_fn_api_pb2.InstructionResponse) -> None
     with self._condition:
       self._response = response
       self._condition.notify_all()
 
   def get(self, timeout=None):
+    # type: (Optional[float]) -> beam_fn_api_pb2.InstructionResponse
     if not self._response and not self._exception:
       with self._condition:
         if not self._response and not self._exception:
@@ -1087,9 +1134,11 @@
     if self._exception:
       raise self._exception
     else:
+      assert self._response is not None
       return self._response
 
   def abort(self, exception):
+    # type: (Exception) -> None
     with self._condition:
       self._exception = exception
       self._condition.notify_all()
diff --git a/sdks/python/apache_beam/runners/portability/portable_runner.py b/sdks/python/apache_beam/runners/portability/portable_runner.py
index 713c762..15c8606 100644
--- a/sdks/python/apache_beam/runners/portability/portable_runner.py
+++ b/sdks/python/apache_beam/runners/portability/portable_runner.py
@@ -16,6 +16,7 @@
 #
 
 # pytype: skip-file
+# mypy: check-untyped-defs
 
 from __future__ import absolute_import
 from __future__ import division
@@ -27,7 +28,6 @@
 import threading
 import time
 from typing import TYPE_CHECKING
-from typing import Any
 from typing import Iterator
 from typing import Optional
 from typing import Tuple
@@ -197,8 +197,12 @@
             pipeline_options=self.get_pipeline_options()),
         timeout=self.timeout)
 
-  def stage(self, pipeline, artifact_staging_endpoint, staging_session_token):
-    # type: (...) -> Optional[Any]
+  def stage(self,
+            proto_pipeline,  # type: beam_runner_api_pb2.Pipeline
+            artifact_staging_endpoint,
+            staging_session_token
+           ):
+    # type: (...) -> None
 
     """Stage artifacts"""
     if artifact_staging_endpoint:
@@ -288,6 +292,7 @@
         'use, such as --runner=FlinkRunner or --runner=SparkRunner.')
 
   def create_job_service_handle(self, job_service, options):
+    # type: (...) -> JobServiceHandle
     return JobServiceHandle(job_service, options)
 
   def create_job_service(self, options):
@@ -299,7 +304,7 @@
     job_endpoint = options.view_as(PortableOptions).job_endpoint
     if job_endpoint:
       if job_endpoint == 'embed':
-        server = job_server.EmbeddedJobServer()
+        server = job_server.EmbeddedJobServer()  # type: job_server.JobServer
       else:
         job_server_timeout = options.view_as(PortableOptions).job_server_timeout
         server = job_server.ExternalJobServer(job_endpoint, job_server_timeout)
@@ -463,6 +468,7 @@
     self._runtime_exception = None
 
   def cancel(self):
+    # type: () -> None
     try:
       self._job_service.Cancel(
           beam_job_api_pb2.CancelJobRequest(job_id=self._job_id))
@@ -496,6 +502,7 @@
     return self._metrics
 
   def _last_error_message(self):
+    # type: () -> str
     # Filter only messages with the "message_response" and error messages.
     messages = [
         m.message_response for m in self._messages
@@ -517,6 +524,7 @@
     :return: The result of the pipeline, i.e. PipelineResult.
     """
     def read_messages():
+      # type: () -> None
       previous_state = -1
       for message in self._message_stream:
         if message.HasField('message_response'):
@@ -576,6 +584,7 @@
       self._cleanup()
 
   def _cleanup(self, on_exit=False):
+    # type: (bool) -> None
     if on_exit and self._cleanup_callbacks:
       _LOGGER.info(
           'Running cleanup on exit. If your pipeline should continue running, '
diff --git a/sdks/python/apache_beam/runners/portability/stager.py b/sdks/python/apache_beam/runners/portability/stager.py
index e6288fc..f1a820f 100644
--- a/sdks/python/apache_beam/runners/portability/stager.py
+++ b/sdks/python/apache_beam/runners/portability/stager.py
@@ -57,6 +57,7 @@
 import tempfile
 from typing import List
 from typing import Optional
+from typing import Tuple
 
 import pkg_resources
 from future.moves.urllib.parse import urlparse
diff --git a/sdks/python/apache_beam/runners/worker/sdk_worker.py b/sdks/python/apache_beam/runners/worker/sdk_worker.py
index c03e431..b3c095c 100644
--- a/sdks/python/apache_beam/runners/worker/sdk_worker.py
+++ b/sdks/python/apache_beam/runners/worker/sdk_worker.py
@@ -160,19 +160,20 @@
 class SdkHarness(object):
   REQUEST_METHOD_PREFIX = '_request_'
 
-  def __init__(self,
-               control_address,  # type: str
-               credentials=None,  # type: Optional[grpc.ChannelCredentials]
-               worker_id=None,  # type: Optional[str]
-               # Caching is disabled by default
-               state_cache_size=0,  # type: int
-               # time-based data buffering is disabled by default
-               data_buffer_time_limit_ms=0,  # type: int
-               profiler_factory=None,  # type: Optional[Callable[..., Profile]]
-               status_address=None,  # type: Optional[str]
-               # Heap dump through status api is disabled by default
-               enable_heap_dump=False, # type: bool
-               ):
+  def __init__(
+      self,
+      control_address,  # type: str
+      credentials=None,  # type: Optional[grpc.ChannelCredentials]
+      worker_id=None,  # type: Optional[str]
+      # Caching is disabled by default
+      state_cache_size=0,  # type: int
+      # time-based data buffering is disabled by default
+      data_buffer_time_limit_ms=0,  # type: int
+      profiler_factory=None,  # type: Optional[Callable[..., Profile]]
+      status_address=None,  # type: Optional[str]
+      # Heap dump through status api is disabled by default
+      enable_heap_dump=False,  # type: bool
+  ):
     # type: (...) -> None
     self._alive = True
     self._worker_index = 0
@@ -276,10 +277,11 @@
       self._status_handler.close()
     _LOGGER.info('Done consuming work.')
 
-  def _execute(self,
-               task,  # type: Callable[[], beam_fn_api_pb2.InstructionResponse]
-               request  # type:  beam_fn_api_pb2.InstructionRequest
-              ):
+  def _execute(
+      self,
+      task,  # type: Callable[[], beam_fn_api_pb2.InstructionResponse]
+      request  # type:  beam_fn_api_pb2.InstructionRequest
+  ):
     # type: (...) -> None
     with statesampler.instruction_id(request.instruction_id):
       try:
@@ -367,11 +369,12 @@
   """
   periodic_shutdown = None  # type: Optional[PeriodicThread]
 
-  def __init__(self,
-               state_handler_factory,  # type: StateHandlerFactory
-               data_channel_factory,  # type: data_plane.DataChannelFactory
-               fns  # type: MutableMapping[str, beam_fn_api_pb2.ProcessBundleDescriptor]
-              ):
+  def __init__(
+      self,
+      state_handler_factory,  # type: StateHandlerFactory
+      data_channel_factory,  # type: data_plane.DataChannelFactory
+      fns  # type: MutableMapping[str, beam_fn_api_pb2.ProcessBundleDescriptor]
+  ):
     # type: (...) -> None
     self.fns = fns
     self.state_handler_factory = state_handler_factory
@@ -554,13 +557,13 @@
 
 
 class SdkWorker(object):
-
-  def __init__(self,
-               bundle_processor_cache,  # type: BundleProcessorCache
-               state_cache_metrics_fn=list,  # type: Callable[[], Iterable[metrics_pb2.MonitoringInfo]]
-               profiler_factory=None,  # type: Optional[Callable[..., Profile]]
-               log_lull_timeout_ns=None,  # type: Optional[int]
-              ):
+  def __init__(
+      self,
+      bundle_processor_cache,  # type: BundleProcessorCache
+      state_cache_metrics_fn=list,  # type: Callable[[], Iterable[metrics_pb2.MonitoringInfo]]
+      profiler_factory=None,  # type: Optional[Callable[..., Profile]]
+      log_lull_timeout_ns=None,  # type: Optional[int]
+  ):
     # type: (...) -> None
     self.bundle_processor_cache = bundle_processor_cache
     self.state_cache_metrics_fn = state_cache_metrics_fn
@@ -579,10 +582,11 @@
     else:
       raise NotImplementedError
 
-  def register(self,
-               request,  # type: beam_fn_api_pb2.RegisterRequest
-               instruction_id  # type: str
-              ):
+  def register(
+      self,
+      request,  # type: beam_fn_api_pb2.RegisterRequest
+      instruction_id  # type: str
+  ):
     # type: (...) -> beam_fn_api_pb2.InstructionResponse
 
     """Registers a set of ``beam_fn_api_pb2.ProcessBundleDescriptor``s.
@@ -598,10 +602,11 @@
         instruction_id=instruction_id,
         register=beam_fn_api_pb2.RegisterResponse())
 
-  def process_bundle(self,
-                     request,  # type: beam_fn_api_pb2.ProcessBundleRequest
-                     instruction_id  # type: str
-                    ):
+  def process_bundle(
+      self,
+      request,  # type: beam_fn_api_pb2.ProcessBundleRequest
+      instruction_id  # type: str
+  ):
     # type: (...) -> beam_fn_api_pb2.InstructionResponse
     bundle_processor = self.bundle_processor_cache.get(
         instruction_id, request.process_bundle_descriptor_id)
@@ -632,10 +637,11 @@
       self.bundle_processor_cache.discard(instruction_id)
       raise
 
-  def process_bundle_split(self,
-                           request,  # type: beam_fn_api_pb2.ProcessBundleSplitRequest
-                           instruction_id  # type: str
-                          ):
+  def process_bundle_split(
+      self,
+      request,  # type: beam_fn_api_pb2.ProcessBundleSplitRequest
+      instruction_id  # type: str
+  ):
     # type: (...) -> beam_fn_api_pb2.InstructionResponse
     try:
       processor = self.bundle_processor_cache.lookup(request.instruction_id)
@@ -700,10 +706,11 @@
     # type: () -> None
     thread_dump()
 
-  def process_bundle_progress(self,
-                              request,  # type: beam_fn_api_pb2.ProcessBundleProgressRequest
-                              instruction_id  # type: str
-                             ):
+  def process_bundle_progress(
+      self,
+      request,  # type: beam_fn_api_pb2.ProcessBundleProgressRequest
+      instruction_id  # type: str
+  ):
     # type: (...) -> beam_fn_api_pb2.InstructionResponse
     try:
       processor = self.bundle_processor_cache.lookup(request.instruction_id)
@@ -726,22 +733,23 @@
                 for info in monitoring_infos
             }))
 
-  def process_bundle_progress_metadata_request(self,
-                                               request,  # type: beam_fn_api_pb2.ProcessBundleProgressMetadataRequest
-                                               instruction_id  # type: str
-                                              ):
+  def monitoring_infos_request(
+      self,
+      request,  # type: beam_fn_api_pb2.MonitoringInfosMetadataRequest
+      instruction_id  # type: str
+  ):
     # type: (...) -> beam_fn_api_pb2.InstructionResponse
     return beam_fn_api_pb2.InstructionResponse(
         instruction_id=instruction_id,
-        process_bundle_progress_metadata=beam_fn_api_pb2.
-        ProcessBundleProgressMetadataResponse(
+        monitoring_infos=beam_fn_api_pb2.MonitoringInfosMetadataResponse(
             monitoring_info=SHORT_ID_CACHE.getInfos(
                 request.monitoring_info_id)))
 
-  def finalize_bundle(self,
-                      request,  # type: beam_fn_api_pb2.FinalizeBundleRequest
-                      instruction_id  # type: str
-                     ):
+  def finalize_bundle(
+      self,
+      request,  # type: beam_fn_api_pb2.FinalizeBundleRequest
+      instruction_id  # type: str
+  ):
     # type: (...) -> beam_fn_api_pb2.InstructionResponse
     try:
       processor = self.bundle_processor_cache.lookup(request.instruction_id)
@@ -780,10 +788,11 @@
 class StateHandler(with_metaclass(abc.ABCMeta, object)):  # type: ignore[misc]
   """An abstract object representing a ``StateHandler``."""
   @abc.abstractmethod
-  def get_raw(self,
-              state_key,  # type: beam_fn_api_pb2.StateKey
-              continuation_token=None  # type: Optional[bytes]
-             ):
+  def get_raw(
+      self,
+      state_key,  # type: beam_fn_api_pb2.StateKey
+      continuation_token=None  # type: Optional[bytes]
+  ):
     # type: (...) -> Tuple[bytes, Optional[bytes]]
     raise NotImplementedError(type(self))
 
@@ -874,10 +883,11 @@
 
 class ThrowingStateHandler(StateHandler):
   """A state handler that errors on any requests."""
-  def get_raw(self,
-              state_key,  # type: beam_fn_api_pb2.StateKey
-              continuation_token=None  # type: Optional[bytes]
-             ):
+  def get_raw(
+      self,
+      state_key,  # type: beam_fn_api_pb2.StateKey
+      continuation_token=None  # type: Optional[bytes]
+  ):
     # type: (...) -> Tuple[bytes, Optional[bytes]]
     raise RuntimeError(
         'Unable to handle state requests for ProcessBundleDescriptor without '
@@ -964,10 +974,11 @@
     self._done = True
     self._requests.put(self._DONE)
 
-  def get_raw(self,
-              state_key,  # type: beam_fn_api_pb2.StateKey
-              continuation_token=None  # type: Optional[bytes]
-             ):
+  def get_raw(
+      self,
+      state_key,  # type: beam_fn_api_pb2.StateKey
+      continuation_token=None  # type: Optional[bytes]
+  ):
     # type: (...) -> Tuple[bytes, Optional[bytes]]
     response = self._blocking_request(
         beam_fn_api_pb2.StateRequest(
@@ -976,10 +987,11 @@
                 continuation_token=continuation_token)))
     return response.get.data, response.get.continuation_token
 
-  def append_raw(self,
-                 state_key,  # type: Optional[beam_fn_api_pb2.StateKey]
-                 data  # type: bytes
-                ):
+  def append_raw(
+      self,
+      state_key,  # type: Optional[beam_fn_api_pb2.StateKey]
+      data  # type: bytes
+  ):
     # type: (...) -> _Future
     return self._request(
         beam_fn_api_pb2.StateRequest(
@@ -1036,11 +1048,11 @@
    If activated but no cache token is supplied, caching is done at the bundle
    level.
   """
-
-  def __init__(self,
-               global_state_cache,  # type: StateCache
-               underlying_state  # type: StateHandler
-              ):
+  def __init__(
+      self,
+      global_state_cache,  # type: StateCache
+      underlying_state  # type: StateHandler
+  ):
     # type: (...) -> None
     self._underlying = underlying_state
     self._state_cache = global_state_cache
@@ -1078,10 +1090,11 @@
       self._context.user_state_cache_token = None
       self._context.bundle_cache_token = None
 
-  def blocking_get(self,
-                   state_key,  # type: beam_fn_api_pb2.StateKey
-                   coder,  # type: coder_impl.CoderImpl
-                  ):
+  def blocking_get(
+      self,
+      state_key,  # type: beam_fn_api_pb2.StateKey
+      coder,  # type: coder_impl.CoderImpl
+  ):
     # type: (...) -> Iterable[Any]
     cache_token = self._get_cache_token(state_key)
     if not cache_token:
@@ -1107,11 +1120,12 @@
             state_key)
     return cached_value
 
-  def extend(self,
-             state_key,  # type: beam_fn_api_pb2.StateKey
-             coder,  # type: coder_impl.CoderImpl
-             elements,  # type: Iterable[Any]
-            ):
+  def extend(
+      self,
+      state_key,  # type: beam_fn_api_pb2.StateKey
+      coder,  # type: coder_impl.CoderImpl
+      elements,  # type: Iterable[Any]
+  ):
     # type: (...) -> _Future
     cache_token = self._get_cache_token(state_key)
     if cache_token:
@@ -1161,7 +1175,7 @@
       state_key,  # type: beam_fn_api_pb2.StateKey
       coder,  # type: coder_impl.CoderImpl
       continuation_token=None  # type: Optional[bytes]
-    ):
+  ):
     # type: (...) -> Iterator[Any]
 
     """Materializes the state lazily, one element at a time.
@@ -1196,7 +1210,7 @@
       self,
       state_key,  # type: beam_fn_api_pb2.StateKey
       coder  # type: coder_impl.CoderImpl
-    ):
+  ):
     # type: (...) -> Iterable[Any]
 
     """Materialized the first page of data, concatenated with a lazy iterable
diff --git a/sdks/python/apache_beam/utils/histogram.py b/sdks/python/apache_beam/utils/histogram.py
index 952546e..93ae292 100644
--- a/sdks/python/apache_beam/utils/histogram.py
+++ b/sdks/python/apache_beam/utils/histogram.py
@@ -21,6 +21,7 @@
 import logging
 import math
 import threading
+from collections import Counter
 
 _LOGGER = logging.getLogger(__name__)
 
@@ -34,18 +35,42 @@
   def __init__(self, bucket_type):
     self._lock = threading.Lock()
     self._bucket_type = bucket_type
-    self._buckets = {}
+    self._buckets = Counter()
     self._num_records = 0
     self._num_top_records = 0
     self._num_bot_records = 0
 
   def clear(self):
     with self._lock:
-      self._buckets = {}
+      self._buckets = Counter()
       self._num_records = 0
       self._num_top_records = 0
       self._num_bot_records = 0
 
+  def copy(self):
+    with self._lock:
+      histogram = Histogram(self._bucket_type)
+      histogram._num_records = self._num_records
+      histogram._num_top_records = self._num_top_records
+      histogram._num_bot_records = self._num_bot_records
+      histogram._buckets = self._buckets.copy()
+      return histogram
+
+  def combine(self, other):
+    if not isinstance(other,
+                      Histogram) or self._bucket_type != other._bucket_type:
+      raise RuntimeError('failed to combine histogram.')
+    other_histogram = other.copy()
+    with self._lock:
+      histogram = Histogram(self._bucket_type)
+      histogram._num_records = self._num_records + other_histogram._num_records
+      histogram._num_top_records = (
+          self._num_top_records + other_histogram._num_top_records)
+      histogram._num_bot_records = (
+          self._num_bot_records + other_histogram._num_bot_records)
+      histogram._buckets = self._buckets + other_histogram._buckets
+      return histogram
+
   def record(self, *args):
     for arg in args:
       self._record(arg)
@@ -78,27 +103,23 @@
   def p50(self):
     return self.get_linear_interpolation(0.50)
 
-  def get_percentile_info(self, elem_type, unit):
+  def get_percentile_info(self):
     def _format(f):
       if f == float('-inf'):
         return '<%s' % self._bucket_type.range_from()
       elif f == float('inf'):
         return '>=%s' % self._bucket_type.range_to()
       else:
-        return str(round(f))  # pylint: disable=round-builtin
+        return str(int(round(f)))  # pylint: disable=round-builtin
 
     with self._lock:
       return (
-          'Total number of %s: %s, '
-          'P99: %s%s, P90: %s%s, P50: %s%s' % (
-              elem_type,
+          'Total count: %s, '
+          'P99: %s, P90: %s, P50: %s' % (
               self.total_count(),
               _format(self._get_linear_interpolation(0.99)),
-              unit,
               _format(self._get_linear_interpolation(0.90)),
-              unit,
-              _format(self._get_linear_interpolation(0.50)),
-              unit))
+              _format(self._get_linear_interpolation(0.50))))
 
   def get_linear_interpolation(self, percentile):
     """Calculate percentile estimation based on linear interpolation.
@@ -141,6 +162,24 @@
         self._bucket_type.range_from() +
         self._bucket_type.accumulated_bucket_size(index) + frac_bucket_size)
 
+  def __eq__(self, other):
+    if not isinstance(other, Histogram):
+      return False
+    return (
+        self._bucket_type == other._bucket_type and
+        self._num_records == other._num_records and
+        self._num_top_records == other._num_top_records and
+        self._num_bot_records == other._num_bot_records and
+        self._buckets == other._buckets)
+
+  def __hash__(self):
+    return hash((
+        self._bucket_type,
+        self._num_records,
+        self._num_top_records,
+        self._num_bot_records,
+        frozenset(self._buckets.items())))
+
 
 class BucketType(object):
   def range_from(self):
@@ -205,3 +244,13 @@
 
   def accumulated_bucket_size(self, end_index):
     return self._width * end_index
+
+  def __eq__(self, other):
+    if not isinstance(other, LinearBucket):
+      return False
+    return (
+        self._start == other._start and self._width == other._width and
+        self._num_buckets == other._num_buckets)
+
+  def __hash__(self):
+    return hash((self._start, self._width, self._num_buckets))
diff --git a/sdks/python/apache_beam/utils/histogram_test.py b/sdks/python/apache_beam/utils/histogram_test.py
index 1e29355..93cb7ae 100644
--- a/sdks/python/apache_beam/utils/histogram_test.py
+++ b/sdks/python/apache_beam/utils/histogram_test.py
@@ -106,13 +106,13 @@
     histogram = Histogram(LinearBucket(0, 0.2, 50))
     histogram.record(-1, -2, -3, -4, -5, 0, 1, 2, 3, 4)
     self.assertEqual(histogram.p50(), float('-inf'))
-    self.assertIn('P50: <0', histogram.get_percentile_info('integers', ''))
+    self.assertIn('P50: <0', histogram.get_percentile_info())
 
   def test_p50_positive_infinity(self):
     histogram = Histogram(LinearBucket(0, 0.2, 50))
     histogram.record(6, 7, 8, 9, 10, 11, 12, 13, 14, 15)
     self.assertEqual(histogram.p50(), float('inf'))
-    self.assertIn('P50: >=10', histogram.get_percentile_info('integers', ''))
+    self.assertIn('P50: >=10', histogram.get_percentile_info())
 
   def test_empty_p99(self):
     histogram = Histogram(LinearBucket(0, 0.2, 50))
@@ -129,6 +129,49 @@
     self.assertEqual(histogram.total_count(), 0)
     self.assertEqual(histogram._buckets.get(5, 0), 0)
 
+  def test_equal(self):
+    histogram1 = Histogram(LinearBucket(0, 0.2, 50))
+    histogram2 = Histogram(LinearBucket(0, 0.2, 50))
+    self.assertEqual(histogram1, histogram2)
+    self.assertEqual(hash(histogram1), hash(histogram2))
+
+    histogram1 = Histogram(LinearBucket(0, 0.2, 5))
+    histogram2 = Histogram(LinearBucket(0, 0.2, 50))
+    self.assertNotEqual(histogram1, histogram2)
+
+    histogram1 = Histogram(LinearBucket(0, 0.2, 50))
+    histogram2 = Histogram(LinearBucket(0, 0.2, 50))
+    histogram1.record(1)
+    histogram2.record(1)
+    self.assertEqual(histogram1, histogram2)
+    self.assertEqual(hash(histogram1), hash(histogram2))
+
+    histogram1 = Histogram(LinearBucket(0, 0.2, 50))
+    histogram2 = Histogram(LinearBucket(0, 0.2, 50))
+    histogram1.record(1)
+    histogram2.record(10)
+    self.assertNotEqual(histogram1, histogram2)
+
+  def test_copy(self):
+    histogram1 = Histogram(LinearBucket(0, 0.2, 50))
+    histogram1.record(0, 1, 2, 3, 4)
+    histogram2 = histogram1.copy()
+    self.assertEqual(histogram1, histogram2)
+
+    histogram1.record(5, 6, 7, 8, 9)
+    self.assertNotEqual(histogram1, histogram2)
+
+  def test_combine(self):
+    histogram1 = Histogram(LinearBucket(0, 0.2, 50))
+    histogram1.record(0, 1, 2, 3, 4)
+    histogram2 = Histogram(LinearBucket(0, 0.2, 50))
+    histogram2.record(5, 6, 7, 8, 9)
+
+    histogram = Histogram(LinearBucket(0, 0.2, 50))
+    histogram.record(0, 1, 2, 3, 4, 5, 6, 7, 8, 9)
+
+    self.assertEqual(histogram1.combine(histogram2), histogram)
+
 
 if __name__ == '__main__':
   unittest.main()
diff --git a/sdks/python/apache_beam/utils/interactive_utils.py b/sdks/python/apache_beam/utils/interactive_utils.py
index a416bb4..ef3e16d 100644
--- a/sdks/python/apache_beam/utils/interactive_utils.py
+++ b/sdks/python/apache_beam/utils/interactive_utils.py
@@ -91,9 +91,9 @@
 
 def _extract_pipeline_of_pvalueish(pvalueish):
   """Extracts the pipeline that the given pvalueish belongs to."""
-  if isinstance(pvalueish, tuple):
+  if isinstance(pvalueish, tuple) and len(pvalueish) > 0:
     pvalue = pvalueish[0]
-  elif isinstance(pvalueish, dict):
+  elif isinstance(pvalueish, dict) and len(pvalueish) > 0:
     pvalue = next(iter(pvalueish.values()))
   else:
     pvalue = pvalueish
diff --git a/sdks/python/apache_beam/utils/profiler.py b/sdks/python/apache_beam/utils/profiler.py
index dae625a..81bd578 100644
--- a/sdks/python/apache_beam/utils/profiler.py
+++ b/sdks/python/apache_beam/utils/profiler.py
@@ -21,6 +21,7 @@
 """
 
 # pytype: skip-file
+# mypy: check-untyped-defs
 
 from __future__ import absolute_import
 
@@ -47,6 +48,9 @@
 
   SORTBY = 'cumulative'
 
+  profile_output = None  # type: str
+  stats = None  # type: pstats.Stats
+
   def __init__(
       self,
       profile_id, # type: str
@@ -72,13 +76,11 @@
         profiling session, the profiler only records the newly allocated objects
         in this session.
     """
-    self.stats = None
     self.profile_id = str(profile_id)
     self.profile_location = profile_location
     self.log_results = log_results
     self.file_copy_fn = file_copy_fn or self.default_file_copy_fn
     self.time_prefix = time_prefix
-    self.profile_output = None
     self.enable_cpu_profiling = enable_cpu_profiling
     self.enable_memory_profiling = enable_memory_profiling
 
@@ -104,7 +106,8 @@
       if self.enable_cpu_profiling:
         self.profile.create_stats()
         self.profile_output = self._upload_profile_data(
-            'cpu_profile', self.profile.stats)
+            # typing: seems stats attr is missing from typeshed
+            self.profile_location, 'cpu_profile', self.profile.stats)  # type: ignore[attr-defined]
 
       if self.enable_memory_profiling:
         if not self.hpy:
@@ -113,7 +116,10 @@
           h = self.hpy.heap()
           heap_dump_data = '%s\n%s' % (h, h.more)
           self._upload_profile_data(
-              'memory_profile', heap_dump_data, write_binary=False)
+              self.profile_location,
+              'memory_profile',
+              heap_dump_data,
+              write_binary=False)
 
     if self.log_results:
       if self.enable_cpu_profiling:
@@ -156,18 +162,20 @@
       return create_profiler
     return None
 
-  def _upload_profile_data(self, dir, data, write_binary=True):
+  def _upload_profile_data(
+      self, profile_location, dir, data, write_binary=True):
+    # type: (...) -> str
     dump_location = os.path.join(
-        self.profile_location,
+        profile_location,
         dir,
         time.strftime(self.time_prefix + self.profile_id))
     fd, filename = tempfile.mkstemp()
     try:
       os.close(fd)
       if write_binary:
-        with open(filename, 'wb') as f:
+        with open(filename, 'wb') as fb:
           import marshal
-          marshal.dump(data, f)
+          marshal.dump(data, fb)
       else:
         with open(filename, 'w') as f:
           f.write(data)
diff --git a/sdks/python/apache_beam/utils/sharded_key.py b/sdks/python/apache_beam/utils/sharded_key.py
new file mode 100644
index 0000000..d411055
--- /dev/null
+++ b/sdks/python/apache_beam/utils/sharded_key.py
@@ -0,0 +1,58 @@
+#
+# 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.
+#
+
+# pytype: skip-file
+
+from __future__ import absolute_import
+
+
+class ShardedKey(object):
+  """
+  A sharded key consisting of a user key and an opaque shard id represented by
+  bytes.
+
+  Attributes:
+    key: The user key.
+    shard_id: An opaque byte string that uniquely represents a shard of the key.
+  """
+  def __init__(
+      self,
+      key,
+      shard_id,  # type: bytes
+  ):
+    # type: (...) -> None
+    assert shard_id is not None
+    self._key = key
+    self._shard_id = shard_id
+
+  @property
+  def key(self):
+    return self._key
+
+  def __repr__(self):
+    return '(%s, %s)' % (repr(self.key), self._shard_id)
+
+  def __eq__(self, other):
+    return (
+        type(self) == type(other) and self.key == other.key and
+        self._shard_id == other._shard_id)
+
+  def __hash__(self):
+    return hash((self.key, self._shard_key))
+
+  def __reduce__(self):
+    return ShardedKey, (self.key, self._shard_id)
diff --git a/sdks/python/container/base_image_requirements.txt b/sdks/python/container/base_image_requirements.txt
index 0ae1945..3e2095c 100644
--- a/sdks/python/container/base_image_requirements.txt
+++ b/sdks/python/container/base_image_requirements.txt
@@ -38,6 +38,7 @@
 protobuf==3.12.2
 pyarrow==0.16.0
 pydot==1.4.1
+pymongo==3.9.0
 pytz==2019.3
 pyvcf==0.6.8;python_version<"3.0"
 pyyaml==5.1
@@ -70,7 +71,6 @@
 python-gflags==3.0.6
 tensorflow==2.1.0;python_version<="2.7"
 tensorflow==2.3.0;python_version>="3.5"
-pymongo==3.9.0
 
 # Packages needed for testing.
 tenacity>=5.0.2
diff --git a/sdks/python/mypy.ini b/sdks/python/mypy.ini
index 121d1b4..6095c02 100644
--- a/sdks/python/mypy.ini
+++ b/sdks/python/mypy.ini
@@ -61,8 +61,6 @@
 
 
 # TODO(BEAM-7746): Remove the lines below.
-[mypy-apache_beam.coders.coders]
-ignore_errors = true
 
 [mypy-apache_beam.io.*]
 ignore_errors = true
@@ -88,18 +86,6 @@
 [mypy-apache_beam.runners.interactive.*]
 ignore_errors = true
 
-[mypy-apache_beam.runners.portability.artifact_service]
-ignore_errors = true
-
-[mypy-apache_beam.runners.portability.fn_api_runner.*]
-ignore_errors = true
-
-[mypy-apache_beam.runners.portability.portable_runner]
-ignore_errors = true
-
-[mypy-apache_beam.runners.portability.stager]
-ignore_errors = true
-
 [mypy-apache_beam.testing.synthetic_pipeline]
 ignore_errors = true
 
diff --git a/sdks/python/setup.py b/sdks/python/setup.py
index e2b9b18..b2ffb96 100644
--- a/sdks/python/setup.py
+++ b/sdks/python/setup.py
@@ -128,12 +128,8 @@
   cythonize = lambda *args, **kwargs: []
 
 REQUIRED_PACKAGES = [
-    # Apache Avro does not follow semantic versioning, so we should not auto
-    # upgrade on minor versions. Due to AVRO-2429, Dataflow still
-    # requires Avro 1.8.x.
-    'avro>=1.8.1,<1.10.0; python_version < "3.0"',
     # Avro 1.9.2 for python3 was broken. The issue was fixed in version 1.9.2.1
-    'avro-python3>=1.8.1,!=1.9.2,<1.10.0; python_version >= "3.0"',
+    'avro-python3>=1.8.1,!=1.9.2,<1.10.0',
     'crcmod>=1.7,<2.0',
     # Dill doesn't have forwards-compatibility guarantees within minor version.
     # Pickles created with a new version of dill may not unpickle using older
@@ -142,9 +138,7 @@
     # See: https://github.com/uqfoundation/dill/issues/341.
     'dill>=0.3.1.1,<0.3.2',
     'fastavro>=0.21.4,<2',
-    'funcsigs>=1.0.2,<2; python_version < "3.0"',
     'future>=0.18.2,<1.0.0',
-    'futures>=3.2.0,<4.0.0; python_version < "3.0"',
     'grpcio>=1.29.0,<2',
     'hdfs>=2.1.0,<3.0.0',
     'httplib2>=0.8,<0.18.0',
@@ -153,17 +147,11 @@
     'pymongo>=3.8.0,<4.0.0',
     'oauth2client>=2.0.1,<5',
     'protobuf>=3.12.2,<4',
-    # [BEAM-6287] pyarrow is not supported on Windows for Python 2
-    ('pyarrow>=0.15.1,<0.18.0; python_version >= "3.0" or '
-     'platform_system != "Windows"'),
+    'pyarrow>=0.15.1,<0.18.0',
     'pydot>=1.2.0,<2',
     'python-dateutil>=2.8.0,<3',
     'pytz>=2018.3',
-    # [BEAM-5628] Beam VCF IO is not supported in Python 3.
-    'pyvcf>=0.6.8,<0.7.0; python_version < "3.0"',
-    # fixes and additions have been made since typing 3.5
     'requests>=2.24.0,<3.0.0',
-    'typing>=3.7.0,<3.8.0; python_full_version < "3.5.3"',
     'typing-extensions>=3.7.0,<3.8.0',
     ]
 
@@ -177,11 +165,8 @@
     'freezegun>=0.3.12',
     'nose>=1.3.7',
     'nose_xunitmp>=0.4.1',
-    'pandas>=0.24.2,<2; python_full_version < "3.5.3"',
-    'pandas>=0.25.2,<2; python_full_version >= "3.5.3"',
+    'pandas>=0.25.2,<2',
     'parameterized>=0.7.1,<0.8.0',
-    # pyhamcrest==1.10.0 doesn't work on Py2. Beam still supports Py2.
-    # See: https://github.com/hamcrest/PyHamcrest/issues/131.
     'pyhamcrest>=1.9,!=1.10.0,<2.0.0',
     'pyyaml>=3.12,<6.0.0',
     'requests_mock>=1.7,<2.0',
@@ -189,13 +174,9 @@
     'pytest>=4.4.0,<5.0',
     'pytest-xdist>=1.29.0,<2',
     'pytest-timeout>=1.3.3,<2',
-    'rsa<4.1; python_version < "3.0"',
-    # sqlalchemy is used only for running xlang jdbc test so limit to Py3
-    'sqlalchemy>=1.3,<2.0; python_version >= "3.5"',
-    # psycopg is used only for running xlang jdbc test so limit to Py3
-    'psycopg2-binary>=2.8.5,<3.0.0; python_version >= "3.5"',
-    # testcontainers is used only for running xlang jdbc test so limit to Py3
-    'testcontainers>=3.0.3,<4.0.0; python_version >= "3.5"',
+    'sqlalchemy>=1.3,<2.0',
+    'psycopg2-binary>=2.8.5,<3.0.0',
+    'testcontainers>=3.0.3,<4.0.0',
     ]
 
 GCP_REQUIREMENTS = [
@@ -216,7 +197,7 @@
     'google-cloud-videointelligence>=1.8.0,<2',
     'google-cloud-vision>=0.38.0,<2',
     # GCP packages required by prebuild sdk container functionality.
-    'google-cloud-build>=2.0.0,<3; python_version >= "3.6"',
+    'google-cloud-build>=2.0.0,<3',
 ]
 
 INTERACTIVE_BEAM = [
@@ -234,7 +215,7 @@
     # headless chrome based integration tests
     'selenium>=3.141.0,<4',
     'needle>=0.5.0,<1',
-    'chromedriver-binary>=85,<86',
+    'chromedriver-binary>=86,<87',
     # use a fixed major version of PIL for different python versions
     'pillow>=7.1.1,<8',
 ]
diff --git a/website/build.gradle b/website/build.gradle
index c203758..6ea9e2a 100644
--- a/website/build.gradle
+++ b/website/build.gradle
@@ -267,7 +267,7 @@
     assert !file("${buildContentDir('apache')}/documentation/sdks/javadoc").exists()
     assert !file("${buildContentDir('apache')}/documentation/sdks/pydoc").exists()
 
-    def git = grgit.open()
+    def git = grgit.open(currentDir: project.rootDir)
     // get the latest commit on master
     def latestCommit = grgit.log(maxCommits: 1)[0].abbreviatedId
 
@@ -320,7 +320,7 @@
   doLast {
     assert grgit : "Cannot publish website outside of git repository"
 
-    def git = grgit.open()
+    def git = grgit.open(currentDir: project.rootDir)
     git.checkout(branch: 'asf-site')
     if (!commitedChanges) {
       println 'No changes to push'
diff --git a/website/www/site/config.toml b/website/www/site/config.toml
index 6b22137..b334ee6 100644
--- a/website/www/site/config.toml
+++ b/website/www/site/config.toml
@@ -104,7 +104,7 @@
 
 [params]
 description = "Apache Beam is an open source, unified model and set of language-specific SDKs for defining and executing data processing workflows, and also data ingestion and integration flows, supporting Enterprise Integration Patterns (EIPs) and Domain Specific Languages (DSLs). Dataflow pipelines simplify the mechanics of large-scale batch and streaming data processing and can run on a number of runtimes like Apache Flink, Apache Spark, and Google Cloud Dataflow (a cloud service). Beam also brings DSL in different languages, allowing users to easily implement their data integration processes."
-release_latest = "2.24.0"
+release_latest = "2.25.0"
 # The repository and branch where the files live in Github or Colab. This is used
 # to serve and stage from your local branch, but publish to the master branch.
 #   e.g. https://github.com/{{< param branch_repo >}}/path/to/notebook.ipynb
diff --git a/website/www/site/content/en/blog/beam-2.24.0.md b/website/www/site/content/en/blog/beam-2.24.0.md
index 6ec264f..461514c 100644
--- a/website/www/site/content/en/blog/beam-2.24.0.md
+++ b/website/www/site/content/en/blog/beam-2.24.0.md
@@ -55,6 +55,9 @@
   --temp_location, or pass method="STREAMING_INSERTS" to WriteToBigQuery ([BEAM-6928](https://issues.apache.org/jira/browse/BEAM-6928)).
 * Python SDK now understands `typing.FrozenSet` type hints, which are not interchangeable with `typing.Set`. You may need to update your pipelines if type checking fails. ([BEAM-10197](https://issues.apache.org/jira/browse/BEAM-10197))
 
+## Known Issues
+
+* ([BEAM-11113](https://issues.apache.org/jira/browse/BEAM-11113)) Default compressor change breaks dataflow python streaming job update compatibility.
 
 ## List of Contributors
 
diff --git a/website/www/site/content/en/blog/beam-2.25.0.md b/website/www/site/content/en/blog/beam-2.25.0.md
new file mode 100644
index 0000000..d757147
--- /dev/null
+++ b/website/www/site/content/en/blog/beam-2.25.0.md
@@ -0,0 +1,85 @@
+---
+title:  "Apache Beam 2.25.0"
+date:   2020-10-23 14:00:00 -0800
+categories:
+  - blog
+authors:
+  - Robin Qiu
+---
+<!--
+Licensed 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.
+-->
+We are happy to present the new 2.25.0 release of Apache Beam. This release includes both improvements and new functionality.
+See the [download page](/get-started/downloads/#2250-2020-10-23) for this release.
+For more information on changes in 2.25.0, check out the
+[detailed release notes](https://issues.apache.org/jira/secure/ReleaseNote.jspa?projectId=12319527&version=12347147).
+
+## Highlights
+
+* Splittable DoFn is now the default for executing the Read transform for Java based runners (Direct, Flink, Jet, Samza, Twister2). The expected output of the Read transform is unchanged. Users can opt-out using `--experiments=use_deprecated_read`. The Apache Beam community is looking for feedback for this change as the community is planning to make this change permanent with no opt-out. If you run into an issue requiring the opt-out, please send an e-mail to [user@beam.apache.org](mailto:user@beam.apache.org) specifically referencing BEAM-10670 in the subject line and why you needed to opt-out. (Java) ([BEAM-10670](https://issues.apache.org/jira/browse/BEAM-10670))
+
+## I/Os
+
+* Added cross-language support to Java's KinesisIO, now available in the Python module `apache_beam.io.kinesis` ([BEAM-10138](https://issues.apache.org/jira/browse/BEAM-10138), [BEAM-10137](https://issues.apache.org/jira/browse/BEAM-10137)).
+* Update Snowflake JDBC dependency for SnowflakeIO ([BEAM-10864](https://issues.apache.org/jira/browse/BEAM-10864))
+* Added cross-language support to Java's SnowflakeIO.Write, now available in the Python module `apache_beam.io.snowflake` ([BEAM-9898](https://issues.apache.org/jira/browse/BEAM-9898)).
+* Added delete function to Java's `ElasticsearchIO#Write`. Now, Java's ElasticsearchIO can be used to selectively delete documents using `withIsDeleteFn` function ([BEAM-5757](https://issues.apache.org/jira/browse/BEAM-5757)).
+* Java SDK: Added new IO connector for InfluxDB - InfluxDbIO ([BEAM-2546](https://issues.apache.org/jira/browse/BEAM-2546)).
+
+## New Features / Improvements
+
+* Support for repeatable fields in JSON decoder for `ReadFromBigQuery` added. (Python) ([BEAM-10524](https://issues.apache.org/jira/browse/BEAM-10524))
+* Added an opt-in, performance-driven runtime type checking system for the Python SDK ([BEAM-10549](https://issues.apache.org/jira/browse/BEAM-10549)).
+    More details will be in an upcoming [blog post](https://beam.apache.org/blog/python-performance-runtime-type-checking/index.html).
+* Added support for Python 3 type annotations on PTransforms using typed PCollections ([BEAM-10258](https://issues.apache.org/jira/browse/BEAM-10258)).
+    More details will be in an upcoming [blog post](https://beam.apache.org/blog/python-improved-annotations/index.html).
+* Improved the Interactive Beam API where recording streaming jobs now start a long running background recording job. Running ib.show() or ib.collect() samples from the recording ([BEAM-10603](https://issues.apache.org/jira/browse/BEAM-10603)).
+* In Interactive Beam, ib.show() and ib.collect() now have "n" and "duration" as parameters. These mean read only up to "n" elements and up to "duration" seconds of data read from the recording ([BEAM-10603](https://issues.apache.org/jira/browse/BEAM-10603)).
+* Initial preview of [Dataframes](https://s.apache.org/simpler-python-pipelines-2020#slide=id.g905ac9257b_1_21) support.
+    See also example at apache_beam/examples/wordcount_dataframe.py
+* Fixed support for type hints on `@ptransform_fn` decorators in the Python SDK.
+  ([BEAM-4091](https://issues.apache.org/jira/browse/BEAM-4091))
+  This has not enabled by default to preserve backwards compatibility; use the
+  `--type_check_additional=ptransform_fn` flag to enable. It may be enabled by
+  default in future versions of Beam.
+
+## Breaking Changes
+
+* Python 2 and Python 3.5 support dropped ([BEAM-10644](https://issues.apache.org/jira/browse/BEAM-10644), [BEAM-9372](https://issues.apache.org/jira/browse/BEAM-9372)).
+* Pandas 1.x allowed.  Older version of Pandas may still be used, but may not be as well tested.
+
+## Deprecations
+
+* Python transform ReadFromSnowflake has been moved from `apache_beam.io.external.snowflake` to `apache_beam.io.snowflake`. The previous path will be removed in the future versions.
+
+## Known Issues
+
+* Dataflow streaming timers once against not strictly time ordered when set earlier mid-bundle, as the fix for  [BEAM-8543](https://issues.apache.org/jira/browse/BEAM-8543) introduced more severe bugs and has been rolled back.
+
+
+## List of Contributors
+
+According to git shortlog, the following people contributed to the 2.25.0 release. Thank you to all contributors!
+
+Ahmet Altay, Alan Myrvold, Aldair Coronel Ruiz, Alexey Romanenko, Andrew Pilloud, Ankur Goenka,
+Ayoub ENNASSIRI, Bipin Upadhyaya, Boyuan Zhang, Brian Hulette, Brian Michalski, Chad Dombrova,
+Chamikara Jayalath, Damon Douglas, Daniel Oliveira, David Cavazos, David Janicek, Doug Roeper, Eric
+Roshan-Eisner, Etta Rapp, Eugene Kirpichov, Filipe Regadas, Heejong Lee, Ihor Indyk, Irvi Firqotul
+Aini, Ismaël Mejía, Jan Lukavský, Jayendra, Jiadai Xia, Jithin Sukumar, Jozsef Bartok, Kamil
+Gałuszka, Kamil Wasilewski, Kasia Kucharczyk, Kenneth Jung, Kenneth Knowles, Kevin Puthusseri, Kevin
+Sijo Puthusseri, KevinGG, Kyle Weaver, Leiyi Zhang, Lourens Naudé, Luke Cwik, Matthew Ouyang,
+Maximilian Michels, Michal Walenia, Milan Cermak, Monica Song, Nelson Osacky, Neville Li, Ning Kang,
+Pablo Estrada, Piotr Szuberski, Qihang, Rehman, Reuven Lax, Robert Bradshaw, Robert Burke, Rui Wang,
+Saavan Nanavati, Sam Bourne, Sam Rohde, Sam Whittle, Sergiy Kolesnikov, Sindy Li, Siyuan Chen, Steve
+Niemitz, Terry Xian, Thomas Weise, Tobiasz Kędzierski, Truc Le, Tyson Hamilton, Udi Meiri, Valentyn
+Tymofieiev, Yichi Zhang, Yifan Mai, Yueyang Qiu, annaqin418, danielxjd, dennis, dp, fuyuwei,
+lostluck, nehsyc, odeshpande, odidev, pulasthi, purbanow, rworley-monster, sclukas77, terryxian78,
+tvalentyn, yoshiki.obata
diff --git a/website/www/site/content/en/contribute/release-guide.md b/website/www/site/content/en/contribute/release-guide.md
index d23b5d2..d10fbf8 100644
--- a/website/www/site/content/en/contribute/release-guide.md
+++ b/website/www/site/content/en/contribute/release-guide.md
@@ -110,19 +110,11 @@
 
 __NOTE__: When generating the key, please make sure you choose the key type as __RSA and RSA (default)__ and key size as __4096 bit__.
 
+* To run the commands manually, refer to the contents of `preparation_before_release.sh`.
 
-##### Run all commands manually
+##### Key ID
 
-* Get more entropy for creating a GPG key
-
-      sudo apt-get install rng-tools
-      sudo rngd -r /dev/urandom
-
-* Create a GPG key
-
-      gpg --full-generate-key
-
-* Determine your Apache GPG Key and Key ID, as follows:
+* You may need your Key ID for future steps. Determine your Apache GPG Key and Key ID as follows:
 
       gpg --list-sigs --keyid-format LONG
 
@@ -135,20 +127,6 @@
 
   Here, the key ID is the 8-digit hex string in the `pub` line: `845E6689`.
 
-  Now, add your Apache GPG key to the Beam’s `KEYS` file both in [`dev`](https://dist.apache.org/repos/dist/dev/beam/KEYS) and [`release`](https://dist.apache.org/repos/dist/release/beam/KEYS) repositories at `dist.apache.org`. Follow the instructions listed at the top of these files. (Note: Only PMC members have write access to the release repository. If you end up getting 403 errors ask on the mailing list for assistance.)
-
-* Configure `git` to use this key when signing code by giving it your key ID, as follows:
-
-      git config --global user.signingkey 845E6689
-
-  You may drop the `--global` option if you’d prefer to use this key for the current repository only.
-
-* Start GPG agent in order to unlock your GPG key
-
-      eval $(gpg-agent --daemon --no-grab --write-env-file $HOME/.gpg-agent-info)
-      export GPG_TTY=$(tty)
-      export GPG_AGENT_INFO
-
 #### Access to Apache Nexus repository
 
 Configure access to the [Apache Nexus repository](https://repository.apache.org/), which enables final deployment of releases to the Maven Central Repository.
@@ -219,20 +197,8 @@
    "auth": "xxxxxx"
 }
 ```
-Release managers should have push permission; please ask for help at dev@.
-```
-From: Release Manager
-To: dev@beam.apache.org
-Subject: DockerHub Push Permission
+Release managers should have push permission; request membership in the [`beammaintainers` team](https://hub.docker.com/orgs/apache/teams/beammaintainers) by filing a JIRA with the Apache Infrastructure team, like [INFRA-20900](https://issues.apache.org/jira/browse/INFRA-20900).
 
-Hi DockerHub Admins
-
-I need push permission to proceed with release, can you please add me to maintainer team?
-My docker hub ID is: xxx
-
-Thanks,
-Release Manager
-```
 ### Create a new version in JIRA
 
 When contributors resolve an issue in JIRA, they are tagging it with a release that will contain their changes. With the release currently underway, new issues should be resolved against a subsequent future release. Therefore, you should create a release item for this subsequent release, as follows:
@@ -563,11 +529,11 @@
 * JIRA release item for the subsequent release has been created;
 * All test failures from branch verification have associated JIRA issues;
 * There are no release blocking JIRA issues;
-* Combined javadoc has the appropriate contents;
 * Release branch has been created;
 * There are no open pull requests to release branch;
 * Originating branch has the version information updated to the new version;
 * Nightly snapshot is in progress (do revisit it continually);
+* Set `JAVA_HOME` to JDK 8 (Example: `export JAVA_HOME=/example/path/to/java/jdk8`).
 
 The core of the release process is the build-vote-fix cycle. Each cycle produces one release candidate. The Release Manager repeats this cycle until the community approves one release candidate, which is then finalized.
 
@@ -607,7 +573,7 @@
           docker run -it --entrypoint=/bin/bash apache/beam_python${ver}_sdk:${RELEASE}_rc{RC_NUM}
           ls -al /opt/apache/beam/third_party_licenses/ | wc -l
           ```
-          - For Java SDK images, there should be around 1400 dependencies.
+          - For Java SDK images, there should be around 200 dependencies.
           ```
           docker run -it --entrypoint=/bin/bash apache/beam_java${ver}_sdk:${RELEASE}_rc{RC_NUM}
           ls -al /opt/apache/beam/third_party_licenses/ | wc -l
@@ -686,7 +652,7 @@
 
 ### Blog post
 
-Write a blog post similar to [beam-2.20.0.md](https://github.com/apache/beam/blob/master/website/www/site/content/en/blog/beam-2.20.0.md).
+Write a blog post similar to [beam-2.23.0.md](https://github.com/apache/beam/commit/b976e7be0744a32e99c841ad790c54920c8737f5#diff-8b1c3fd0d4a6765c16dfd18509182f9d).
 
 - Update `CHANGES.md` by adding a new section for the next release.
 - Copy the changes for the current release from `CHANGES.md` to the blog post and edit as necessary.
@@ -743,7 +709,7 @@
 
 ## List of Contributors
 
-According to git shortlog, the following people contributed to the 2.XX.0 release. Thank you to all contributors!
+According to git shortlog, the following people contributed to the {$RELEASE_VERSION} release. Thank you to all contributors!
 
 ${CONTRIBUTORS}
 ```
diff --git a/website/www/site/content/en/documentation/io/built-in/google-bigquery.md b/website/www/site/content/en/documentation/io/built-in/google-bigquery.md
index bf252d7..a4f61cc 100644
--- a/website/www/site/content/en/documentation/io/built-in/google-bigquery.md
+++ b/website/www/site/content/en/documentation/io/built-in/google-bigquery.md
@@ -594,11 +594,6 @@
 
 ### Setting the insertion method
 
-{{< paragraph class="language-py" >}}
-> The Beam SDK for Python does not currently support specifying the insertion
-method.
-{{< /paragraph >}}
-
 BigQueryIO supports two methods of inserting data into BigQuery: load jobs and
 streaming inserts. Each insertion method provides different tradeoffs of cost,
 quota, and data consistency. See the BigQuery documentation for
@@ -725,10 +720,6 @@
 
 ### Using dynamic destinations
 
-{{< paragraph class="language-py" >}}
-> The Beam SDK for Python does not currently support dynamic destinations.
-{{< /paragraph >}}
-
 You can use the dynamic destinations feature to write elements in a
 `PCollection` to different BigQuery tables, possibly with different schemas.
 
@@ -769,15 +760,11 @@
 {{< /highlight >}}
 
 {{< highlight py >}}
-# The Beam SDK for Python does not currently support dynamic destinations.
+{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets.py" model_bigqueryio_write_dynamic_destinations>}}
 {{< /highlight >}}
 
 ### Using time partitioning
 
-{{< paragraph class="language-py" >}}
-> The Beam SDK for Python does not currently support time partitioning.
-{{< /paragraph >}}
-
 BigQuery time partitioning divides your table into smaller partitions, which is
 called a [partitioned table](https://cloud.google.com/bigquery/docs/partitioned-tables).
 Partitioned tables make it easier for you to manage and query your data.
@@ -805,7 +792,7 @@
 {{< /highlight >}}
 
 {{< highlight py >}}
-# The Beam SDK for Python does not currently support time partitioning.
+{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets.py" model_bigqueryio_time_partitioning>}}
 {{< /highlight >}}
 
 
diff --git a/website/www/site/content/en/documentation/programming-guide.md b/website/www/site/content/en/documentation/programming-guide.md
index 8371a48..abd682e 100644
--- a/website/www/site/content/en/documentation/programming-guide.md
+++ b/website/www/site/content/en/documentation/programming-guide.md
@@ -2645,12 +2645,12 @@
 
 
 {{< paragraph class="language-py" >}}
-**beam.Row**
+**beam.Row and Select**
 {{< /paragraph >}}
 
 {{< paragraph class="language-py" >}}
-It's also possible to create ad-hoc schema declarations with a simple lambda
-that returns instances of `beam.Row`:
+There are also methods for creating ad-hoc schema declarations. First, you can
+use a lambda that returns instances of `beam.Row`:
 {{< /paragraph >}}
 
 {{< highlight py >}}
@@ -2660,17 +2660,28 @@
 {{< /highlight >}}
 
 {{< paragraph class="language-py" >}}
-Note that this declaration doesn't include any specific information about the
-types of the `bank` and `purchase_amount` fields. Beam will attempt to infer
-type information, if it's unable to it will fall back to the generic type
+Sometimes it can be more concise to express the same logic with the
+[`Select`](https://beam.apache.org/releases/pydoc/current/apache_beam.transforms.core.html#apache_beam.transforms.core.Select) transform:
+{{< /paragraph >}}
+
+{{< highlight py >}}
+input_pc = ... # {"bank": ..., "purchase_amount": ...}
+output_pc = input_pc | beam.Select(bank=lambda item: item["bank"],
+                                   purchase_amount=lambda item: item["purchase_amount"])
+{{< /highlight >}}
+
+{{< paragraph class="language-py" >}}
+Note that these declaration don't include any specific information about the
+types of the `bank` and `purchase_amount` fields, so Beam will attempt to infer
+type information. If it's unable to it will fall back to the generic type
 `Any`. Sometimes this is not ideal, you can use casts to make sure Beam
-correctly infers types with `beam.Row`:
+correctly infers types with `beam.Row` or with `Select`:
 {{< /paragraph >}}
 
 {{< highlight py >}}
 input_pc = ... # {"bank": ..., "purchase_amount": ...}
 output_pc = input_pc | beam.Map(lambda item: beam.Row(bank=str(item["bank"]),
-                                                      purchase_amount=float(item["purchase_amount"]))
+                                                      purchase_amount=float(item["purchase_amount"])))
 {{< /highlight >}}
 
 
@@ -4320,7 +4331,7 @@
 *   Retrieve an accurate count of the number of elements that have been processed;
 *   ...and so on.
 
-### 10.1 The main concepts of Beam metrics
+### 10.1. The main concepts of Beam metrics
 *   **Named**. Each metric has a name which consists of a namespace and an actual name. The
     namespace can be used to differentiate between multiple metrics with the same name and also
     allows querying for all metrics within a specific namespace.
@@ -4340,7 +4351,7 @@
 > **Note:** It is runner-dependent whether metrics are accessible during pipeline execution or only
 after jobs have completed.
 
-### 10.2 Types of metrics {#types-of-metrics}
+### 10.2. Types of metrics {#types-of-metrics}
 There are three types of metrics that are supported for the moment: `Counter`, `Distribution` and
 `Gauge`.
 
@@ -4386,7 +4397,7 @@
 }
 {{< /highlight >}}
 
-### 10.3 Querying metrics {#querying-metrics}
+### 10.3. Querying metrics {#querying-metrics}
 `PipelineResult` has a method `metrics()` which returns a `MetricResults` object that allows
 accessing metrics. The main method available in `MetricResults` allows querying for all metrics
 matching a given filter.
@@ -4414,7 +4425,7 @@
 }
 {{< /highlight >}}
 
-### 10.4 Using metrics in pipeline {#using-metrics}
+### 10.4. Using metrics in pipeline {#using-metrics}
 Below, there is a simple example of how to use a `Counter` metric in a user pipeline.
 
 {{< highlight java >}}
@@ -4452,7 +4463,7 @@
 }
 {{< /highlight >}}
 
-### 10.5 Export metrics {#export-metrics}
+### 10.5. Export metrics {#export-metrics}
 
 Beam metrics can be exported to external sinks. If a metrics sink is set up in the configuration, the runner will push metrics to it at a default 5s period.
 The configuration is held in the [MetricsOptions](https://beam.apache.org/releases/javadoc/2.19.0/org/apache/beam/sdk/metrics/MetricsOptions.html) class.
@@ -4496,7 +4507,7 @@
 to other nodes in the graph. A `DoFn` can declare multiple state variables.
 {{< /paragraph >}}
 
-### 11.1 Types of state {#types-of-state}
+### 11.1. Types of state {#types-of-state}
 
 Beam provides several types of state:
 
@@ -4603,7 +4614,7 @@
        | 'Bag state pardo' >> beam.ParDo(BagStateDoFn()))
 {{< /highlight >}}
 
-### 11.2 Deferred state reads {#deferred-state-reads}
+### 11.2. Deferred state reads {#deferred-state-reads}
 
 When a `DoFn` contains multiple state specifications, reading each one in order can be slow. Calling the `read()` function
 on a state can cause the runner to perform a blocking read. Performing multiple blocking reads in sequence adds latency
@@ -4657,14 +4668,14 @@
 }));
 {{< /highlight >}}
 
-### 11.3 Timers {#timers}
+### 11.3. Timers {#timers}
 
 Beam provides a per-key timer callback API. This allows for delayed processing of data stored using the state API.
 Timers can be set to callback at either an event-time or a processing-time timestamp. Every timer is identified with a
 TimerId. A given timer for a key can only be set for a single timestamp. Calling set on a timer overwrites the previous
 firing time for that key's timer.
 
-#### 11.3.1 Event-time timers {#event-time-timers}
+#### 11.3.1. Event-time timers {#event-time-timers}
 
 Event-time timers fire when the input watermark for the DoFn passes the time at which the timer is set, meaning that
 the runner believes that there are no more elements to be processed with timestamps before the timer timestamp. This
@@ -4714,7 +4725,7 @@
        | 'EventTime timer pardo' >> beam.ParDo(EventTimerDoFn()))
 {{< /highlight >}}
 
-#### 11.3.2 Processing-time timers {#processing-time-timers}
+#### 11.3.2. Processing-time timers {#processing-time-timers}
 
 Processing-time timers fire when the real wall-clock time passes. This is often used to create larger batches of data
 before processing. It can also be used to schedule events that should occur at a specific time. Just like with
@@ -4762,7 +4773,7 @@
        | 'ProcessingTime timer pardo' >> beam.ParDo(ProcessingTimerDoFn()))
 {{< /highlight >}}
 
-#### 11.3.3 Dynamic timer tags {#dynamic-timer-tags}
+#### 11.3.3. Dynamic timer tags {#dynamic-timer-tags}
 
 Beam also supports dynamically setting a timer tag using `TimerMap`. This allows for setting multiple different timers
 in a `DoFn` and allowing for the timer tags to be dynamically chosen - e.g. based on data in the input elements. A
@@ -4792,7 +4803,7 @@
 {{< highlight python >}}
 To be supported, See BEAM-9602
 {{< /highlight >}}
-#### 11.3.4 Timer output timestamps {#timer-output-timestamps}
+#### 11.3.4. Timer output timestamps {#timer-output-timestamps}
 
 By default, event-time timers will hold the output watermark of the `ParDo` to the timestamp of the timer. This means
 that if a timer is set to 12pm, any windowed aggregations or event-time timers later in the pipeline graph that finish
@@ -4895,11 +4906,11 @@
 }));
 {{< /highlight >}}
 
-### 11.4 Garbage collecting state {#garbage-collecting-state}
+### 11.4. Garbage collecting state {#garbage-collecting-state}
 Per-key state needs to be garbage collected, or eventually the increasing size of state may negatively impact
 performance. There are two common strategies for garbage collecting state.
 
-##### 11.4.1 **Using windows for garbage collection** {#using-windows-for-garbage-collection}
+##### 11.4.1. **Using windows for garbage collection** {#using-windows-for-garbage-collection}
 All state and timers for a key is scoped to the window it is in. This means that depending on the timestamp of the
 input element the ParDo will see different values for the state depending on the window that element falls into. In
 addition, once the input watermark passes the end of the window, the runner should garbage collect all state for that
@@ -4940,7 +4951,7 @@
 This `ParDo` stores state per day. Once the pipeline is done processing data for a given day, all the state for that
 day is garbage collected.
 
-##### 11.4.1 **Using timers For garbage collection** {#using-timers-for-garbage-collection}
+##### 11.4.1. **Using timers For garbage collection** {#using-timers-for-garbage-collection}
 
 In some cases, it is difficult to find a windowing strategy that models the desired garbage-collection strategy. For
 example, a common desire is to garbage collect state for a key once no activity has been seen on the key for some time.
@@ -5017,7 +5028,7 @@
        | 'User DoFn' >> beam.ParDo(UserDoFn()))
 {{< /highlight >}}
 
-### 11.5 State and timers examples {#state-timers-examples}
+### 11.5. State and timers examples {#state-timers-examples}
 
 Following are some example uses of state and timers
 
@@ -5103,7 +5114,7 @@
  }));
 {{< /highlight >}}
 
-#### 11.5.2 Batching RPCs {#batching-rpcs}
+#### 11.5.2. Batching RPCs {#batching-rpcs}
 
 In this example, input elements are being forwarded to an external RPC service. The RPC accepts batch requests -
 multiple events for the same user can be batched in a single RPC call. Since this RPC service also imposes rate limits,
@@ -5143,3 +5154,281 @@
   }
 }));
 {{< /highlight >}}
+
+## 12. Splittable `DoFns` {#splittable-dofns}
+
+A Splittable `DoFn` (SDF) enables users to create modular components containing I/Os (and some advanced
+[non I/O use cases](https://s.apache.org/splittable-do-fn#heading=h.5cep9s8k4fxv)). Having modular
+I/O components that can be connected to each other simplify typical patterns that users want.
+For example, a popular use case is to read filenames from a message queue followed by parsing those
+files. Traditionally, users were required to either write a single I/O connector that contained the
+logic for the message queue and the file reader (increased complexity) or choose to reuse a message
+queue I/O followed by a regular `DoFn` that read the file (decreased performance). With SDF,
+we bring the richness of Apache Beam’s I/O APIs to a `DoFn` enabling modularity while maintaining the
+performance of traditional I/O connectors.
+
+### 12.1. SDF basics {#sdf-basics}
+
+At a high level, an SDF is responsible for processing element and restriction pairs. A
+restriction represents a subset of work that would have been necessary to have been done when
+processing the element.
+
+Executing an SDF follows the following steps:
+
+1. Each element is paired with a restriction (e.g. filename is paired with offset range representing the whole file).
+2. Each element and restriction pair is split (e.g. offset ranges are broken up into smaller pieces).
+3. The runner redistributes the element and restriction pairs to several workers.
+4. Element and restriction pairs are processed in parallel (e.g. the file is read). Within this last step,
+the element and restriction pair can pause its own processing and/or be split into further element and
+restriction pairs.
+
+![Diagram of steps that an SDF is composed of](/images/sdf_high_level_overview.svg)
+
+
+#### 12.1.1. A basic SDF {#a-basic-sdf}
+
+A basic SDF is composed of three parts: a restriction, a restriction provider, and a
+restriction tracker. The restriction is used to represent a subset of work for a given element.
+The restriction provider lets SDF authors override default implementations for splitting, sizing,
+watermark estimation, and so forth. In [Java](https://github.com/apache/beam/blob/f4c2734261396858e388ebef2eef50e7d48231a8/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java#L92)
+and [Go](https://github.com/apache/beam/blob/0f466e6bcd4ac8677c2bd9ecc8e6af3836b7f3b8/sdks/go/pkg/beam/pardo.go#L226),
+this is the `DoFn`. [Python](https://github.com/apache/beam/blob/f4c2734261396858e388ebef2eef50e7d48231a8/sdks/python/apache_beam/transforms/core.py#L213)
+has a dedicated RestrictionProvider type. The restriction tracker is responsible for tracking
+what subset of the restriction has been completed during processing.
+
+To define an SDF, you must choose whether the SDF is bounded (default) or
+unbounded and define a way to initialize an initial restriction for an element.
+
+{{< highlight java >}}
+{{< code_sample "examples/java/src/main/java/org/apache/beam/examples/snippets/Snippets.java" SDF_BasicExample >}}
+{{< /highlight >}}
+
+{{< highlight py >}}
+{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets.py" SDF_BasicExample >}}
+{{< /highlight >}}
+
+{{< highlight go >}}
+func (fn *splittableDoFn) CreateInitialRestriction(filename string) offsetrange.Restriction {
+	return offsetrange.Restriction{
+		Start: 0,
+		End:   getFileLength(filename),
+	}
+}
+
+func (fn *splittableDoFn) CreateTracker(rest offsetrange.Restriction) *sdf.LockRTracker {
+	return sdf.NewLockRTracker(offsetrange.NewTracker(rest))
+}
+
+func (fn *splittableDoFn) ProcessElement(rt *sdf.LockRTracker, filename string, emit func(int)) error {
+            file, err := os.Open(filename)
+	if err != nil {
+		return err
+	}
+	offset, err := seekToNextRecordBoundaryInFile(file, rt.GetRestriction().(offsetrange.Restriction).Start)
+
+	if err != nil {
+		return err
+	}
+	for rt.TryClaim(offset) {
+		record, newOffset := readNextRecord(file)
+		emit(record)
+		offset = newOffset
+	}
+	return nil
+}
+{{< /highlight >}}
+
+At this point, we have an SDF that supports [runner-initiated splits](#runner-initiated-split)
+enabling dynamic work rebalancing. To increase the rate at which initial parallelization of work occurs
+or for those runners that do not support runner-initiated splitting, we recommend providing
+a set of initial splits:
+
+{{< highlight java >}}
+{{< code_sample "examples/java/src/main/java/org/apache/beam/examples/snippets/Snippets.java" SDF_BasicExampleWithSplitting >}}
+{{< /highlight >}}
+
+{{< highlight py >}}
+{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets.py" SDF_BasicExampleWithSplitting >}}
+{{< /highlight >}}
+
+{{< highlight go >}}
+func (fn *splittableDoFn) SplitRestriction(filename string, rest offsetrange.Restriction) (splits []offsetrange.Restriction) {
+	size := 64 * (1 << 20)
+	i := rest.Start
+	for i < rest.End - size {
+		// Compute and output 64 MiB size ranges to process in parallel
+		end := i + size
+     		splits = append(splits, offsetrange.Restriction{i, end})
+		i = end
+	}
+	// Output the last range
+	splits = append(splits, offsetrange.Restriction{i, rest.End})
+	return splits
+}
+{{< /highlight >}}
+
+### 12.2. Sizing and progress {#sizing-and-progress}
+
+Sizing and progress are used during execution of an SDF to inform runners so that they may
+perform intelligent decisions about which restrictions to split and how to parallelize work.
+
+Before processing an element and restriction, an initial size may be used by a runner to choose
+how and who processes the restrictions attempting to improve initial balancing and parallelization
+of work. During the processing of an element and restriction, sizing and progress are used to choose
+which restrictions to split and who should process them.
+
+By default, we use the restriction tracker’s estimate for work remaining falling back to assuming
+that all restrictions have an equal cost. To override the default, SDF authors can provide the
+appropriate method within the restriction provider.
+
+{{< highlight java >}}
+{{< code_sample "examples/java/src/main/java/org/apache/beam/examples/snippets/Snippets.java" SDF_GetSize >}}
+{{< /highlight >}}
+
+{{< highlight py >}}
+{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets.py" SDF_GetSize >}}
+{{< /highlight >}}
+
+{{< highlight go >}}
+func (fn *splittableDoFn) RestrictionSize(filename string, rest offsetrange.Restriction) float64 {
+	weight := float64(1)
+	if strings.Contains(filename, “expensiveRecords”) {
+		weight = 2
+	}
+	return weight * (rest.End - rest.Start)
+}
+{{< /highlight >}}
+
+### 12.3. User-initiated checkpoint {#user-initiated-checkpoint}
+
+Some I/Os cannot produce all of the data necessary to complete a restriction within the lifetime of a
+single bundle. This typically happens with unbounded restrictions, but can also happen with bounded
+restrictions. For example, there could be more data that needs to be ingested but is not available yet.
+Another cause of this scenario is the source system throttling your data.
+
+Your SDF can signal to you that you are not done processing the current restriction. This
+signal can suggest a time to resume at. While the runner tries to honor the resume time, this is not
+guaranteed. This allows execution to continue on a restriction that has available work improving
+resource utilization.
+
+{{< highlight java >}}
+{{< code_sample "examples/java/src/main/java/org/apache/beam/examples/snippets/Snippets.java" SDF_UserInitiatedCheckpoint >}}
+{{< /highlight >}}
+
+{{< highlight py >}}
+{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets.py" SDF_UserInitiatedCheckpoint >}}
+{{< /highlight >}}
+
+### 12.4. Runner-initiated split {#runner-initiated-split}
+
+A runner at any time may attempt to split a restriction while it is being processed. This allows the
+runner to either pause processing of the restriction so that other work may be done (common for
+unbounded restrictions to limit the amount of output and/or improve latency) or split the restriction
+into two pieces, increasing the available parallelism within the system. It is important to author a
+SDF with this in mind since the end of the restriction may change. Thus when writing the
+processing loop, it is important to use the result from trying to claim a piece of the restriction
+instead of assuming one can process till the end.
+
+{{< highlight java >}}
+{{< code_sample "examples/java/src/main/java/org/apache/beam/examples/snippets/Snippets.java" SDF_BadTryClaimLoop >}}
+{{< /highlight >}}
+
+{{< highlight py >}}
+{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets.py" SDF_BadTryClaimLoop >}}
+{{< /highlight >}}
+
+{{< highlight go >}}
+func (fn *badTryClaimLoop) ProcessElement(rt *sdf.LockRTracker, filename string, emit func(int)) error {
+            file, err := os.Open(filename)
+	if err != nil {
+		return err
+	}
+	offset, err := seekToNextRecordBoundaryInFile(file, rt.GetRestriction().(offsetrange.Restriction).Start)
+
+	if err != nil {
+		return err
+	}
+
+	// The restriction tracker can be modified by another thread in parallel
+	// so storing state locally is ill advised.
+	end = rt.GetRestriction().(offsetrange.Restriction).End
+	for offset < end {
+		// Only after successfully claiming should we produce any output and/or
+		// perform side effects.
+    	rt.TryClaim(offset)
+		record, newOffset := readNextRecord(file)
+		emit(record)
+		offset = newOffset
+	}
+	return nil
+}
+{{< /highlight >}}
+
+### 12.5. Watermark estimation {#watermark-estimation}
+
+The default watermark estimator does not produce a watermark estimate. Therefore, the output watermark
+is solely computed by the minimum of upstream watermarks.
+
+An SDF can advance the output watermark by specifying a lower bound for all future output
+that this element and restriction pair will produce. The runner computes the minimum output watermark
+by taking the minimum over all upstream watermarks and the minimum reported by each element and
+restriction pair. The reported watermark must monotonically increase for each element and restriction
+pair across bundle boundaries. When an element and restriction pair stops processing its watermark,
+it is no longer considered part of the above calculation.
+
+Tips:
+*   If you author an SDF that outputs records with timestamps, you should expose ways to allow users of
+this SDF to configure which watermark estimator to use.
+*   Any data produced before the watermark may be considered late. See
+[watermarks and late data](#watermarks-and-late-data) for more details.
+
+#### 12.5.1. Controlling the watermark {#controlling-the-watermark}
+
+There are two general types of watermark estimators: timestamp observing and external clock observing.
+Timestamp observing watermark estimators use the output timestamp of each record to compute the watermark
+estimate while external clock observing watermark estimators control the watermark by using a clock that
+is not associated to any individual output, such as the local clock of the machine or a clock exposed
+through an external service.
+
+The restriction provider lets you override the default watermark estimation logic and use an existing
+watermark estimator implementation. You can also provide your own watermark estimator implementation.
+
+{{< highlight java >}}
+{{< code_sample "examples/java/src/main/java/org/apache/beam/examples/snippets/Snippets.java" SDF_CustomWatermarkEstimator >}}
+{{< /highlight >}}
+
+{{< highlight py >}}
+{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets.py" SDF_CustomWatermarkEstimator >}}
+{{< /highlight >}}
+
+### 12.6. Truncating during drain {#truncating-during-drain}
+
+Runners which support draining pipelines need the ability to drain SDFs; otherwise, the
+pipeline may never stop. By default, bounded restrictions process the remainder of the restriction while
+unbounded restrictions finish processing at the next SDF-initiated checkpoint or runner-initiated split.
+You are able to override this default behavior by defining the appropriate method on the restriction
+provider.
+
+{{< highlight java >}}
+{{< code_sample "examples/java/src/main/java/org/apache/beam/examples/snippets/Snippets.java" SDF_Truncate >}}
+{{< /highlight >}}
+
+{{< highlight py >}}
+{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets.py" SDF_Truncate >}}
+{{< /highlight >}}
+
+### 12.7. Bundle finalization {#bundle-finalization}
+
+Bundle finalization enables a `DoFn` to perform side effects by registering a callback.
+The callback is invoked once the runner has acknowledged that it has durably persisted the output.
+For example, a message queue might need to acknowledge messages that it has ingested into the pipeline.
+Bundle finalization is not limited to SDFs but is called out here since this is the primary
+use case.
+
+{{< highlight java >}}
+{{< code_sample "examples/java/src/main/java/org/apache/beam/examples/snippets/Snippets.java" BundleFinalize >}}
+{{< /highlight >}}
+
+{{< highlight py >}}
+{{< code_sample "sdks/python/apache_beam/examples/snippets/snippets.py" BundleFinalize >}}
+{{< /highlight >}}
\ No newline at end of file
diff --git a/website/www/site/content/en/get-started/downloads.md b/website/www/site/content/en/get-started/downloads.md
index d0395d5..722fa1a 100644
--- a/website/www/site/content/en/get-started/downloads.md
+++ b/website/www/site/content/en/get-started/downloads.md
@@ -88,6 +88,14 @@
 
 ## Releases
 
+### 2.25.0 (2020-10-23)
+Official [source code download](http://www.apache.org/dyn/closer.cgi/beam/2.25.0/apache-beam-2.25.0-source-release.zip).
+[SHA-512](https://downloads.apache.org/beam/2.25.0/apache-beam-2.25.0-source-release.zip.sha512).
+[signature](https://downloads.apache.org/beam/2.25.0/apache-beam-2.25.0-source-release.zip.asc).
+
+[Release notes](https://issues.apache.org/jira/secure/ReleaseNote.jspa?projectId=12319527&version=12347147).
+[Blog post](/blog/beam-2.25.0).
+
 ### 2.24.0 (2020-09-18)
 Official [source code download](http://www.apache.org/dyn/closer.cgi/beam/2.24.0/apache-beam-2.24.0-source-release.zip).
 [SHA-512](https://downloads.apache.org/beam/2.24.0/apache-beam-2.24.0-source-release.zip.sha512).
diff --git a/website/www/site/content/en/roadmap/go-sdk.md b/website/www/site/content/en/roadmap/go-sdk.md
index 4a4f9c4..4a5c28d 100644
--- a/website/www/site/content/en/roadmap/go-sdk.md
+++ b/website/www/site/content/en/roadmap/go-sdk.md
@@ -42,7 +42,7 @@
 
 ## Usability
 
-* Beam Schema Support [BEAM-9615](https://issues.apache.org/jira/browse/BEAM-4124)
+* Beam Schema Support [BEAM-9615](https://issues.apache.org/jira/browse/BEAM-9615)
 * Improvements to starcgen [BEAM-9616](https://issues.apache.org/jira/browse/BEAM-9616)
 
 ## Integrating with the Go ecosystem
diff --git a/website/www/site/layouts/partials/section-menu/en/documentation.html b/website/www/site/layouts/partials/section-menu/en/documentation.html
index 05264bc..a7ba2d4 100644
--- a/website/www/site/layouts/partials/section-menu/en/documentation.html
+++ b/website/www/site/layouts/partials/section-menu/en/documentation.html
@@ -158,6 +158,20 @@
         <li><a href="/documentation/programming-guide/#state-timers-examples">State and timers examples</a></li>
       </ul>
     </li>
+
+    <li class="section-nav-item--collapsible">
+      <span class="section-nav-list-title">Splittable DoFns</span>
+
+      <ul class="section-nav-list">
+        <li><a href="/documentation/programming-guide/#sdf-basics">Basics</a></li>
+        <li><a href="/documentation/programming-guide/#sizing-and-progress">Sizing and progress</a></li>
+        <li><a href="/documentation/programming-guide/#user-initiated-checkpoint">User-initiated checkpoint</a></li>
+        <li><a href="/documentation/programming-guide/#runner-initiated-split">Runner initiated split</a></li>
+        <li><a href="/documentation/programming-guide/#watermark-estimation">Watermark estimation</a></li>
+        <li><a href="/documentation/programming-guide/#truncating-during-drain">Truncating during drain</a></li>
+        <li><a href="/documentation/programming-guide/#bundle-finalization">Bundle finalization</a></li>
+      </ul>
+    </li>
   </ul>
 </li>
 
diff --git a/website/www/site/static/.htaccess b/website/www/site/static/.htaccess
index 08e2532..b95666c 100644
--- a/website/www/site/static/.htaccess
+++ b/website/www/site/static/.htaccess
@@ -21,6 +21,6 @@
 # The following redirect maintains the previously supported URLs.
 RedirectMatch permanent "/documentation/sdks/(javadoc|pydoc)(.*)" "https://beam.apache.org/releases/$1$2"
 # Keep this updated to point to the current release.
-RedirectMatch "/releases/([^/]+)/current(.*)" "https://beam.apache.org/releases/$1/2.24.0$2"
+RedirectMatch "/releases/([^/]+)/current(.*)" "https://beam.apache.org/releases/$1/2.25.0$2"
 
 RedirectMatch "/contribute/design-documents" "https://cwiki.apache.org/confluence/display/BEAM/Design+Documents"
diff --git a/website/www/site/static/images/sdf_high_level_overview.svg b/website/www/site/static/images/sdf_high_level_overview.svg
new file mode 100644
index 0000000..9ad596f
--- /dev/null
+++ b/website/www/site/static/images/sdf_high_level_overview.svg
@@ -0,0 +1,415 @@
+<?xml version="1.0" encoding="UTF-8" standalone="no"?>
+<!--
+ 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.
+-->
+<svg
+   xmlns:dc="http://purl.org/dc/elements/1.1/"
+   xmlns:cc="http://creativecommons.org/ns#"
+   xmlns:rdf="http://www.w3.org/1999/02/22-rdf-syntax-ns#"
+   xmlns:svg="http://www.w3.org/2000/svg"
+   xmlns="http://www.w3.org/2000/svg"
+   xmlns:sodipodi="http://sodipodi.sourceforge.net/DTD/sodipodi-0.dtd"
+   xmlns:inkscape="http://www.inkscape.org/namespaces/inkscape"
+   inkscape:version="1.0 (4035a4fb49, 2020-05-01)"
+   height="241.89746"
+   width="776.61823"
+   sodipodi:docname="SplittableDoFn 2020 Blog.svg"
+   id="svg121"
+   stroke-miterlimit="10"
+   stroke-linecap="square"
+   stroke="none"
+   fill="none"
+   viewBox="0 0 776.61823 241.89746"
+   version="1.1">
+  <metadata
+     id="metadata127">
+    <rdf:RDF>
+      <cc:Work
+         rdf:about="">
+        <dc:format>image/svg+xml</dc:format>
+        <dc:type
+           rdf:resource="http://purl.org/dc/dcmitype/StillImage" />
+      </cc:Work>
+    </rdf:RDF>
+  </metadata>
+  <defs
+     id="defs125" />
+  <sodipodi:namedview
+     inkscape:current-layer="svg121"
+     inkscape:window-maximized="1"
+     inkscape:window-y="0"
+     inkscape:window-x="0"
+     inkscape:cy="480"
+     inkscape:cx="513.17521"
+     inkscape:zoom="1.0210937"
+     showgrid="false"
+     id="namedview123"
+     inkscape:window-height="1246"
+     inkscape:window-width="2560"
+     inkscape:pageshadow="2"
+     inkscape:pageopacity="0"
+     guidetolerance="10"
+     gridtolerance="10"
+     objecttolerance="10"
+     borderopacity="1"
+     bordercolor="#666666"
+     pagecolor="#ffffff" />
+  <clipPath
+     id="p.0">
+    <path
+       id="path2"
+       clip-rule="nonzero"
+       d="M 0,0 H 1280 V 960 H 0 Z" />
+  </clipPath>
+  <g
+     id="g119"
+     clip-path="url(#p.0)">
+    <path
+       id="path5"
+       fill-rule="evenodd"
+       d="M 0,0 H 1280 V 960 H 0 Z"
+       fill-opacity="0"
+       fill="#000000" />
+    <path
+       id="path7"
+       fill-rule="evenodd"
+       d="m 46.39895,43.145138 v 0 C 46.39895,21.566464 63.891923,4.07349 85.470598,4.07349 H 696.9336 c 10.36242,0 20.30047,4.116465 27.62781,11.443821 7.32739,7.327356 11.44384,17.26538 11.44384,27.627827 V 199.42704 c 0,21.57867 -17.49298,39.07164 -39.07165,39.07164 H 85.470599 c -21.578674,0 -39.071648,-17.49297 -39.071648,-39.07164 z"
+       fill-opacity="0"
+       fill="#000000" />
+    <path
+       id="path9"
+       fill-rule="evenodd"
+       d="m 46.39895,43.145138 v 0 C 46.39895,21.566464 63.891923,4.07349 85.470598,4.07349 H 696.9336 c 10.36242,0 20.30047,4.116465 27.62781,11.443821 7.32739,7.327356 11.44384,17.26538 11.44384,27.627827 V 199.42704 c 0,21.57867 -17.49298,39.07164 -39.07165,39.07164 H 85.470599 c -21.578674,0 -39.071648,-17.49297 -39.071648,-39.07164 z"
+       stroke-linecap="butt"
+       stroke-linejoin="round"
+       stroke-width="2"
+       stroke="#b7b7b7" />
+    <path
+       id="path11"
+       fill-rule="evenodd"
+       d="m 72,87.2021 h 112 v 78.99213 H 72 Z"
+       fill="#ff9900" />
+    <path
+       id="path13"
+       fill-rule="nonzero"
+       d="m 90.46094,109.10504 v 4.45313 h -1.5 v -11.375 h 4.1875 q 1.875,0 2.921875,0.95312 1.0625,0.95313 1.0625,2.51563 0,1.65625 -1.03125,2.5625 -1.03125,0.89062 -2.96875,0.89062 z m 0,-1.23437 h 2.6875 q 1.203125,0 1.84375,-0.5625 0.640625,-0.5625 0.640625,-1.64063 0,-1.01562 -0.640625,-1.625 -0.640625,-0.60937 -1.75,-0.625 h -2.78125 z m 13.49219,5.6875 q -0.125,-0.25 -0.20313,-0.89063 -1,1.04688 -2.40625,1.04688 -1.25,0 -2.046873,-0.70313 -0.796875,-0.71875 -0.796875,-1.79687 0,-1.32813 1,-2.04688 0.999998,-0.73437 2.828128,-0.73437 h 1.40625 v -0.67188 q 0,-0.75 -0.45313,-1.20312 -0.45312,-0.45313 -1.34375,-0.45313 -0.76562,0 -1.29687,0.39063 -0.51563,0.39062 -0.51563,0.95312 h -1.453123 q 0,-0.64062 0.4375,-1.21875 0.453125,-0.59375 1.218753,-0.9375 0.78125,-0.34375 1.70312,-0.34375 1.45313,0 2.28125,0.73438 0.82813,0.71875 0.85938,2 v 3.89062 q 0,1.17188 0.29687,1.85938 v 0.125 z m -2.39063,-1.10938 q 0.67188,0 1.28125,-0.34375 0.60938,-0.35937 0.89063,-0.92187 v -1.73438 h -1.14063 q -2.656248,0 -2.656248,1.5625 0,0.67188 0.453128,1.0625 0.45312,0.375 1.17187,0.375 z m 7.45313,1.10938 h -1.45313 v -8.45313 h 1.45313 z m -1.5625,-10.70313 q 0,-0.34375 0.20312,-0.57812 0.21875,-0.25 0.64063,-0.25 0.42187,0 0.64062,0.25 0.21875,0.23437 0.21875,0.59375 0,0.34375 -0.21875,0.57812 -0.21875,0.23438 -0.64062,0.23438 -0.42188,0 -0.64063,-0.23438 -0.20312,-0.23437 -0.20312,-0.59375 z m 7.96093,3.54688 q -0.32812,-0.0625 -0.71875,-0.0625 -1.42187,0 -1.92187,1.21875 v 6 h -1.45313 v -8.45313 h 1.40625 l 0.0312,0.96875 q 0.70313,-1.125 2.01563,-1.125 0.42187,0 0.64062,0.10938 z m 8.79688,7.3125 q -1.71875,0 -2.79688,-1.125 -1.07812,-1.14063 -1.07812,-3.03125 v -0.26563 q 0,-1.25 0.46875,-2.23437 0.48437,-1 1.34375,-1.54688 0.875,-0.5625 1.875,-0.5625 1.65625,0 2.5625,1.09375 0.92187,1.07813 0.92187,3.09375 v 0.60938 h -5.73437 q 0.0312,1.25 0.73437,2.01562 0.70313,0.76563 1.78125,0.76563 0.76563,0 1.29688,-0.3125 0.53125,-0.3125 0.92187,-0.82813 l 0.89063,0.6875 q -1.0625,1.64063 -3.1875,1.64063 z m -0.1875,-7.57813 q -0.875,0 -1.46875,0.64063 -0.59375,0.625 -0.73438,1.78125 h 4.23438 v -0.10938 q -0.0625,-1.10937 -0.59375,-1.70312 -0.53125,-0.60938 -1.4375,-0.60938 z m 10.36719,7.42188 q -0.125,-0.25 -0.20313,-0.89063 -1,1.04688 -2.40625,1.04688 -1.25,0 -2.04687,-0.70313 -0.79688,-0.71875 -0.79688,-1.79687 0,-1.32813 1,-2.04688 1,-0.73437 2.82813,-0.73437 h 1.40625 v -0.67188 q 0,-0.75 -0.45313,-1.20312 -0.45312,-0.45313 -1.34375,-0.45313 -0.76562,0 -1.29687,0.39063 -0.51563,0.39062 -0.51563,0.95312 h -1.45312 q 0,-0.64062 0.4375,-1.21875 0.45312,-0.59375 1.21875,-0.9375 0.78125,-0.34375 1.70312,-0.34375 1.45313,0 2.28125,0.73438 0.82813,0.71875 0.85938,2 v 3.89062 q 0,1.17188 0.29687,1.85938 v 0.125 z M 132,112.44879 q 0.67188,0 1.28125,-0.34375 0.60938,-0.35937 0.89063,-0.92187 v -1.73438 h -1.14063 q -2.65625,0 -2.65625,1.5625 0,0.67188 0.45313,1.0625 0.45312,0.375 1.17187,0.375 z m 9.26563,0.0781 q 0.78125,0 1.35937,-0.46875 0.57813,-0.46875 0.64063,-1.17188 h 1.35937 q -0.0312,0.73438 -0.5,1.39063 -0.45312,0.65625 -1.23437,1.04687 -0.76563,0.39063 -1.625,0.39063 -1.71875,0 -2.75,-1.15625 Q 137.5,111.40192 137.5,109.40192 v -0.23438 q 0,-1.23437 0.45313,-2.1875 0.45312,-0.96875 1.29687,-1.5 0.85938,-0.53125 2.01563,-0.53125 1.42187,0 2.35937,0.85938 0.9375,0.84375 1,2.20312 h -1.35937 q -0.0625,-0.82812 -0.625,-1.34375 -0.5625,-0.53125 -1.375,-0.53125 -1.10938,0 -1.71875,0.79688 -0.59375,0.78125 -0.59375,2.28125 v 0.28125 q 0,1.45312 0.59375,2.25 0.60937,0.78125 1.71875,0.78125 z m 6.4375,-6.40625 q 0.95312,-1.17188 2.5,-1.17188 2.67187,0 2.70312,3.01563 v 5.59375 h -1.45312 v -5.59375 q 0,-0.92188 -0.42188,-1.35938 -0.40625,-0.4375 -1.26562,-0.4375 -0.70313,0 -1.23438,0.375 -0.53125,0.375 -0.82812,0.98438 v 6.03125 H 146.25 v -12 h 1.45313 z m 17.83594,4.46875 h -4.76563 l -1.07812,2.96875 h -1.54688 l 4.34375,-11.375 h 1.3125 l 4.35938,11.375 h -1.54688 z m -4.3125,-1.23438 h 3.85937 l -1.9375,-5.32812 z"
+       fill="#ffffff" />
+    <path
+       id="path15"
+       fill-rule="nonzero"
+       d="m 100.48047,130.55817 1.625,-6.45313 h 1.45312 l -2.46875,8.45313 h -1.17187 l -2.046875,-6.40625 -2,6.40625 H 94.69922 l -2.453125,-8.45313 h 1.4375 l 1.65625,6.32813 1.96875,-6.32813 h 1.171875 z m 6.11719,2 h -1.45313 v -8.45313 h 1.45313 z m -1.5625,-10.70313 q 0,-0.34375 0.20312,-0.57812 0.21875,-0.25 0.64063,-0.25 0.42187,0 0.64062,0.25 0.21875,0.23437 0.21875,0.59375 0,0.34375 -0.21875,0.57812 -0.21875,0.23438 -0.64062,0.23438 -0.42188,0 -0.64063,-0.23438 -0.20312,-0.23437 -0.20312,-0.59375 z m 5.83593,0.20313 v 2.04687 h 1.57813 v 1.10938 h -1.57813 v 5.25 q 0,0.5 0.20313,0.76562 0.21875,0.25 0.71875,0.25 0.25,0 0.6875,-0.0937 v 1.17188 q -0.5625,0.15625 -1.10938,0.15625 -0.96875,0 -1.46875,-0.57813 -0.48437,-0.59375 -0.48437,-1.67187 v -5.25 h -1.53125 v -1.10938 h 1.53125 v -2.04687 z m 4.71094,3.0625 q 0.95313,-1.17188 2.5,-1.17188 2.67188,0 2.70313,3.01563 v 5.59375 h -1.45313 v -5.59375 q 0,-0.92188 -0.42187,-1.35938 -0.40625,-0.4375 -1.26563,-0.4375 -0.70312,0 -1.23437,0.375 -0.53125,0.375 -0.82813,0.98438 v 6.03125 h -1.45312 v -12 h 1.45312 z m 12.89844,7.4375 h -1.45313 v -8.45313 h 1.45313 z m -1.5625,-10.70313 q 0,-0.34375 0.20312,-0.57812 0.21875,-0.25 0.64063,-0.25 0.42187,0 0.64062,0.25 0.21875,0.23437 0.21875,0.59375 0,0.34375 -0.21875,0.57812 -0.21875,0.23438 -0.64062,0.23438 -0.42188,0 -0.64063,-0.23438 -0.20312,-0.23437 -0.20312,-0.59375 z m 5.24219,2.25 0.0469,1.0625 q 0.96875,-1.21875 2.53125,-1.21875 2.67188,0 2.70313,3.01563 v 5.59375 h -1.45313 v -5.59375 q 0,-0.92188 -0.42187,-1.35938 -0.40625,-0.4375 -1.26563,-0.4375 -0.70312,0 -1.23437,0.375 -0.53125,0.375 -0.82813,0.98438 v 6.03125 h -1.45312 v -8.45313 z m 9.03125,8.45313 h -1.45313 v -8.45313 h 1.45313 z m -1.5625,-10.70313 q 0,-0.34375 0.20312,-0.57812 0.21875,-0.25 0.64063,-0.25 0.42187,0 0.64062,0.25 0.21875,0.23437 0.21875,0.59375 0,0.34375 -0.21875,0.57812 -0.21875,0.23438 -0.64062,0.23438 -0.42188,0 -0.64063,-0.23438 -0.20312,-0.23437 -0.20312,-0.59375 z m 5.83593,0.20313 v 2.04687 h 1.57813 v 1.10938 h -1.57813 v 5.25 q 0,0.5 0.20313,0.76562 0.21875,0.25 0.71875,0.25 0.25,0 0.6875,-0.0937 v 1.17188 q -0.5625,0.15625 -1.10938,0.15625 -0.96875,0 -1.46875,-0.57813 -0.48437,-0.59375 -0.48437,-1.67187 v -5.25 h -1.53125 v -1.10938 h 1.53125 v -2.04687 z m 4.83594,10.5 h -1.45312 v -8.45313 h 1.45312 z m -1.5625,-10.70313 q 0,-0.34375 0.20313,-0.57812 0.21875,-0.25 0.64062,-0.25 0.42188,0 0.64063,0.25 0.21875,0.23437 0.21875,0.59375 0,0.34375 -0.21875,0.57812 -0.21875,0.23438 -0.64063,0.23438 -0.42187,0 -0.64062,-0.23438 -0.20313,-0.23437 -0.20313,-0.59375 z m 9.08594,10.70313 q -0.125,-0.25 -0.20313,-0.89063 -1,1.04688 -2.40625,1.04688 -1.25,0 -2.04687,-0.70313 -0.79688,-0.71875 -0.79688,-1.79687 0,-1.32813 1,-2.04688 1,-0.73437 2.82813,-0.73437 h 1.40625 v -0.67188 q 0,-0.75 -0.45313,-1.20312 -0.45312,-0.45313 -1.34375,-0.45313 -0.76562,0 -1.29687,0.39063 -0.51563,0.39062 -0.51563,0.95312 h -1.45312 q 0,-0.64062 0.4375,-1.21875 0.45312,-0.59375 1.21875,-0.9375 0.78125,-0.34375 1.70312,-0.34375 1.45313,0 2.28125,0.73438 0.82813,0.71875 0.85938,2 v 3.89062 q 0,1.17188 0.29687,1.85938 v 0.125 z m -2.39063,-1.10938 q 0.67188,0 1.28125,-0.34375 0.60938,-0.35937 0.89063,-0.92187 v -1.73438 h -1.14063 q -2.65625,0 -2.65625,1.5625 0,0.67188 0.45313,1.0625 0.45312,0.375 1.17187,0.375 z m 7.45313,1.10938 h -1.45313 v -12 h 1.45313 z"
+       fill="#ffffff" />
+    <path
+       id="path17"
+       fill-rule="nonzero"
+       d="m 90.30469,144.40192 q -0.328125,-0.0625 -0.71875,-0.0625 -1.421875,0 -1.921875,1.21875 v 6 H 86.21094 v -8.45313 h 1.40625 l 0.03125,0.96875 q 0.703125,-1.125 2.015625,-1.125 0.421875,0 0.640625,0.10938 z m 4.6875,7.3125 q -1.71875,0 -2.796875,-1.125 -1.078125,-1.14063 -1.078125,-3.03125 v -0.26563 q 0,-1.25 0.46875,-2.23437 0.484375,-1 1.34375,-1.54688 0.875,-0.5625 1.875,-0.5625 1.65625,0 2.5625,1.09375 0.921875,1.07813 0.921875,3.09375 v 0.60938 H 92.55469 q 0.03125,1.25 0.734375,2.01562 0.703125,0.76563 1.78125,0.76563 0.765625,0 1.296875,-0.3125 0.53125,-0.3125 0.921875,-0.82813 l 0.890625,0.6875 q -1.0625,1.64063 -3.1875,1.64063 z m -0.1875,-7.57813 q -0.875,0 -1.46875,0.64063 -0.59375,0.625 -0.734375,1.78125 h 4.234375 v -0.10938 q -0.0625,-1.10937 -0.59375,-1.70312 -0.53125,-0.60938 -1.4375,-0.60938 z m 10.07031,5.17188 q 0,-0.57813 -0.4375,-0.90625 -0.4375,-0.32813 -1.54687,-0.5625 -1.09375,-0.23438 -1.73438,-0.5625 -0.64062,-0.32813 -0.95312,-0.78125 -0.312503,-0.45313 -0.312503,-1.07813 0,-1.03125 0.875003,-1.75 0.89062,-0.71875 2.25,-0.71875 1.4375,0 2.32812,0.75 0.90625,0.73438 0.90625,1.89063 h -1.45312 q 0,-0.59375 -0.51563,-1.01563 -0.5,-0.4375 -1.26562,-0.4375 -0.78125,0 -1.23438,0.34375 -0.4375,0.34375 -0.4375,0.89063 0,0.53125 0.40625,0.79687 0.42188,0.26563 1.5,0.51563 1.07813,0.23437 1.75,0.57812 0.67188,0.32813 1,0.79688 0.32813,0.46875 0.32813,1.15625 0,1.125 -0.90625,1.8125 -0.90625,0.6875 -2.35938,0.6875 -1.01562,0 -1.79687,-0.35938 -0.78125,-0.35937 -1.21875,-1 -0.437503,-0.65625 -0.437503,-1.40625 h 1.437503 q 0.0469,0.73438 0.57812,1.15625 0.54688,0.42188 1.4375,0.42188 0.82813,0 1.3125,-0.32813 0.5,-0.32812 0.5,-0.89062 z m 5.29688,-8.25 v 2.04687 H 111.75 v 1.10938 h -1.57812 v 5.25 q 0,0.5 0.20312,0.76562 0.21875,0.25 0.71875,0.25 0.25,0 0.6875,-0.0937 v 1.17188 q -0.5625,0.15625 -1.10937,0.15625 -0.96875,0 -1.46875,-0.57813 -0.48438,-0.59375 -0.48438,-1.67187 v -5.25 h -1.53125 v -1.10938 h 1.53125 v -2.04687 z m 7.35156,3.34375 q -0.32813,-0.0625 -0.71875,-0.0625 -1.42188,0 -1.92188,1.21875 v 6 h -1.45312 v -8.45313 h 1.40625 l 0.0312,0.96875 q 0.70312,-1.125 2.01562,-1.125 0.42188,0 0.64063,0.10938 z m 2.89844,7.15625 h -1.45313 v -8.45313 h 1.45313 z m -1.5625,-10.70313 q 0,-0.34375 0.20312,-0.57812 0.21875,-0.25 0.64063,-0.25 0.42187,0 0.64062,0.25 0.21875,0.23437 0.21875,0.59375 0,0.34375 -0.21875,0.57812 -0.21875,0.23438 -0.64062,0.23438 -0.42188,0 -0.64063,-0.23438 -0.20312,-0.23437 -0.20312,-0.59375 z m 7.25781,9.67188 q 0.78125,0 1.35937,-0.46875 0.57813,-0.46875 0.64063,-1.17188 h 1.35937 q -0.0312,0.73438 -0.5,1.39063 -0.45312,0.65625 -1.23437,1.04687 -0.76563,0.39063 -1.625,0.39063 -1.71875,0 -2.75,-1.15625 -1.01563,-1.15625 -1.01563,-3.15625 v -0.23438 q 0,-1.23437 0.45313,-2.1875 0.45312,-0.96875 1.29687,-1.5 0.85938,-0.53125 2.01563,-0.53125 1.42187,0 2.35937,0.85938 0.9375,0.84375 1,2.20312 h -1.35937 q -0.0625,-0.82812 -0.625,-1.34375 -0.5625,-0.53125 -1.375,-0.53125 -1.10938,0 -1.71875,0.79688 -0.59375,0.78125 -0.59375,2.28125 v 0.28125 q 0,1.45312 0.59375,2.25 0.60937,0.78125 1.71875,0.78125 z m 6.95312,-9.46875 v 2.04687 h 1.57813 v 1.10938 h -1.57813 v 5.25 q 0,0.5 0.20313,0.76562 0.21875,0.25 0.71875,0.25 0.25,0 0.6875,-0.0937 v 1.17188 q -0.5625,0.15625 -1.10938,0.15625 -0.96875,0 -1.46875,-0.57813 -0.48437,-0.59375 -0.48437,-1.67187 v -5.25 h -1.53125 v -1.10938 h 1.53125 v -2.04687 z m 4.83594,10.5 h -1.45312 v -8.45313 h 1.45312 z m -1.5625,-10.70313 q 0,-0.34375 0.20313,-0.57812 0.21875,-0.25 0.64062,-0.25 0.42188,0 0.64063,0.25 0.21875,0.23437 0.21875,0.59375 0,0.34375 -0.21875,0.57812 -0.21875,0.23438 -0.64063,0.23438 -0.42187,0 -0.64062,-0.23438 -0.20313,-0.23437 -0.20313,-0.59375 z m 3.49219,6.39063 q 0,-1.23438 0.48437,-2.21875 0.48438,-1 1.35938,-1.53125 0.875,-0.54688 1.98437,-0.54688 1.73438,0 2.79688,1.20313 1.0625,1.1875 1.0625,3.17187 v 0.0937 q 0,1.23438 -0.46875,2.21875 -0.46875,0.98438 -1.35938,1.53125 -0.875,0.54688 -2.01562,0.54688 -1.71875,0 -2.78125,-1.1875 -1.0625,-1.20313 -1.0625,-3.17188 z m 1.45312,0.17187 q 0,1.40625 0.64063,2.26563 0.65625,0.84375 1.75,0.84375 1.10937,0 1.75,-0.85938 0.65625,-0.85937 0.65625,-2.42187 0,-1.39063 -0.67188,-2.25 -0.65625,-0.85938 -1.75,-0.85938 -1.0625,0 -1.71875,0.85938 -0.65625,0.84375 -0.65625,2.42187 z m 9.42188,-4.3125 0.0469,1.0625 q 0.96875,-1.21875 2.53125,-1.21875 2.67188,0 2.70313,3.01563 v 5.59375 h -1.45313 v -5.59375 q 0,-0.92188 -0.42187,-1.35938 -0.40625,-0.4375 -1.26563,-0.4375 -0.70312,0 -1.23437,0.375 -0.53125,0.375 -0.82813,0.98438 v 6.03125 h -1.45312 v -8.45313 z m 15.82031,3.84375 h -2.67187 v 4.60938 h -1.51563 v -11.375 h 3.76563 q 1.92187,0 2.95312,0.875 1.04688,0.875 1.04688,2.54687 0,1.0625 -0.57813,1.85938 -0.57812,0.78125 -1.59375,1.17187 l 2.67188,4.82813 v 0.0937 H 169 Z m -2.67187,-1.21875 h 2.29687 q 1.125,0 1.78125,-0.57812 0.65625,-0.57813 0.65625,-1.54688 0,-1.0625 -0.625,-1.625 -0.625,-0.5625 -1.8125,-0.5625 h -2.29687 z"
+       fill="#ffffff" />
+    <path
+       id="path19"
+       fill-rule="evenodd"
+       d="M 0,126.53281 H 72"
+       fill-opacity="0"
+       fill="#000000" />
+    <path
+       id="path21"
+       fill-rule="evenodd"
+       d="M 0,126.53281 H 65.14583"
+       stroke-linecap="butt"
+       stroke-linejoin="round"
+       stroke-width="2"
+       stroke="#4a86e8" />
+    <path
+       id="path23"
+       fill-rule="evenodd"
+       d="m 65.14583,126.53281 -2.249168,2.24918 6.179535,-2.24918 -6.179535,-2.24917 z"
+       stroke-linecap="butt"
+       stroke-width="2"
+       stroke="#4a86e8"
+       fill="#4a86e8" />
+    <path
+       id="path25"
+       fill-rule="evenodd"
+       d="m 16,94.2021 h 40 v 63.33858 H 16 Z"
+       fill-opacity="0"
+       fill="#000000" />
+    <path
+       id="path27"
+       fill-rule="nonzero"
+       d="M 33.71875,118.5621 H 32.203125 L 31.5,116.32773 h -4.25 l -0.71875,2.23437 H 25.078125 L 28.46875,108.34335 H 30.375 Z m -2.625,-3.46875 -1.71875,-5.46875 -1.734375,5.46875 z"
+       fill="#000000" />
+    <path
+       id="path29"
+       fill-rule="evenodd"
+       d="m 184,126.69817 h 96"
+       fill-opacity="0"
+       fill="#000000" />
+    <path
+       id="path31"
+       fill-rule="evenodd"
+       d="m 184,126.69817 h 89.14584"
+       stroke-linecap="butt"
+       stroke-linejoin="round"
+       stroke-width="2"
+       stroke="#4a86e8" />
+    <path
+       id="path33"
+       fill-rule="evenodd"
+       d="m 273.1458,126.69817 -2.24915,2.24916 6.17954,-2.24916 -6.17954,-2.24917 z"
+       stroke-linecap="butt"
+       stroke-width="2"
+       stroke="#4a86e8"
+       fill="#4a86e8" />
+    <path
+       id="path35"
+       fill-rule="evenodd"
+       d="m 184,94.2021 h 104 v 31.33858 H 184 Z"
+       fill-opacity="0"
+       fill="#000000" />
+    <path
+       id="path37"
+       fill-rule="nonzero"
+       d="m 201.1875,118.5621 h -1.75 l -3.82813,-5.01562 v 5.01562 h -1.39062 v -10.21875 h 1.39062 v 4.75 l 3.75,-4.75 H 201 l -4.03125,4.85938 z m 9.375,-10.21875 -3.46875,10.21875 h -1.875 l -3.39063,-10.21875 h 1.57813 l 2.20312,6.89063 0.625,2.01562 0.64063,-2.01562 2.20312,-6.89063 z m 7.20312,9.48438 -0.84375,0.85937 -5.25,-4.39062 5.25,-4.375 0.84375,0.85937 -4.25,3.5 z m 10.34375,0.73437 h -1.51562 l -0.70313,-2.23437 h -4.25 l -0.71875,2.23437 h -1.45312 l 3.39062,-10.21875 h 1.90625 z m -2.625,-3.46875 -1.71875,-5.46875 -1.73437,5.46875 z m 4.5625,5.07813 q 0.40625,0.0156 0.78125,-0.0781 0.375,-0.0781 0.65625,-0.25 0.28125,-0.17187 0.45313,-0.42187 0.17187,-0.25 0.17187,-0.5625 0,-0.34375 -0.10937,-0.54688 -0.10938,-0.20312 -0.25,-0.375 -0.14063,-0.17187 -0.26563,-0.35937 -0.10937,-0.20313 -0.10937,-0.53125 0,-0.15625 0.0625,-0.32813 0.0625,-0.1875 0.1875,-0.32812 0.14062,-0.15625 0.32812,-0.25 0.20313,-0.0937 0.48438,-0.0937 0.26562,0 0.51562,0.10937 0.26563,0.10938 0.45313,0.34375 0.1875,0.23438 0.29687,0.59375 0.125,0.34375 0.125,0.82813 0,0.65625 -0.25,1.25 -0.23437,0.59375 -0.70312,1.04687 -0.46875,0.46875 -1.1875,0.73438 -0.70313,0.28125 -1.64063,0.28125 z M 254,118.5621 h -1.57813 l -1.51562,-3.26562 q -0.17188,-0.375 -0.35938,-0.625 -0.17187,-0.25 -0.39062,-0.39063 -0.20313,-0.14062 -0.45313,-0.20312 -0.23437,-0.0625 -0.54687,-0.0625 H 248.5 v 4.54687 h -1.39063 v -10.21875 h 2.73438 q 0.89062,0 1.53125,0.20313 0.64062,0.1875 1.04687,0.54687 0.42188,0.34375 0.60938,0.84375 0.1875,0.5 0.1875,1.09375 0,0.48438 -0.14063,0.90625 -0.14062,0.42188 -0.42187,0.78125 -0.26563,0.34375 -0.6875,0.59375 -0.40625,0.25 -0.9375,0.375 0.4375,0.15625 0.73437,0.53125 0.29688,0.35938 0.60938,0.98438 z m -2.23438,-7.40625 q 0,-0.82812 -0.51562,-1.23437 -0.5,-0.40625 -1.4375,-0.40625 H 248.5 v 3.375 h 1.125 q 0.5,0 0.89062,-0.10938 0.39063,-0.10937 0.67188,-0.32812 0.28125,-0.23438 0.42187,-0.54688 0.15625,-0.32812 0.15625,-0.75 z m 4.45313,-0.375 0.84375,-0.85937 5.25,4.375 -5.25,4.39062 -0.84375,-0.85937 4.25,-3.51563 z"
+       fill="#000000" />
+    <path
+       id="path39"
+       fill-rule="evenodd"
+       d="m 280,87.2021 h 176 v 78.99213 H 280 Z"
+       fill="#ff9900" />
+    <path
+       id="path41"
+       fill-rule="nonzero"
+       d="m 330.8711,108.48004 q -1.92187,-0.54687 -2.8125,-1.35937 -0.875,-0.8125 -0.875,-2 0,-1.34375 1.07813,-2.21875 1.07812,-0.875 2.79687,-0.875 1.17188,0 2.07813,0.45312 0.92187,0.45313 1.42187,1.25 0.51563,0.79688 0.51563,1.73438 h -1.51563 q 0,-1.03125 -0.65625,-1.60938 -0.65625,-0.59375 -1.84375,-0.59375 -1.10937,0 -1.73437,0.48438 -0.625,0.48437 -0.625,1.35937 0,0.6875 0.59375,1.17188 0.59375,0.48437 2,0.89062 1.42187,0.39063 2.21875,0.875 0.8125,0.46875 1.1875,1.10938 0.39062,0.64062 0.39062,1.51562 0,1.375 -1.07812,2.21875 -1.07813,0.82813 -2.875,0.82813 -1.17188,0 -2.1875,-0.45313 -1.01563,-0.45312 -1.57813,-1.23437 -0.54687,-0.78125 -0.54687,-1.76563 h 1.51562 q 0,1.03125 0.75,1.625 0.76563,0.59375 2.04688,0.59375 1.1875,0 1.8125,-0.48437 0.64062,-0.48438 0.64062,-1.3125 0,-0.84375 -0.59375,-1.29688 -0.57812,-0.45312 -2.125,-0.90625 z m 13.05469,0.9375 q 0,1.9375 -0.89063,3.125 -0.875,1.17188 -2.375,1.17188 -1.54687,0 -2.42187,-0.98438 v 4.07813 h -1.45313 v -11.70313 h 1.32813 l 0.0625,0.9375 q 0.89062,-1.09375 2.45312,-1.09375 1.53125,0 2.40625,1.15625 0.89063,1.14063 0.89063,3.1875 z m -1.4375,-0.15625 q 0,-1.4375 -0.60938,-2.26562 -0.60937,-0.82813 -1.67187,-0.82813 -1.3125,0 -1.96875,1.17188 v 4.03125 q 0.64062,1.15625 1.98437,1.15625 1.03125,0 1.64063,-0.8125 0.625,-0.82813 0.625,-2.45313 z m 4.85156,4.29688 h -1.45312 v -12 h 1.45312 z m 3.88281,0 h -1.45312 v -8.45313 h 1.45312 z m -1.5625,-10.70313 q 0,-0.34375 0.20313,-0.57812 0.21875,-0.25 0.64062,-0.25 0.42188,0 0.64063,0.25 0.21875,0.23437 0.21875,0.59375 0,0.34375 -0.21875,0.57812 -0.21875,0.23438 -0.64063,0.23438 -0.42187,0 -0.64062,-0.23438 -0.20313,-0.23437 -0.20313,-0.59375 z m 5.83594,0.20313 v 2.04687 h 1.57813 v 1.10938 h -1.57813 v 5.25 q 0,0.5 0.20313,0.76562 0.21875,0.25 0.71875,0.25 0.25,0 0.6875,-0.0937 v 1.17188 q -0.5625,0.15625 -1.10938,0.15625 -0.96875,0 -1.46875,-0.57813 -0.48437,-0.59375 -0.48437,-1.67187 v -5.25 h -1.53125 v -1.10938 h 1.53125 v -2.04687 z m 10.73438,10.65625 q -1.71875,0 -2.79688,-1.125 -1.07812,-1.14063 -1.07812,-3.03125 v -0.26563 q 0,-1.25 0.46875,-2.23437 0.48437,-1 1.34375,-1.54688 0.875,-0.5625 1.875,-0.5625 1.65625,0 2.5625,1.09375 0.92187,1.07813 0.92187,3.09375 v 0.60938 h -5.73437 q 0.0312,1.25 0.73437,2.01562 0.70313,0.76563 1.78125,0.76563 0.76563,0 1.29688,-0.3125 0.53125,-0.3125 0.92187,-0.82813 l 0.89063,0.6875 q -1.0625,1.64063 -3.1875,1.64063 z m -0.1875,-7.57813 q -0.875,0 -1.46875,0.64063 -0.59375,0.625 -0.73438,1.78125 h 4.23438 v -0.10938 q -0.0625,-1.10937 -0.59375,-1.70312 -0.53125,-0.60938 -1.4375,-0.60938 z m 10.36718,7.42188 q -0.125,-0.25 -0.20312,-0.89063 -1,1.04688 -2.40625,1.04688 -1.25,0 -2.04688,-0.70313 -0.79687,-0.71875 -0.79687,-1.79687 0,-1.32813 1,-2.04688 1,-0.73437 2.82812,-0.73437 h 1.40625 v -0.67188 q 0,-0.75 -0.45312,-1.20312 -0.45313,-0.45313 -1.34375,-0.45313 -0.76563,0 -1.29688,0.39063 -0.51562,0.39062 -0.51562,0.95312 h -1.45313 q 0,-0.64062 0.4375,-1.21875 0.45313,-0.59375 1.21875,-0.9375 0.78125,-0.34375 1.70313,-0.34375 1.45312,0 2.28125,0.73438 0.82812,0.71875 0.85937,2 v 3.89062 q 0,1.17188 0.29688,1.85938 v 0.125 z m -2.39062,-1.10938 q 0.67187,0 1.28125,-0.34375 0.60937,-0.35937 0.89062,-0.92187 v -1.73438 h -1.14062 q -2.65625,0 -2.65625,1.5625 0,0.67188 0.45312,1.0625 0.45313,0.375 1.17188,0.375 z m 9.26562,0.0781 q 0.78125,0 1.35938,-0.46875 0.57812,-0.46875 0.64062,-1.17188 h 1.35938 q -0.0312,0.73438 -0.5,1.39063 -0.45313,0.65625 -1.23438,1.04687 -0.76562,0.39063 -1.625,0.39063 -1.71875,0 -2.75,-1.15625 -1.01562,-1.15625 -1.01562,-3.15625 v -0.23438 q 0,-1.23437 0.45312,-2.1875 0.45313,-0.96875 1.29688,-1.5 0.85937,-0.53125 2.01562,-0.53125 1.42188,0 2.35938,0.85938 0.9375,0.84375 1,2.20312 h -1.35938 q -0.0625,-0.82812 -0.625,-1.34375 -0.5625,-0.53125 -1.375,-0.53125 -1.10937,0 -1.71875,0.79688 -0.59375,0.78125 -0.59375,2.28125 v 0.28125 q 0,1.45312 0.59375,2.25 0.60938,0.78125 1.71875,0.78125 z m 6.4375,-6.40625 q 0.95313,-1.17188 2.5,-1.17188 2.67188,0 2.70313,3.01563 v 5.59375 h -1.45313 v -5.59375 q 0,-0.92188 -0.42187,-1.35938 -0.40625,-0.4375 -1.26563,-0.4375 -0.70312,0 -1.23437,0.375 -0.53125,0.375 -0.82813,0.98438 v 6.03125 h -1.45312 v -12 h 1.45312 z m 15.72657,2.82812 h -2.67188 v 4.60938 h -1.51562 v -11.375 h 3.76562 q 1.92188,0 2.95313,0.875 1.04687,0.875 1.04687,2.54687 0,1.0625 -0.57812,1.85938 -0.57813,0.78125 -1.59375,1.17187 l 2.67187,4.82813 v 0.0937 h -1.60937 z m -2.67188,-1.21875 h 2.29688 q 1.125,0 1.78125,-0.57812 0.65625,-0.57813 0.65625,-1.54688 0,-1.0625 -0.625,-1.625 -0.625,-0.5625 -1.8125,-0.5625 h -2.29688 z"
+       fill="#ffffff" />
+    <path
+       id="path43"
+       fill-rule="nonzero"
+       d="m 326.09766,122.05817 v 2.04688 h 1.57813 v 1.10937 h -1.57813 v 5.25 q 0,0.5 0.20313,0.76563 0.21875,0.25 0.71875,0.25 0.25,0 0.6875,-0.0937 v 1.17187 q -0.5625,0.15625 -1.10938,0.15625 -0.96875,0 -1.46875,-0.57812 -0.48437,-0.59375 -0.48437,-1.67188 v -5.25 h -1.53125 v -1.10937 h 1.53125 v -2.04688 z m 2.72656,6.1875 q 0,-1.23437 0.48438,-2.21875 0.48437,-1 1.35937,-1.53125 0.875,-0.54687 1.98438,-0.54687 1.73437,0 2.79687,1.20312 1.0625,1.1875 1.0625,3.17188 v 0.0937 q 0,1.23437 -0.46875,2.21875 -0.46875,0.98437 -1.35937,1.53125 -0.875,0.54687 -2.01563,0.54687 -1.71875,0 -2.78125,-1.1875 -1.0625,-1.20312 -1.0625,-3.17187 z m 1.45313,0.17188 q 0,1.40625 0.64062,2.26562 0.65625,0.84375 1.75,0.84375 1.10938,0 1.75,-0.85937 0.65625,-0.85938 0.65625,-2.42188 0,-1.39062 -0.67187,-2.25 -0.65625,-0.85937 -1.75,-0.85937 -1.0625,0 -1.71875,0.85937 -0.65625,0.84375 -0.65625,2.42188 z m 19.14844,0 q 0,1.9375 -0.89063,3.125 -0.875,1.17187 -2.375,1.17187 -1.54687,0 -2.42187,-0.98437 v 4.07812 h -1.45313 v -11.70312 h 1.32813 l 0.0625,0.9375 q 0.89062,-1.09375 2.45312,-1.09375 1.53125,0 2.40625,1.15625 0.89063,1.14062 0.89063,3.1875 z m -1.4375,-0.15625 q 0,-1.4375 -0.60938,-2.26563 -0.60937,-0.82812 -1.67187,-0.82812 -1.3125,0 -1.96875,1.17187 v 4.03125 q 0.64062,1.15625 1.98437,1.15625 1.03125,0 1.64063,-0.8125 0.625,-0.82812 0.625,-2.45312 z m 8.49218,4.29687 q -0.125,-0.25 -0.20312,-0.89062 -1,1.04687 -2.40625,1.04687 -1.25,0 -2.04688,-0.70312 -0.79687,-0.71875 -0.79687,-1.79688 0,-1.32812 1,-2.04687 1,-0.73438 2.82812,-0.73438 h 1.40625 v -0.67187 q 0,-0.75 -0.45312,-1.20313 -0.45313,-0.45312 -1.34375,-0.45312 -0.76563,0 -1.29688,0.39062 -0.51562,0.39063 -0.51562,0.95313 h -1.45313 q 0,-0.64063 0.4375,-1.21875 0.45313,-0.59375 1.21875,-0.9375 0.78125,-0.34375 1.70313,-0.34375 1.45312,0 2.28125,0.73437 0.82812,0.71875 0.85937,2 v 3.89063 q 0,1.17187 0.29688,1.85937 v 0.125 z m -2.39062,-1.10937 q 0.67187,0 1.28125,-0.34375 0.60937,-0.35938 0.89062,-0.92188 v -1.73437 h -1.14062 q -2.65625,0 -2.65625,1.5625 0,0.67187 0.45312,1.0625 0.45313,0.375 1.17188,0.375 z m 9.96875,-6.04688 q -0.32813,-0.0625 -0.71875,-0.0625 -1.42188,0 -1.92188,1.21875 v 6 h -1.45312 v -8.45312 h 1.40625 l 0.0312,0.96875 q 0.70312,-1.125 2.01562,-1.125 0.42188,0 0.64063,0.10937 z m 6.22656,7.15625 q -0.125,-0.25 -0.20312,-0.89062 -1,1.04687 -2.40625,1.04687 -1.25,0 -2.04688,-0.70312 -0.79687,-0.71875 -0.79687,-1.79688 0,-1.32812 1,-2.04687 1,-0.73438 2.82812,-0.73438 h 1.40625 v -0.67187 q 0,-0.75 -0.45312,-1.20313 -0.45313,-0.45312 -1.34375,-0.45312 -0.76563,0 -1.29688,0.39062 -0.51562,0.39063 -0.51562,0.95313 h -1.45313 q 0,-0.64063 0.4375,-1.21875 0.45313,-0.59375 1.21875,-0.9375 0.78125,-0.34375 1.70313,-0.34375 1.45312,0 2.28125,0.73437 0.82812,0.71875 0.85937,2 v 3.89063 q 0,1.17187 0.29688,1.85937 v 0.125 z m -2.39062,-1.10937 q 0.67187,0 1.28125,-0.34375 0.60937,-0.35938 0.89062,-0.92188 v -1.73437 h -1.14062 q -2.65625,0 -2.65625,1.5625 0,0.67187 0.45312,1.0625 0.45313,0.375 1.17188,0.375 z m 7.45312,1.10937 h -1.45312 v -12 h 1.45312 z m 3.88281,0 h -1.45312 v -12 h 1.45312 z m 5.82032,0.15625 q -1.71875,0 -2.79688,-1.125 -1.07812,-1.14062 -1.07812,-3.03125 v -0.26562 q 0,-1.25 0.46875,-2.23438 0.48437,-1 1.34375,-1.54687 0.875,-0.5625 1.875,-0.5625 1.65625,0 2.5625,1.09375 0.92187,1.07812 0.92187,3.09375 v 0.60937 h -5.73437 q 0.0312,1.25 0.73437,2.01563 0.70313,0.76562 1.78125,0.76562 0.76563,0 1.29688,-0.3125 0.53125,-0.3125 0.92187,-0.82812 l 0.89063,0.6875 q -1.0625,1.64062 -3.1875,1.64062 z m -0.1875,-7.57812 q -0.875,0 -1.46875,0.64062 -0.59375,0.625 -0.73438,1.78125 h 4.23438 v -0.10937 q -0.0625,-1.10938 -0.59375,-1.70313 -0.53125,-0.60937 -1.4375,-0.60937 z m 6.72656,7.42187 h -1.45313 v -12 h 1.45313 z m 3.88281,0 h -1.45312 v -8.45312 h 1.45312 z m -1.5625,-10.70312 q 0,-0.34375 0.20313,-0.57813 0.21875,-0.25 0.64062,-0.25 0.42188,0 0.64063,0.25 0.21875,0.23438 0.21875,0.59375 0,0.34375 -0.21875,0.57813 -0.21875,0.23437 -0.64063,0.23437 -0.42187,0 -0.64062,-0.23437 -0.20313,-0.23438 -0.20313,-0.59375 z m 5.22656,9.51562 h 4.95313 v 1.1875 h -6.71875 v -1.0625 l 4.67187,-6.20312 h -4.59375 v -1.1875 h 6.40625 v 1.01562 z m 9.96094,1.34375 q -1.71875,0 -2.79687,-1.125 -1.07813,-1.14062 -1.07813,-3.03125 v -0.26562 q 0,-1.25 0.46875,-2.23438 0.48438,-1 1.34375,-1.54687 0.875,-0.5625 1.875,-0.5625 1.65625,0 2.5625,1.09375 0.92188,1.07812 0.92188,3.09375 v 0.60937 h -5.73438 q 0.0312,1.25 0.73438,2.01563 0.70312,0.76562 1.78125,0.76562 0.76562,0 1.29687,-0.3125 0.53125,-0.3125 0.92188,-0.82812 l 0.89062,0.6875 q -1.0625,1.64062 -3.1875,1.64062 z m -0.1875,-7.57812 q -0.875,0 -1.46875,0.64062 -0.59375,0.625 -0.73437,1.78125 h 4.23437 v -0.10937 q -0.0625,-1.10938 -0.59375,-1.70313 -0.53125,-0.60937 -1.4375,-0.60937 z"
+       fill="#ffffff" />
+    <path
+       id="path45"
+       fill-rule="nonzero"
+       d="m 320.51562,147.41754 q 0,1.9375 -0.89062,3.125 -0.875,1.17187 -2.375,1.17187 -1.54688,0 -2.42188,-0.98437 v 4.07812 H 313.375 v -11.70312 h 1.32812 l 0.0625,0.9375 q 0.89063,-1.09375 2.45313,-1.09375 1.53125,0 2.40625,1.15625 0.89062,1.14062 0.89062,3.1875 z m -1.4375,-0.15625 q 0,-1.4375 -0.60937,-2.26563 -0.60938,-0.82812 -1.67188,-0.82812 -1.3125,0 -1.96875,1.17187 v 4.03125 q 0.64063,1.15625 1.98438,1.15625 1.03125,0 1.64062,-0.8125 0.625,-0.82812 0.625,-2.45312 z m 7.36719,-2.85938 q -0.32813,-0.0625 -0.71875,-0.0625 -1.42188,0 -1.92188,1.21875 v 6 h -1.45312 v -8.45312 h 1.40625 l 0.0312,0.96875 q 0.70312,-1.125 2.01562,-1.125 0.42188,0 0.64063,0.10937 z m 0.78906,2.84375 q 0,-1.23437 0.48438,-2.21875 0.48437,-1 1.35937,-1.53125 0.875,-0.54687 1.98438,-0.54687 1.73437,0 2.79687,1.20312 1.0625,1.1875 1.0625,3.17188 v 0.0937 q 0,1.23437 -0.46875,2.21875 -0.46875,0.98437 -1.35937,1.53125 -0.875,0.54687 -2.01563,0.54687 -1.71875,0 -2.78125,-1.1875 -1.0625,-1.20312 -1.0625,-3.17187 z m 1.45313,0.17188 q 0,1.40625 0.64062,2.26562 0.65625,0.84375 1.75,0.84375 1.10938,0 1.75,-0.85937 0.65625,-0.85938 0.65625,-2.42188 0,-1.39062 -0.67187,-2.25 -0.65625,-0.85937 -1.75,-0.85937 -1.0625,0 -1.71875,0.85937 -0.65625,0.84375 -0.65625,2.42188 z m 11.4375,3.10937 q 0.78125,0 1.35937,-0.46875 0.57813,-0.46875 0.64063,-1.17187 h 1.35937 q -0.0312,0.73437 -0.5,1.39062 -0.45312,0.65625 -1.23437,1.04688 -0.76563,0.39062 -1.625,0.39062 -1.71875,0 -2.75,-1.15625 -1.01563,-1.15625 -1.01563,-3.15625 v -0.23437 q 0,-1.23438 0.45313,-2.1875 0.45312,-0.96875 1.29687,-1.5 0.85938,-0.53125 2.01563,-0.53125 1.42187,0 2.35937,0.85937 0.9375,0.84375 1,2.20313 H 342.125 q -0.0625,-0.82813 -0.625,-1.34375 -0.5625,-0.53125 -1.375,-0.53125 -1.10938,0 -1.71875,0.79687 -0.59375,0.78125 -0.59375,2.28125 v 0.28125 q 0,1.45313 0.59375,2.25 0.60937,0.78125 1.71875,0.78125 z m 8.5,1.1875 q -1.71875,0 -2.79688,-1.125 -1.07812,-1.14062 -1.07812,-3.03125 v -0.26562 q 0,-1.25 0.46875,-2.23438 0.48437,-1 1.34375,-1.54687 0.875,-0.5625 1.875,-0.5625 1.65625,0 2.5625,1.09375 0.92187,1.07812 0.92187,3.09375 v 0.60937 h -5.73437 q 0.0312,1.25 0.73437,2.01563 0.70313,0.76562 1.78125,0.76562 0.76563,0 1.29688,-0.3125 0.53125,-0.3125 0.92187,-0.82812 l 0.89063,0.6875 q -1.0625,1.64062 -3.1875,1.64062 z m -0.1875,-7.57812 q -0.875,0 -1.46875,0.64062 -0.59375,0.625 -0.73438,1.78125 h 4.23438 v -0.10937 q -0.0625,-1.10938 -0.59375,-1.70313 -0.53125,-0.60937 -1.4375,-0.60937 z m 10.07031,5.17187 q 0,-0.57812 -0.4375,-0.90625 -0.4375,-0.32812 -1.54688,-0.5625 -1.09375,-0.23437 -1.73437,-0.5625 -0.64063,-0.32812 -0.95313,-0.78125 -0.3125,-0.45312 -0.3125,-1.07812 0,-1.03125 0.875,-1.75 0.89063,-0.71875 2.25,-0.71875 1.4375,0 2.32813,0.75 0.90625,0.73437 0.90625,1.89062 h -1.45313 q 0,-0.59375 -0.51562,-1.01562 -0.5,-0.4375 -1.26563,-0.4375 -0.78125,0 -1.23437,0.34375 -0.4375,0.34375 -0.4375,0.89062 0,0.53125 0.40625,0.79688 0.42187,0.26562 1.5,0.51562 1.07812,0.23438 1.75,0.57813 0.67187,0.32812 1,0.79687 0.32812,0.46875 0.32812,1.15625 0,1.125 -0.90625,1.8125 -0.90625,0.6875 -2.35937,0.6875 -1.01563,0 -1.79688,-0.35937 -0.78125,-0.35938 -1.21875,-1 -0.4375,-0.65625 -0.4375,-1.40625 h 1.4375 q 0.0469,0.73437 0.57813,1.15625 0.54687,0.42187 1.4375,0.42187 0.82812,0 1.3125,-0.32812 0.5,-0.32813 0.5,-0.89063 z m 8.25,0 q 0,-0.57812 -0.4375,-0.90625 -0.4375,-0.32812 -1.54688,-0.5625 -1.09375,-0.23437 -1.73437,-0.5625 -0.64063,-0.32812 -0.95313,-0.78125 -0.3125,-0.45312 -0.3125,-1.07812 0,-1.03125 0.875,-1.75 0.89063,-0.71875 2.25,-0.71875 1.4375,0 2.32813,0.75 0.90625,0.73437 0.90625,1.89062 h -1.45313 q 0,-0.59375 -0.51562,-1.01562 -0.5,-0.4375 -1.26563,-0.4375 -0.78125,0 -1.23437,0.34375 -0.4375,0.34375 -0.4375,0.89062 0,0.53125 0.40625,0.79688 0.42187,0.26562 1.5,0.51562 1.07812,0.23438 1.75,0.57813 0.67187,0.32812 1,0.79687 0.32812,0.46875 0.32812,1.15625 0,1.125 -0.90625,1.8125 -0.90625,0.6875 -2.35937,0.6875 -1.01563,0 -1.79688,-0.35937 -0.78125,-0.35938 -1.21875,-1 -0.4375,-0.65625 -0.4375,-1.40625 h 1.4375 q 0.0469,0.73437 0.57813,1.15625 0.54687,0.42187 1.4375,0.42187 0.82812,0 1.3125,-0.32812 0.5,-0.32813 0.5,-0.89063 z m 4.90625,2.25 h -1.45313 v -8.45312 h 1.45313 z m -1.5625,-10.70312 q 0,-0.34375 0.20312,-0.57813 0.21875,-0.25 0.64063,-0.25 0.42187,0 0.64062,0.25 0.21875,0.23438 0.21875,0.59375 0,0.34375 -0.21875,0.57813 -0.21875,0.23437 -0.64062,0.23437 -0.42188,0 -0.64063,-0.23437 -0.20312,-0.23438 -0.20312,-0.59375 z m 5.24219,2.25 0.0469,1.0625 q 0.96875,-1.21875 2.53125,-1.21875 2.67188,0 2.70313,3.01562 v 5.59375 h -1.45313 v -5.59375 q 0,-0.92187 -0.42187,-1.35937 -0.40625,-0.4375 -1.26563,-0.4375 -0.70312,0 -1.23437,0.375 -0.53125,0.375 -0.82813,0.98437 v 6.03125 h -1.45312 v -8.45312 z m 7.10937,4.15625 q 0,-1.98438 0.90625,-3.14063 0.92188,-1.17187 2.4375,-1.17187 1.54688,0 2.40625,1.09375 l 0.0781,-0.9375 h 1.3125 v 8.25 q 0,1.64062 -0.96875,2.57812 -0.96875,0.95313 -2.60938,0.95313 -0.92187,0 -1.79687,-0.39063 -0.875,-0.39062 -1.32813,-1.0625 l 0.75,-0.875 q 0.92188,1.15625 2.26563,1.15625 1.0625,0 1.64062,-0.59375 0.59375,-0.59375 0.59375,-1.6875 v -0.71875 q -0.85937,1 -2.35937,1 -1.48438,0 -2.40625,-1.1875 -0.92188,-1.20312 -0.92188,-3.26562 z m 1.45313,0.15625 q 0,1.4375 0.57812,2.25 0.59375,0.8125 1.65625,0.8125 1.35938,0 2,-1.23438 v -3.85937 q -0.65625,-1.21875 -1.98437,-1.21875 -1.0625,0 -1.65625,0.82812 -0.59375,0.8125 -0.59375,2.42188 z m 11.45312,-0.17188 q 0,-1.23437 0.48438,-2.21875 0.48437,-1 1.35937,-1.53125 0.875,-0.54687 1.98438,-0.54687 1.73437,0 2.79687,1.20312 1.0625,1.1875 1.0625,3.17188 v 0.0937 q 0,1.23437 -0.46875,2.21875 -0.46875,0.98437 -1.35937,1.53125 -0.875,0.54687 -2.01563,0.54687 -1.71875,0 -2.78125,-1.1875 -1.0625,-1.20312 -1.0625,-3.17187 z m 1.45313,0.17188 q 0,1.40625 0.64062,2.26562 0.65625,0.84375 1.75,0.84375 1.10938,0 1.75,-0.85937 0.65625,-0.85938 0.65625,-2.42188 0,-1.39062 -0.67187,-2.25 -0.65625,-0.85937 -1.75,-0.85937 -1.0625,0 -1.71875,0.85937 -0.65625,0.84375 -0.65625,2.42188 z m 8.76562,4.14062 v -7.34375 h -1.34375 v -1.10937 h 1.34375 v -0.875 q 0,-1.35938 0.71875,-2.09375 0.73438,-0.75 2.0625,-0.75 0.5,0 0.98438,0.14062 l -0.0781,1.17188 q -0.35937,-0.0781 -0.78125,-0.0781 -0.70312,0 -1.09375,0.42188 -0.375,0.40625 -0.375,1.17187 v 0.89063 h 1.8125 v 1.10937 h -1.8125 v 7.34375 z m 15.3125,-2.96875 H 416.125 l -1.07813,2.96875 H 413.5 l 4.34375,-11.375 h 1.3125 l 4.35937,11.375 h -1.54687 z m -4.3125,-1.23437 h 3.85938 L 418.5,142.02691 Z"
+       fill="#ffffff" />
+    <path
+       id="path47"
+       fill-rule="evenodd"
+       d="M 456,126.53281 H 568"
+       fill-opacity="0"
+       fill="#000000" />
+    <path
+       id="path49"
+       fill-rule="evenodd"
+       d="M 456,126.53281 H 561.14581"
+       stroke-linecap="butt"
+       stroke-linejoin="round"
+       stroke-width="2"
+       stroke="#4a86e8" />
+    <path
+       id="path51"
+       fill-rule="evenodd"
+       d="m 561.1458,126.53281 -2.24915,2.24918 6.17951,-2.24918 -6.17951,-2.24917 z"
+       stroke-linecap="butt"
+       stroke-width="2"
+       stroke="#4a86e8"
+       fill="#4a86e8" />
+    <path
+       id="path53"
+       fill-rule="evenodd"
+       d="M 460,55.217846 H 564 V 125.86352 H 460 Z"
+       fill-opacity="0"
+       fill="#000000" />
+    <path
+       id="path55"
+       fill-rule="nonzero"
+       d="m 477.1875,79.57785 h -1.75 l -3.82813,-5.015625 v 5.015625 h -1.39062 V 69.3591 h 1.39062 v 4.75 l 3.75,-4.75 H 477 l -4.03125,4.859375 z m 9.375,-10.21875 -3.46875,10.21875 h -1.875 L 477.82812,69.3591 h 1.57813 l 2.20312,6.890625 0.625,2.015625 0.64063,-2.015625 2.20312,-6.890625 z m 7.20312,9.484375 -0.84375,0.859375 -5.25,-4.390625 5.25,-4.375 0.84375,0.859375 -4.25,3.5 z m 10.34375,0.734375 h -1.51562 l -0.70313,-2.234375 h -4.25 l -0.71875,2.234375 h -1.45312 l 3.39062,-10.21875 h 1.90625 z m -2.625,-3.46875 -1.71875,-5.46875 -1.73437,5.46875 z m 4.5625,5.078125 q 0.40625,0.01563 0.78125,-0.07813 0.375,-0.07813 0.65625,-0.25 0.28125,-0.171875 0.45313,-0.421875 0.17187,-0.25 0.17187,-0.5625 0,-0.34375 -0.10937,-0.546875 -0.10938,-0.203125 -0.25,-0.375 -0.14063,-0.171875 -0.26563,-0.359375 -0.10937,-0.203125 -0.10937,-0.53125 0,-0.15625 0.0625,-0.328125 0.0625,-0.1875 0.1875,-0.328125 0.14062,-0.15625 0.32812,-0.25 0.20313,-0.09375 0.48438,-0.09375 0.26562,0 0.51562,0.109375 0.26563,0.109375 0.45313,0.34375 0.1875,0.234375 0.29687,0.59375 0.125,0.34375 0.125,0.828125 0,0.65625 -0.25,1.25 -0.23437,0.59375 -0.70312,1.046875 -0.46875,0.46875 -1.1875,0.734375 -0.70313,0.28125 -1.64063,0.28125 z M 530,79.57785 h -1.57813 l -1.51562,-3.265625 q -0.17188,-0.375 -0.35938,-0.625 -0.17187,-0.25 -0.39062,-0.390625 -0.20313,-0.140625 -0.45313,-0.203125 -0.23437,-0.0625 -0.54687,-0.0625 H 524.5 v 4.546875 h -1.39063 V 69.3591 h 2.73438 q 0.89062,0 1.53125,0.203125 0.64062,0.1875 1.04687,0.546875 0.42188,0.34375 0.60938,0.84375 0.1875,0.5 0.1875,1.09375 0,0.484375 -0.14063,0.90625 -0.14062,0.421875 -0.42187,0.78125 -0.26563,0.34375 -0.6875,0.59375 -0.40625,0.25 -0.9375,0.375 0.4375,0.15625 0.73437,0.53125 0.29688,0.359375 0.60938,0.984375 z m -2.23438,-7.40625 q 0,-0.828125 -0.51562,-1.234375 -0.5,-0.40625 -1.4375,-0.40625 H 524.5 v 3.375 h 1.125 q 0.5,0 0.89062,-0.109375 0.39063,-0.109375 0.67188,-0.328125 0.28125,-0.234375 0.42187,-0.546875 0.15625,-0.328125 0.15625,-0.75 z"
+       fill="#000000" />
+    <path
+       id="path57"
+       fill-rule="nonzero"
+       d="m 535.7656,85.57785 h -4.26563 V 84.7341 h 1.75 v -4.96875 l -1.64062,0.890625 -0.32813,-0.765625 2.15625,-1.140625 h 0.8125 V 84.7341 h 1.51563 z"
+       fill="#000000" />
+    <path
+       id="path59"
+       fill-rule="nonzero"
+       d="m 538.07764,71.7966 0.84375,-0.859375 5.25,4.375 -5.25,4.390625 -0.84375,-0.859375 4.25,-3.515625 z"
+       fill="#000000" />
+    <path
+       id="path61"
+       fill-rule="nonzero"
+       d="m 473.34375,96.1091 q 0.26562,0 0.5,0.109375 0.23437,0.09375 0.40625,0.28125 0.1875,0.171875 0.28125,0.421875 0.10937,0.234375 0.10937,0.5 0,0.265625 -0.10937,0.5 -0.0937,0.234375 -0.28125,0.421875 -0.17188,0.171875 -0.40625,0.265625 -0.23438,0.109375 -0.5,0.109375 -0.28125,0 -0.51563,-0.109375 -0.23437,-0.09375 -0.40625,-0.265625 -0.17187,-0.1875 -0.28125,-0.421875 -0.0937,-0.234375 -0.0937,-0.5 0,-0.265625 0.0937,-0.5 0.10938,-0.25 0.28125,-0.421875 0.17188,-0.1875 0.40625,-0.28125 0.23438,-0.109375 0.51563,-0.109375 z m 8.79687,0 q 0.26563,0 0.5,0.109375 0.23438,0.09375 0.40625,0.28125 0.1875,0.171875 0.28125,0.421875 0.10938,0.234375 0.10938,0.5 0,0.265625 -0.10938,0.5 -0.0937,0.234375 -0.28125,0.421875 -0.17187,0.171875 -0.40625,0.265625 -0.23437,0.109375 -0.5,0.109375 -0.28125,0 -0.51562,-0.109375 -0.23438,-0.09375 -0.40625,-0.265625 -0.17188,-0.1875 -0.28125,-0.421875 -0.0937,-0.234375 -0.0937,-0.5 0,-0.265625 0.0937,-0.5 0.10937,-0.25 0.28125,-0.421875 0.17187,-0.1875 0.40625,-0.28125 0.23437,-0.109375 0.51562,-0.109375 z m 8.79688,0 q 0.26562,0 0.5,0.109375 0.23437,0.09375 0.40625,0.28125 0.1875,0.171875 0.28125,0.421875 0.10937,0.234375 0.10937,0.5 0,0.265625 -0.10937,0.5 -0.0937,0.234375 -0.28125,0.421875 -0.17188,0.171875 -0.40625,0.265625 -0.23438,0.109375 -0.5,0.109375 -0.28125,0 -0.51563,-0.109375 -0.23437,-0.09375 -0.40625,-0.265625 -0.17187,-0.1875 -0.28125,-0.421875 -0.0937,-0.234375 -0.0937,-0.5 0,-0.265625 0.0937,-0.5 0.10938,-0.25 0.28125,-0.421875 0.17188,-0.1875 0.40625,-0.28125 0.23438,-0.109375 0.51563,-0.109375 z"
+       fill="#000000" />
+    <path
+       id="path63"
+       fill-rule="nonzero"
+       d="m 477.1875,117.57785 h -1.75 l -3.82813,-5.01563 v 5.01563 h -1.39062 V 107.3591 h 1.39062 v 4.75 l 3.75,-4.75 H 477 l -4.03125,4.85937 z m 9.375,-10.21875 -3.46875,10.21875 h -1.875 l -3.39063,-10.21875 h 1.57813 l 2.20312,6.89062 0.625,2.01563 0.64063,-2.01563 2.20312,-6.89062 z m 7.20312,9.48437 -0.84375,0.85938 -5.25,-4.39063 5.25,-4.375 0.84375,0.85938 -4.25,3.5 z m 10.34375,0.73438 h -1.51562 l -0.70313,-2.23438 h -4.25 l -0.71875,2.23438 h -1.45312 l 3.39062,-10.21875 h 1.90625 z m -2.625,-3.46875 -1.71875,-5.46875 -1.73437,5.46875 z m 4.5625,5.07812 q 0.40625,0.0156 0.78125,-0.0781 0.375,-0.0781 0.65625,-0.25 0.28125,-0.17188 0.45313,-0.42188 0.17187,-0.25 0.17187,-0.5625 0,-0.34375 -0.10937,-0.54687 -0.10938,-0.20313 -0.25,-0.375 -0.14063,-0.17188 -0.26563,-0.35938 -0.10937,-0.20312 -0.10937,-0.53125 0,-0.15625 0.0625,-0.32812 0.0625,-0.1875 0.1875,-0.32813 0.14062,-0.15625 0.32812,-0.25 0.20313,-0.0937 0.48438,-0.0937 0.26562,0 0.51562,0.10938 0.26563,0.10937 0.45313,0.34375 0.1875,0.23437 0.29687,0.59375 0.125,0.34375 0.125,0.82812 0,0.65625 -0.25,1.25 -0.23437,0.59375 -0.70312,1.04688 -0.46875,0.46875 -1.1875,0.73437 -0.70313,0.28125 -1.64063,0.28125 z M 530,117.57785 h -1.57813 l -1.51562,-3.26563 q -0.17188,-0.375 -0.35938,-0.625 -0.17187,-0.25 -0.39062,-0.39062 -0.20313,-0.14063 -0.45313,-0.20313 -0.23437,-0.0625 -0.54687,-0.0625 H 524.5 v 4.54688 h -1.39063 V 107.3591 h 2.73438 q 0.89062,0 1.53125,0.20312 0.64062,0.1875 1.04687,0.54688 0.42188,0.34375 0.60938,0.84375 0.1875,0.5 0.1875,1.09375 0,0.48437 -0.14063,0.90625 -0.14062,0.42187 -0.42187,0.78125 -0.26563,0.34375 -0.6875,0.59375 -0.40625,0.25 -0.9375,0.375 0.4375,0.15625 0.73437,0.53125 0.29688,0.35937 0.60938,0.98437 z m -2.23438,-7.40625 q 0,-0.82813 -0.51562,-1.23438 -0.5,-0.40625 -1.4375,-0.40625 H 524.5 v 3.375 h 1.125 q 0.5,0 0.89062,-0.10937 0.39063,-0.10938 0.67188,-0.32813 0.28125,-0.23437 0.42187,-0.54687 0.15625,-0.32813 0.15625,-0.75 z"
+       fill="#000000" />
+    <path
+       id="path65"
+       fill-rule="nonzero"
+       d="m 535.8281,123.57785 h -1.21875 l -1.96875,-4.23438 -0.57813,-1.375 v 3.4375 2.17188 h -0.85937 v -6.79688 h 1.1875 l 1.89062,4.01563 0.67188,1.54687 v -3.64062 -1.92188 h 0.875 z"
+       fill="#000000" />
+    <path
+       id="path67"
+       fill-rule="nonzero"
+       d="m 538.07764,109.7966 0.84375,-0.85938 5.25,4.375 -5.25,4.39063 -0.84375,-0.85938 4.25,-3.51562 z"
+       fill="#000000" />
+    <path
+       id="path69"
+       fill-rule="evenodd"
+       d="m 568,87.2021 h 152 v 78.99213 H 568 Z"
+       fill="#ff9900" />
+    <path
+       id="path71"
+       fill-rule="nonzero"
+       d="m 580.03906,118.60504 v 4.45313 h -1.5 v -11.375 h 4.1875 q 1.875,0 2.92187,0.95312 1.0625,0.95313 1.0625,2.51563 0,1.65625 -1.03125,2.5625 -1.03125,0.89062 -2.96875,0.89062 z m 0,-1.23437 h 2.6875 q 1.20312,0 1.84375,-0.5625 0.64062,-0.5625 0.64062,-1.64063 0,-1.01562 -0.64062,-1.625 -0.64063,-0.60937 -1.75,-0.625 h -2.78125 z m 12.45312,-1.46875 q -0.32812,-0.0625 -0.71875,-0.0625 -1.42187,0 -1.92187,1.21875 v 6 h -1.45313 v -8.45313 h 1.40625 l 0.0312,0.96875 q 0.70313,-1.125 2.01563,-1.125 0.42187,0 0.64062,0.10938 z m 0.78907,2.84375 q 0,-1.23438 0.48437,-2.21875 0.48438,-1 1.35938,-1.53125 0.875,-0.54688 1.98437,-0.54688 1.73438,0 2.79688,1.20313 1.0625,1.1875 1.0625,3.17187 v 0.0937 q 0,1.23438 -0.46875,2.21875 -0.46875,0.98438 -1.35938,1.53125 -0.875,0.54688 -2.01562,0.54688 -1.71875,0 -2.78125,-1.1875 -1.0625,-1.20313 -1.0625,-3.17188 z m 1.45312,0.17187 q 0,1.40625 0.64063,2.26563 0.65625,0.84375 1.75,0.84375 1.10937,0 1.75,-0.85938 0.65625,-0.85937 0.65625,-2.42187 0,-1.39063 -0.67188,-2.25 -0.65625,-0.85938 -1.75,-0.85938 -1.0625,0 -1.71875,0.85938 -0.65625,0.84375 -0.65625,2.42187 z m 11.4375,3.10938 q 0.78125,0 1.35938,-0.46875 0.57812,-0.46875 0.64062,-1.17188 h 1.35938 q -0.0312,0.73438 -0.5,1.39063 -0.45313,0.65625 -1.23438,1.04687 -0.76562,0.39063 -1.625,0.39063 -1.71875,0 -2.75,-1.15625 -1.01562,-1.15625 -1.01562,-3.15625 v -0.23438 q 0,-1.23437 0.45312,-2.1875 0.45313,-0.96875 1.29688,-1.5 0.85937,-0.53125 2.01562,-0.53125 1.42188,0 2.35938,0.85938 0.9375,0.84375 1,2.20312 h -1.35938 q -0.0625,-0.82812 -0.625,-1.34375 -0.5625,-0.53125 -1.375,-0.53125 -1.10937,0 -1.71875,0.79688 -0.59375,0.78125 -0.59375,2.28125 v 0.28125 q 0,1.45312 0.59375,2.25 0.60938,0.78125 1.71875,0.78125 z m 8.5,1.1875 q -1.71875,0 -2.79687,-1.125 -1.07813,-1.14063 -1.07813,-3.03125 v -0.26563 q 0,-1.25 0.46875,-2.23437 0.48438,-1 1.34375,-1.54688 0.875,-0.5625 1.875,-0.5625 1.65625,0 2.5625,1.09375 0.92188,1.07813 0.92188,3.09375 v 0.60938 h -5.73438 q 0.0312,1.25 0.73438,2.01562 0.70312,0.76563 1.78125,0.76563 0.76562,0 1.29687,-0.3125 0.53125,-0.3125 0.92188,-0.82813 l 0.89062,0.6875 q -1.0625,1.64063 -3.1875,1.64063 z m -0.1875,-7.57813 q -0.875,0 -1.46875,0.64063 -0.59375,0.625 -0.73437,1.78125 h 4.23437 v -0.10938 q -0.0625,-1.10937 -0.59375,-1.70312 -0.53125,-0.60938 -1.4375,-0.60938 z m 10.07031,5.17188 q 0,-0.57813 -0.4375,-0.90625 -0.4375,-0.32813 -1.54687,-0.5625 -1.09375,-0.23438 -1.73438,-0.5625 -0.64062,-0.32813 -0.95312,-0.78125 -0.3125,-0.45313 -0.3125,-1.07813 0,-1.03125 0.875,-1.75 0.89062,-0.71875 2.25,-0.71875 1.4375,0 2.32812,0.75 0.90625,0.73438 0.90625,1.89063 h -1.45312 q 0,-0.59375 -0.51563,-1.01563 -0.5,-0.4375 -1.26562,-0.4375 -0.78125,0 -1.23438,0.34375 -0.4375,0.34375 -0.4375,0.89063 0,0.53125 0.40625,0.79687 0.42188,0.26563 1.5,0.51563 1.07813,0.23437 1.75,0.57812 0.67188,0.32813 1,0.79688 0.32813,0.46875 0.32813,1.15625 0,1.125 -0.90625,1.8125 -0.90625,0.6875 -2.35938,0.6875 -1.01562,0 -1.79687,-0.35938 -0.78125,-0.35937 -1.21875,-1 -0.4375,-0.65625 -0.4375,-1.40625 h 1.4375 q 0.0469,0.73438 0.57812,1.15625 0.54688,0.42188 1.4375,0.42188 0.82813,0 1.3125,-0.32813 0.5,-0.32812 0.5,-0.89062 z m 8.25,0 q 0,-0.57813 -0.4375,-0.90625 -0.4375,-0.32813 -1.54687,-0.5625 -1.09375,-0.23438 -1.73438,-0.5625 -0.64062,-0.32813 -0.95312,-0.78125 -0.3125,-0.45313 -0.3125,-1.07813 0,-1.03125 0.875,-1.75 0.89062,-0.71875 2.25,-0.71875 1.4375,0 2.32812,0.75 0.90625,0.73438 0.90625,1.89063 h -1.45312 q 0,-0.59375 -0.51563,-1.01563 -0.5,-0.4375 -1.26562,-0.4375 -0.78125,0 -1.23438,0.34375 -0.4375,0.34375 -0.4375,0.89063 0,0.53125 0.40625,0.79687 0.42188,0.26563 1.5,0.51563 1.07813,0.23437 1.75,0.57812 0.67188,0.32813 1,0.79688 0.32813,0.46875 0.32813,1.15625 0,1.125 -0.90625,1.8125 -0.90625,0.6875 -2.35938,0.6875 -1.01562,0 -1.79687,-0.35938 -0.78125,-0.35937 -1.21875,-1 -0.4375,-0.65625 -0.4375,-1.40625 h 1.4375 q 0.0469,0.73438 0.57812,1.15625 0.54688,0.42188 1.4375,0.42188 0.82813,0 1.3125,-0.32813 0.5,-0.32812 0.5,-0.89062 z m 10.80469,2.40625 q -1.71875,0 -2.79687,-1.125 -1.07813,-1.14063 -1.07813,-3.03125 v -0.26563 q 0,-1.25 0.46875,-2.23437 0.48438,-1 1.34375,-1.54688 0.875,-0.5625 1.875,-0.5625 1.65625,0 2.5625,1.09375 0.92188,1.07813 0.92188,3.09375 v 0.60938 h -5.73438 q 0.0312,1.25 0.73438,2.01562 0.70312,0.76563 1.78125,0.76563 0.76562,0 1.29687,-0.3125 0.53125,-0.3125 0.92188,-0.82813 l 0.89062,0.6875 q -1.0625,1.64063 -3.1875,1.64063 z m -0.1875,-7.57813 q -0.875,0 -1.46875,0.64063 -0.59375,0.625 -0.73437,1.78125 h 4.23437 v -0.10938 q -0.0625,-1.10937 -0.59375,-1.70312 -0.53125,-0.60938 -1.4375,-0.60938 z m 6.72656,7.42188 h -1.45312 v -12 h 1.45312 z m 5.82032,0.15625 q -1.71875,0 -2.79688,-1.125 -1.07812,-1.14063 -1.07812,-3.03125 v -0.26563 q 0,-1.25 0.46875,-2.23437 0.48437,-1 1.34375,-1.54688 0.875,-0.5625 1.875,-0.5625 1.65625,0 2.5625,1.09375 0.92187,1.07813 0.92187,3.09375 v 0.60938 h -5.73437 q 0.0312,1.25 0.73437,2.01562 0.70313,0.76563 1.78125,0.76563 0.76563,0 1.29688,-0.3125 0.53125,-0.3125 0.92187,-0.82813 l 0.89063,0.6875 q -1.0625,1.64063 -3.1875,1.64063 z m -0.1875,-7.57813 q -0.875,0 -1.46875,0.64063 -0.59375,0.625 -0.73438,1.78125 h 4.23438 v -0.10938 q -0.0625,-1.10937 -0.59375,-1.70312 -0.53125,-0.60938 -1.4375,-0.60938 z m 6.50781,-1.03125 0.0469,0.9375 q 0.92188,-1.09375 2.5,-1.09375 1.78125,0 2.42188,1.35938 0.42187,-0.60938 1.09375,-0.98438 0.67187,-0.375 1.59375,-0.375 2.78125,0 2.82812,2.9375 v 5.67188 h -1.4375 v -5.57813 q 0,-0.90625 -0.42187,-1.35937 -0.40625,-0.45313 -1.39063,-0.45313 -0.79687,0 -1.32812,0.48438 -0.53125,0.48437 -0.625,1.29687 v 5.60938 h -1.45313 v -5.54688 q 0,-1.84375 -1.79687,-1.84375 -1.42188,0 -1.95313,1.21875 v 6.17188 h -1.4375 v -8.45313 z m 16.17969,8.60938 q -1.71875,0 -2.79688,-1.125 -1.07812,-1.14063 -1.07812,-3.03125 v -0.26563 q 0,-1.25 0.46875,-2.23437 0.48437,-1 1.34375,-1.54688 0.875,-0.5625 1.875,-0.5625 1.65625,0 2.5625,1.09375 0.92187,1.07813 0.92187,3.09375 v 0.60938 h -5.73437 q 0.0312,1.25 0.73437,2.01562 0.70313,0.76563 1.78125,0.76563 0.76563,0 1.29688,-0.3125 0.53125,-0.3125 0.92187,-0.82813 l 0.89063,0.6875 q -1.0625,1.64063 -3.1875,1.64063 z m -0.1875,-7.57813 q -0.875,0 -1.46875,0.64063 -0.59375,0.625 -0.73438,1.78125 h 4.23438 v -0.10938 q -0.0625,-1.10937 -0.59375,-1.70312 -0.53125,-0.60938 -1.4375,-0.60938 z m 6.52344,-1.03125 0.0469,1.0625 q 0.96875,-1.21875 2.53125,-1.21875 2.67188,0 2.70313,3.01563 v 5.59375 h -1.45313 v -5.59375 q 0,-0.92188 -0.42187,-1.35938 -0.40625,-0.4375 -1.26563,-0.4375 -0.70312,0 -1.23437,0.375 -0.53125,0.375 -0.82813,0.98438 v 6.03125 h -1.45312 v -8.45313 z m 9.42187,-2.04687 v 2.04687 h 1.57813 v 1.10938 h -1.57813 v 5.25 q 0,0.5 0.20313,0.76562 0.21875,0.25 0.71875,0.25 0.25,0 0.6875,-0.0937 v 1.17188 q -0.5625,0.15625 -1.10938,0.15625 -0.96875,0 -1.46875,-0.57813 -0.48437,-0.59375 -0.48437,-1.67187 v -5.25 h -1.53125 v -1.10938 h 1.53125 v -2.04687 z m 13.73438,7.53125 h -4.76563 l -1.07812,2.96875 h -1.54688 l 4.34375,-11.375 h 1.3125 l 4.35938,11.375 h -1.54688 z m -4.3125,-1.23438 h 3.85937 l -1.9375,-5.32812 z"
+       fill="#ffffff" />
+    <path
+       id="path73"
+       fill-rule="nonzero"
+       d="m 626.78845,142.05817 q -0.125,-0.25 -0.20313,-0.89063 -1,1.04688 -2.40625,1.04688 -1.25,0 -2.04687,-0.70313 -0.79688,-0.71875 -0.79688,-1.79687 0,-1.32813 1,-2.04688 1,-0.73437 2.82813,-0.73437 h 1.40625 v -0.67188 q 0,-0.75 -0.45313,-1.20312 -0.45312,-0.45313 -1.34375,-0.45313 -0.76562,0 -1.29687,0.39063 -0.51563,0.39062 -0.51563,0.95312 h -1.45312 q 0,-0.64062 0.4375,-1.21875 0.45312,-0.59375 1.21875,-0.9375 0.78125,-0.34375 1.70312,-0.34375 1.45313,0 2.28125,0.73438 0.82813,0.71875 0.85938,2 v 3.89062 q 0,1.17188 0.29687,1.85938 v 0.125 z m -2.39063,-1.10938 q 0.67188,0 1.28125,-0.34375 0.60938,-0.35937 0.89063,-0.92187 v -1.73438 h -1.14063 q -2.65625,0 -2.65625,1.5625 0,0.67188 0.45313,1.0625 0.45312,0.375 1.17187,0.375 z m 7.25,-7.34375 0.0469,1.0625 q 0.96875,-1.21875 2.53125,-1.21875 2.67187,0 2.70312,3.01563 v 5.59375 h -1.45312 v -5.59375 q 0,-0.92188 -0.42188,-1.35938 -0.40625,-0.4375 -1.26562,-0.4375 -0.70313,0 -1.23438,0.375 -0.53125,0.375 -0.82812,0.98438 v 6.03125 h -1.45313 v -8.45313 z m 7.10938,4.15625 q 0,-1.95312 0.92187,-3.125 0.92188,-1.1875 2.40625,-1.1875 1.48438,0 2.35938,1.01563 v -4.40625 h 1.4375 v 12 h -1.32813 l -0.0625,-0.90625 q -0.875,1.0625 -2.42187,1.0625 -1.46875,0 -2.39063,-1.20313 -0.92187,-1.20312 -0.92187,-3.14062 z m 1.4375,0.15625 q 0,1.4375 0.59375,2.25 0.59375,0.8125 1.64062,0.8125 1.375,0 2.01563,-1.23437 v -3.875 q -0.65625,-1.20313 -2,-1.20313 -1.0625,0 -1.65625,0.82813 -0.59375,0.8125 -0.59375,2.42187 z m 16.29687,-0.46875 h -2.67187 v 4.60938 h -1.51563 v -11.375 h 3.76563 q 1.92187,0 2.95312,0.875 1.04688,0.875 1.04688,2.54687 0,1.0625 -0.57813,1.85938 -0.57812,0.78125 -1.59375,1.17187 l 2.67188,4.82813 v 0.0937 h -1.60938 z m -2.67187,-1.21875 h 2.29687 q 1.125,0 1.78125,-0.57812 0.65625,-0.57813 0.65625,-1.54688 0,-1.0625 -0.625,-1.625 -0.625,-0.5625 -1.8125,-0.5625 h -2.29687 z"
+       fill="#ffffff" />
+    <path
+       id="path75"
+       fill-rule="nonzero"
+       d="m 664.1869,143.38629 1.82813,-2.90625 h 1.17187 l -2.39062,3.76562 2.45312,3.8125 h -1.1875 l -1.875,-2.95312 -1.85937,2.95312 h -1.1875 l 2.45312,-3.8125 -2.39062,-3.76562 h 1.17187 z"
+       fill="#ffffff" />
+    <path
+       id="path77"
+       fill-rule="evenodd"
+       d="m 736,94.2021 h 32 v 31.33858 h -32 z"
+       fill-opacity="0"
+       fill="#000000" />
+    <path
+       id="path79"
+       fill-rule="nonzero"
+       d="m 752.9375,115.4996 q 0,0.73438 -0.28125,1.3125 -0.28125,0.5625 -0.8125,0.95313 -0.53125,0.39062 -1.28125,0.59375 -0.73438,0.20312 -1.65625,0.20312 h -2.67188 v -10.21875 h 2.92188 q 3.42187,0 3.42187,2.48438 0,0.82812 -0.40625,1.42187 -0.39062,0.59375 -1.28125,0.89063 0.42188,0.0781 0.78125,0.26562 0.375,0.1875 0.65625,0.48438 0.28125,0.29687 0.4375,0.70312 0.17188,0.40625 0.17188,0.90625 z m -1.8125,-4.48437 q 0,-0.3125 -0.0937,-0.57813 -0.0937,-0.28125 -0.32813,-0.48437 -0.23437,-0.20313 -0.64062,-0.3125 -0.39063,-0.125 -1,-0.125 h -1.4375 v 3.20312 h 1.39062 q 0.48438,0 0.85938,-0.0937 0.39062,-0.10937 0.67187,-0.3125 0.28125,-0.21875 0.42188,-0.53125 0.15625,-0.32812 0.15625,-0.76562 z m 0.34375,4.53125 q 0,-0.39063 -0.17188,-0.70313 -0.15625,-0.3125 -0.46875,-0.51562 -0.3125,-0.21875 -0.76562,-0.32813 -0.45313,-0.125 -1.01563,-0.125 H 747.625 v 3.51563 h 1.46875 q 1.20312,0 1.78125,-0.4375 0.59375,-0.45313 0.59375,-1.40625 z"
+       fill="#000000" />
+    <path
+       id="path81"
+       fill-rule="evenodd"
+       d="m 720,126.69817 56,-0.31497"
+       fill-opacity="0"
+       fill="#000000" />
+    <path
+       id="path83"
+       fill-rule="evenodd"
+       d="m 720,126.69817 49.14593,-0.27642"
+       stroke-linecap="butt"
+       stroke-linejoin="round"
+       stroke-width="2"
+       stroke="#4a86e8" />
+    <path
+       id="path85"
+       fill-rule="evenodd"
+       d="m 769.14594,126.42175 -2.23651,2.26178 6.16681,-2.28389 -6.19208,-2.21437 z"
+       stroke-linecap="butt"
+       stroke-width="2"
+       stroke="#4a86e8"
+       fill="#4a86e8" />
+    <path
+       id="path87"
+       fill-rule="evenodd"
+       d="M 670.224,165.70341 V 190.2021 H 528 v -63"
+       fill-opacity="0"
+       fill="#000000" />
+    <path
+       id="path89"
+       fill-rule="evenodd"
+       d="M 670.224,165.70341 V 190.2021 H 528 v -56.14583"
+       stroke-linecap="butt"
+       stroke-linejoin="round"
+       stroke-width="2"
+       stroke="#4a86e8" />
+    <path
+       id="path91"
+       fill-rule="evenodd"
+       d="m 528,134.05627 2.24915,2.24916 L 528,130.1259 l -2.24915,6.17953 z"
+       stroke-linecap="butt"
+       stroke-width="2"
+       stroke="#4a86e8"
+       fill="#4a86e8" />
+    <path
+       id="path93"
+       fill-rule="evenodd"
+       d="m 548.3436,182.09636 v 0 c -0.99908,-2.58107 2.28101,-5.13618 8.4483,-6.5811 6.1673,-1.44492 14.14026,-1.52623 20.53565,-0.20944 v 0 c 2.26544,-1.50074 6.4118,-2.5369 11.185,-2.79506 4.77319,-0.25817 9.61248,0.29201 13.05413,1.48408 v 0 c 1.92987,-1.36067 5.71918,-2.27487 10.02332,-2.41819 4.30414,-0.14331 8.51392,0.50453 11.1355,1.71365 v 0 c 3.48657,-1.44231 9.03381,-2.04959 14.24139,-1.55905 5.20752,0.49054 9.14014,1.99077 10.09607,3.85154 v 0 c 4.27161,0.40963 7.82983,1.45095 9.75525,2.85491 1.92541,1.40396 2.02917,3.03285 0.28448,4.4658 v 0 c 4.2063,1.92463 5.19025,4.48916 2.58466,6.73656 -2.60559,2.24741 -8.40924,3.84004 -15.24524,4.18356 -0.0482,2.10927 -3.33862,4.04471 -8.60309,5.06032 -5.26446,1.01561 -11.68091,0.95279 -16.77606,-0.16423 -2.17029,2.52622 -8.27887,4.38498 -15.68664,4.77324 -7.40778,0.38826 -14.78675,-0.76358 -18.94892,-2.9579 -5.10193,1.08159 -11.22388,1.39316 -16.9848,0.86443 -5.76087,-0.52874 -10.67548,-1.85323 -13.63519,-3.6747 v 0 c -5.21357,0.21449 -10.25434,-0.73513 -12.62067,-2.37755 -2.36627,-1.64241 -1.55432,-3.62801 2.03284,-4.97133 v 0 c -4.65064,-0.96229 -7.02362,-2.87181 -5.88166,-4.7328 1.14203,-1.861 5.54029,-3.25176 10.90131,-3.44707 z"
+       fill="#ff9900" />
+    <path
+       id="path95"
+       fill-rule="evenodd"
+       d="m 543.22095,190.37572 v 0 c 2.19464,0.4541 4.72998,0.6601 7.26556,0.59032 m 3.32111,6.75872 c 1.09039,-0.0449 2.15918,-0.13984 3.17883,-0.28253 m 27.43964,3.09269 c -0.76691,-0.40433 -1.409,-0.83638 -1.91535,-1.28881 m 36.55225,-0.52661 v 0 c 0.39563,-0.46057 0.65204,-0.93458 0.76477,-1.41415 m 24.6134,-3.48175 c 0.0513,-2.24566 -3.57678,-4.30179 -9.32586,-5.28525 m 21.98626,-5.63465 c -0.93103,0.7647 -2.35235,1.44303 -4.15258,1.98185 m -5.88623,-9.30275 v 0 c 0.15863,0.3088 0.23205,0.62223 0.21923,0.93597 m -24.55627,-3.22838 v 0 c -0.86969,0.35979 -1.58624,0.76184 -2.12732,1.19365 m -19.03168,-0.48926 v 0 c -0.4635,0.32679 -0.80963,0.6726 -1.03033,1.02944 m -23.20917,0.28157 v 0 c 1.35321,0.27862 2.60517,0.614 3.72833,0.99872 m -32.71191,5.79185 v 0 c 0.1377,0.35575 0.35523,0.70709 0.65064,1.05078"
+       fill-opacity="0"
+       fill="#000000" />
+    <path
+       id="path97"
+       fill-rule="nonzero"
+       d="m 572.07404,181.51132 q -0.23438,1.32812 -1.1875,2.10937 -0.95313,0.76563 -2.28125,0.73438 -0.79688,-0.0312 -1.39063,-0.39063 -0.59375,-0.375 -0.92187,-1.07812 -0.32813,-0.70313 -0.35938,-1.54688 -0.0156,-0.48437 0.0469,-0.9375 l 0.0937,-0.70312 q 0.28125,-1.92188 1.34375,-3.03125 1.07812,-1.10938 2.625,-1.09375 1.20312,0.0312 1.90625,0.78125 0.70312,0.75 0.73437,2.09375 h -1.09375 q -0.0781,-1.79688 -1.40625,-1.9375 l -0.1875,-0.0156 q -1.10937,-0.0312 -1.85937,0.82813 -0.73438,0.85937 -0.95313,2.32812 l -0.125,0.89063 -0.0312,0.51562 q -0.0312,1.07813 0.375,1.71875 0.42188,0.625 1.25,0.65625 0.89063,0.0156 1.46875,-0.4375 0.59375,-0.45312 0.89063,-1.46875 z m 2.94727,-2.84375 q 0.84375,-0.90625 1.92187,-0.89063 0.89063,0.0156 1.3125,0.625 0.4375,0.59375 0.32813,1.67188 l -0.6875,4.15625 h -1.0625 l 0.70312,-4.17188 q 0.0312,-0.28125 0,-0.53125 -0.0937,-0.82812 -0.96875,-0.84375 -0.96875,-0.0156 -1.67187,1.01563 l -0.78125,4.53125 h -1.0625 l 1.5625,-9 h 1.0625 z m 7.23437,5.6875 q -1.1875,-0.0312 -1.85937,-0.875 -0.65625,-0.85938 -0.54688,-2.1875 l 0.0312,-0.25 q 0.0937,-0.9375 0.54688,-1.71875 0.46875,-0.78125 1.125,-1.17188 0.67187,-0.39062 1.4375,-0.375 0.96875,0.0156 1.53125,0.64063 0.5625,0.625 0.60937,1.70312 0.0156,0.375 -0.0312,0.76563 l -0.0625,0.48437 h -4.14062 q -0.10938,0.85938 0.28125,1.46875 0.39062,0.59375 1.17187,0.625 0.9375,0.0156 1.73438,-0.84375 l 0.60937,0.51563 q -0.40625,0.57812 -1.03125,0.90625 -0.625,0.3125 -1.40625,0.3125 z m 0.67188,-5.70313 q -0.65625,-0.0156 -1.14063,0.45313 -0.48437,0.45312 -0.78125,1.375 h 3.09375 l 0.0156,-0.0937 q 0.0937,-0.73438 -0.23438,-1.21875 -0.32812,-0.5 -0.95312,-0.51563 z m 5.48828,4.8125 q 0.5625,0.0156 1.01562,-0.32812 0.46875,-0.34375 0.60938,-0.90625 h 1 q -0.0937,0.59375 -0.48438,1.09375 -0.375,0.48437 -0.96875,0.76562 -0.59375,0.28125 -1.21875,0.26563 -0.79687,-0.0156 -1.34375,-0.42188 -0.54687,-0.40625 -0.8125,-1.125 -0.25,-0.71875 -0.15625,-1.60937 l 0.0156,-0.29688 q 0.125,-0.90625 0.54687,-1.625 0.42188,-0.71875 1.09375,-1.125 0.6875,-0.40625 1.5,-0.375 1,0.0156 1.59375,0.67188 0.59375,0.64062 0.59375,1.625 h -0.98437 q -0.0156,-0.625 -0.35938,-1 -0.32812,-0.39063 -0.89062,-0.42188 -0.82813,-0.0156 -1.375,0.625 -0.54688,0.625 -0.6875,1.76563 l -0.0156,0.15625 q -0.0312,0.375 0,0.76562 0.0312,0.6875 0.375,1.09375 0.34375,0.39063 0.95313,0.40625 z m 5.74609,-2.15625 -0.82812,0.73438 -0.375,2.1875 h -1.0625 l 1.5625,-9 h 1.0625 l -0.9375,5.375 0.65625,-0.65625 2.25,-2.0625 h 1.375 l -2.92188,2.70312 2.01563,3.64063 h -1.17188 z m 9.42773,-0.21875 q -0.10937,0.96875 -0.51562,1.75 -0.40625,0.76563 -1.01563,1.15625 -0.59375,0.375 -1.3125,0.35938 -1.10937,-0.0312 -1.6875,-0.75 l -0.5625,3.0625 h -1.0625 l 1.51563,-8.78125 h 0.96875 l -0.125,0.70312 q 0.78125,-0.84375 1.89062,-0.8125 0.90625,0.0156 1.40625,0.67188 0.51563,0.64062 0.54688,1.79687 0.0156,0.375 -0.0312,0.71875 z m -1.04687,-0.125 0.0312,-0.48437 q 0,-0.85938 -0.32813,-1.3125 -0.3125,-0.45313 -0.9375,-0.48438 -0.92187,-0.0156 -1.57812,0.875 l -0.53125,3.04688 q 0.34375,0.84375 1.28125,0.85937 0.78125,0.0156 1.32812,-0.60937 0.5625,-0.64063 0.73438,-1.89063 z m 5.41992,-3.1875 q 0.79688,0 1.35938,0.42188 0.57812,0.42187 0.84375,1.17187 0.28125,0.73438 0.1875,1.625 l -0.0156,0.125 q -0.10937,0.92188 -0.54687,1.67188 -0.4375,0.75 -1.14063,1.17187 -0.6875,0.40625 -1.53125,0.39063 -0.78125,-0.0156 -1.35937,-0.4375 -0.5625,-0.42188 -0.82813,-1.14063 -0.26562,-0.73437 -0.1875,-1.60937 0.0781,-0.98438 0.51563,-1.76563 0.45312,-0.79687 1.15625,-1.21875 0.70312,-0.4375 1.54687,-0.40625 z m -2.15625,3.39063 q -0.0469,0.375 0,0.76562 0.0625,0.70313 0.40625,1.10938 0.35938,0.40625 0.95313,0.42187 0.53125,0.0156 0.96875,-0.25 0.45312,-0.28125 0.75,-0.84375 0.3125,-0.5625 0.39062,-1.25 0.0469,-0.54687 0.0312,-0.90625 -0.0781,-0.71875 -0.4375,-1.125 -0.35937,-0.40625 -0.95312,-0.4375 -0.8125,-0.0156 -1.39063,0.65625 -0.57812,0.65625 -0.70312,1.73438 z m 6.47852,3.0625 h -1.04688 l 1.09375,-6.34375 h 1.0625 z m 0.875,-8.65625 q 0.26562,-0.0156 0.4375,0.15625 0.1875,0.17187 0.1875,0.45312 0,0.28125 -0.17188,0.46875 -0.17187,0.17188 -0.45312,0.1875 -0.28125,0 -0.45313,-0.17187 -0.17187,-0.17188 -0.17187,-0.4375 0.0156,-0.28125 0.17187,-0.45313 0.17188,-0.1875 0.45313,-0.20312 z m 2.96093,2.3125 -0.14062,0.79687 q 0.84375,-0.9375 1.9375,-0.90625 0.89062,0.0156 1.3125,0.625 0.4375,0.59375 0.32812,1.67188 l -0.6875,4.15625 h -1.0625 l 0.70313,-4.17188 q 0.0312,-0.28125 0,-0.53125 -0.0937,-0.82812 -0.96875,-0.84375 -0.95313,-0.0156 -1.67188,1.01563 l -0.78125,4.53125 h -1.0625 l 1.09375,-6.34375 z m 7.1836,-1.53125 -0.26563,1.53125 h 1.15625 l -0.15625,0.84375 h -1.14062 l -0.67188,3.9375 q -0.0156,0.15625 0,0.28125 0.0469,0.45312 0.5,0.45312 0.1875,0 0.51563,-0.0469 l -0.0781,0.875 q -0.39062,0.10937 -0.78125,0.10937 -0.625,-0.0156 -0.95312,-0.46875 -0.3125,-0.45312 -0.23438,-1.21875 l 0.64063,-3.92187 h -1.125 l 0.15625,-0.84375 h 1.10937 l 0.26563,-1.53125 z"
+       fill="#ffffff" />
+    <path
+       id="path99"
+       fill-rule="nonzero"
+       d="m 579.96466,198.96445 h -0.95313 l 5.0625,-9.26563 h 0.95313 z m 11.61133,-2.45313 q 0.0937,-0.65625 -0.71875,-0.92187 l -1.10938,-0.32813 q -1.39062,-0.48437 -1.34375,-1.59375 0.0469,-0.82812 0.76563,-1.35937 0.71875,-0.54688 1.70312,-0.53125 0.95313,0 1.54688,0.5625 0.60937,0.54687 0.59375,1.42187 l -1.0625,-0.0156 q 0.0156,-0.46875 -0.29688,-0.76563 -0.3125,-0.29687 -0.82812,-0.3125 -0.54688,-0.0156 -0.9375,0.26563 -0.375,0.26562 -0.4375,0.70312 -0.0781,0.54688 0.67187,0.79688 l 0.53125,0.14062 q 1.09375,0.28125 1.54688,0.73438 0.46875,0.45312 0.42187,1.125 -0.0312,0.59375 -0.39062,1.03125 -0.34375,0.4375 -0.9375,0.67187 -0.57813,0.23438 -1.23438,0.21875 -1.01562,-0.0156 -1.67187,-0.59375 -0.65625,-0.57812 -0.625,-1.48437 h 1.0625 q 0,0.53125 0.32812,0.85937 0.34375,0.3125 0.9375,0.3125 0.57813,0.0156 1,-0.23437 0.42188,-0.25 0.48438,-0.70313 z m 7.87109,-1.42187 q -0.10937,0.96875 -0.51562,1.75 -0.40625,0.76562 -1.01563,1.15625 -0.59375,0.375 -1.3125,0.35937 -1.10937,-0.0312 -1.6875,-0.75 l -0.5625,3.0625 h -1.0625 l 1.51563,-8.78125 h 0.96875 l -0.125,0.70313 q 0.78125,-0.84375 1.89062,-0.8125 0.90625,0.0156 1.40625,0.67187 0.51563,0.64063 0.54688,1.79688 0.0156,0.375 -0.0312,0.71875 z m -1.04687,-0.125 0.0312,-0.48438 q 0,-0.85937 -0.32813,-1.3125 -0.3125,-0.45312 -0.9375,-0.48437 -0.92187,-0.0156 -1.57812,0.875 l -0.53125,3.04687 q 0.34375,0.84375 1.28125,0.85938 0.78125,0.0156 1.32812,-0.60938 0.5625,-0.64062 0.73438,-1.89062 z m 3.04492,3.26562 h -1.04688 l 1.54688,-9 h 1.0625 z m 2.88281,0 h -1.04687 l 1.09375,-6.34375 h 1.0625 z m 0.875,-8.65625 q 0.26563,-0.0156 0.4375,0.15625 0.1875,0.17188 0.1875,0.45313 0,0.28125 -0.17187,0.46875 -0.17188,0.17187 -0.45313,0.1875 -0.28125,0 -0.45312,-0.17188 -0.17188,-0.17187 -0.17188,-0.4375 0.0156,-0.28125 0.17188,-0.45312 0.17187,-0.1875 0.45312,-0.20313 z m 3.66406,0.78125 -0.26562,1.53125 h 1.15625 l -0.15625,0.84375 h -1.14063 l -0.67187,3.9375 q -0.0156,0.15625 0,0.28125 0.0469,0.45313 0.5,0.45313 0.1875,0 0.51562,-0.0469 l -0.0781,0.875 q -0.39063,0.10938 -0.78125,0.10938 -0.625,-0.0156 -0.95313,-0.46875 -0.3125,-0.45313 -0.23437,-1.21875 l 0.64062,-3.92188 h -1.125 l 0.15625,-0.84375 h 1.10938 l 0.26562,-1.53125 z"
+       fill="#ffffff" />
+    <path
+       id="path101"
+       fill-rule="evenodd"
+       d="m 547.1102,195.32545 h 104 v 31.33859 h -104 z"
+       fill-opacity="0"
+       fill="#000000" />
+    <path
+       id="path103"
+       fill-rule="nonzero"
+       d="m 564.2977,219.68546 h -1.75 l -3.82813,-5.01563 v 5.01563 h -1.39062 v -10.21875 h 1.39062 v 4.75 l 3.75,-4.75 h 1.64063 l -4.03125,4.85937 z m 9.375,-10.21875 -3.46875,10.21875 h -1.875 l -3.39063,-10.21875 h 1.57813 l 2.20312,6.89062 0.625,2.01563 0.64063,-2.01563 2.20312,-6.89062 z m 7.20312,9.48437 -0.84375,0.85938 -5.25,-4.39063 5.25,-4.375 0.84375,0.85938 -4.25,3.5 z m 10.34375,0.73438 h -1.51562 l -0.70313,-2.23438 h -4.25 l -0.71875,2.23438 h -1.45312 l 3.39062,-10.21875 h 1.90625 z m -2.625,-3.46875 -1.71875,-5.46875 -1.73437,5.46875 z m 4.5625,5.07812 q 0.40625,0.0156 0.78125,-0.0781 0.375,-0.0781 0.65625,-0.25 0.28125,-0.17188 0.45313,-0.42188 0.17187,-0.25 0.17187,-0.5625 0,-0.34375 -0.10937,-0.54687 -0.10938,-0.20313 -0.25,-0.375 -0.14063,-0.17188 -0.26563,-0.35938 -0.10937,-0.20312 -0.10937,-0.53125 0,-0.15625 0.0625,-0.32812 0.0625,-0.1875 0.1875,-0.32813 0.14062,-0.15625 0.32812,-0.25 0.20313,-0.0937 0.48438,-0.0937 0.26562,0 0.51562,0.10938 0.26563,0.10937 0.45313,0.34375 0.1875,0.23437 0.29687,0.59375 0.125,0.34375 0.125,0.82812 0,0.65625 -0.25,1.25 -0.23437,0.59375 -0.70312,1.04688 -0.46875,0.46875 -1.1875,0.73437 -0.70313,0.28125 -1.64063,0.28125 z m 23.95313,-1.60937 h -1.57813 l -1.51562,-3.26563 q -0.17188,-0.375 -0.35938,-0.625 -0.17187,-0.25 -0.39062,-0.39062 -0.20313,-0.14063 -0.45313,-0.20313 -0.23437,-0.0625 -0.54687,-0.0625 h -0.65625 v 4.54688 h -1.39063 v -10.21875 h 2.73438 q 0.89062,0 1.53125,0.20312 0.64062,0.1875 1.04687,0.54688 0.42188,0.34375 0.60938,0.84375 0.1875,0.5 0.1875,1.09375 0,0.48437 -0.14063,0.90625 -0.14062,0.42187 -0.42187,0.78125 -0.26563,0.34375 -0.6875,0.59375 -0.40625,0.25 -0.9375,0.375 0.4375,0.15625 0.73437,0.53125 0.29688,0.35937 0.60938,0.98437 z m -2.23438,-7.40625 q 0,-0.82813 -0.51562,-1.23438 -0.5,-0.40625 -1.4375,-0.40625 h -1.3125 v 3.375 h 1.125 q 0.5,0 0.89062,-0.10937 0.39063,-0.10938 0.67188,-0.32813 0.28125,-0.23437 0.42187,-0.54687 0.15625,-0.32813 0.15625,-0.75 z"
+       fill="#000000" />
+    <path
+       id="path105"
+       fill-rule="nonzero"
+       d="m 623.4227,225.68546 h -1.17188 l -1.65625,-2.75 -1.65625,2.75 h -1.15625 l 2.26563,-3.46875 -2.07813,-3.32813 h 1.09375 l 1.54688,2.5625 1.57812,-2.5625 h 1.0625 l -2.09375,3.28125 z m 3.65576,-7.35938 -0.15625,2.45313 h -0.89062 l -0.14063,-2.45313 z"
+       fill="#000000" />
+    <path
+       id="path107"
+       fill-rule="nonzero"
+       d="m 631.0467,211.9042 0.84375,-0.85938 5.25,4.375 -5.25,4.39063 -0.84375,-0.85938 4.25,-3.51562 z"
+       fill="#000000" />
+    <path
+       id="path109"
+       fill-rule="evenodd"
+       d="m 433.0334,25.323118 v 0 c -1.55396,-3.918263 3.548,-7.797104 13.14096,-9.99059 9.59299,-2.193486 21.99454,-2.316926 31.94223,-0.317941 v 0 c 3.5238,-2.27824 9.97333,-3.851207 17.3978,-4.24311 7.42444,-0.391903 14.95178,0.443288 20.30508,2.252947 v 0 c 3.00183,-2.065599 8.89594,-3.4534183 15.59076,-3.6709857 6.69489,-0.2175684 13.24304,0.7659057 17.3208,2.6014337 v 0 c 5.42322,-2.1895293 14.0517,-3.1114129 22.1518,-2.3667449 8.10015,0.7446669 14.2171,3.0221369 15.70397,5.8469339 v 0 c 6.64435,0.621835 12.17896,2.202625 15.17383,4.333942 2.99494,2.131315 3.15631,4.604082 0.44251,6.779417 v 0 c 6.54272,2.921711 8.07324,6.814854 4.02038,10.226582 -4.05291,3.411724 -13.08026,5.82946 -23.71325,6.350944 -0.075,3.202027 -5.19312,6.140167 -13.38178,7.681938 -8.18866,1.541768 -18.16907,1.446408 -26.09436,-0.249321 -3.37573,3.83498 -12.87738,6.656723 -24.39984,7.246128 -11.52246,0.589409 -23.00009,-1.159183 -29.47415,-4.490314 -7.93588,1.641926 -17.45822,2.11491 -26.41907,1.312263 -8.96081,-0.80265 -16.60528,-2.813324 -21.20895,-5.578453 v 0 c -8.10944,0.3256 -15.95017,-1.115978 -19.63083,-3.609291 -3.68067,-2.493309 -2.41776,-5.507583 3.16195,-7.546848 v 0 c -7.23382,-1.460834 -10.92495,-4.359611 -9.14862,-7.184732 1.77634,-2.825123 8.61765,-4.936407 16.95642,-5.232903 z"
+       fill="#ff9900" />
+    <path
+       id="path111"
+       fill-rule="evenodd"
+       d="m 425.06528,37.891766 v 0 c 3.4137,0.689373 7.35727,1.002087 11.3013,0.89616 m 5.1658,10.260231 c 1.69608,-0.0681 3.35855,-0.212307 4.94455,-0.428902 m 42.68109,4.694912 c -1.1929,-0.613792 -2.19168,-1.269683 -2.97928,-1.956493 m 56.85532,-0.799435 v 0 c 0.61548,-0.699173 1.01422,-1.418773 1.18964,-2.146797 m 38.28503,-5.285549 c 0.0798,-3.40905 -5.56354,-6.530414 -14.50598,-8.023369 m 34.19867,-8.553799 c -1.44818,1.160845 -3.65906,2.190617 -6.45917,3.008572 m -9.15582,-14.122235 v 0 c 0.24676,0.468776 0.36096,0.944598 0.341,1.420869 m -38.1961,-4.900916 v 0 c -1.35285,0.546178 -2.46735,1.156526 -3.30896,1.812038 m -29.60291,-0.742725 v 0 c -0.72095,0.49609 -1.25928,1.021039 -1.60266,1.562752 m -36.1008,0.427452 v 0 c 2.10495,0.422985 4.05228,0.932091 5.79925,1.516137 m -50.88186,8.79245 v 0 c 0.21417,0.540039 0.55258,1.073417 1.01205,1.595148"
+       fill-opacity="0"
+       fill="#000000" />
+    <path
+       id="path113"
+       fill-rule="nonzero"
+       d="m 456.44363,19.39394 h -1.96875 l -0.59375,3.453125 h -1.10938 l 1.48438,-8.531251 h 2.5625 q 1.34375,0.01563 2.04687,0.6875 0.70313,0.671875 0.59375,1.828126 -0.0625,0.8125 -0.59375,1.421875 -0.51562,0.609375 -1.375,0.90625 l 1.32813,3.609375 -0.0156,0.07813 h -1.17187 z m -1.8125,-0.921875 1.625,0.01563 q 0.84375,0 1.42187,-0.453125 0.57813,-0.453125 0.67188,-1.203125 0.0937,-0.734375 -0.29688,-1.140626 -0.375,-0.421875 -1.15625,-0.4375 l -1.70312,-0.01563 z m 9.22461,3.75 q -0.73438,0.765625 -1.89063,0.75 -0.90625,-0.03125 -1.34375,-0.65625 -0.4375,-0.625 -0.32812,-1.71875 l 0.6875,-4.09375 h 1.0625 l -0.6875,4.109375 q -0.0312,0.265625 -0.0156,0.515625 0.0312,0.421875 0.25,0.65625 0.21875,0.234375 0.625,0.265625 1.14063,0.03125 1.75,-0.921875 l 0.79688,-4.625 h 1.0625 l -1.09375,6.34375 h -1 z m 4.55273,-5.71875 -0.14062,0.796875 q 0.84375,-0.9375 1.9375,-0.90625 0.89062,0.01563 1.3125,0.625 0.4375,0.59375 0.32812,1.671875 l -0.6875,4.15625 h -1.0625 l 0.70313,-4.171875 q 0.0312,-0.28125 0,-0.53125 -0.0937,-0.828125 -0.96875,-0.84375 -0.95313,-0.01563 -1.67188,1.015625 l -0.78125,4.53125 h -1.0625 l 1.09375,-6.34375 z m 6.48047,0 -0.14062,0.796875 q 0.84375,-0.9375 1.9375,-0.90625 0.89062,0.01563 1.3125,0.625 0.4375,0.59375 0.32812,1.671875 l -0.6875,4.15625 h -1.0625 l 0.70313,-4.171875 q 0.0312,-0.28125 0,-0.53125 -0.0937,-0.828125 -0.96875,-0.84375 -0.95313,-0.01563 -1.67188,1.015625 l -0.78125,4.53125 h -1.0625 l 1.09375,-6.34375 z m 7.1211,6.46875 q -1.1875,-0.03125 -1.85938,-0.875 -0.65625,-0.859375 -0.54687,-2.1875 l 0.0312,-0.25 q 0.0937,-0.9375 0.54687,-1.71875 0.46875,-0.78125 1.125,-1.171875 0.67188,-0.390625 1.4375,-0.375 0.96875,0.01563 1.53125,0.640625 0.5625,0.625 0.60938,1.703125 0.0156,0.375 -0.0312,0.765625 l -0.0625,0.484375 h -4.14063 q -0.10937,0.859375 0.28125,1.46875 0.39063,0.59375 1.17188,0.625 0.9375,0.01563 1.73437,-0.84375 l 0.60938,0.515625 q -0.40625,0.578125 -1.03125,0.90625 -0.625,0.3125 -1.40625,0.3125 z m 0.67187,-5.703125 q -0.65625,-0.01563 -1.14062,0.453125 -0.48438,0.453125 -0.78125,1.375 h 3.09375 l 0.0156,-0.09375 q 0.0937,-0.734375 -0.23437,-1.21875 -0.32813,-0.5 -0.95313,-0.515625 z m 6.75391,0.203125 -0.5,-0.04687 q -1.01563,0 -1.60938,0.921875 l -0.76562,4.5 h -1.0625 l 1.09375,-6.34375 h 1.03125 l -0.15625,0.734375 q 0.67187,-0.890625 1.57812,-0.859375 0.20313,0 0.51563,0.09375 z m 6.9375,0 -0.5,-0.04687 q -1.01563,0 -1.60938,0.921875 l -0.76562,4.5 h -1.0625 l 1.09375,-6.34375 h 1.03125 l -0.15625,0.734375 q 0.67187,-0.890625 1.57812,-0.859375 0.20313,0 0.51563,0.09375 z m 2.68164,5.5 q -1.1875,-0.03125 -1.85938,-0.875 -0.65625,-0.859375 -0.54687,-2.1875 l 0.0312,-0.25 q 0.0937,-0.9375 0.54687,-1.71875 0.46875,-0.78125 1.125,-1.171875 0.67188,-0.390625 1.4375,-0.375 0.96875,0.01563 1.53125,0.640625 0.5625,0.625 0.60938,1.703125 0.0156,0.375 -0.0312,0.765625 l -0.0625,0.484375 h -4.14063 q -0.10937,0.859375 0.28125,1.46875 0.39063,0.59375 1.17188,0.625 0.9375,0.01563 1.73437,-0.84375 l 0.60938,0.515625 q -0.40625,0.578125 -1.03125,0.90625 -0.625,0.3125 -1.40625,0.3125 z m 0.67187,-5.703125 q -0.65625,-0.01563 -1.14062,0.453125 -0.48438,0.453125 -0.78125,1.375 h 3.09375 l 0.0156,-0.09375 q 0.0937,-0.734375 -0.23437,-1.21875 -0.32813,-0.5 -0.95313,-0.515625 z m 3.11328,2.390625 q 0.125,-0.96875 0.53125,-1.734375 0.42188,-0.78125 1.03125,-1.171875 0.60938,-0.390625 1.34375,-0.375 1.0625,0.03125 1.625,0.78125 l 0.60938,-3.312501 h 1.0625 l -1.5625,9.000001 h -0.96875 l 0.10937,-0.6875 q -0.75,0.828125 -1.84375,0.8125 -0.875,-0.03125 -1.40625,-0.6875 -0.51562,-0.671875 -0.5625,-1.796875 -0.0156,-0.34375 0.0312,-0.828125 z m 1.01563,0.609375 q -0.0156,0.84375 0.29687,1.3125 0.32813,0.453125 0.9375,0.46875 0.92188,0.04687 1.625,-0.921875 l 0.51563,-2.9375 q -0.35938,-0.859375 -1.28125,-0.890625 -0.625,-0.01563 -1.10938,0.390625 -0.46875,0.390625 -0.71875,1.109375 -0.25,0.703125 -0.26562,1.46875 z m 6.44922,2.578125 h -1.04688 l 1.09375,-6.34375 h 1.0625 z m 0.875,-8.656251 q 0.26562,-0.01563 0.4375,0.15625 0.1875,0.171875 0.1875,0.453125 0,0.28125 -0.17188,0.46875 -0.17187,0.171875 -0.45312,0.1875 -0.28125,0 -0.45313,-0.171875 -0.17187,-0.171875 -0.17187,-0.4375 0.0156,-0.28125 0.17187,-0.453125 0.17188,-0.1875 0.45313,-0.203125 z m 4.78903,6.937501 q 0.0937,-0.65625 -0.71875,-0.921875 l -1.10938,-0.328125 q -1.39062,-0.484375 -1.34375,-1.59375 0.0469,-0.828125 0.76563,-1.359375 0.71875,-0.546875 1.70312,-0.53125 0.95313,0 1.54688,0.5625 0.60937,0.546875 0.59375,1.421875 l -1.0625,-0.01563 q 0.0156,-0.46875 -0.29688,-0.765625 -0.3125,-0.296875 -0.82812,-0.3125 -0.54688,-0.01563 -0.9375,0.265625 -0.375,0.265625 -0.4375,0.703125 -0.0781,0.546875 0.67187,0.796875 l 0.53125,0.140625 q 1.09375,0.28125 1.54688,0.734375 0.46875,0.453125 0.42187,1.125 -0.0312,0.59375 -0.39062,1.03125 -0.34375,0.4375 -0.9375,0.671875 -0.57813,0.234375 -1.23438,0.21875 -1.01562,-0.01563 -1.67187,-0.59375 -0.65625,-0.578125 -0.625,-1.484375 h 1.0625 q 0,0.53125 0.32812,0.859375 0.34375,0.3125 0.9375,0.3125 0.57813,0.01563 1,-0.234375 0.42188,-0.25 0.48438,-0.703125 z m 4.93359,-6.156251 -0.26562,1.531251 h 1.15625 l -0.15625,0.84375 h -1.14063 l -0.67187,3.9375 q -0.0156,0.15625 0,0.28125 0.0469,0.453125 0.5,0.453125 0.1875,0 0.51562,-0.04687 l -0.0781,0.875 q -0.39063,0.109375 -0.78125,0.109375 -0.625,-0.01563 -0.95313,-0.46875 -0.3125,-0.453125 -0.23437,-1.21875 l 0.64062,-3.921875 h -1.125 l 0.15625,-0.84375 h 1.10938 l 0.26562,-1.531251 z m 5.00196,2.500001 -0.5,-0.04687 q -1.01563,0 -1.60938,0.921875 l -0.76562,4.5 h -1.0625 l 1.09375,-6.34375 h 1.03125 l -0.15625,0.734375 q 0.67187,-0.890625 1.57812,-0.859375 0.20313,0 0.51563,0.09375 z m 1.19922,5.375 h -1.04688 l 1.09375,-6.34375 h 1.0625 z m 0.875,-8.656251 q 0.26562,-0.01563 0.4375,0.15625 0.1875,0.171875 0.1875,0.453125 0,0.28125 -0.17188,0.46875 -0.17187,0.171875 -0.45312,0.1875 -0.28125,0 -0.45313,-0.171875 -0.17187,-0.171875 -0.17187,-0.4375 0.0156,-0.28125 0.17187,-0.453125 0.17188,-0.1875 0.45313,-0.203125 z m 6.61718,5.515626 q -0.125,0.96875 -0.54687,1.75 -0.40625,0.78125 -1,1.15625 -0.59375,0.375 -1.3125,0.359375 -1.14063,-0.03125 -1.70313,-0.859375 l -0.21875,0.734375 h -0.96875 l 1.5625,-9.000001 h 1.0625 l -0.64062,3.359376 q 0.76562,-0.84375 1.84375,-0.8125 0.92187,0.01563 1.42187,0.671875 0.51563,0.640625 0.54688,1.796875 0,0.375 -0.0312,0.71875 z m -1.01562,-0.609375 q 0.0312,-0.859375 -0.28125,-1.3125 -0.29688,-0.453125 -0.9375,-0.484375 -0.98438,-0.01563 -1.67188,1 l -0.46875,2.765625 q 0.34375,0.953125 1.3125,0.984375 0.625,0.01563 1.09375,-0.375 0.46875,-0.390625 0.70313,-1.09375 0.23437,-0.703125 0.25,-1.484375 z m 5.87304,3.125 q -0.73437,0.765625 -1.89062,0.75 -0.90625,-0.03125 -1.34375,-0.65625 -0.4375,-0.625 -0.32813,-1.71875 l 0.6875,-4.09375 h 1.0625 l -0.6875,4.109375 q -0.0312,0.265625 -0.0156,0.515625 0.0312,0.421875 0.25,0.65625 0.21875,0.234375 0.625,0.265625 1.14062,0.03125 1.75,-0.921875 l 0.79687,-4.625 h 1.0625 l -1.09375,6.34375 h -1 z m 5.25586,-7.250001 -0.26562,1.531251 h 1.15625 l -0.15625,0.84375 h -1.14063 l -0.67187,3.9375 q -0.0156,0.15625 0,0.28125 0.0469,0.453125 0.5,0.453125 0.1875,0 0.51562,-0.04687 l -0.0781,0.875 q -0.39063,0.109375 -0.78125,0.109375 -0.625,-0.01563 -0.95313,-0.46875 -0.3125,-0.453125 -0.23437,-1.21875 l 0.64062,-3.921875 h -1.125 l 0.15625,-0.84375 h 1.10938 l 0.26562,-1.531251 z m 3.79883,8.000001 q -1.1875,-0.03125 -1.85937,-0.875 -0.65625,-0.859375 -0.54688,-2.1875 l 0.0312,-0.25 q 0.0937,-0.9375 0.54688,-1.71875 0.46875,-0.78125 1.125,-1.171875 0.67187,-0.390625 1.4375,-0.375 0.96875,0.01563 1.53125,0.640625 0.5625,0.625 0.60937,1.703125 0.0156,0.375 -0.0312,0.765625 l -0.0625,0.484375 h -4.14062 q -0.10938,0.859375 0.28125,1.46875 0.39062,0.59375 1.17187,0.625 0.9375,0.01563 1.73438,-0.84375 l 0.60937,0.515625 q -0.40625,0.578125 -1.03125,0.90625 -0.625,0.3125 -1.40625,0.3125 z m 0.67188,-5.703125 q -0.65625,-0.01563 -1.14063,0.453125 -0.48437,0.453125 -0.78125,1.375 h 3.09375 l 0.0156,-0.09375 q 0.0937,-0.734375 -0.23438,-1.21875 -0.32812,-0.5 -0.95312,-0.515625 z m 6.73828,3.859375 q 0.0937,-0.65625 -0.71875,-0.921875 l -1.10938,-0.328125 q -1.39062,-0.484375 -1.34375,-1.59375 0.0469,-0.828125 0.76563,-1.359375 0.71875,-0.546875 1.70312,-0.53125 0.95313,0 1.54688,0.5625 0.60937,0.546875 0.59375,1.421875 l -1.0625,-0.01563 q 0.0156,-0.46875 -0.29688,-0.765625 -0.3125,-0.296875 -0.82812,-0.3125 -0.54688,-0.01563 -0.9375,0.265625 -0.375,0.265625 -0.4375,0.703125 -0.0781,0.546875 0.67187,0.796875 l 0.53125,0.140625 q 1.09375,0.28125 1.54688,0.734375 0.46875,0.453125 0.42187,1.125 -0.0312,0.59375 -0.39062,1.03125 -0.34375,0.4375 -0.9375,0.671875 -0.57813,0.234375 -1.23438,0.21875 -1.01562,-0.01563 -1.67187,-0.59375 -0.65625,-0.578125 -0.625,-1.484375 h 1.0625 q 0,0.53125 0.32812,0.859375 0.34375,0.3125 0.9375,0.3125 0.57813,0.01563 1,-0.234375 0.42188,-0.25 0.48438,-0.703125 z"
+       fill="#ffffff" />
+    <path
+       id="path115"
+       fill-rule="nonzero"
+       d="m 469.0911,36.847065 q -0.0469,-0.171875 -0.0469,-0.328125 l 0.0156,-0.34375 q -0.85938,0.8125 -1.84375,0.796875 -0.84375,-0.01563 -1.375,-0.53125 -0.51563,-0.53125 -0.46875,-1.3125 0.0625,-1 0.875,-1.5625 0.82812,-0.5625 2.17187,-0.5625 h 1.10938 l 0.0781,-0.5 q 0.0469,-0.578127 -0.23437,-0.906252 -0.26563,-0.328125 -0.82813,-0.34375 -0.54687,-0.01563 -0.96875,0.265625 -0.40625,0.28125 -0.5,0.734377 l -1.07812,0.01563 q 0.0625,-0.562502 0.4375,-0.984377 0.375,-0.4375 0.96875,-0.671875 0.60937,-0.234375 1.25,-0.234375 0.98437,0.03125 1.53125,0.609375 0.5625,0.578125 0.46875,1.531252 l -0.51563,3.140625 -0.0312,0.453125 q -0.0156,0.3125 0.0781,0.640625 l -0.0156,0.09375 z m -1.67188,-0.828125 q 0.51563,0.01563 0.96875,-0.234375 0.45313,-0.25 0.78125,-0.703125 l 0.23438,-1.3125 h -0.8125 q -1,0 -1.53125,0.3125 -0.53125,0.3125 -0.59375,0.890625 -0.0625,0.453125 0.20312,0.75 0.26563,0.296875 0.75,0.296875 z m 6.86133,0.0625 q 0.5625,0.01563 1.01563,-0.328125 0.46875,-0.34375 0.60937,-0.90625 h 1 q -0.0937,0.59375 -0.48437,1.09375 -0.375,0.484375 -0.96875,0.765625 -0.59375,0.28125 -1.21875,0.265625 -0.79688,-0.01563 -1.34375,-0.421875 -0.54688,-0.40625 -0.8125,-1.125 -0.25,-0.71875 -0.15625,-1.609375 l 0.0156,-0.296875 q 0.125,-0.90625 0.54688,-1.625002 0.42187,-0.71875 1.09375,-1.125 0.6875,-0.40625 1.5,-0.375 1,0.01563 1.59375,0.671875 0.59375,0.640625 0.59375,1.625002 h -0.98438 q -0.0156,-0.625 -0.35937,-1.000002 -0.32813,-0.390625 -0.89063,-0.421875 -0.82812,-0.01563 -1.375,0.625 -0.54687,0.625002 -0.6875,1.765627 l -0.0156,0.15625 q -0.0312,0.375 0,0.765625 0.0312,0.6875 0.375,1.09375 0.34375,0.390625 0.95312,0.40625 z m 7.41797,-4.609377 -0.5,-0.04687 q -1.01562,0 -1.60937,0.921877 l -0.76563,4.5 h -1.0625 l 1.09375,-6.343752 h 1.03125 l -0.15625,0.734375 q 0.67188,-0.890625 1.57813,-0.859375 0.20312,0 0.51562,0.09375 z m 3.45703,-1.078125 q 0.79688,0 1.35938,0.421875 0.57812,0.421875 0.84375,1.171875 0.28125,0.734377 0.1875,1.625002 l -0.0156,0.125 q -0.10937,0.921875 -0.54687,1.671875 -0.4375,0.75 -1.14063,1.171875 -0.6875,0.40625 -1.53125,0.390625 -0.78125,-0.01563 -1.35937,-0.4375 -0.5625,-0.421875 -0.82813,-1.140625 -0.26562,-0.734375 -0.1875,-1.609375 0.0781,-0.984375 0.51563,-1.765625 0.45312,-0.796877 1.15625,-1.218752 0.70312,-0.4375 1.54687,-0.40625 z m -2.15625,3.390627 q -0.0469,0.375 0,0.765625 0.0625,0.703125 0.40625,1.109375 0.35938,0.40625 0.95313,0.421875 0.53125,0.01563 0.96875,-0.25 0.45312,-0.28125 0.75,-0.84375 0.3125,-0.5625 0.39062,-1.25 0.0469,-0.546875 0.0312,-0.90625 -0.0781,-0.71875 -0.4375,-1.125002 -0.35937,-0.40625 -0.95312,-0.4375 -0.8125,-0.01563 -1.39063,0.65625 -0.57812,0.656252 -0.70312,1.734377 z m 9.25977,1.34375 q 0.0937,-0.65625 -0.71875,-0.921875 l -1.10938,-0.328125 q -1.39062,-0.484375 -1.34375,-1.59375 0.0469,-0.828127 0.76563,-1.359377 0.71875,-0.546875 1.70312,-0.53125 0.95313,0 1.54688,0.5625 0.60937,0.546875 0.59375,1.421877 l -1.0625,-0.01563 q 0.0156,-0.468752 -0.29688,-0.765627 -0.3125,-0.296875 -0.82812,-0.3125 -0.54688,-0.01563 -0.9375,0.265625 -0.375,0.265625 -0.4375,0.703127 -0.0781,0.546875 0.67187,0.796875 l 0.53125,0.140625 q 1.09375,0.28125 1.54688,0.734375 0.46875,0.453125 0.42187,1.125 -0.0312,0.59375 -0.39062,1.03125 -0.34375,0.4375 -0.9375,0.671875 -0.57813,0.234375 -1.23438,0.21875 -1.01562,-0.01563 -1.67187,-0.59375 -0.65625,-0.578125 -0.625,-1.484375 h 1.0625 q 0,0.53125 0.32812,0.859375 0.34375,0.3125 0.9375,0.3125 0.57813,0.01563 1,-0.234375 0.42188,-0.25 0.48438,-0.703125 z m 6.05859,0 q 0.0937,-0.65625 -0.71875,-0.921875 l -1.10937,-0.328125 q -1.39063,-0.484375 -1.34375,-1.59375 0.0469,-0.828127 0.76562,-1.359377 0.71875,-0.546875 1.70313,-0.53125 0.95312,0 1.54687,0.5625 0.60938,0.546875 0.59375,1.421877 l -1.0625,-0.01563 q 0.0156,-0.468752 -0.29687,-0.765627 -0.3125,-0.296875 -0.82813,-0.3125 -0.54687,-0.01563 -0.9375,0.265625 -0.375,0.265625 -0.4375,0.703127 -0.0781,0.546875 0.67188,0.796875 l 0.53125,0.140625 q 1.09375,0.28125 1.54687,0.734375 0.46875,0.453125 0.42188,1.125 -0.0312,0.59375 -0.39063,1.03125 -0.34375,0.4375 -0.9375,0.671875 -0.57812,0.234375 -1.23437,0.21875 -1.01563,-0.01563 -1.67188,-0.59375 -0.65625,-0.578125 -0.625,-1.484375 h 1.0625 q 0,0.53125 0.32813,0.859375 0.34375,0.3125 0.9375,0.3125 0.57812,0.01563 1,-0.234375 0.42187,-0.25 0.48437,-0.703125 z m 7.17188,-4.625002 -0.14063,0.703125 q 0.79688,-0.84375 1.92188,-0.8125 0.60937,0 1.03125,0.265625 0.42187,0.265625 0.57812,0.75 0.89063,-1.046875 2.09375,-1.015625 0.95313,0.01563 1.40622,0.640625 0.45313,0.609375 0.32813,1.640627 l -0.6875,4.171875 h -1.06247 l 0.6875,-4.171875 q 0.0469,-0.3125 0,-0.5625 -0.10938,-0.796877 -1.03125,-0.812502 -0.57813,-0.01563 -1.0625,0.359375 -0.46875,0.375002 -0.59375,0.968752 l -0.73438,4.21875 h -1.0625 l 0.70313,-4.171875 q 0.0781,-0.65625 -0.20313,-1.000002 -0.26562,-0.359375 -0.84375,-0.359375 -0.98437,-0.03125 -1.5625,0.890627 l -0.8125,4.640625 h -1.0625 l 1.10938,-6.343752 z m 12.18161,5.718752 q -0.73438,0.765625 -1.89063,0.75 -0.90625,-0.03125 -1.34375,-0.65625 -0.4375,-0.625 -0.32812,-1.71875 l 0.6875,-4.093752 h 1.0625 l -0.6875,4.109377 q -0.0312,0.265625 -0.0156,0.515625 0.0312,0.421875 0.25,0.65625 0.21875,0.234375 0.625,0.265625 1.14063,0.03125 1.75,-0.921875 l 0.79688,-4.625002 h 1.0625 l -1.09375,6.343752 h -1 z m 5.25586,-7.250002 -0.26563,1.53125 h 1.15625 l -0.15625,0.84375 h -1.14062 l -0.67188,3.937502 q -0.0156,0.15625 0,0.28125 0.0469,0.453125 0.5,0.453125 0.1875,0 0.51563,-0.04687 l -0.0781,0.875 q -0.39062,0.109375 -0.78125,0.109375 -0.625,-0.01563 -0.95312,-0.46875 -0.3125,-0.453125 -0.23438,-1.21875 l 0.64063,-3.921877 h -1.125 l 0.15625,-0.84375 h 1.10937 l 0.26563,-1.53125 z m 2.20508,7.875002 h -1.04688 l 1.54688,-9.000002 h 1.0625 z m 2.88281,0 h -1.04688 l 1.09375,-6.343752 h 1.0625 z m 0.875,-8.656252 q 0.26562,-0.01563 0.4375,0.15625 0.1875,0.171875 0.1875,0.453125 0,0.28125 -0.17188,0.46875 -0.17187,0.171875 -0.45312,0.1875 -0.28125,0 -0.45313,-0.171875 -0.17187,-0.171875 -0.17187,-0.4375 0.0156,-0.28125 0.17187,-0.453125 0.17188,-0.1875 0.45313,-0.203125 z m 6.60156,5.515627 q -0.10937,0.96875 -0.51562,1.75 -0.40625,0.765625 -1.01563,1.15625 -0.59375,0.375 -1.3125,0.359375 -1.10937,-0.03125 -1.6875,-0.75 l -0.5625,3.0625 h -1.0625 l 1.51563,-8.781252 h 0.96875 l -0.125,0.703125 q 0.78125,-0.84375 1.89062,-0.8125 0.90625,0.01563 1.40625,0.671875 0.51563,0.640625 0.54688,1.796877 0.0156,0.375 -0.0312,0.71875 z m -1.04687,-0.125 0.0312,-0.484375 q 0,-0.859375 -0.32813,-1.312502 -0.3125,-0.453125 -0.9375,-0.484375 -0.92187,-0.01563 -1.57812,0.875002 l -0.53125,3.046875 q 0.34375,0.84375 1.28125,0.859375 0.78125,0.01563 1.32812,-0.609375 0.5625,-0.640625 0.73438,-1.890625 z m 3.04492,3.265625 h -1.04688 l 1.54688,-9.000002 h 1.0625 z m 4.47656,0.125 q -1.1875,-0.03125 -1.85938,-0.875 -0.65625,-0.859375 -0.54687,-2.1875 l 0.0312,-0.25 q 0.0937,-0.9375 0.54687,-1.718752 0.46875,-0.78125 1.125,-1.171875 0.67188,-0.390625 1.4375,-0.375 0.96875,0.01563 1.53125,0.640625 0.5625,0.625 0.60938,1.703127 0.0156,0.375 -0.0312,0.765625 l -0.0625,0.484375 h -4.14063 q -0.10937,0.859375 0.28125,1.46875 0.39063,0.59375 1.17188,0.625 0.9375,0.01563 1.73437,-0.84375 l 0.60938,0.515625 q -0.40625,0.578125 -1.03125,0.90625 -0.625,0.3125 -1.40625,0.3125 z m 0.67187,-5.703127 q -0.65625,-0.01563 -1.14062,0.453125 -0.48438,0.453127 -0.78125,1.375002 h 3.09375 l 0.0156,-0.09375 q 0.0937,-0.734375 -0.23437,-1.218752 -0.32813,-0.5 -0.95313,-0.515625 z"
+       fill="#ffffff" />
+    <path
+       id="path117"
+       fill-rule="nonzero"
+       d="m 490.21317,49.347065 2.01562,-4.84375 h 1.09375 l -2.89062,6.34375 h -0.85938 l -0.65625,-4.796875 -2.28125,4.796875 h -0.85937 l -0.6875,-6.34375 h 1.01562 l 0.39063,4.71875 2.25,-4.71875 h 0.84375 z m 6.63477,-4.953125 q 0.79687,0 1.35937,0.421875 0.57813,0.421875 0.84375,1.171875 0.28125,0.734375 0.1875,1.625 l -0.0156,0.125 q -0.10938,0.921875 -0.54688,1.671875 -0.4375,0.75 -1.14062,1.171875 -0.6875,0.40625 -1.53125,0.390625 -0.78125,-0.01563 -1.35938,-0.4375 -0.5625,-0.421875 -0.82812,-1.140625 -0.26563,-0.734375 -0.1875,-1.609375 0.0781,-0.984375 0.51562,-1.765625 0.45313,-0.796875 1.15625,-1.21875 0.70313,-0.4375 1.54688,-0.40625 z m -2.15625,3.390625 q -0.0469,0.375 0,0.765625 0.0625,0.703125 0.40625,1.109375 0.35937,0.40625 0.95312,0.421875 0.53125,0.01563 0.96875,-0.25 0.45313,-0.28125 0.75,-0.84375 0.3125,-0.5625 0.39063,-1.25 0.0469,-0.546875 0.0312,-0.90625 -0.0781,-0.71875 -0.4375,-1.125 -0.35938,-0.40625 -0.95313,-0.4375 -0.8125,-0.01563 -1.39062,0.65625 -0.57813,0.65625 -0.70313,1.734375 z m 9.27539,-2.3125 -0.5,-0.04687 q -1.01563,0 -1.60938,0.921875 l -0.76562,4.5 h -1.0625 l 1.09375,-6.34375 h 1.03125 l -0.15625,0.734375 q 0.67187,-0.890625 1.57812,-0.859375 0.20313,0 0.51563,0.09375 z m 2.32422,2.453125 -0.82813,0.734375 -0.375,2.1875 h -1.0625 l 1.5625,-9 h 1.0625 l -0.9375,5.375 0.65625,-0.65625 2.25,-2.0625 h 1.375 l -2.92187,2.703125 2.01562,3.640625 h -1.17187 z m 6.31051,3.046875 q -1.18747,-0.03125 -1.85934,-0.875 -0.65625,-0.859375 -0.54688,-2.1875 l 0.0312,-0.25 q 0.0937,-0.9375 0.54688,-1.71875 0.46875,-0.78125 1.125,-1.171875 0.67184,-0.390625 1.43747,-0.375 0.96875,0.01563 1.53125,0.640625 0.5625,0.625 0.60937,1.703125 0.0156,0.375 -0.0312,0.765625 l -0.0625,0.484375 h -4.14059 q -0.10938,0.859375 0.28125,1.46875 0.39062,0.59375 1.17184,0.625 0.9375,0.01563 1.73438,-0.84375 l 0.60937,0.515625 q -0.40625,0.578125 -1.03125,0.90625 -0.625,0.3125 -1.40625,0.3125 z m 0.67188,-5.703125 q -0.65625,-0.01563 -1.14063,0.453125 -0.48434,0.453125 -0.78122,1.375 h 3.09372 l 0.0156,-0.09375 q 0.0937,-0.734375 -0.23438,-1.21875 -0.32812,-0.5 -0.95312,-0.515625 z m 6.7539,0.203125 -0.5,-0.04687 q -1.01562,0 -1.60937,0.921875 l -0.76563,4.5 h -1.0625 l 1.09375,-6.34375 h 1.03125 l -0.15625,0.734375 q 0.67188,-0.890625 1.57813,-0.859375 0.20312,0 0.51562,0.09375 z m 3.98047,3.65625 q 0.0937,-0.65625 -0.71875,-0.921875 l -1.10937,-0.328125 q -1.39063,-0.484375 -1.34375,-1.59375 0.0469,-0.828125 0.76562,-1.359375 0.71875,-0.546875 1.70313,-0.53125 0.95312,0 1.54687,0.5625 0.60938,0.546875 0.59375,1.421875 l -1.0625,-0.01563 q 0.0156,-0.46875 -0.29687,-0.765625 -0.3125,-0.296875 -0.82813,-0.3125 -0.54687,-0.01563 -0.9375,0.265625 -0.375,0.265625 -0.4375,0.703125 -0.0781,0.546875 0.67188,0.796875 l 0.53125,0.140625 q 1.09375,0.28125 1.54687,0.734375 0.46875,0.453125 0.42188,1.125 -0.0312,0.59375 -0.39063,1.03125 -0.34375,0.4375 -0.9375,0.671875 -0.57812,0.234375 -1.23437,0.21875 -1.01563,-0.01563 -1.67188,-0.59375 -0.65625,-0.578125 -0.625,-1.484375 h 1.0625 q 0,0.53125 0.32813,0.859375 0.34375,0.3125 0.9375,0.3125 0.57812,0.01563 1,-0.234375 0.42187,-0.25 0.48437,-0.703125 z"
+       fill="#ffffff" />
+  </g>
+</svg>