Merge pull request #10590: [BEAM-7746] Fix a typing issue where SourceBase was assumed to have a coder attribute

diff --git a/.asf.yaml b/.asf.yaml
new file mode 100644
index 0000000..3da2ec1
--- /dev/null
+++ b/.asf.yaml
@@ -0,0 +1,177 @@
+#
+# 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.
+#
+
+github:
+  description: "Apache Beam is a unified programming model for Batch and Streaming"
+  homepage: https://beam.apache.org/
+
+jenkins:
+  github_whitelist:
+    - 11moon11
+    - aaltay
+    - acrites
+    - ageron
+    - aijamalnk
+    - akedin
+    - alanmyrvold
+    - alexvanboxel
+    - amaliujia
+    - andrefaraujo
+    - angoenka
+    - anguillanneuf
+    - angulartist
+    - apilloud
+    - appaquet
+    - Ardagan
+    - aromanenko-dev
+    - aryann
+    - blackml
+    - bmv126
+    - boyuanzz
+    - bradgwest
+    - brunocvcunha
+    - bumblebee-coming
+    - chadrik
+    - chamikaramj
+    - charithe
+    - ChethanUK
+    - chunyang
+    - ClaireMcGinty
+    - cmachgodaddy
+    - cmm08
+    - coheigea
+    - CraigChambersG
+    - DAAC
+    - davidcavazos
+    - davidyan74
+    - dependabot[bot]
+    - derekunimarket
+    - dmvk
+    - dpcollins-google
+    - drobert
+    - dsdinter
+    - dsnet
+    - ecanzonieri
+    - echauchot
+    - elharo
+    - gxercavins
+    - Hannah-Jiang
+    - happygiraffe
+    - henryken
+    - htyleo
+    - HuangLED
+    - ianlancetaylor
+    - ibzib
+    - iemejia
+    - ihji
+    - iht
+    - jackwhelpton
+    - jagthebeetle
+    - jbartok
+    - je-ik
+    - jesusrv1103
+    - jhalaria
+    - jklukas
+    - Jofre
+    - kamilwu
+    - KangZhiDong
+    - kanterov
+    - kennknowles
+    - KevinGG
+    - kkucharc
+    - kmjung
+    - kyle-winkelman
+    - lazylynx
+    - leonardoam
+    - lgajowy
+    - lhaiesp
+    - liumomo315
+    - lloigor
+    - lostluck
+    - Luis-MX
+    - lukecwik
+    - markflyhigh
+    - MattMorgis
+    - milantracy
+    - mrociorg
+    - mwalenia
+    - mxm
+    - nahuellofeudo
+    - nielm
+    - olegbonar
+    - ostrokach
+    - ozturkberkay
+    - pabloem
+    - pawelpasterz
+    - Pehat
+    - pgudlani
+    - piter75
+    - Primevenn
+    - ra1861
+    - rahul8383
+    - rainwoodman
+    - rakeshcusat
+    - rehmanmuradali
+    - reuvenlax
+    - riazela
+    - robertwb
+    - robinyqiu
+    - RochesterinNYC
+    - rohdesamuel
+    - rosetn
+    - RusOr10n
+    - RyanBerti
+    - RyanSkraba
+    - sadovnychyi
+    - salmanVD
+    - samaitra
+    - sambvfx
+    - saulchavez93
+    - scwhittle
+    - sgrj
+    - shusso
+    - snallapa
+    - sorensenjs
+    - soyrice
+    - stefanondisponibile
+    - stephydx
+    - stevekoonce
+    - steveniemitz
+    - sunjincheng121
+    - suztomo
+    - tamanobi
+    - tchiarato
+    - the1plummie
+    - TheNeuralBit
+    - TimvdLippe
+    - ttanay
+    - tudorm
+    - tvalentyn
+    - tweise
+    - tysonjh
+    - udim
+    - vectorijk
+    - violalyu
+    - wcn3
+    - wintermelons
+    - xubii
+    - y1chi
+    - yifanzou
+    - yirutang
+    - youngoli
+    - ziel
+
diff --git a/.test-infra/jenkins/job_CancelStaleDataflowJobs.groovy b/.test-infra/jenkins/job_CancelStaleDataflowJobs.groovy
index a03a1d0..e32d14a 100644
--- a/.test-infra/jenkins/job_CancelStaleDataflowJobs.groovy
+++ b/.test-infra/jenkins/job_CancelStaleDataflowJobs.groovy
@@ -37,7 +37,6 @@
   steps {
     gradle {
       rootBuildScriptDir(commonJobProperties.checkoutDir)
-      tasks(':beam-test-tools:check')
       tasks(':beam-test-tools:cancelStaleDataflowJobs')
       commonJobProperties.setGradleSwitches(delegate)
     }
diff --git a/.test-infra/tools/build.gradle b/.test-infra/tools/build.gradle
index aabeca0..53445b6 100644
--- a/.test-infra/tools/build.gradle
+++ b/.test-infra/tools/build.gradle
@@ -16,27 +16,6 @@
  * limitations under the License.
  */
 
-plugins {
-  id 'org.apache.beam.module'
-}
-
-applyGoNature()
-
-repositories { mavenCentral() }
-
-clean {
-  delete '.gogradle'
-}
-
-golang {
-  packagePath = 'github.com/apache/beam/.test-infra/tools'
-}
-
-check.dependsOn goTest
-
-task cancelStaleDataflowJobs(type: com.github.blindpirate.gogradle.Go) {
-  dependsOn goVendor
-  go('get golang.org/x/oauth2/google')
-  go('get google.golang.org/api/dataflow/v1b3')
-  go('run stale_dataflow_jobs_cleaner.go')
+task cancelStaleDataflowJobs(type: Exec) {
+  commandLine './stale_dataflow_jobs_cleaner.sh'
 }
diff --git a/.test-infra/tools/stale_dataflow_jobs_cleaner.go b/.test-infra/tools/stale_dataflow_jobs_cleaner.go
deleted file mode 100644
index 6361e27..0000000
--- a/.test-infra/tools/stale_dataflow_jobs_cleaner.go
+++ /dev/null
@@ -1,117 +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 main
-
-import (
-	"context"
-	"log"
-	"strings"
-	"time"
-
-	"golang.org/x/oauth2/google"
-	df "google.golang.org/api/dataflow/v1b3"
-)
-
-const (
-	longRunningPrefix = "long-running-"
-)
-
-// client contains methods for listing and cancelling jobs, extracted to allow easier testing.
-type client interface {
-	CurrentTime() time.Time
-	ListJobs(projectId string) ([]*df.Job, error)
-	CancelJob(job *df.Job) error
-}
-
-// dataflowClient implements the client interface for Google Cloud Dataflow.
-type dataflowClient struct {
-	s *df.ProjectsJobsService
-}
-
-// newDataflowClient creates a new Dataflow ProjectsJobsService.
-func newDataflowClient() (*dataflowClient, error) {
-	ctx := context.Background()
-	cl, err := google.DefaultClient(ctx, df.CloudPlatformScope)
-	if err != nil {
-		return nil, err
-	}
-	service, err := df.New(cl)
-	if err != nil {
-		return nil, err
-	}
-	return &dataflowClient{s: df.NewProjectsJobsService(service)}, nil
-}
-
-// CurrentTime gets the time Now.
-func (c dataflowClient) CurrentTime() time.Time {
-	return time.Now()
-}
-
-// ListJobs lists the active Dataflow jobs for a project.
-func (c dataflowClient) ListJobs(projectId string) ([]*df.Job, error) {
-	resp, err := c.s.Aggregated(projectId).Filter("ACTIVE").Fields("jobs(id,name,projectId,createTime)").Do()
-	if err != nil {
-		return nil, err
-	}
-	return resp.Jobs, nil
-}
-
-// CancelJob requests the cancellation od a Dataflow job.
-func (c dataflowClient) CancelJob(job *df.Job) error {
-	jobDone := df.Job{
-		RequestedState: "JOB_STATE_DONE",
-	}
-	_, err := c.s.Update(job.ProjectId, job.Id, &jobDone).Do()
-	return err
-}
-
-// cleanDataflowJobs cancels stale Dataflow jobs, excluding the longRunningPrefix prefixed jobs.
-func cleanDataflowJobs(c client, projectId string, hoursStale float64) error {
-	now := c.CurrentTime()
-	jobs, err := c.ListJobs(projectId)
-	if err != nil {
-		return err
-	}
-	for _, j := range jobs {
-		t, err := time.Parse(time.RFC3339, j.CreateTime)
-		if err != nil {
-			return err
-		}
-		hoursSinceCreate := now.Sub(t).Hours()
-		log.Printf("Job %v %v %v %v %.2f\n", j.ProjectId, j.Id, j.Name, j.CreateTime, hoursSinceCreate)
-		if hoursSinceCreate > hoursStale && !strings.HasPrefix(j.Name, longRunningPrefix) {
-			log.Printf("Attempting to cancel %v\n", j.Id)
-			c.CancelJob(j)
-		}
-	}
-	return nil
-}
-
-func main() {
-	client, err := newDataflowClient()
-	if err != nil {
-		log.Fatalf("Error creating dataflow client, %v", err)
-	}
-	// Cancel any jobs older than 3 hours.
-	err = cleanDataflowJobs(client, "apache-beam-testing", 3.0)
-	if err != nil {
-		log.Fatalf("Error cleaning dataflow jobs, %v", err)
-	}
-	log.Printf("Done")
-}
diff --git a/.test-infra/tools/stale_dataflow_jobs_cleaner.sh b/.test-infra/tools/stale_dataflow_jobs_cleaner.sh
new file mode 100755
index 0000000..66bf880
--- /dev/null
+++ b/.test-infra/tools/stale_dataflow_jobs_cleaner.sh
@@ -0,0 +1,23 @@
+#!/usr/bin/env bash
+#
+#    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.
+#
+#    Cancels active Dataflow jobs older than 3 hours.
+#
+set -euo pipefail
+
+gcloud dataflow jobs list --created-before=-P3H --format='value(JOB_ID)' \
+--status=active --region=us-central1 | xargs gcloud dataflow jobs cancel
diff --git a/.test-infra/tools/stale_dataflow_jobs_cleaner_test.go b/.test-infra/tools/stale_dataflow_jobs_cleaner_test.go
deleted file mode 100644
index 342052a..0000000
--- a/.test-infra/tools/stale_dataflow_jobs_cleaner_test.go
+++ /dev/null
@@ -1,74 +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 main
-
-import (
-	df "google.golang.org/api/dataflow/v1b3"
-	"reflect"
-	"testing"
-	"time"
-)
-
-var (
-	currentTime   time.Time = time.Now()
-	jobsReturned            = []*df.Job{}
-	cancelledJobs           = []*df.Job{}
-)
-
-type fakeClient struct{}
-
-func (c fakeClient) ListJobs(projectId string) ([]*df.Job, error) {
-	return jobsReturned, nil
-}
-
-func (c fakeClient) CancelJob(job *df.Job) error {
-	cancelledJobs = append(cancelledJobs, job)
-	return nil
-}
-
-func (c fakeClient) CurrentTime() time.Time {
-	return currentTime
-}
-
-func helperForJobCancel(t *testing.T, hoursStale float64, jobList []*df.Job, expectedCancelled []*df.Job) {
-	var c fakeClient
-	jobsReturned = jobList
-	cancelledJobs = []*df.Job{}
-	cleanDataflowJobs(c, "some-project-id", 2.0)
-	if !reflect.DeepEqual(cancelledJobs, expectedCancelled) {
-		t.Errorf("Cancelled arrays not as expected actual=%v, expected=%v", cancelledJobs, expectedCancelled)
-	}
-}
-
-func TestEmptyJobList(t *testing.T) {
-	helperForJobCancel(t, 2.0, []*df.Job{}, []*df.Job{})
-}
-
-func TestNotExpiredJob(t *testing.T) {
-	// Just under 2 hours.
-	createTime := currentTime.Add(-(2*time.Hour - time.Second))
-	helperForJobCancel(t, 2.0, []*df.Job{&df.Job{CreateTime: createTime.Format(time.RFC3339)}}, []*df.Job{})
-}
-
-func TestExpiredJob(t *testing.T) {
-	// Just over 2 hours.
-	createTime := currentTime.Add(-(2*time.Hour + time.Second))
-	job := &df.Job{CreateTime: createTime.Format(time.RFC3339)}
-	helperForJobCancel(t, 2.0, []*df.Job{job}, []*df.Job{job})
-}
diff --git a/build.gradle b/build.gradle
index b1702fa..b63b08e 100644
--- a/build.gradle
+++ b/build.gradle
@@ -303,7 +303,7 @@
 
   configurations { linkageCheckerJava }
   dependencies {
-    linkageCheckerJava "com.google.cloud.tools:dependencies:1.0.1"
+    linkageCheckerJava "com.google.cloud.tools:dependencies:1.1.2"
   }
 
   // We need to evaluate all the projects first so that we can find depend on all the
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 d6cf5e1..8939bf2 100644
--- a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy
+++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy
@@ -297,7 +297,7 @@
 
     // Automatically use the official release version if we are performing a release
     // otherwise append '-SNAPSHOT'
-    project.version = '2.19.0'
+    project.version = '2.20.0'
     if (!isRelease(project)) {
       project.version += '-SNAPSHOT'
     }
@@ -364,6 +364,7 @@
     def aws_java_sdk2_version = "2.5.71"
     def cassandra_driver_version = "3.8.0"
     def classgraph_version = "4.8.56"
+    def gax_version = "1.52.0"
     def generated_grpc_beta_version = "0.44.0"
     def generated_grpc_ga_version = "1.83.0"
     def generated_grpc_dc_beta_version = "0.27.0-alpha"
@@ -372,7 +373,8 @@
     def google_cloud_bigdataoss_version = "1.9.16"
     def google_cloud_core_version = "1.61.0"
     def google_cloud_spanner_version = "1.6.0"
-    def grpc_version = "1.17.1"
+    def google_http_clients_version = "1.34.0"
+    def grpc_version = "1.25.0"
     def guava_version = "25.1-jre"
     def hadoop_version = "2.8.5"
     def hamcrest_version = "2.1"
@@ -434,7 +436,8 @@
         commons_lang3                               : "org.apache.commons:commons-lang3:3.9",
         commons_math3                               : "org.apache.commons:commons-math3:3.6.1",
         error_prone_annotations                     : "com.google.errorprone:error_prone_annotations:2.0.15",
-        gax_grpc                                    : "com.google.api:gax-grpc:1.38.0",
+        gax                                         : "com.google.api:gax:$gax_version",
+        gax_grpc                                    : "com.google.api:gax-grpc:$gax_version",
         google_api_client                           : "com.google.api-client:google-api-client:$google_clients_version",
         google_api_client_jackson2                  : "com.google.api-client:google-api-client-jackson2:$google_clients_version",
         google_api_client_java6                     : "com.google.api-client:google-api-client-java6:$google_clients_version",
@@ -455,15 +458,16 @@
         google_cloud_dataflow_java_proto_library_all: "com.google.cloud.dataflow:google-cloud-dataflow-java-proto-library-all:0.5.160304",
         google_cloud_datastore_v1_proto_client      : "com.google.cloud.datastore:datastore-v1-proto-client:1.6.3",
         google_cloud_spanner                        : "com.google.cloud:google-cloud-spanner:$google_cloud_spanner_version",
-        google_http_client                          : "com.google.http-client:google-http-client:$google_clients_version",
-        google_http_client_jackson                  : "com.google.http-client:google-http-client-jackson:$google_clients_version",
-        google_http_client_jackson2                 : "com.google.http-client:google-http-client-jackson2:$google_clients_version",
-        google_http_client_protobuf                 : "com.google.http-client:google-http-client-protobuf:$google_clients_version",
+        google_http_client                          : "com.google.http-client:google-http-client:$google_http_clients_version",
+        google_http_client_jackson                  : "com.google.http-client:google-http-client-jackson:1.29.2",
+        google_http_client_jackson2                 : "com.google.http-client:google-http-client-jackson2:$google_http_clients_version",
+        google_http_client_protobuf                 : "com.google.http-client:google-http-client-protobuf:$google_http_clients_version",
         google_oauth_client                         : "com.google.oauth-client:google-oauth-client:$google_clients_version",
         google_oauth_client_java6                   : "com.google.oauth-client:google-oauth-client-java6:$google_clients_version",
         grpc_all                                    : "io.grpc:grpc-all:$grpc_version",
         grpc_auth                                   : "io.grpc:grpc-auth:$grpc_version",
         grpc_core                                   : "io.grpc:grpc-core:$grpc_version",
+        grpc_context                                : "io.grpc:grpc-context:$grpc_version",
         grpc_google_cloud_datacatalog_v1beta1       : "com.google.api.grpc:grpc-google-cloud-datacatalog-v1beta1:$generated_grpc_dc_beta_version",
         grpc_google_cloud_pubsub_v1                 : "com.google.api.grpc:grpc-google-cloud-pubsub-v1:$generated_grpc_ga_version",
         grpc_protobuf                               : "io.grpc:grpc-protobuf:$grpc_version",
@@ -524,7 +528,7 @@
         spark_streaming                             : "org.apache.spark:spark-streaming_2.11:$spark_version",
         stax2_api                                   : "org.codehaus.woodstox:stax2-api:3.1.4",
         vendored_bytebuddy_1_9_3                    : "org.apache.beam:beam-vendor-bytebuddy-1_9_3:0.1",
-        vendored_grpc_1_21_0                        : "org.apache.beam:beam-vendor-grpc-1_21_0:0.1",
+        vendored_grpc_1_26_0                        : "org.apache.beam:beam-vendor-grpc-1_26_0:0.1",
         vendored_guava_26_0_jre                     : "org.apache.beam:beam-vendor-guava-26_0-jre:0.1",
         vendored_calcite_1_20_0                     : "org.apache.beam:beam-vendor-calcite-1_20_0:0.1",
         woodstox_core_asl                           : "org.codehaus.woodstox:woodstox-core-asl:4.4.1",
@@ -1432,19 +1436,21 @@
 
     /** ***********************************************************************************************/
 
+    // applyGrpcNature should only be applied to projects who wish to use
+    // unvendored gRPC / protobuf dependencies.
     project.ext.applyGrpcNature = {
       project.apply plugin: "com.google.protobuf"
       project.protobuf {
         protoc {
           // The artifact spec for the Protobuf Compiler
-          artifact = "com.google.protobuf:protoc:3.6.0" }
+          artifact = "com.google.protobuf:protoc:$protobuf_version" }
 
         // Configure the codegen plugins
         plugins {
           // An artifact spec for a protoc plugin, with "grpc" as
           // the identifier, which can be referred to in the "plugins"
           // container of the "generateProtoTasks" closure.
-          grpc { artifact = "io.grpc:protoc-gen-grpc-java:1.13.1" }
+          grpc { artifact = "io.grpc:protoc-gen-grpc-java:$grpc_version" }
         }
 
         generateProtoTasks {
@@ -1481,6 +1487,8 @@
 
     /** ***********************************************************************************************/
 
+    // applyPortabilityNature should only be applied to projects that want to use
+    // vendored gRPC / protobuf dependencies.
     project.ext.applyPortabilityNature = {
       PortabilityNatureConfiguration configuration = it ? it as PortabilityNatureConfiguration : new PortabilityNatureConfiguration()
 
@@ -1495,10 +1503,10 @@
               archivesBaseName: configuration.archivesBaseName,
               automaticModuleName: configuration.automaticModuleName,
               shadowJarValidationExcludes: it.shadowJarValidationExcludes,
-              shadowClosure: GrpcVendoring_1_21_0.shadowClosure() << {
+              shadowClosure: GrpcVendoring_1_26_0.shadowClosure() << {
                 // We perform all the code relocations but don't include
                 // any of the actual dependencies since they will be supplied
-                // by org.apache.beam:beam-vendor-grpc-v1p21p0:0.1
+                // by org.apache.beam:beam-vendor-grpc-v1p26p0:0.1
                 dependencies {
                   include(dependency { return false })
                 }
@@ -1515,14 +1523,14 @@
       project.protobuf {
         protoc {
           // The artifact spec for the Protobuf Compiler
-          artifact = "com.google.protobuf:protoc:3.7.1" }
+          artifact = "com.google.protobuf:protoc:${GrpcVendoring_1_26_0.protobuf_version}" }
 
         // Configure the codegen plugins
         plugins {
           // An artifact spec for a protoc plugin, with "grpc" as
           // the identifier, which can be referred to in the "plugins"
           // container of the "generateProtoTasks" closure.
-          grpc { artifact = "io.grpc:protoc-gen-grpc-java:1.21.0" }
+          grpc { artifact = "io.grpc:protoc-gen-grpc-java:${GrpcVendoring_1_26_0.grpc_version}" }
         }
 
         generateProtoTasks {
@@ -1536,7 +1544,7 @@
         }
       }
 
-      project.dependencies GrpcVendoring_1_21_0.dependenciesClosure() << { shadow project.ext.library.java.vendored_grpc_1_21_0 }
+      project.dependencies GrpcVendoring_1_26_0.dependenciesClosure() << { shadow project.ext.library.java.vendored_grpc_1_26_0 }
     }
 
     /** ***********************************************************************************************/
diff --git a/buildSrc/src/main/groovy/org/apache/beam/gradle/GrpcVendoring_1_21_0.groovy b/buildSrc/src/main/groovy/org/apache/beam/gradle/GrpcVendoring_1_21_0.groovy
deleted file mode 100644
index 3c34a6d..0000000
--- a/buildSrc/src/main/groovy/org/apache/beam/gradle/GrpcVendoring_1_21_0.groovy
+++ /dev/null
@@ -1,150 +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.gradle
-
-import org.gradle.api.Project
-
-/**
- * Utilities for working with our vendored version of gRPC.
- */
-class GrpcVendoring_1_21_0 {
-  /** Returns the list of compile time dependencies. */
-  static List<String> dependencies() {
-    return [
-      'com.google.guava:guava:26.0-jre',
-      'com.google.protobuf:protobuf-java:3.7.1',
-      'com.google.protobuf:protobuf-java-util:3.7.1',
-      'com.google.code.gson:gson:2.7',
-      'io.grpc:grpc-auth:1.21.0',
-      'io.grpc:grpc-core:1.21.0',
-      'io.grpc:grpc-context:1.21.0',
-      'io.grpc:grpc-netty:1.21.0',
-      'io.grpc:grpc-protobuf:1.21.0',
-      'io.grpc:grpc-stub:1.21.0',
-      'io.netty:netty-transport-native-epoll:4.1.34.Final',
-      // tcnative version from https://github.com/grpc/grpc-java/blob/master/SECURITY.md#netty
-      'io.netty:netty-tcnative-boringssl-static:2.0.22.Final',
-      'com.google.auth:google-auth-library-credentials:0.13.0',
-      'io.grpc:grpc-testing:1.21.0',
-      'com.google.api.grpc:proto-google-common-protos:1.12.0',
-      'io.opencensus:opencensus-api:0.21.0',
-      'io.opencensus:opencensus-contrib-grpc-metrics:0.21.0',
-    ]
-  }
-
-  /**
-   * Returns the list of runtime time dependencies that should be exported as runtime
-   * dependencies within the vendored jar.
-   */
-  static List<String> runtimeDependencies() {
-    return [
-      'com.google.errorprone:error_prone_annotations:2.3.2',
-    ]
-  }
-
-  static Map<String, String> relocations() {
-    // The relocation paths below specifically use gRPC and the full version string as
-    // the code relocation prefix. See https://lists.apache.org/thread.html/4c12db35b40a6d56e170cd6fc8bb0ac4c43a99aa3cb7dbae54176815@%3Cdev.beam.apache.org%3E
-    // for further details.
-
-    // To produce the list of necessary relocations, one needs to start with a set of target
-    // packages that one wants to vendor, find all necessary transitive dependencies of that
-    // set and provide relocations for each such that all necessary packages and their
-    // dependencies are relocated. Any optional dependency that doesn't need relocation
-    // must be excluded via an 'exclude' rule. There is additional complexity of libraries that use
-    // JNI or reflection and have to be handled on case by case basis by learning whether
-    // they support relocation and how would one go about doing it by reading any documentation
-    // those libraries may provide. The 'validateShadedJarDoesntLeakNonOrgApacheBeamClasses'
-    // ensures that there are no classes outside of the 'org.apache.beam' namespace.
-
-    String version = "v1p21p0";
-    String prefix = "org.apache.beam.vendor.grpc.${version}";
-    List<String> packagesToRelocate = [
-      // guava uses the com.google.common and com.google.thirdparty package namespaces
-      "com.google.common",
-      "com.google.thirdparty",
-      "com.google.protobuf",
-      "com.google.gson",
-      "io.grpc",
-      "com.google.auth",
-      "com.google.api",
-      "com.google.cloud",
-      "com.google.logging",
-      "com.google.longrunning",
-      "com.google.rpc",
-      "com.google.type",
-      "io.opencensus",
-      "io.netty"
-    ]
-
-    return packagesToRelocate.collectEntries {
-      [ (it): "${prefix}.${it}" ]
-    } + [
-      // Adapted from https://github.com/grpc/grpc-java/blob/e283f70ad91f99c7fee8b31b605ef12a4f9b1690/netty/shaded/build.gradle#L41
-      // We       "io.netty": "${prefix}.io.netty",have to be careful with these replacements as they must not match any
-      // string in NativeLibraryLoader, else they cause corruption. Note that
-      // this includes concatenation of string literals and constants.
-      'META-INF/native/libnetty': "META-INF/native/liborg_apache_beam_vendor_grpc_${version}_netty",
-      'META-INF/native/netty': "META-INF/native/org_apache_beam_vendor_grpc_${version}_netty",
-    ]
-  }
-
-  /** Returns the list of shading exclusions. */
-  static List<String> exclusions() {
-    return [
-      // Don't include android annotations, errorprone, checkerframework, JDK8 annotations, objenesis, junit, and mockito in the vendored jar
-      "android/annotation/**/",
-      "com/google/errorprone/**",
-      "com/google/instrumentation/**",
-      "com/google/j2objc/annotations/**",
-      "javax/annotation/**",
-      "junit/**",
-      "org/checkerframework/**",
-      "org/codehaus/mojo/animal_sniffer/**",
-      "org/hamcrest/**",
-      "org/junit/**",
-      "org/mockito/**",
-      "org/objenesis/**",
-    ]
-  }
-
-  /**
-   * Returns a closure contaning the dependencies map used for shading gRPC within the main
-   * Apache Beam project.
-   */
-  static Object dependenciesClosure() {
-    return {
-      dependencies().each { compile it }
-      runtimeDependencies().each { shadow it }
-    }
-  }
-
-  /**
-   * Returns a closure with the code relocation configuration for shading gRPC within the main
-   * Apache Beam project.
-   */
-  static Object shadowClosure() {
-    return {
-      relocations().each { srcNamespace, destNamespace ->
-        relocate srcNamespace, destNamespace
-      }
-      exclusions().each { exclude it }
-    }
-  }
-}
diff --git a/buildSrc/src/main/groovy/org/apache/beam/gradle/GrpcVendoring_1_26_0.groovy b/buildSrc/src/main/groovy/org/apache/beam/gradle/GrpcVendoring_1_26_0.groovy
index 8c70aa2..de87bdf 100644
--- a/buildSrc/src/main/groovy/org/apache/beam/gradle/GrpcVendoring_1_26_0.groovy
+++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/GrpcVendoring_1_26_0.groovy
@@ -24,40 +24,62 @@
  * Utilities for working with our vendored version of gRPC.
  */
 class GrpcVendoring_1_26_0 {
+
+  static def guava_version = "26.0-jre"
+  static def protobuf_version = "3.11.0"
+  static def grpc_version = "1.26.0"
+  static def gson_version = "2.8.6"
+  static def netty_version = "4.1.42.Final"
+  static def google_auth_version = "0.18.0"
+  static def proto_google_common_protos_version = "1.12.0"
+  static def opencensus_version = "0.24.0"
+  static def perfmark_version = "0.19.0"
+  static def lzma_java_version = "1.3"
+  static def protobuf_javanano_version = "3.0.0-alpha-5"
+  static def jzlib_version = "1.1.3"
+  static def compress_lzf_version = "1.0.3"
+  static def lz4_version = "1.3.0"
+  static def bouncycastle_version = "1.54"
+  static def conscrypt_version = "1.3.0"
+  static def alpn_api_version = "1.1.2.v20150522"
+  static def npn_api_version = "1.1.1.v20141010"
+  static def jboss_marshalling_version = "1.4.11.Final"
+  static def jboss_modules_version = "1.1.0.Beta1"
+
   /** Returns the list of compile time dependencies. */
   static List<String> dependencies() {
     return [
-      'com.google.guava:guava:26.0-jre',
-      'com.google.protobuf:protobuf-java:3.11.0',
-      'com.google.protobuf:protobuf-java-util:3.11.0',
-      'com.google.code.gson:gson:2.8.6',
-      'io.grpc:grpc-auth:1.26.0',
-      'io.grpc:grpc-core:1.26.0',
-      'io.grpc:grpc-context:1.26.0',
-      'io.grpc:grpc-netty:1.26.0',
-      'io.grpc:grpc-protobuf:1.26.0',
-      'io.grpc:grpc-stub:1.26.0',
-      'io.netty:netty-transport-native-epoll:4.1.42.Final',
+      "com.google.guava:guava:$guava_version",
+      "com.google.protobuf:protobuf-java:$protobuf_version",
+      "com.google.protobuf:protobuf-java-util:$protobuf_version",
+      "com.google.code.gson:gson:$gson_version",
+      "io.grpc:grpc-auth:$grpc_version",
+      "io.grpc:grpc-core:$grpc_version",
+      "io.grpc:grpc-context:$grpc_version",
+      "io.grpc:grpc-netty:$grpc_version",
+      "io.grpc:grpc-protobuf:$grpc_version",
+      "io.grpc:grpc-stub:$grpc_version",
+      "io.netty:netty-transport-native-epoll:$netty_version",
       // tcnative version from https://github.com/grpc/grpc-java/blob/master/SECURITY.md#netty
-      'io.netty:netty-tcnative-boringssl-static:2.0.26.Final',
-      'com.google.auth:google-auth-library-credentials:0.18.0',
-      'io.grpc:grpc-testing:1.26.0',
-      'com.google.api.grpc:proto-google-common-protos:1.12.0',
-      'io.opencensus:opencensus-api:0.24.0',
-      'io.opencensus:opencensus-contrib-grpc-metrics:0.24.0',
-      'io.perfmark:perfmark-api:0.19.0',
-      'com.github.jponge:lzma-java:1.3',
-      'com.google.protobuf.nano:protobuf-javanano:3.0.0-alpha-5',
-      'com.jcraft:jzlib:1.1.3',
-      'com.ning:compress-lzf:1.0.3',
-      'net.jpountz.lz4:lz4:1.3.0',
-      'org.bouncycastle:bcpkix-jdk15on:1.54',
-      'org.bouncycastle:bcprov-jdk15on:1.54',
-      'org.conscrypt:conscrypt-openjdk-uber:1.3.0',
-      'org.eclipse.jetty.alpn:alpn-api:1.1.2.v20150522',
-      'org.eclipse.jetty.npn:npn-api:1.1.1.v20141010',
-      'org.jboss.marshalling:jboss-marshalling:1.4.11.Final',
-      'org.jboss.modules:jboss-modules:1.1.0.Beta1'
+      "io.netty:netty-tcnative-boringssl-static:2.0.26.Final",
+      "com.google.auth:google-auth-library-credentials:$google_auth_version",
+      "io.grpc:grpc-testing:$grpc_version",
+      "com.google.api.grpc:proto-google-common-protos:$proto_google_common_protos_version",
+      "io.opencensus:opencensus-api:$opencensus_version",
+      "io.opencensus:opencensus-contrib-grpc-metrics:$opencensus_version",
+      "io.perfmark:perfmark-api:$perfmark_version",
+      "com.github.jponge:lzma-java:$lzma_java_version",
+      "com.google.protobuf.nano:protobuf-javanano:$protobuf_javanano_version",
+      "com.jcraft:jzlib:$jzlib_version",
+      "com.ning:compress-lzf:$compress_lzf_version",
+      "net.jpountz.lz4:lz4:$lz4_version",
+      "org.bouncycastle:bcpkix-jdk15on:$bouncycastle_version",
+      "org.bouncycastle:bcprov-jdk15on:$bouncycastle_version",
+      "org.conscrypt:conscrypt-openjdk-uber:$conscrypt_version",
+      "org.eclipse.jetty.alpn:alpn-api:$alpn_api_version",
+      "org.eclipse.jetty.npn:npn-api:$npn_api_version",
+      "org.jboss.marshalling:jboss-marshalling:$jboss_marshalling_version",
+      "org.jboss.modules:jboss-modules:$jboss_modules_version"
     ]
   }
 
diff --git a/examples/notebooks/get-started/try-apache-beam-java.ipynb b/examples/notebooks/get-started/try-apache-beam-java.ipynb
index 40d648a..101df82 100644
--- a/examples/notebooks/get-started/try-apache-beam-java.ipynb
+++ b/examples/notebooks/get-started/try-apache-beam-java.ipynb
@@ -593,8 +593,8 @@
             "\n", 
             "> Task :runShadow\n", 
             "WARNING: An illegal reflective access operation has occurred\n", 
-            "WARNING: Illegal reflective access by org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.UnsafeUtil (file:/content/build/install/content-shadow/lib/WordCount.jar) to field java.nio.Buffer.address\n", 
-            "WARNING: Please consider reporting this to the maintainers of org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.UnsafeUtil\n", 
+            "WARNING: Illegal reflective access by org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.UnsafeUtil (file:/content/build/install/content-shadow/lib/WordCount.jar) to field java.nio.Buffer.address\n",
+            "WARNING: Please consider reporting this to the maintainers of org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.UnsafeUtil\n",
             "WARNING: Use --illegal-access=warn to enable warnings of further illegal reflective access operations\n", 
             "WARNING: All illegal access operations will be denied in a future release\n", 
             "Mar 04, 2019 11:00:24 PM org.apache.beam.sdk.io.FileBasedSource getEstimatedSizeBytes\n", 
@@ -735,8 +735,8 @@
             "\n", 
             ">> java -jar WordCount.jar\n", 
             "WARNING: An illegal reflective access operation has occurred\n", 
-            "WARNING: Illegal reflective access by org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.UnsafeUtil (file:/content/WordCount.jar) to field java.nio.Buffer.address\n", 
-            "WARNING: Please consider reporting this to the maintainers of org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.UnsafeUtil\n", 
+            "WARNING: Illegal reflective access by org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.UnsafeUtil (file:/content/WordCount.jar) to field java.nio.Buffer.address\n", 
+            "WARNING: Please consider reporting this to the maintainers of org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.UnsafeUtil\n", 
             "WARNING: Use --illegal-access=warn to enable warnings of further illegal reflective access operations\n", 
             "WARNING: All illegal access operations will be denied in a future release\n", 
             "Mar 04, 2019 11:00:49 PM org.apache.beam.sdk.io.FileBasedSource getEstimatedSizeBytes\n", 
@@ -981,8 +981,8 @@
             "\n", 
             "> Task :runShadow\n", 
             "WARNING: An illegal reflective access operation has occurred\n", 
-            "WARNING: Illegal reflective access by org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.UnsafeUtil (file:/content/build/install/content-shadow/lib/WordCount.jar) to field java.nio.Buffer.address\n", 
-            "WARNING: Please consider reporting this to the maintainers of org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.UnsafeUtil\n", 
+            "WARNING: Illegal reflective access by org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.UnsafeUtil (file:/content/build/install/content-shadow/lib/WordCount.jar) to field java.nio.Buffer.address\n", 
+            "WARNING: Please consider reporting this to the maintainers of org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.UnsafeUtil\n", 
             "WARNING: Use --illegal-access=warn to enable warnings of further illegal reflective access operations\n", 
             "WARNING: All illegal access operations will be denied in a future release\n", 
             "Mar 04, 2019 11:01:26 PM org.apache.beam.sdk.io.FileBasedSource getEstimatedSizeBytes\n", 
@@ -1096,4 +1096,4 @@
       ]
     }
   ]
-}
\ No newline at end of file
+}
diff --git a/gradle.properties b/gradle.properties
index d758ae4..2b9eb79 100644
--- a/gradle.properties
+++ b/gradle.properties
@@ -23,7 +23,7 @@
 signing.gnupg.executable=gpg
 signing.gnupg.useLegacyGpg=true
 
-version=2.19.0-SNAPSHOT
-python_sdk_version=2.19.0.dev
+version=2.20.0-SNAPSHOT
+sdk_version=2.20.0.dev
 
 javaVersion=1.8
diff --git a/model/pipeline/src/main/proto/beam_runner_api.proto b/model/pipeline/src/main/proto/beam_runner_api.proto
index df5d59d..57c5295 100644
--- a/model/pipeline/src/main/proto/beam_runner_api.proto
+++ b/model/pipeline/src/main/proto/beam_runner_api.proto
@@ -395,6 +395,9 @@
 
   // (Optional) Only set when this ParDo can request bundle finalization.
   bool requests_finalization = 8;
+
+  // (Optional) A mapping of local timer family names to timer specifications.
+  map<string, TimerFamilySpec> timer_family_specs = 9;
 }
 
 // Parameters that a UDF might require.
@@ -461,6 +464,11 @@
   string timer_coder_id = 2;
 }
 
+message TimerFamilySpec {
+  TimeDomain.Enum time_domain = 1;
+  string timer_family_coder_id = 2;
+}
+
 message IsBounded {
   enum Enum {
     UNSPECIFIED = 0;
@@ -1079,21 +1087,6 @@
   FunctionSpec window_mapping_fn = 3;
 }
 
-// Settings that decide the coder type of wire coder.
-message WireCoderSetting {
-  // (Required) The URN of the wire coder.
-  // Note that only windowed value coder or parameterized windowed value coder are supported.
-  string urn = 1;
-
-  // (Optional) The data specifying any parameters to the URN. If
-  // the URN is beam:coder:windowed_value:v1, this may be omitted. If the URN is
-  // beam:coder:param_windowed_value:v1, the payload is an encoded windowed
-  // value using the beam:coder:windowed_value:v1 coder parameterized by
-  // a beam:coder:bytes:v1 element coder and the window coder that this
-  // param_windowed_value coder uses.
-  bytes payload = 2;
-}
-
 // An environment for executing UDFs. By default, an SDK container URL, but
 // can also be a process forked by a command, or an externally managed process.
 message Environment {
@@ -1302,8 +1295,8 @@
   // because ExecutableStages use environments directly. This may change in the future.
   Environment environment = 1;
 
-  // set the wire coder of this executable stage
-  WireCoderSetting wire_coder_setting = 9;
+  // The wire coder settings of this executable stage
+  repeated WireCoderSetting wire_coder_settings = 9;
 
   // (Required) Input PCollection id. This must be present as a value in the inputs of any
   // PTransform the ExecutableStagePayload is the payload of.
@@ -1333,6 +1326,10 @@
   // this ExecutableStagePayload must be represented within this field.
   repeated TimerId timers = 8;
 
+  // The timerfamilies required for this executable stage. Each timer familyof each PTransform within
+  // this ExecutableStagePayload must be represented within this field.
+  repeated TimerFamilyId timerFamilies = 10;
+
   // A reference to a side input. Side inputs are uniquely identified by PTransform id and
   // local name.
   message SideInputId {
@@ -1362,4 +1359,36 @@
     // (Required) The local name of this timer for the PTransform that references it.
     string local_name = 2;
   }
+
+  // A reference to a timer. Timers are uniquely identified by PTransform id and
+  // local name.
+  message TimerFamilyId {
+    // (Required) The id of the PTransform that references this timer family.
+    string transform_id = 1;
+
+    // (Required) The local name of this timer family for the PTransform that references it.
+    string local_name = 2;
+  }
+  // Settings that decide the coder type of wire coder.
+  message WireCoderSetting {
+    // (Required) The URN of the wire coder.
+    // Note that only windowed value coder or parameterized windowed value coder are supported.
+    string urn = 1;
+
+    // (Optional) The data specifying any parameters to the URN. If
+    // the URN is beam:coder:windowed_value:v1, this may be omitted. If the URN is
+    // beam:coder:param_windowed_value:v1, the payload is an encoded windowed
+    // value using the beam:coder:windowed_value:v1 coder parameterized by
+    // a beam:coder:bytes:v1 element coder and the window coder that this
+    // param_windowed_value coder uses.
+    bytes payload = 2;
+
+    // (Required) The target(PCollection or Timer) this setting applies to.
+    oneof target {
+      // The input or output PCollection id this setting applies to.
+      string input_or_output_id = 3;
+      // The timer id this setting applies to.
+      TimerId timer = 4;
+    }
+  }
 }
diff --git a/release/src/main/scripts/set_version.sh b/release/src/main/scripts/set_version.sh
index 5844b73..b52dfc9 100755
--- a/release/src/main/scripts/set_version.sh
+++ b/release/src/main/scripts/set_version.sh
@@ -67,7 +67,7 @@
   sed -i -e "s/version=.*/version=$TARGET_VERSION/" gradle.properties
   sed -i -e "s/project.version = .*/project.version = '$TARGET_VERSION'/" buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy
   sed -i -e "s/^__version__ = .*/__version__ = '${TARGET_VERSION}'/" sdks/python/apache_beam/version.py
-  sed -i -e "s/python_sdk_version=.*/python_sdk_version=$TARGET_VERSION/" gradle.properties
+  sed -i -e "s/sdk_version=.*/sdk_version=$TARGET_VERSION/" gradle.properties
   # TODO: [BEAM-4767]
   sed -i -e "s/'dataflow.container_version' : .*/'dataflow.container_version' : 'beam-${RELEASE}'/" runners/google-cloud-dataflow-java/build.gradle
 else
@@ -79,7 +79,7 @@
   sed -i -e "s/version=.*/version=$TARGET_VERSION-SNAPSHOT/" gradle.properties
   sed -i -e "s/project.version = .*/project.version = '$TARGET_VERSION'/" buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy
   sed -i -e "s/^__version__ = .*/__version__ = '${TARGET_VERSION}.dev'/" sdks/python/apache_beam/version.py
-  sed -i -e "s/python_sdk_version=.*/python_sdk_version=$TARGET_VERSION.dev/" gradle.properties
+  sed -i -e "s/sdk_version=.*/sdk_version=$TARGET_VERSION.dev/" gradle.properties
   sed -i -e "s/'dataflow.container_version' : .*/'dataflow.container_version' : 'beam-master-.*'/" runners/google-cloud-dataflow-java/build.gradle
 fi
 
diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexTimerInternals.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexTimerInternals.java
index 682cbed..886f153 100644
--- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexTimerInternals.java
+++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexTimerInternals.java
@@ -99,7 +99,7 @@
   }
 
   @Override
-  public void deleteTimer(StateNamespace namespace, String timerId) {
+  public void deleteTimer(StateNamespace namespace, String timerId, String timerFamilyId) {
     this.eventTimeTimeTimers.deleteTimer(getKeyBytes(this.currentKey), namespace, timerId);
     this.processingTimeTimers.deleteTimer(getKeyBytes(this.currentKey), namespace, timerId);
   }
diff --git a/runners/apex/src/test/java/org/apache/beam/runners/apex/translation/operators/ApexTimerInternalsTest.java b/runners/apex/src/test/java/org/apache/beam/runners/apex/translation/operators/ApexTimerInternalsTest.java
index df3f600..180d2fe 100644
--- a/runners/apex/src/test/java/org/apache/beam/runners/apex/translation/operators/ApexTimerInternalsTest.java
+++ b/runners/apex/src/test/java/org/apache/beam/runners/apex/translation/operators/ApexTimerInternalsTest.java
@@ -109,7 +109,7 @@
     assertEquals(1, timerMap.size());
     assertEquals(2, timerMap.values().iterator().next().size());
 
-    timerInternals.deleteTimer(timerData0.getNamespace(), timerData0.getTimerId());
+    timerInternals.deleteTimer(timerData0.getNamespace(), timerData0.getTimerId(), "");
     assertEquals(1, timerMap.size());
     assertEquals(1, timerMap.values().iterator().next().size());
 
diff --git a/runners/core-construction-java/build.gradle b/runners/core-construction-java/build.gradle
index b9c842f..3de798a 100644
--- a/runners/core-construction-java/build.gradle
+++ b/runners/core-construction-java/build.gradle
@@ -36,7 +36,7 @@
   compile project(path: ":model:pipeline", configuration: "shadow")
   compile project(path: ":model:job-management", configuration: "shadow")
   compile project(path: ":sdks:java:core", configuration: "shadow")
-  compile library.java.vendored_grpc_1_21_0
+  compile library.java.vendored_grpc_1_26_0
   compile library.java.vendored_guava_26_0_jre
   compile library.java.classgraph
   compile library.java.jackson_core
diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ArtifactServiceStager.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ArtifactServiceStager.java
index 4212916..29e47b8 100644
--- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ArtifactServiceStager.java
+++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ArtifactServiceStager.java
@@ -50,9 +50,9 @@
 import org.apache.beam.model.jobmanagement.v1.ArtifactStagingServiceGrpc.ArtifactStagingServiceStub;
 import org.apache.beam.sdk.util.MoreFutures;
 import org.apache.beam.sdk.util.ThrowingSupplier;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.Channel;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.StreamObserver;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.Channel;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.stub.StreamObserver;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.hash.Hasher;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.hash.Hashing;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.util.concurrent.ListeningExecutorService;
diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/BeamUrns.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/BeamUrns.java
index f1f30dc..e4fc6d7 100644
--- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/BeamUrns.java
+++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/BeamUrns.java
@@ -18,7 +18,7 @@
 package org.apache.beam.runners.core.construction;
 
 import org.apache.beam.model.pipeline.v1.RunnerApi;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ProtocolMessageEnum;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ProtocolMessageEnum;
 
 /** Returns the standard URN of a given enum annotated with [(standard_urn)]. */
 public class BeamUrns {
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 8e1021d..86f0178 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
@@ -28,7 +28,7 @@
 import org.apache.beam.model.pipeline.v1.RunnerApi.FunctionSpec;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.util.SerializableUtils;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
+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.collect.BiMap;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableBiMap;
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 6f156dd..81b7922 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
@@ -33,7 +33,7 @@
 import org.apache.beam.sdk.util.InstanceBuilder;
 import org.apache.beam.sdk.util.WindowedValue;
 import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.InvalidProtocolBufferException;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
 
 /** {@link CoderTranslator} implementations for known coder types. */
diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CombineTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CombineTranslation.java
index e5edc35..5ea9ef7 100644
--- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CombineTranslation.java
+++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CombineTranslation.java
@@ -39,7 +39,7 @@
 import org.apache.beam.sdk.util.SerializableUtils;
 import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
+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.collect.ImmutableMap;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables;
diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CreatePCollectionViewTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CreatePCollectionViewTranslation.java
index b89c5b6..5027bb4 100644
--- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CreatePCollectionViewTranslation.java
+++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CreatePCollectionViewTranslation.java
@@ -34,7 +34,7 @@
 import org.apache.beam.sdk.util.SerializableUtils;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PCollectionView;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
 
 /**
  * Utility methods for translating a {@link View} transforms to and from {@link RunnerApi}
diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/DefaultExpansionServiceClientFactory.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/DefaultExpansionServiceClientFactory.java
index 1586be8..a25007e 100644
--- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/DefaultExpansionServiceClientFactory.java
+++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/DefaultExpansionServiceClientFactory.java
@@ -23,7 +23,7 @@
 import org.apache.beam.model.expansion.v1.ExpansionApi;
 import org.apache.beam.model.expansion.v1.ExpansionServiceGrpc;
 import org.apache.beam.model.pipeline.v1.Endpoints;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.ManagedChannel;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.ManagedChannel;
 
 /** Default factory for ExpansionServiceClient used by External transform. */
 public class DefaultExpansionServiceClientFactory implements ExpansionServiceClientFactory {
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 c7cd235..57836b9 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
@@ -19,8 +19,8 @@
 
 import org.apache.beam.model.pipeline.v1.RunnerApi;
 import org.apache.beam.sdk.transforms.display.DisplayData;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.Any;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.BoolValue;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.Any;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.BoolValue;
 
 /** Utilities for going to/from DisplayData protos. */
 public class DisplayDataTranslation {
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 ed94642..73bf534 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
@@ -32,7 +32,7 @@
 import org.apache.beam.model.pipeline.v1.RunnerApi.StandardEnvironments;
 import org.apache.beam.sdk.util.ReleaseInfo;
 import org.apache.beam.sdk.util.common.ReflectHelpers;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
 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.base.Strings;
 
@@ -55,7 +55,7 @@
    * container.
    */
   private static final String JAVA_SDK_HARNESS_CONTAINER_URL =
-      "apachebeam/java_sdk:" + ReleaseInfo.getReleaseInfo().getVersion();
+      "apachebeam/java_sdk:" + ReleaseInfo.getReleaseInfo().getSdkVersion();
   public static final Environment JAVA_SDK_HARNESS_ENVIRONMENT =
       createDockerEnvironment(JAVA_SDK_HARNESS_CONTAINER_URL);
 
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 d58346b..45665fd 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
@@ -38,8 +38,8 @@
 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.grpc.v1p21p0.com.google.protobuf.ByteString;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.ManagedChannelBuilder;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.ManagedChannelBuilder;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Strings;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables;
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 3d6d4dd..929d7a8 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
@@ -25,7 +25,7 @@
 import org.apache.beam.model.pipeline.v1.RunnerApi.Coder;
 import org.apache.beam.model.pipeline.v1.RunnerApi.FunctionSpec;
 import org.apache.beam.model.pipeline.v1.RunnerApi.StandardCoders;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableSet;
 
 /** Utilities and constants ot interact with coders that are part of the Beam Model. */
diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PCollectionViewTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PCollectionViewTranslation.java
index d9ddb93..402e7d6 100644
--- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PCollectionViewTranslation.java
+++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PCollectionViewTranslation.java
@@ -30,7 +30,7 @@
 import org.apache.beam.sdk.values.PCollectionView;
 import org.apache.beam.sdk.values.TupleTag;
 import org.apache.beam.sdk.values.WindowingStrategy;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.InvalidProtocolBufferException;
 
 /** Utilities for interacting with PCollection view protos. */
 public class PCollectionViewTranslation {
diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ParDoTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ParDoTranslation.java
index 8a191e5..7e6ba7b 100644
--- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ParDoTranslation.java
+++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ParDoTranslation.java
@@ -79,8 +79,8 @@
 import org.apache.beam.sdk.values.PCollectionView;
 import org.apache.beam.sdk.values.TupleTag;
 import org.apache.beam.sdk.values.TupleTagList;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.InvalidProtocolBufferException;
 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.Iterables;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Sets;
@@ -259,10 +259,26 @@
                   translateTimerSpec(getTimerSpecOrThrow(timer.getValue(), doFn), newComponents);
               timerSpecs.put(timer.getKey(), spec);
             }
+
             return timerSpecs;
           }
 
           @Override
+          public Map<String, RunnerApi.TimerFamilySpec> translateTimerFamilySpecs(
+              SdkComponents newComponents) {
+            Map<String, RunnerApi.TimerFamilySpec> timerFamilySpecs = new HashMap<>();
+            for (Map.Entry<String, DoFnSignature.TimerFamilyDeclaration> timerFamily :
+                signature.timerFamilyDeclarations().entrySet()) {
+              RunnerApi.TimerFamilySpec spec =
+                  translateTimerFamilySpec(
+                      DoFnSignatures.getTimerFamilySpecOrThrow(timerFamily.getValue(), doFn),
+                      newComponents);
+              timerFamilySpecs.put(timerFamily.getKey(), spec);
+            }
+            return timerFamilySpecs;
+          }
+
+          @Override
           public boolean isSplittable() {
             return signature.processElement().isSplittable();
           }
@@ -580,6 +596,14 @@
         .build();
   }
 
+  public static RunnerApi.TimerFamilySpec translateTimerFamilySpec(
+      TimerSpec timer, SdkComponents components) {
+    return RunnerApi.TimerFamilySpec.newBuilder()
+        .setTimeDomain(translateTimeDomain(timer.getTimeDomain()))
+        .setTimerFamilyCoderId(registerCoderOrThrow(components, Timer.Coder.of(VoidCoder.of())))
+        .build();
+  }
+
   private static RunnerApi.TimeDomain.Enum translateTimeDomain(TimeDomain timeDomain) {
     switch (timeDomain) {
       case EVENT_TIME:
@@ -697,7 +721,9 @@
 
   public static boolean usesStateOrTimers(AppliedPTransform<?, ?, ?> transform) throws IOException {
     ParDoPayload payload = getParDoPayload(transform);
-    return payload.getStateSpecsCount() > 0 || payload.getTimerSpecsCount() > 0;
+    return payload.getStateSpecsCount() > 0
+        || payload.getTimerSpecsCount() > 0
+        || payload.getTimerFamilySpecsCount() > 0;
   }
 
   public static boolean isSplittable(AppliedPTransform<?, ?, ?> transform) throws IOException {
@@ -726,6 +752,8 @@
 
     Map<String, RunnerApi.TimerSpec> translateTimerSpecs(SdkComponents newComponents);
 
+    Map<String, RunnerApi.TimerFamilySpec> translateTimerFamilySpecs(SdkComponents newComponents);
+
     boolean isSplittable();
 
     String translateRestrictionCoderId(SdkComponents newComponents);
@@ -739,6 +767,7 @@
         .addAllParameters(parDo.translateParameters())
         .putAllStateSpecs(parDo.translateStateSpecs(components))
         .putAllTimerSpecs(parDo.translateTimerSpecs(components))
+        .putAllTimerFamilySpecs(parDo.translateTimerFamilySpecs(components))
         .putAllSideInputs(parDo.translateSideInputs(components))
         .setSplittable(parDo.isSplittable())
         .setRestrictionCoderId(parDo.translateRestrictionCoderId(components))
diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PipelineOptionsTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PipelineOptionsTranslation.java
index baf7c36..56e5d06 100644
--- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PipelineOptionsTranslation.java
+++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PipelineOptionsTranslation.java
@@ -27,9 +27,9 @@
 import java.util.Map;
 import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.util.common.ReflectHelpers;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.InvalidProtocolBufferException;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.Struct;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.util.JsonFormat;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.Struct;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.util.JsonFormat;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.CaseFormat;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
 
diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ReadTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ReadTranslation.java
index 81bafab..94288e5 100644
--- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ReadTranslation.java
+++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ReadTranslation.java
@@ -38,8 +38,8 @@
 import org.apache.beam.sdk.util.SerializableUtils;
 import org.apache.beam.sdk.values.PBegin;
 import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.InvalidProtocolBufferException;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
 
 /**
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 f89874e..2b700d5 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
@@ -32,6 +32,7 @@
 import org.apache.beam.model.pipeline.v1.RunnerApi.Parameter;
 import org.apache.beam.model.pipeline.v1.RunnerApi.SideInput;
 import org.apache.beam.model.pipeline.v1.RunnerApi.StateSpec;
+import org.apache.beam.model.pipeline.v1.RunnerApi.TimerFamilySpec;
 import org.apache.beam.model.pipeline.v1.RunnerApi.TimerSpec;
 import org.apache.beam.runners.core.construction.PTransformTranslation.TransformPayloadTranslator;
 import org.apache.beam.runners.core.construction.ParDoTranslation.ParDoLike;
@@ -399,6 +400,13 @@
                 }
 
                 @Override
+                public Map<String, TimerFamilySpec> translateTimerFamilySpecs(
+                    SdkComponents newComponents) {
+                  // SDFs don't have timers.
+                  return ImmutableMap.of();
+                }
+
+                @Override
                 public boolean isSplittable() {
                   return true;
                 }
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 8a41d7e..ce888c9 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
@@ -29,6 +29,7 @@
 import org.apache.beam.sdk.state.State;
 import org.apache.beam.sdk.state.TimeDomain;
 import org.apache.beam.sdk.state.Timer;
+import org.apache.beam.sdk.state.TimerMap;
 import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.transforms.PTransform;
 import org.apache.beam.sdk.transforms.ParDo;
@@ -243,6 +244,11 @@
       }
 
       @Override
+      public TimerMap timerFamily(String tagId) {
+        throw new UnsupportedOperationException();
+      }
+
+      @Override
       public Object schemaElement(int index) {
         throw new UnsupportedOperationException();
       }
@@ -253,6 +259,11 @@
       }
 
       @Override
+      public String timerId(DoFn<InputT, OutputT> doFn) {
+        throw new UnsupportedOperationException();
+      }
+
+      @Override
       public OutputReceiver<OutputT> outputReceiver(DoFn<InputT, OutputT> doFn) {
         return new OutputReceiver<OutputT>() {
           @Override
diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TestStreamTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TestStreamTranslation.java
index 1b747c1..abbc328 100644
--- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TestStreamTranslation.java
+++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TestStreamTranslation.java
@@ -36,7 +36,7 @@
 import org.apache.beam.sdk.values.PBegin;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.TimestampedValue;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
 import org.joda.time.Duration;
 import org.joda.time.Instant;
 
diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WindowIntoTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WindowIntoTranslation.java
index 0d72861..d7fafcd 100644
--- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WindowIntoTranslation.java
+++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WindowIntoTranslation.java
@@ -33,7 +33,7 @@
 import org.apache.beam.sdk.transforms.windowing.Window;
 import org.apache.beam.sdk.transforms.windowing.Window.Assign;
 import org.apache.beam.sdk.transforms.windowing.WindowFn;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.InvalidProtocolBufferException;
 
 /**
  * Utility methods for translating a {@link Window.Assign} to and from {@link RunnerApi}
diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WindowingStrategyTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WindowingStrategyTranslation.java
index 63f662f..bbb31a7 100644
--- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WindowingStrategyTranslation.java
+++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WindowingStrategyTranslation.java
@@ -41,10 +41,10 @@
 import org.apache.beam.sdk.util.SerializableUtils;
 import org.apache.beam.sdk.values.WindowingStrategy;
 import org.apache.beam.sdk.values.WindowingStrategy.AccumulationMode;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.InvalidProtocolBufferException;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.util.Durations;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.util.Timestamps;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.util.Durations;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.util.Timestamps;
 import org.joda.time.Duration;
 
 /** Utilities for working with {@link WindowingStrategy WindowingStrategies}. */
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 0df16a2..986a585 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
@@ -45,7 +45,7 @@
 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.grpc.v1p21p0.com.google.protobuf.ByteString;
+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.MoreObjects;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Lists;
diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/expansion/ExpansionServer.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/expansion/ExpansionServer.java
index 12b52f4..5859d69 100644
--- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/expansion/ExpansionServer.java
+++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/expansion/ExpansionServer.java
@@ -20,8 +20,8 @@
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.util.concurrent.TimeUnit;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.Server;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.netty.NettyServerBuilder;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.Server;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.netty.NettyServerBuilder;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions;
 
 /** A {@link Server gRPC Server} for an ExpansionService. */
diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/expansion/ExpansionService.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/expansion/ExpansionService.java
index aab393b..45559bb 100644
--- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/expansion/ExpansionService.java
+++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/expansion/ExpansionService.java
@@ -54,9 +54,9 @@
 import org.apache.beam.sdk.values.PInput;
 import org.apache.beam.sdk.values.POutput;
 import org.apache.beam.sdk.values.TupleTag;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.Server;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.ServerBuilder;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.StreamObserver;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.Server;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.ServerBuilder;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.stub.StreamObserver;
 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.CaseFormat;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Converter;
diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/ExecutableStage.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/ExecutableStage.java
index dd2d374..21d252d 100644
--- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/ExecutableStage.java
+++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/ExecutableStage.java
@@ -17,9 +17,8 @@
  */
 package org.apache.beam.runners.core.construction.graph;
 
-import static org.apache.beam.runners.core.construction.BeamUrns.getUrn;
-
 import java.util.Collection;
+import java.util.Collections;
 import java.util.List;
 import java.util.stream.Collectors;
 import org.apache.beam.model.pipeline.v1.RunnerApi;
@@ -29,11 +28,11 @@
 import org.apache.beam.model.pipeline.v1.RunnerApi.ExecutableStagePayload.SideInputId;
 import org.apache.beam.model.pipeline.v1.RunnerApi.ExecutableStagePayload.TimerId;
 import org.apache.beam.model.pipeline.v1.RunnerApi.ExecutableStagePayload.UserStateId;
+import org.apache.beam.model.pipeline.v1.RunnerApi.ExecutableStagePayload.WireCoderSetting;
 import org.apache.beam.model.pipeline.v1.RunnerApi.FunctionSpec;
 import org.apache.beam.model.pipeline.v1.RunnerApi.PCollection;
 import org.apache.beam.model.pipeline.v1.RunnerApi.PTransform;
 import org.apache.beam.model.pipeline.v1.RunnerApi.Pipeline;
-import org.apache.beam.model.pipeline.v1.RunnerApi.WireCoderSetting;
 import org.apache.beam.runners.core.construction.graph.PipelineNode.PCollectionNode;
 import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode;
 
@@ -73,12 +72,12 @@
   Environment getEnvironment();
 
   /**
-   * Returns the {@link WireCoderSetting} this stage executes in.
+   * Returns a set of {@link WireCoderSetting}s this stage executes in.
    *
    * <p>A {@link WireCoderSetting} consists of settings which is used to configure the type of the
-   * wire coder.
+   * wire coder for a dedicated PCollection.
    */
-  WireCoderSetting getWireCoderSetting();
+  Collection<WireCoderSetting> getWireCoderSettings();
 
   /**
    * Returns the root {@link PCollectionNode} of this {@link ExecutableStage}. This {@link
@@ -145,7 +144,7 @@
     ExecutableStagePayload.Builder payload = ExecutableStagePayload.newBuilder();
 
     payload.setEnvironment(getEnvironment());
-    payload.setWireCoderSetting(getWireCoderSetting());
+    payload.addAllWireCoderSettings(getWireCoderSettings());
 
     // Populate inputs and outputs of the stage payload and outer PTransform simultaneously.
     PCollectionNode input = getInputPCollection();
@@ -220,7 +219,7 @@
   static ExecutableStage fromPayload(ExecutableStagePayload payload) {
     Components components = payload.getComponents();
     Environment environment = payload.getEnvironment();
-    WireCoderSetting wireCoderSetting = payload.getWireCoderSetting();
+    Collection<WireCoderSetting> wireCoderSettings = payload.getWireCoderSettingsList();
 
     PCollectionNode input =
         PipelineNode.pCollection(
@@ -254,12 +253,12 @@
         timers,
         transforms,
         outputs,
-        wireCoderSetting);
+        wireCoderSettings);
   }
 
-  /** The default wire coder, i.e., WINDOWED_VALUE coder. */
-  WireCoderSetting DEFAULT_WIRE_CODER_SETTING =
-      WireCoderSetting.newBuilder()
-          .setUrn(getUrn(RunnerApi.StandardCoders.Enum.WINDOWED_VALUE))
-          .build();
+  /**
+   * The default wire coder settings which returns an empty list, i.e., the WireCoder for each
+   * PCollection and timer will be a WINDOWED_VALUE coder.
+   */
+  Collection<WireCoderSetting> DEFAULT_WIRE_CODER_SETTINGS = Collections.emptyList();
 }
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 cecbee9..3d7d414 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
@@ -31,7 +31,7 @@
 import org.apache.beam.runners.core.construction.graph.PipelineNode.PCollectionNode;
 import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode;
 import org.apache.beam.sdk.transforms.Flatten;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.InvalidProtocolBufferException;
 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.Maps;
 import org.slf4j.Logger;
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 00cca32..4eaad5a 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
@@ -416,7 +416,7 @@
         stage.getTimers(),
         pTransformNodes,
         stage.getOutputPCollections(),
-        stage.getWireCoderSetting());
+        stage.getWireCoderSettings());
   }
 
   /**
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 87f2076..6ec3da2 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
@@ -17,7 +17,7 @@
  */
 package org.apache.beam.runners.core.construction.graph;
 
-import static org.apache.beam.runners.core.construction.graph.ExecutableStage.DEFAULT_WIRE_CODER_SETTING;
+import static org.apache.beam.runners.core.construction.graph.ExecutableStage.DEFAULT_WIRE_CODER_SETTINGS;
 import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument;
 
 import java.util.ArrayDeque;
@@ -140,7 +140,7 @@
         timers,
         fusedTransforms.build(),
         materializedPCollections,
-        DEFAULT_WIRE_CODER_SETTING);
+        DEFAULT_WIRE_CODER_SETTINGS);
   }
 
   private static Environment getStageEnvironment(
diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/ImmutableExecutableStage.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/ImmutableExecutableStage.java
index 0092056..a66958f 100644
--- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/ImmutableExecutableStage.java
+++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/graph/ImmutableExecutableStage.java
@@ -22,7 +22,7 @@
 import java.util.stream.Collectors;
 import org.apache.beam.model.pipeline.v1.RunnerApi.Components;
 import org.apache.beam.model.pipeline.v1.RunnerApi.Environment;
-import org.apache.beam.model.pipeline.v1.RunnerApi.WireCoderSetting;
+import org.apache.beam.model.pipeline.v1.RunnerApi.ExecutableStagePayload.WireCoderSetting;
 import org.apache.beam.runners.core.construction.graph.PipelineNode.PCollectionNode;
 import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableSet;
@@ -39,7 +39,7 @@
       Collection<TimerReference> timers,
       Collection<PTransformNode> transforms,
       Collection<PCollectionNode> outputs,
-      WireCoderSetting wireCoderSetting) {
+      Collection<WireCoderSetting> wireCoderSettings) {
     Components prunedComponents =
         components
             .toBuilder()
@@ -57,7 +57,7 @@
         timers,
         transforms,
         outputs,
-        wireCoderSetting);
+        wireCoderSettings);
   }
 
   public static ImmutableExecutableStage of(
@@ -69,7 +69,7 @@
       Collection<TimerReference> timers,
       Collection<PTransformNode> transforms,
       Collection<PCollectionNode> outputs,
-      WireCoderSetting wireCoderSetting) {
+      Collection<WireCoderSetting> wireCoderSettings) {
     return new AutoValue_ImmutableExecutableStage(
         components,
         environment,
@@ -79,7 +79,7 @@
         ImmutableSet.copyOf(timers),
         ImmutableSet.copyOf(transforms),
         ImmutableSet.copyOf(outputs),
-        wireCoderSetting);
+        wireCoderSettings);
   }
 
   @Override
@@ -108,5 +108,5 @@
   public abstract Collection<PCollectionNode> getOutputPCollections();
 
   @Override
-  public abstract WireCoderSetting getWireCoderSetting();
+  public abstract Collection<WireCoderSetting> getWireCoderSettings();
 }
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 def7de8..cd5d8ba 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
@@ -309,7 +309,7 @@
         stage.getTimers(),
         updatedTransforms,
         updatedOutputs,
-        stage.getWireCoderSetting());
+        stage.getWireCoderSettings());
   }
 
   /**
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 4ed19da..099294d 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
@@ -62,7 +62,7 @@
 import org.apache.beam.runners.core.construction.PTransformTranslation;
 import org.apache.beam.runners.core.construction.graph.PipelineNode.PCollectionNode;
 import org.apache.beam.runners.core.construction.graph.PipelineNode.PTransformNode;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.InvalidProtocolBufferException;
 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.ImmutableSet;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables;
diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ArtifactServiceStagerTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ArtifactServiceStagerTest.java
index 99c14e5..4cc2b68 100644
--- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ArtifactServiceStagerTest.java
+++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ArtifactServiceStagerTest.java
@@ -33,9 +33,9 @@
 import java.util.Set;
 import org.apache.beam.model.jobmanagement.v1.ArtifactApi.ArtifactMetadata;
 import org.apache.beam.runners.core.construction.ArtifactServiceStager.StagedFile;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.Server;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.inprocess.InProcessChannelBuilder;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.inprocess.InProcessServerBuilder;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.Server;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.inprocess.InProcessChannelBuilder;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.inprocess.InProcessServerBuilder;
 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.hash.Hashing;
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 56fae2f..4ec1a7d 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
@@ -71,7 +71,7 @@
 import org.apache.beam.sdk.util.WindowedValue;
 import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.Row;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.InvalidProtocolBufferException;
 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.base.Splitter;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ExternalTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ExternalTest.java
index b399472..1cedfc5 100644
--- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ExternalTest.java
+++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ExternalTest.java
@@ -38,11 +38,11 @@
 import org.apache.beam.sdk.values.TupleTag;
 import org.apache.beam.sdk.values.TupleTagList;
 import org.apache.beam.sdk.values.TypeDescriptors;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.ConnectivityState;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.ManagedChannel;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.ManagedChannelBuilder;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.Server;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.ServerBuilder;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.ConnectivityState;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.ManagedChannel;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.ManagedChannelBuilder;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.Server;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.ServerBuilder;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/InMemoryArtifactStagerService.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/InMemoryArtifactStagerService.java
index cb850bd..34431b6 100644
--- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/InMemoryArtifactStagerService.java
+++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/InMemoryArtifactStagerService.java
@@ -35,7 +35,7 @@
 import org.apache.beam.model.jobmanagement.v1.ArtifactApi.PutArtifactRequest.ContentCase;
 import org.apache.beam.model.jobmanagement.v1.ArtifactApi.PutArtifactResponse;
 import org.apache.beam.model.jobmanagement.v1.ArtifactStagingServiceGrpc.ArtifactStagingServiceImplBase;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.StreamObserver;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.stub.StreamObserver;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.hash.Hashing;
 
 /**
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 a482d02..14d8c1c 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
@@ -30,9 +30,9 @@
 import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.options.PipelineOptionsFactory;
 import org.apache.beam.sdk.util.common.ReflectHelpers;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.NullValue;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.Struct;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.Value;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.NullValue;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.Struct;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.Value;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
 import org.junit.Test;
 import org.junit.runner.RunWith;
diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/WindowIntoTranslationTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/WindowIntoTranslationTest.java
index 2db4d70..4b3f7aa 100644
--- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/WindowIntoTranslationTest.java
+++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/WindowIntoTranslationTest.java
@@ -38,7 +38,7 @@
 import org.apache.beam.sdk.transforms.windowing.Window;
 import org.apache.beam.sdk.transforms.windowing.Window.Assign;
 import org.apache.beam.sdk.transforms.windowing.WindowFn;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.InvalidProtocolBufferException;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
 import org.joda.time.Duration;
 import org.joda.time.Instant;
diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/expansion/ExpansionServiceTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/expansion/ExpansionServiceTest.java
index 6024c10..b78f1da 100644
--- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/expansion/ExpansionServiceTest.java
+++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/expansion/ExpansionServiceTest.java
@@ -46,7 +46,7 @@
 import org.apache.beam.sdk.transforms.Count;
 import org.apache.beam.sdk.transforms.Impulse;
 import org.apache.beam.sdk.values.KV;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
 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.ImmutableList;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/ExecutableStageTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/ExecutableStageTest.java
index 863165c..9a69ed8 100644
--- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/ExecutableStageTest.java
+++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/ExecutableStageTest.java
@@ -17,7 +17,7 @@
  */
 package org.apache.beam.runners.core.construction.graph;
 
-import static org.apache.beam.runners.core.construction.graph.ExecutableStage.DEFAULT_WIRE_CODER_SETTING;
+import static org.apache.beam.runners.core.construction.graph.ExecutableStage.DEFAULT_WIRE_CODER_SETTINGS;
 import static org.hamcrest.Matchers.allOf;
 import static org.hamcrest.Matchers.contains;
 import static org.hamcrest.Matchers.containsInAnyOrder;
@@ -106,7 +106,7 @@
             Collections.singleton(timerRef),
             Collections.singleton(PipelineNode.pTransform("pt", pt)),
             Collections.singleton(PipelineNode.pCollection("output.out", output)),
-            DEFAULT_WIRE_CODER_SETTING);
+            DEFAULT_WIRE_CODER_SETTINGS);
 
     PTransform stagePTransform = stage.toPTransform("foo");
     assertThat(stagePTransform.getOutputsMap(), hasValue("output.out"));
diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/ImmutableExecutableStageTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/ImmutableExecutableStageTest.java
index 578d506..69336b7 100644
--- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/ImmutableExecutableStageTest.java
+++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/ImmutableExecutableStageTest.java
@@ -17,7 +17,7 @@
  */
 package org.apache.beam.runners.core.construction.graph;
 
-import static org.apache.beam.runners.core.construction.graph.ExecutableStage.DEFAULT_WIRE_CODER_SETTING;
+import static org.apache.beam.runners.core.construction.graph.ExecutableStage.DEFAULT_WIRE_CODER_SETTINGS;
 import static org.hamcrest.Matchers.allOf;
 import static org.hamcrest.Matchers.contains;
 import static org.hamcrest.Matchers.equalTo;
@@ -100,7 +100,7 @@
             Collections.singleton(timerRef),
             Collections.singleton(PipelineNode.pTransform("pt", pt)),
             Collections.singleton(PipelineNode.pCollection("output.out", output)),
-            DEFAULT_WIRE_CODER_SETTING);
+            DEFAULT_WIRE_CODER_SETTINGS);
 
     assertThat(stage.getComponents().containsTransforms("pt"), is(true));
     assertThat(stage.getComponents().containsTransforms("other_pt"), is(false));
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 8e83eef..8d45782 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
@@ -17,7 +17,7 @@
  */
 package org.apache.beam.runners.core.construction.graph;
 
-import static org.apache.beam.runners.core.construction.graph.ExecutableStage.DEFAULT_WIRE_CODER_SETTING;
+import static org.apache.beam.runners.core.construction.graph.ExecutableStage.DEFAULT_WIRE_CODER_SETTINGS;
 import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables.getOnlyElement;
 import static org.hamcrest.Matchers.containsInAnyOrder;
 import static org.hamcrest.Matchers.empty;
@@ -121,7 +121,7 @@
             ImmutableList.of(),
             ImmutableList.of(PipelineNode.pTransform("one", one)),
             ImmutableList.of(PipelineNode.pCollection(oneOut.getUniqueName(), oneOut)),
-            DEFAULT_WIRE_CODER_SETTING);
+            DEFAULT_WIRE_CODER_SETTINGS);
     ExecutableStage twoStage =
         ImmutableExecutableStage.of(
             components,
@@ -132,7 +132,7 @@
             ImmutableList.of(),
             ImmutableList.of(PipelineNode.pTransform("two", two)),
             ImmutableList.of(PipelineNode.pCollection(twoOut.getUniqueName(), twoOut)),
-            DEFAULT_WIRE_CODER_SETTING);
+            DEFAULT_WIRE_CODER_SETTINGS);
     PTransformNode redTransform = PipelineNode.pTransform("red", red);
     PTransformNode blueTransform = PipelineNode.pTransform("blue", blue);
     QueryablePipeline pipeline = QueryablePipeline.forPrimitivesIn(components);
@@ -241,7 +241,7 @@
             ImmutableList.of(
                 PipelineNode.pTransform("one", one), PipelineNode.pTransform("shared", shared)),
             ImmutableList.of(PipelineNode.pCollection(sharedOut.getUniqueName(), sharedOut)),
-            DEFAULT_WIRE_CODER_SETTING);
+            DEFAULT_WIRE_CODER_SETTINGS);
     ExecutableStage twoStage =
         ImmutableExecutableStage.of(
             components,
@@ -253,7 +253,7 @@
             ImmutableList.of(
                 PipelineNode.pTransform("two", two), PipelineNode.pTransform("shared", shared)),
             ImmutableList.of(PipelineNode.pCollection(sharedOut.getUniqueName(), sharedOut)),
-            DEFAULT_WIRE_CODER_SETTING);
+            DEFAULT_WIRE_CODER_SETTINGS);
     PTransformNode redTransform = PipelineNode.pTransform("red", red);
     PTransformNode blueTransform = PipelineNode.pTransform("blue", blue);
     QueryablePipeline pipeline = QueryablePipeline.forPrimitivesIn(components);
@@ -373,7 +373,7 @@
             ImmutableList.of(),
             ImmutableList.of(PipelineNode.pTransform("one", one), sharedTransform),
             ImmutableList.of(PipelineNode.pCollection(sharedOut.getUniqueName(), sharedOut)),
-            DEFAULT_WIRE_CODER_SETTING);
+            DEFAULT_WIRE_CODER_SETTINGS);
     PTransformNode redTransform = PipelineNode.pTransform("red", red);
     PTransformNode blueTransform = PipelineNode.pTransform("blue", blue);
     QueryablePipeline pipeline = QueryablePipeline.forPrimitivesIn(components);
@@ -547,7 +547,7 @@
             ImmutableList.of(
                 PipelineNode.pCollection(sharedOut.getUniqueName(), sharedOut),
                 PipelineNode.pCollection(otherSharedOut.getUniqueName(), otherSharedOut)),
-            DEFAULT_WIRE_CODER_SETTING);
+            DEFAULT_WIRE_CODER_SETTINGS);
     ExecutableStage oneStage =
         ImmutableExecutableStage.of(
             components,
@@ -559,7 +559,7 @@
             ImmutableList.of(
                 PipelineNode.pTransform("one", one), PipelineNode.pTransform("shared", shared)),
             ImmutableList.of(PipelineNode.pCollection(sharedOut.getUniqueName(), sharedOut)),
-            DEFAULT_WIRE_CODER_SETTING);
+            DEFAULT_WIRE_CODER_SETTINGS);
     ExecutableStage twoStage =
         ImmutableExecutableStage.of(
             components,
@@ -573,7 +573,7 @@
                 PipelineNode.pTransform("otherShared", otherShared)),
             ImmutableList.of(
                 PipelineNode.pCollection(otherSharedOut.getUniqueName(), otherSharedOut)),
-            DEFAULT_WIRE_CODER_SETTING);
+            DEFAULT_WIRE_CODER_SETTINGS);
     PTransformNode redTransform = PipelineNode.pTransform("red", red);
     PTransformNode blueTransform = PipelineNode.pTransform("blue", blue);
     QueryablePipeline pipeline = QueryablePipeline.forPrimitivesIn(components);
diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/ProtoOverridesTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/ProtoOverridesTest.java
index 5215d72..f27f38d 100644
--- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/ProtoOverridesTest.java
+++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/graph/ProtoOverridesTest.java
@@ -36,7 +36,7 @@
 import org.apache.beam.model.pipeline.v1.RunnerApi.Pipeline;
 import org.apache.beam.model.pipeline.v1.RunnerApi.WindowingStrategy;
 import org.apache.beam.runners.core.construction.graph.ProtoOverrides.TransformReplacement;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
 import org.junit.Test;
 import org.junit.runner.RunWith;
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 286e60b..7fbfaf0 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
@@ -122,9 +122,14 @@
     WindowTracing.trace("{}.setTimer: {}", getClass().getSimpleName(), timerData);
 
     @Nullable
-    TimerData existing = existingTimers.get(timerData.getNamespace(), timerData.getTimerId());
+    TimerData existing =
+        existingTimers.get(
+            timerData.getNamespace(), timerData.getTimerId() + '+' + timerData.getTimerFamilyId());
     if (existing == null) {
-      existingTimers.put(timerData.getNamespace(), timerData.getTimerId(), timerData);
+      existingTimers.put(
+          timerData.getNamespace(),
+          timerData.getTimerId() + '+' + timerData.getTimerFamilyId(),
+          timerData);
       timersForDomain(timerData.getDomain()).add(timerData);
     } else {
       checkArgument(
@@ -138,7 +143,10 @@
         NavigableSet<TimerData> timers = timersForDomain(timerData.getDomain());
         timers.remove(existing);
         timers.add(timerData);
-        existingTimers.put(timerData.getNamespace(), timerData.getTimerId(), timerData);
+        existingTimers.put(
+            timerData.getNamespace(),
+            timerData.getTimerId() + '+' + timerData.getTimerFamilyId(),
+            timerData);
       }
     }
   }
@@ -151,8 +159,8 @@
   /** @deprecated use {@link #deleteTimer(StateNamespace, String, TimeDomain)}. */
   @Deprecated
   @Override
-  public void deleteTimer(StateNamespace namespace, String timerId) {
-    TimerData existing = existingTimers.get(namespace, timerId);
+  public void deleteTimer(StateNamespace namespace, String timerId, String timerFamilyId) {
+    TimerData existing = existingTimers.get(namespace, timerId + '+' + timerFamilyId);
     if (existing != null) {
       deleteTimer(existing);
     }
@@ -163,7 +171,8 @@
   @Override
   public void deleteTimer(TimerData timer) {
     WindowTracing.trace("{}.deleteTimer: {}", getClass().getSimpleName(), timer);
-    existingTimers.remove(timer.getNamespace(), timer.getTimerId());
+    existingTimers.remove(
+        timer.getNamespace(), timer.getTimerId() + '+' + timer.getTimerFamilyId());
     timersForDomain(timer.getDomain()).remove(timer);
   }
 
@@ -321,7 +330,8 @@
 
     if (!timers.isEmpty() && currentTime.isAfter(timers.first().getTimestamp())) {
       TimerData timer = timers.pollFirst();
-      existingTimers.remove(timer.getNamespace(), timer.getTimerId());
+      existingTimers.remove(
+          timer.getNamespace(), timer.getTimerId() + '+' + timer.getTimerFamilyId());
       return timer;
     } else {
       return null;
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 9edc558..8f09c5d 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
@@ -30,6 +30,7 @@
 import org.apache.beam.sdk.state.State;
 import org.apache.beam.sdk.state.TimeDomain;
 import org.apache.beam.sdk.state.Timer;
+import org.apache.beam.sdk.state.TimerMap;
 import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.transforms.DoFn.FinishBundleContext;
 import org.apache.beam.sdk.transforms.DoFn.MultiOutputReceiver;
@@ -137,6 +138,12 @@
               }
 
               @Override
+              public String timerId(DoFn<InputT, OutputT> doFn) {
+                throw new UnsupportedOperationException(
+                    "Cannot access timerId as parameter outside of @OnTimer method.");
+              }
+
+              @Override
               public TimeDomain timeDomain(DoFn<InputT, OutputT> doFn) {
                 throw new UnsupportedOperationException(
                     "Access to time domain not supported in ProcessElement");
@@ -213,6 +220,12 @@
                 throw new UnsupportedOperationException(
                     "Access to timers not supported in Splittable DoFn");
               }
+
+              @Override
+              public TimerMap timerFamily(String tagId) {
+                throw new UnsupportedOperationException(
+                    "Access to timerFamily not supported in Splittable DoFn");
+              }
             });
     processContext.cancelScheduledCheckpoint();
     @Nullable KV<RestrictionT, Instant> residual = processContext.getTakenCheckpoint();
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 8af4d13..082cb28 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
@@ -21,6 +21,7 @@
 import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkNotNull;
 
 import java.util.Collection;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -33,6 +34,7 @@
 import org.apache.beam.sdk.state.StateSpec;
 import org.apache.beam.sdk.state.TimeDomain;
 import org.apache.beam.sdk.state.Timer;
+import org.apache.beam.sdk.state.TimerMap;
 import org.apache.beam.sdk.state.TimerSpec;
 import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.transforms.DoFn.MultiOutputReceiver;
@@ -212,8 +214,8 @@
     }
 
     OnTimerArgumentProvider argumentProvider =
-        new OnTimerArgumentProvider(window, effectiveTimestamp, timeDomain);
-    invoker.invokeOnTimer(timerId, argumentProvider);
+        new OnTimerArgumentProvider(timerId, window, effectiveTimestamp, timeDomain);
+    invoker.invokeOnTimer(timerId, timerFamilyId, argumentProvider);
   }
 
   private void invokeProcessElement(WindowedValue<InputT> elem) {
@@ -328,6 +330,11 @@
     }
 
     @Override
+    public String timerId(DoFn<InputT, OutputT> doFn) {
+      throw new UnsupportedOperationException("Cannot access timerId outside of @OnTimer method.");
+    }
+
+    @Override
     public TimeDomain timeDomain(DoFn<InputT, OutputT> doFn) {
       throw new UnsupportedOperationException(
           "Cannot access time domain outside of @ProcessTimer method.");
@@ -374,6 +381,12 @@
       throw new UnsupportedOperationException(
           "Cannot access timers outside of @ProcessElement and @OnTimer methods.");
     }
+
+    @Override
+    public TimerMap timerFamily(String tagId) {
+      throw new UnsupportedOperationException(
+          "Cannot access timer family outside of @ProcessElement and @OnTimer methods");
+    }
   }
 
   /** B A concrete implementation of {@link DoFn.FinishBundleContext}. */
@@ -448,6 +461,12 @@
     }
 
     @Override
+    public String timerId(DoFn<InputT, OutputT> doFn) {
+      throw new UnsupportedOperationException(
+          "Cannot access timerId as parameter outside of @OnTimer method.");
+    }
+
+    @Override
     public TimeDomain timeDomain(DoFn<InputT, OutputT> doFn) {
       throw new UnsupportedOperationException(
           "Cannot access time domain outside of @ProcessTimer method.");
@@ -496,6 +515,12 @@
     }
 
     @Override
+    public TimerMap timerFamily(String tagId) {
+      throw new UnsupportedOperationException(
+          "Cannot access timerFamily outside of @ProcessElement and @OnTimer methods.");
+    }
+
+    @Override
     public void output(OutputT output, Instant timestamp, BoundedWindow window) {
       output(mainOutputTag, output, timestamp, window);
     }
@@ -668,6 +693,12 @@
     }
 
     @Override
+    public String timerId(DoFn<InputT, OutputT> doFn) {
+      throw new UnsupportedOperationException(
+          "Cannot access timerId as parameter outside of @OnTimer method.");
+    }
+
+    @Override
     public TimeDomain timeDomain(DoFn<InputT, OutputT> doFn) {
       throw new UnsupportedOperationException(
           "Cannot access time domain outside of @ProcessTimer method.");
@@ -722,6 +753,18 @@
         throw new RuntimeException(e);
       }
     }
+
+    @Override
+    public TimerMap timerFamily(String timerFamilyId) {
+      try {
+        TimerSpec spec =
+            (TimerSpec) signature.timerFamilyDeclarations().get(timerFamilyId).field().get(fn);
+        return new TimerInternalsTimerMap(
+            timerFamilyId, window(), getNamespace(), spec, stepContext.timerInternals());
+      } catch (IllegalAccessException e) {
+        throw new RuntimeException(e);
+      }
+    }
   }
 
   /**
@@ -733,6 +776,7 @@
     private final BoundedWindow window;
     private final Instant timestamp;
     private final TimeDomain timeDomain;
+    private final String timerId;
 
     /** Lazily initialized; should only be accessed via {@link #getNamespace()}. */
     private @Nullable StateNamespace namespace;
@@ -752,8 +796,9 @@
     }
 
     private OnTimerArgumentProvider(
-        BoundedWindow window, Instant timestamp, TimeDomain timeDomain) {
+        String timerId, BoundedWindow window, Instant timestamp, TimeDomain timeDomain) {
       fn.super();
+      this.timerId = timerId;
       this.window = window;
       this.timestamp = timestamp;
       this.timeDomain = timeDomain;
@@ -822,6 +867,11 @@
     }
 
     @Override
+    public String timerId(DoFn<InputT, OutputT> doFn) {
+      return timerId;
+    }
+
+    @Override
     public TimeDomain timeDomain(DoFn<InputT, OutputT> doFn) {
       return timeDomain();
     }
@@ -876,6 +926,18 @@
     }
 
     @Override
+    public TimerMap timerFamily(String timerFamilyId) {
+      try {
+        TimerSpec spec =
+            (TimerSpec) signature.timerFamilyDeclarations().get(timerFamilyId).field().get(fn);
+        return new TimerInternalsTimerMap(
+            timerFamilyId, window(), getNamespace(), spec, stepContext.timerInternals());
+      } catch (IllegalAccessException e) {
+        throw new RuntimeException(e);
+      }
+    }
+
+    @Override
     public PipelineOptions getPipelineOptions() {
       return options;
     }
@@ -910,6 +972,7 @@
     private final BoundedWindow window;
     private final StateNamespace namespace;
     private final String timerId;
+    private final String timerFamilyId;
     private final TimerSpec spec;
     private Instant target;
     private Instant outputTimestamp;
@@ -925,6 +988,22 @@
       this.window = window;
       this.namespace = namespace;
       this.timerId = timerId;
+      this.timerFamilyId = "";
+      this.spec = spec;
+      this.timerInternals = timerInternals;
+    }
+
+    public TimerInternalsTimer(
+        BoundedWindow window,
+        StateNamespace namespace,
+        String timerId,
+        String timerFamilyId,
+        TimerSpec spec,
+        TimerInternals timerInternals) {
+      this.window = window;
+      this.namespace = namespace;
+      this.timerId = timerId;
+      this.timerFamilyId = timerFamilyId;
       this.spec = spec;
       this.timerInternals = timerInternals;
     }
@@ -1029,7 +1108,7 @@
      */
     private void setUnderlyingTimer() {
       timerInternals.setTimer(
-          namespace, timerId, "", target, outputTimestamp, spec.getTimeDomain());
+          namespace, timerId, timerFamilyId, target, outputTimestamp, spec.getTimeDomain());
     }
 
     private Instant getCurrentTime() {
@@ -1046,4 +1125,46 @@
       }
     }
   }
+
+  private class TimerInternalsTimerMap implements TimerMap {
+
+    Map<String, Timer> timers = new HashMap<>();
+    private final TimerInternals timerInternals;
+    private final BoundedWindow window;
+    private final StateNamespace namespace;
+    private final TimerSpec spec;
+    private final String timerFamilyId;
+
+    public TimerInternalsTimerMap(
+        String timerFamilyId,
+        BoundedWindow window,
+        StateNamespace namespace,
+        TimerSpec spec,
+        TimerInternals timerInternals) {
+      this.window = window;
+      this.namespace = namespace;
+      this.spec = spec;
+      this.timerInternals = timerInternals;
+      this.timerFamilyId = timerFamilyId;
+    }
+
+    @Override
+    public void set(String timerId, Instant absoluteTime) {
+      Timer timer =
+          new TimerInternalsTimer(window, namespace, timerId, timerFamilyId, spec, timerInternals);
+      timer.set(absoluteTime);
+      timers.put(timerId, timer);
+    }
+
+    @Override
+    public Timer get(String timerId) {
+      if (timers.get(timerId) == null) {
+        Timer timer =
+            new TimerInternalsTimer(
+                window, namespace, timerId, timerFamilyId, spec, timerInternals);
+        timers.put(timerId, timer);
+      }
+      return timers.get(timerId);
+    }
+  }
 }
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/TimerInternals.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/TimerInternals.java
index e9083d7..5ea3fb7 100644
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/TimerInternals.java
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/TimerInternals.java
@@ -80,7 +80,7 @@
 
   /** @deprecated use {@link #deleteTimer(StateNamespace, String, TimeDomain)}. */
   @Deprecated
-  void deleteTimer(StateNamespace namespace, String timerId);
+  void deleteTimer(StateNamespace namespace, String timerId, String timerFamilyId);
 
   /** @deprecated use {@link #deleteTimer(StateNamespace, String, TimeDomain)}. */
   @Deprecated
@@ -195,7 +195,7 @@
         Instant outputTimestamp,
         TimeDomain domain) {
       return new AutoValue_TimerInternals_TimerData(
-          timerId, timerId, namespace, timestamp, outputTimestamp, domain);
+          timerId, "", namespace, timestamp, outputTimestamp, domain);
     }
 
     /**
@@ -221,7 +221,17 @@
     public static TimerData of(
         String timerId, StateNamespace namespace, Instant timestamp, TimeDomain domain) {
       return new AutoValue_TimerInternals_TimerData(
-          timerId, timerId, namespace, timestamp, timestamp, domain);
+          timerId, "", namespace, timestamp, timestamp, domain);
+    }
+
+    public static TimerData of(
+        String timerId,
+        String timerFamilyId,
+        StateNamespace namespace,
+        Instant timestamp,
+        TimeDomain domain) {
+      return new AutoValue_TimerInternals_TimerData(
+          timerId, timerFamilyId, namespace, timestamp, timestamp, domain);
     }
 
     /**
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 dd106329..3ab7932 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
@@ -71,15 +71,15 @@
     Instant laterTimestamp = new Instant(42);
 
     underTest.advanceInputWatermark(new Instant(0));
-    underTest.setTimer(NS1, ID1, ID1, earlyTimestamp, earlyTimestamp, TimeDomain.EVENT_TIME);
-    underTest.setTimer(NS1, ID1, ID1, laterTimestamp, laterTimestamp, TimeDomain.EVENT_TIME);
+    underTest.setTimer(NS1, ID1, "", earlyTimestamp, earlyTimestamp, TimeDomain.EVENT_TIME);
+    underTest.setTimer(NS1, ID1, "", laterTimestamp, laterTimestamp, TimeDomain.EVENT_TIME);
     underTest.advanceInputWatermark(earlyTimestamp.plus(1L));
     assertThat(underTest.removeNextEventTimer(), nullValue());
 
     underTest.advanceInputWatermark(laterTimestamp.plus(1L));
     assertThat(
         underTest.removeNextEventTimer(),
-        equalTo(TimerData.of(ID1, NS1, laterTimestamp, TimeDomain.EVENT_TIME)));
+        equalTo(TimerData.of(ID1, "", NS1, laterTimestamp, TimeDomain.EVENT_TIME)));
   }
 
   @Test
@@ -87,8 +87,8 @@
     InMemoryTimerInternals underTest = new InMemoryTimerInternals();
     Instant timestamp = new Instant(42);
     underTest.setTimer(NS1, ID1, ID1, timestamp, timestamp, TimeDomain.EVENT_TIME);
-    underTest.deleteTimer(NS1, ID1);
-    underTest.deleteTimer(NS1, ID1);
+    underTest.deleteTimer(NS1, ID1, ID1);
+    underTest.deleteTimer(NS1, ID1, ID1);
   }
 
   @Test
@@ -98,7 +98,7 @@
 
     underTest.advanceInputWatermark(new Instant(0));
     underTest.setTimer(NS1, ID1, ID1, timestamp, timestamp, TimeDomain.EVENT_TIME);
-    underTest.deleteTimer(NS1, ID1);
+    underTest.deleteTimer(NS1, ID1, ID1);
     underTest.advanceInputWatermark(new Instant(43));
 
     assertThat(underTest.removeNextEventTimer(), nullValue());
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 90bb5aa..f3f1628 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
@@ -244,7 +244,7 @@
     // the method call.
     runner.onTimer(
         DoFnWithTimers.TIMER_ID,
-        DoFnWithTimers.TIMER_ID,
+        "",
         GlobalWindow.INSTANCE,
         currentTime.plus(offset),
         currentTime.plus(offset),
@@ -255,7 +255,7 @@
         contains(
             TimerData.of(
                 DoFnWithTimers.TIMER_ID,
-                DoFnWithTimers.TIMER_ID,
+                "",
                 StateNamespaces.window(windowFn.windowCoder(), GlobalWindow.INSTANCE),
                 currentTime.plus(offset),
                 currentTime.plus(offset),
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 10ac7a8..8f88d41 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
@@ -284,12 +284,7 @@
     // Mocking is not easily compatible with annotation analysis, so we manually record
     // the method call.
     runner.onTimer(
-        timerId,
-        timerId,
-        window,
-        new Instant(timestamp),
-        new Instant(timestamp),
-        TimeDomain.EVENT_TIME);
+        timerId, "", window, new Instant(timestamp), new Instant(timestamp), TimeDomain.EVENT_TIME);
 
     assertThat(
         underlying.firedTimers,
diff --git a/runners/direct-java/build.gradle b/runners/direct-java/build.gradle
index b8836a8..6d46528 100644
--- a/runners/direct-java/build.gradle
+++ b/runners/direct-java/build.gradle
@@ -65,7 +65,7 @@
     compile project(it)
   }
   shadow project(path: ":sdks:java:core", configuration: "shadow")
-  shadow library.java.vendored_grpc_1_21_0
+  shadow library.java.vendored_grpc_1_26_0
   shadow library.java.joda_time
   shadow library.java.slf4j_api
   shadow library.java.args4j
diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectTimerInternals.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectTimerInternals.java
index 0261bf6..7b6dbf1 100644
--- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectTimerInternals.java
+++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectTimerInternals.java
@@ -75,7 +75,7 @@
   /** @deprecated use {@link #deleteTimer(StateNamespace, String, TimeDomain)}. */
   @Deprecated
   @Override
-  public void deleteTimer(StateNamespace namespace, String timerId) {
+  public void deleteTimer(StateNamespace namespace, String timerId, String timerFamilyId) {
     throw new UnsupportedOperationException("Canceling of timer by ID is not yet supported.");
   }
 
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 f99d07f..f30ee41 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
@@ -103,7 +103,8 @@
     if (signature.processElement().isSplittable()) {
       return SplittableParDo.forAppliedParDo((AppliedPTransform) application);
     } else if (signature.stateDeclarations().size() > 0
-        || signature.timerDeclarations().size() > 0) {
+        || signature.timerDeclarations().size() > 0
+        || signature.timerFamilyDeclarations().size() > 0) {
       return new GbkThenStatefulParDo(
           fn,
           ParDoTranslation.getMainOutputTag(application),
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 8972319..0dd826d 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
@@ -358,6 +358,7 @@
   private static StateTag<WatermarkHoldState> setTimerTag(TimerData timerData) {
     return StateTags.makeSystemTagInternal(
         StateTags.watermarkStateInternal(
-            "timer-" + timerData.getTimerId(), TimestampCombiner.EARLIEST));
+            "timer-" + timerData.getTimerId() + "+" + timerData.getTimerFamilyId(),
+            TimestampCombiner.EARLIEST));
   }
 }
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 5df3bb5..265ebb1 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
@@ -342,7 +342,8 @@
         if (TimeDomain.EVENT_TIME.equals(timer.getDomain())) {
           @Nullable
           TimerData existingTimer =
-              existingTimersForKey.get(timer.getNamespace(), timer.getTimerId());
+              existingTimersForKey.get(
+                  timer.getNamespace(), timer.getTimerId() + '+' + timer.getTimerFamilyId());
 
           if (existingTimer == null) {
             pendingTimers.add(timer);
@@ -357,7 +358,8 @@
             keyTimers.add(timer);
           }
 
-          existingTimersForKey.put(timer.getNamespace(), timer.getTimerId(), timer);
+          existingTimersForKey.put(
+              timer.getNamespace(), timer.getTimerId() + '+' + timer.getTimerFamilyId(), timer);
         }
       }
 
@@ -365,12 +367,15 @@
         if (TimeDomain.EVENT_TIME.equals(timer.getDomain())) {
           @Nullable
           TimerData existingTimer =
-              existingTimersForKey.get(timer.getNamespace(), timer.getTimerId());
+              existingTimersForKey.get(
+                  timer.getNamespace(), timer.getTimerId() + '+' + timer.getTimerFamilyId());
 
           if (existingTimer != null) {
             pendingTimers.remove(existingTimer);
             keyTimers.remove(existingTimer);
-            existingTimersForKey.remove(existingTimer.getNamespace(), existingTimer.getTimerId());
+            existingTimersForKey.remove(
+                existingTimer.getNamespace(),
+                existingTimer.getTimerId() + '+' + existingTimer.getTimerFamilyId());
           }
         }
       }
@@ -619,7 +624,9 @@
 
         @Nullable
         TimerData existingTimer =
-            existingTimersForKey.get(addedTimer.getNamespace(), addedTimer.getTimerId());
+            existingTimersForKey.get(
+                addedTimer.getNamespace(),
+                addedTimer.getTimerId() + '+' + addedTimer.getTimerFamilyId());
         if (existingTimer == null) {
           timerQueue.add(addedTimer);
         } else if (!existingTimer.equals(addedTimer)) {
@@ -627,7 +634,10 @@
           timerQueue.add(addedTimer);
         } // else the timer is already set identically, so noop.
 
-        existingTimersForKey.put(addedTimer.getNamespace(), addedTimer.getTimerId(), addedTimer);
+        existingTimersForKey.put(
+            addedTimer.getNamespace(),
+            addedTimer.getTimerId() + '+' + addedTimer.getTimerFamilyId(),
+            addedTimer);
       }
 
       for (TimerData deletedTimer : update.deletedTimers) {
@@ -638,12 +648,16 @@
 
         @Nullable
         TimerData existingTimer =
-            existingTimersForKey.get(deletedTimer.getNamespace(), deletedTimer.getTimerId());
+            existingTimersForKey.get(
+                deletedTimer.getNamespace(),
+                deletedTimer.getTimerId() + '+' + deletedTimer.getTimerFamilyId());
 
         if (existingTimer != null) {
           pendingTimers.remove(deletedTimer);
           timerQueue.remove(deletedTimer);
-          existingTimersForKey.remove(existingTimer.getNamespace(), existingTimer.getTimerId());
+          existingTimersForKey.remove(
+              existingTimer.getNamespace(),
+              existingTimer.getTimerId() + '+' + existingTimer.getTimerFamilyId());
         }
       }
 
@@ -957,7 +971,7 @@
       Map<ExecutableT, Set<String>> transformsWithAlreadyExtractedTimers, ExecutableT executable) {
 
     return update -> {
-      String timerIdWithNs = TimerUpdate.getTimerIdWithNamespace(update);
+      String timerIdWithNs = TimerUpdate.getTimerIdAndTimerFamilyIdWithNamespace(update);
       transformsWithAlreadyExtractedTimers.compute(
           executable,
           (k, v) -> {
@@ -1229,7 +1243,8 @@
                     v = new HashSet<>();
                   }
                   final Set<String> toUpdate = v;
-                  newTimers.forEach(td -> toUpdate.add(TimerUpdate.getTimerIdWithNamespace(td)));
+                  newTimers.forEach(
+                      td -> toUpdate.add(TimerUpdate.getTimerIdAndTimerFamilyIdWithNamespace(td)));
                   return v;
                 });
             allTimers.addAll(firedTimers);
@@ -1584,11 +1599,13 @@
 
     private static Map<String, TimerData> indexTimerData(Iterable<? extends TimerData> timerData) {
       return StreamSupport.stream(timerData.spliterator(), false)
-          .collect(Collectors.toMap(TimerUpdate::getTimerIdWithNamespace, e -> e, (a, b) -> b));
+          .collect(
+              Collectors.toMap(
+                  TimerUpdate::getTimerIdAndTimerFamilyIdWithNamespace, e -> e, (a, b) -> b));
     }
 
-    private static String getTimerIdWithNamespace(TimerData td) {
-      return td.getNamespace() + td.getTimerId();
+    private static String getTimerIdAndTimerFamilyIdWithNamespace(TimerData td) {
+      return td.getNamespace() + td.getTimerId() + td.getTimerFamilyId();
     }
 
     private TimerUpdate(
@@ -1645,7 +1662,7 @@
       Set<TimerData> pushedBack = Sets.newHashSet(pushedBackTimers);
       Map<String, TimerData> newSetTimers = indexTimerData(setTimers);
       for (TimerData td : completedTimers) {
-        String timerIdWithNs = getTimerIdWithNamespace(td);
+        String timerIdWithNs = getTimerIdAndTimerFamilyIdWithNamespace(td);
         if (!pushedBack.contains(td)) {
           timersToComplete.add(td);
         } else if (!newSetTimers.containsKey(timerIdWithNs)) {
diff --git a/runners/flink/flink_runner.gradle b/runners/flink/flink_runner.gradle
index ef6c82f..1722a6a 100644
--- a/runners/flink/flink_runner.gradle
+++ b/runners/flink/flink_runner.gradle
@@ -139,7 +139,7 @@
   compile project(":runners:core-construction-java")
   compile project(":runners:java-fn-execution")
   compile project(":sdks:java:extensions:google-cloud-platform-core")
-  compile library.java.vendored_grpc_1_21_0
+  compile library.java.vendored_grpc_1_26_0
   compile library.java.jackson_annotations
   compile library.java.slf4j_api
   compile library.java.joda_time
@@ -200,6 +200,7 @@
     useJUnit {
       includeCategories 'org.apache.beam.sdk.testing.ValidatesRunner'
       excludeCategories 'org.apache.beam.sdk.testing.UsesTestStreamWithOutputTimestamp'
+      excludeCategories 'org.apache.beam.sdk.testing.UsesTimerMap'
       excludeCategories 'org.apache.beam.sdk.testing.FlattenWithHeterogeneousCoders'
       excludeCategories 'org.apache.beam.sdk.testing.LargeKeys$Above100MB'
       excludeCategories 'org.apache.beam.sdk.testing.UsesCommittedMetrics'
diff --git a/runners/flink/job-server/flink_job_server.gradle b/runners/flink/job-server/flink_job_server.gradle
index 27a116f..12b4d65 100644
--- a/runners/flink/job-server/flink_job_server.gradle
+++ b/runners/flink/job-server/flink_job_server.gradle
@@ -151,6 +151,7 @@
       if (streaming) {
         excludeCategories 'org.apache.beam.sdk.testing.UsesTestStreamWithProcessingTime'
         excludeCategories 'org.apache.beam.sdk.testing.UsesTestStreamWithMultipleStages'
+        excludeCategories 'org.apache.beam.sdk.testing.UsesTestStreamWithOutputTimestamp'
       } else {
         excludeCategories 'org.apache.beam.sdk.testing.UsesTestStream'
       }
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 ee40fb6..84a2e05 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
@@ -75,7 +75,7 @@
 import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.WindowingStrategy;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.InvalidProtocolBufferException;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.BiMap;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables;
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 40257b9..67f664e 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
@@ -30,7 +30,7 @@
 import org.apache.beam.runners.fnexecution.jobsubmission.PortablePipelineRunner;
 import org.apache.beam.runners.fnexecution.provisioning.JobInfo;
 import org.apache.beam.sdk.options.PortablePipelineOptions;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.Struct;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.Struct;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Strings;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.util.concurrent.ListeningExecutorService;
 import org.slf4j.Logger;
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 137aa00..f46bf7b 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
@@ -45,7 +45,7 @@
 import org.apache.beam.sdk.options.PipelineOptionsFactory;
 import org.apache.beam.sdk.options.PortablePipelineOptions;
 import org.apache.beam.sdk.options.PortablePipelineOptions.RetrievalServiceType;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.Struct;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.Struct;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions;
 import org.apache.flink.api.common.JobExecutionResult;
 import org.apache.flink.client.program.DetachedEnvironment;
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 92b07a4..cbc437b 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
@@ -90,7 +90,7 @@
 import org.apache.beam.sdk.values.TypeDescriptors;
 import org.apache.beam.sdk.values.ValueWithRecordId;
 import org.apache.beam.sdk.values.WindowingStrategy;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.InvalidProtocolBufferException;
+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.BiMap;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.HashMultiset;
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 b7c650c..1f7fe00 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
@@ -1162,7 +1162,7 @@
     /** @deprecated use {@link #deleteTimer(StateNamespace, String, TimeDomain)}. */
     @Deprecated
     @Override
-    public void deleteTimer(StateNamespace namespace, String timerId) {
+    public void deleteTimer(StateNamespace namespace, String timerId, String timerFamilyId) {
       throw new UnsupportedOperationException("Canceling of a timer by ID is not yet supported.");
     }
 
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 45fc2a1..23a2a4b 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
@@ -87,7 +87,7 @@
 import org.apache.beam.sdk.values.PCollectionView;
 import org.apache.beam.sdk.values.TupleTag;
 import org.apache.beam.sdk.values.WindowingStrategy;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
 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.base.Preconditions;
 import org.apache.beam.vendor.sdk.v2.sdk.extensions.protobuf.ByteStringCoder;
diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkKeyUtils.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkKeyUtils.java
index ccd10d4..3e27bb1 100644
--- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkKeyUtils.java
+++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkKeyUtils.java
@@ -32,7 +32,7 @@
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.coders.StructuredCoder;
 import org.apache.beam.sdk.util.CoderUtils;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
 
 /**
  * Utility functions for dealing with key encoding. Beam requires keys to be compared in binary
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkJobInvokerTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkJobInvokerTest.java
index 28c2cde..6397d4a 100644
--- a/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkJobInvokerTest.java
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkJobInvokerTest.java
@@ -23,7 +23,7 @@
 import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.io.GenerateSequence;
 import org.apache.beam.sdk.options.PipelineOptionsFactory;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.Struct;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.Struct;
 import org.apache.flink.client.program.OptimizerPlanEnvironment;
 import org.apache.flink.client.program.PackagedProgram;
 import org.apache.flink.client.program.ProgramInvocationException;
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 61d8906..fb79841 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
@@ -45,7 +45,7 @@
 import org.apache.beam.sdk.options.PipelineOptionsFactory;
 import org.apache.beam.sdk.transforms.join.RawUnionValue;
 import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.Struct;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.Struct;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
 import org.apache.flink.api.common.cache.DistributedCache;
 import org.apache.flink.api.common.functions.RuntimeContext;
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 e3b46e6..3c5f44b 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
@@ -96,8 +96,8 @@
 import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.TupleTag;
 import org.apache.beam.sdk.values.WindowingStrategy;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.Struct;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.Struct;
 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.ImmutableList;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkKeyUtilsTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkKeyUtilsTest.java
index 274b2bf..817b5e8 100644
--- a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkKeyUtilsTest.java
+++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkKeyUtilsTest.java
@@ -26,7 +26,7 @@
 import org.apache.beam.sdk.coders.StringUtf8Coder;
 import org.apache.beam.sdk.coders.VoidCoder;
 import org.apache.beam.sdk.util.CoderUtils;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Charsets;
 import org.junit.Test;
 
diff --git a/runners/google-cloud-dataflow-java/build.gradle b/runners/google-cloud-dataflow-java/build.gradle
index d8ddffa..30e91930 100644
--- a/runners/google-cloud-dataflow-java/build.gradle
+++ b/runners/google-cloud-dataflow-java/build.gradle
@@ -81,8 +81,9 @@
   compile library.java.jackson_databind
   compile library.java.joda_time
   compile library.java.slf4j_api
-  compile library.java.vendored_grpc_1_21_0
+  compile library.java.vendored_grpc_1_26_0
   testCompile library.java.hamcrest_core
+  testCompile library.java.guava_testlib
   testCompile library.java.junit
   testCompile project(path: ":sdks:java:io:google-cloud-platform", configuration: "testRuntime")
   testCompile project(path: ":sdks:java:core", configuration: "shadowTest")
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 fa45b7f..0d100ad 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
@@ -108,8 +108,8 @@
 import org.apache.beam.sdk.values.TimestampedValue;
 import org.apache.beam.sdk.values.TupleTag;
 import org.apache.beam.sdk.values.WindowingStrategy;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.TextFormat;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.TextFormat;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Supplier;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables;
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 4db0e3d..eb42be2 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
@@ -18,9 +18,11 @@
 package org.apache.beam.runners.dataflow;
 
 import static org.apache.beam.runners.core.construction.PTransformTranslation.PAR_DO_TRANSFORM_URN;
+import static org.apache.beam.runners.core.construction.ParDoTranslation.translateTimerFamilySpec;
 import static org.apache.beam.runners.core.construction.ParDoTranslation.translateTimerSpec;
 import static org.apache.beam.sdk.options.ExperimentalOptions.hasExperiment;
 import static org.apache.beam.sdk.transforms.reflect.DoFnSignatures.getStateSpecOrThrow;
+import static org.apache.beam.sdk.transforms.reflect.DoFnSignatures.getTimerFamilySpecOrThrow;
 import static org.apache.beam.sdk.transforms.reflect.DoFnSignatures.getTimerSpecOrThrow;
 import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument;
 
@@ -242,6 +244,20 @@
             }
 
             @Override
+            public Map<String, RunnerApi.TimerFamilySpec> translateTimerFamilySpecs(
+                SdkComponents newComponents) {
+              Map<String, RunnerApi.TimerFamilySpec> timerFamilySpecs = new HashMap<>();
+              for (Map.Entry<String, DoFnSignature.TimerFamilyDeclaration> timerFamily :
+                  signature.timerFamilyDeclarations().entrySet()) {
+                RunnerApi.TimerFamilySpec spec =
+                    translateTimerFamilySpec(
+                        getTimerFamilySpecOrThrow(timerFamily.getValue(), doFn), newComponents);
+                timerFamilySpecs.put(timerFamily.getKey(), spec);
+              }
+              return timerFamilySpecs;
+            }
+
+            @Override
             public boolean isSplittable() {
               return signature.processElement().isSplittable();
             }
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 e341004..8e8589d 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
@@ -22,6 +22,7 @@
 import com.google.api.client.json.GenericJson;
 import com.google.api.client.util.Key;
 import java.util.Map;
+import java.util.Objects;
 import javax.annotation.Nullable;
 
 /**
@@ -182,4 +183,18 @@
   public CloudObject clone() {
     return (CloudObject) super.clone();
   }
+
+  @Override
+  public boolean equals(Object otherObject) {
+    if (!(otherObject instanceof CloudObject)) {
+      return false;
+    }
+    CloudObject other = (CloudObject) otherObject;
+    return Objects.equals(className, other.className) && super.equals(otherObject);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(className, super.hashCode());
+  }
 }
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/OutputReference.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/OutputReference.java
index f8b7784..7c1b9e4f 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/OutputReference.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/OutputReference.java
@@ -21,6 +21,7 @@
 
 import com.google.api.client.json.GenericJson;
 import com.google.api.client.util.Key;
+import java.util.Objects;
 
 /**
  * A representation used by {@link com.google.api.services.dataflow.model.Step}s to reference the
@@ -40,4 +41,21 @@
     this.stepName = checkNotNull(stepName);
     this.outputName = checkNotNull(outputName);
   }
+
+  @Override
+  public boolean equals(Object otherObject) {
+    if (!(otherObject instanceof OutputReference)) {
+      return false;
+    }
+    OutputReference other = (OutputReference) otherObject;
+    return Objects.equals(type, other.type)
+        && Objects.equals(stepName, other.stepName)
+        && Objects.equals(outputName, other.outputName)
+        && super.equals(other);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(type, stepName, outputName, super.hashCode());
+  }
 }
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/CloudObjectTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/CloudObjectTest.java
new file mode 100644
index 0000000..118bab8
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/CloudObjectTest.java
@@ -0,0 +1,42 @@
+/*
+ * 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.dataflow.util;
+
+import com.google.common.testing.EqualsTester;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+@RunWith(JUnit4.class)
+public class CloudObjectTest {
+
+  @Test
+  public void testEquality() {
+    new EqualsTester()
+        .addEqualityGroup(CloudObject.forFloat(1.0), CloudObject.forFloat(1.0))
+        .addEqualityGroup(CloudObject.forInteger(3L), CloudObject.forInteger(3L))
+        .addEqualityGroup(CloudObject.forFloat(3.0))
+        .addEqualityGroup(CloudObject.forString("foo"), CloudObject.forString("foo"))
+        .addEqualityGroup(CloudObject.forClassName("foo.Bar"), CloudObject.forClassName("foo.Bar"))
+        .addEqualityGroup(
+            CloudObject.fromSpec(ImmutableMap.of(PropertyNames.OBJECT_TYPE_NAME, "ValuesDoFn")),
+            CloudObject.fromSpec(ImmutableMap.of(PropertyNames.OBJECT_TYPE_NAME, "ValuesDoFn")))
+        .testEquals();
+  }
+}
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/OutputReferenceTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/OutputReferenceTest.java
new file mode 100644
index 0000000..f0817b5
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/OutputReferenceTest.java
@@ -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 org.apache.beam.runners.dataflow.util;
+
+import com.google.common.testing.EqualsTester;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+@RunWith(JUnit4.class)
+public class OutputReferenceTest {
+
+  @Test
+  public void testEquality() {
+    new EqualsTester()
+        .addEqualityGroup(new OutputReference("sA", "oA"), new OutputReference("sA", "oA"))
+        .addEqualityGroup(new OutputReference("sB", "oB"), new OutputReference("sB", "oB"))
+        .testEquals();
+  }
+}
diff --git a/runners/google-cloud-dataflow-java/worker/build.gradle b/runners/google-cloud-dataflow-java/worker/build.gradle
index 6b866d7..a0e3356 100644
--- a/runners/google-cloud-dataflow-java/worker/build.gradle
+++ b/runners/google-cloud-dataflow-java/worker/build.gradle
@@ -76,7 +76,7 @@
   compile project(":runners:java-fn-execution")
   compile project(":sdks:java:fn-execution")
   compile project(path: ":runners:google-cloud-dataflow-java:worker:windmill", configuration: "shadow")
-  compile library.java.vendored_grpc_1_21_0
+  compile library.java.vendored_grpc_1_26_0
   compile google_api_services_dataflow
   compile library.java.avro
   compile library.java.google_api_client
diff --git a/runners/google-cloud-dataflow-java/worker/legacy-worker/build.gradle b/runners/google-cloud-dataflow-java/worker/legacy-worker/build.gradle
index 5ee2d63..d2912e4 100644
--- a/runners/google-cloud-dataflow-java/worker/legacy-worker/build.gradle
+++ b/runners/google-cloud-dataflow-java/worker/legacy-worker/build.gradle
@@ -53,7 +53,7 @@
         library.java.jackson_databind,
         library.java.joda_time,
         library.java.slf4j_api,
-        library.java.vendored_grpc_1_21_0,
+        library.java.vendored_grpc_1_26_0,
 ]
 
 def sdk_provided_shaded_project_dependencies = [
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 a5fb660..5cc0db7 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
@@ -21,7 +21,7 @@
 import com.google.api.services.dataflow.model.WorkItem;
 import java.io.Closeable;
 import java.io.IOException;
-import java.util.List;
+import java.util.Collection;
 import java.util.function.Function;
 import javax.annotation.Nullable;
 import org.apache.beam.model.fnexecution.v1.BeamFnApi.RemoteGrpcPort;
@@ -46,6 +46,7 @@
 import org.apache.beam.runners.dataflow.worker.graph.RegisterNodeFunction;
 import org.apache.beam.runners.dataflow.worker.graph.ReplacePgbkWithPrecombineFunction;
 import org.apache.beam.runners.dataflow.worker.status.DebugCapture;
+import org.apache.beam.runners.dataflow.worker.status.DebugCapture.Capturable;
 import org.apache.beam.runners.dataflow.worker.status.WorkerStatusPages;
 import org.apache.beam.runners.dataflow.worker.util.MemoryMonitor;
 import org.apache.beam.sdk.fn.IdGenerator;
@@ -265,7 +266,7 @@
   }
 
   private static DebugCapture.Manager initializeAndStartDebugCaptureManager(
-      DataflowWorkerHarnessOptions options, List<DebugCapture.Capturable> debugCapturePages) {
+      DataflowWorkerHarnessOptions options, Collection<Capturable> debugCapturePages) {
     DebugCapture.Manager result = new DebugCapture.Manager(options, debugCapturePages);
     result.start();
     return result;
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ByteStringCoder.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ByteStringCoder.java
index 3d9a2c4..eab920f 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ByteStringCoder.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ByteStringCoder.java
@@ -23,7 +23,7 @@
 import org.apache.beam.sdk.coders.AtomicCoder;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.util.VarInt;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.io.ByteStreams;
 
 /**
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 eda6b03..e3de7cb 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
@@ -33,12 +33,14 @@
 import org.apache.beam.runners.dataflow.worker.fn.logging.BeamFnLoggingService;
 import org.apache.beam.runners.dataflow.worker.fn.stream.ServerStreamObserverFactory;
 import org.apache.beam.runners.dataflow.worker.logging.DataflowWorkerLoggingInitializer;
+import org.apache.beam.runners.dataflow.worker.status.SdkWorkerStatusServlet;
 import org.apache.beam.runners.fnexecution.GrpcContextHeaderAccessorProvider;
 import org.apache.beam.runners.fnexecution.ServerFactory;
 import org.apache.beam.runners.fnexecution.control.FnApiControlClient;
 import org.apache.beam.runners.fnexecution.state.GrpcStateService;
+import org.apache.beam.runners.fnexecution.status.BeamWorkerStatusGrpcService;
 import org.apache.beam.sdk.io.FileSystems;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.Server;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.Server;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -62,6 +64,7 @@
     // critical traffic protected from best effort traffic.
     ApiServiceDescriptor controlApiService = DataflowWorkerHarnessHelper.getControlDescriptor();
     ApiServiceDescriptor loggingApiService = DataflowWorkerHarnessHelper.getLoggingDescriptor();
+    ApiServiceDescriptor statusApiService = DataflowWorkerHarnessHelper.getStatusDescriptor();
 
     LOG.info(
         "{} started, using port {} for control, {} for logging.",
@@ -93,6 +96,7 @@
 
     Server servicesServer = null;
     Server loggingServer = null;
+    Server statusServer = null;
     try (BeamFnLoggingService beamFnLoggingService =
             new BeamFnLoggingService(
                 loggingApiService,
@@ -110,6 +114,11 @@
                 controlApiService,
                 streamObserverFactory::from,
                 GrpcContextHeaderAccessorProvider.getHeaderAccessor());
+        BeamWorkerStatusGrpcService beamWorkerStatusGrpcService =
+            statusApiService == null
+                ? null
+                : BeamWorkerStatusGrpcService.create(
+                    statusApiService, GrpcContextHeaderAccessorProvider.getHeaderAccessor());
         GrpcStateService beamFnStateService = GrpcStateService.create()) {
 
       servicesServer =
@@ -120,22 +129,41 @@
       loggingServer =
           serverFactory.create(ImmutableList.of(beamFnLoggingService), loggingApiService);
 
+      // gRPC server for obtaining SDK harness runtime status information.
+      if (beamWorkerStatusGrpcService != null) {
+        statusServer =
+            serverFactory.create(ImmutableList.of(beamWorkerStatusGrpcService), statusApiService);
+      }
+
       start(
           pipeline,
           pipelineOptions,
           beamFnControlService,
           beamFnDataService,
           controlApiService,
-          beamFnStateService);
+          beamFnStateService,
+          beamWorkerStatusGrpcService);
+
+      if (statusServer != null) {
+        statusServer.shutdown();
+      }
       servicesServer.shutdown();
       loggingServer.shutdown();
+
+      // wait 30 secs for outstanding requests to finish.
+      if (statusServer != null) {
+        statusServer.awaitTermination(30, TimeUnit.SECONDS);
+      }
+      servicesServer.awaitTermination(30, TimeUnit.SECONDS);
+      loggingServer.awaitTermination(30, TimeUnit.SECONDS);
     } finally {
-      if (servicesServer != null) {
-        servicesServer.awaitTermination(30, TimeUnit.SECONDS);
+      if (statusServer != null && !statusServer.isTerminated()) {
+        statusServer.shutdownNow();
+      }
+      if (servicesServer != null && !servicesServer.isTerminated()) {
         servicesServer.shutdownNow();
       }
-      if (loggingServer != null) {
-        loggingServer.awaitTermination(30, TimeUnit.SECONDS);
+      if (loggingServer != null && !loggingServer.isTerminated()) {
         loggingServer.shutdownNow();
       }
     }
@@ -148,7 +176,8 @@
       BeamFnControlService beamFnControlService,
       BeamFnDataGrpcService beamFnDataService,
       ApiServiceDescriptor stateApiServiceDescriptor,
-      GrpcStateService beamFnStateService)
+      GrpcStateService beamFnStateService,
+      BeamWorkerStatusGrpcService beamWorkerStatusGrpcService)
       throws Exception {
 
     SdkHarnessRegistry sdkHarnessRegistry =
@@ -161,6 +190,12 @@
       StreamingDataflowWorker worker =
           StreamingDataflowWorker.forStreamingFnWorkerHarness(
               Collections.emptyList(), client, pipelineOptions, pipeline, sdkHarnessRegistry);
+      // Add SDK status servlet and capture page only if Fn worker status server is started.
+      if (beamWorkerStatusGrpcService != null) {
+        SdkWorkerStatusServlet sdkWorkerStatusServlet =
+            new SdkWorkerStatusServlet(beamWorkerStatusGrpcService);
+        worker.addWorkerStatusPage(sdkWorkerStatusServlet);
+      }
       worker.startStatusPages();
       worker.start();
       ExecutorService executor = Executors.newSingleThreadExecutor();
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 5449462..36454db 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
@@ -31,7 +31,7 @@
 import org.apache.beam.runners.dataflow.worker.ExperimentContext.Experiment;
 import org.apache.beam.runners.dataflow.worker.logging.DataflowWorkerLoggingInitializer;
 import org.apache.beam.runners.dataflow.worker.logging.DataflowWorkerLoggingMDC;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.TextFormat;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.TextFormat;
 import org.conscrypt.OpenSSLProvider;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
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 b12f889..bef1466 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
@@ -58,7 +58,7 @@
 import org.apache.beam.sdk.util.WindowedValue.WindowedValueCoder;
 import org.apache.beam.sdk.values.TupleTag;
 import org.apache.beam.sdk.values.WindowingStrategy;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.InvalidProtocolBufferException;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
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 734f49e..f08bc80 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
@@ -30,7 +30,7 @@
 import org.apache.beam.runners.dataflow.worker.windmill.Windmill.KeyedGetDataRequest;
 import org.apache.beam.runners.dataflow.worker.windmill.WindmillServerStub;
 import org.apache.beam.runners.dataflow.worker.windmill.WindmillServerStub.GetDataStream;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.util.concurrent.SettableFuture;
 import org.joda.time.Duration;
 
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 147fd76..4cc73a1 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
@@ -36,7 +36,7 @@
 import org.apache.beam.sdk.util.SerializableUtils;
 import org.apache.beam.sdk.util.WindowedValue;
 import org.apache.beam.sdk.util.WindowedValue.WindowedValueCoder;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
 
 /**
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 6b00560..8b4439d 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
@@ -22,7 +22,7 @@
 import javax.annotation.concurrent.ThreadSafe;
 import org.apache.beam.sdk.io.UnboundedSource;
 import org.apache.beam.sdk.values.KV;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
 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.cache.Cache;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.cache.CacheBuilder;
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 3c804db..1dbbd6c 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
@@ -38,7 +38,7 @@
 import org.apache.beam.sdk.values.PCollectionView;
 import org.apache.beam.sdk.values.TupleTag;
 import org.apache.beam.sdk.values.WindowingStrategy;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
 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.Supplier;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.cache.Cache;
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 d6d017a..f2b0b27 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
@@ -36,6 +36,7 @@
 import java.io.PrintWriter;
 import java.util.ArrayDeque;
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.Deque;
 import java.util.HashMap;
@@ -128,8 +129,8 @@
 import org.apache.beam.sdk.util.Sleeper;
 import org.apache.beam.sdk.util.UserCodeException;
 import org.apache.beam.sdk.util.WindowedValue.WindowedValueCoder;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.TextFormat;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.TextFormat;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.MoreObjects;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Optional;
@@ -818,7 +819,7 @@
           new TimerTask() {
             @Override
             public void run() {
-              List<Capturable> pages = statusPages.getDebugCapturePages();
+              Collection<Capturable> pages = statusPages.getDebugCapturePages();
               if (pages.isEmpty()) {
                 LOG.warn("No captured status pages.");
               }
@@ -871,6 +872,13 @@
     statusPages.start();
   }
 
+  public void addWorkerStatusPage(BaseStatusServlet page) {
+    statusPages.addServlet(page);
+    if (page instanceof Capturable) {
+      statusPages.addCapturePage((Capturable) page);
+    }
+  }
+
   public void stop() {
     try {
       if (globalConfigRefreshTimer != null) {
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 76aa8b0..a761ae4 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
@@ -54,7 +54,7 @@
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
 import org.apache.beam.sdk.values.PCollectionView;
 import org.apache.beam.sdk.values.TupleTag;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
+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.Supplier;
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 2c00c99..eb95afb 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
@@ -48,8 +48,8 @@
 import org.apache.beam.sdk.util.WindowedValue;
 import org.apache.beam.sdk.values.PCollectionView;
 import org.apache.beam.sdk.values.WindowingStrategy;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.Parser;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.Parser;
 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.Iterables;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Lists;
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillNamespacePrefix.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillNamespacePrefix.java
index eba5c5d..2dd7006 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillNamespacePrefix.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillNamespacePrefix.java
@@ -17,7 +17,7 @@
  */
 package org.apache.beam.runners.dataflow.worker;
 
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
 
 /**
  * A prefix for a Windmill state or timer tag to separate user state and timers from system state
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 60ddce5..cd35038 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
@@ -40,7 +40,7 @@
 import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.ValueWithRecordId;
 import org.apache.beam.sdk.values.ValueWithRecordId.ValueWithRecordIdCoder;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
 
 class WindmillSink<T> extends Sink<WindowedValue<T>> {
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 b419a38..eb18ef8 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
@@ -32,7 +32,7 @@
 import org.apache.beam.runners.dataflow.worker.status.StatusDataProvider;
 import org.apache.beam.sdk.state.State;
 import org.apache.beam.sdk.util.Weighted;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Equivalence;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.cache.Cache;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.cache.CacheBuilder;
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 9c9779e..a3619ab 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
@@ -55,7 +55,7 @@
 import org.apache.beam.sdk.transforms.windowing.TimestampCombiner;
 import org.apache.beam.sdk.util.CombineFnUtil;
 import org.apache.beam.sdk.util.Weighted;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
+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.Supplier;
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 0050602..75ee1cb 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
@@ -40,7 +40,7 @@
 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.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
+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;
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 fb33ed3..5d92d2a 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
@@ -29,7 +29,7 @@
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.state.TimeDomain;
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
+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.collect.HashBasedTable;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Table;
@@ -88,8 +88,14 @@
 
   @Override
   public void setTimer(TimerData timerKey) {
-    timers.put(timerKey.getTimerId(), timerKey.getNamespace(), timerKey);
-    timerStillPresent.put(timerKey.getTimerId(), timerKey.getNamespace(), true);
+    timers.put(
+        getTimerDataKey(timerKey.getTimerId(), timerKey.getTimerFamilyId()),
+        timerKey.getNamespace(),
+        timerKey);
+    timerStillPresent.put(
+        getTimerDataKey(timerKey.getTimerId(), timerKey.getTimerFamilyId()),
+        timerKey.getNamespace(),
+        true);
   }
 
   @Override
@@ -101,20 +107,31 @@
       Instant outputTimestamp,
       TimeDomain timeDomain) {
     timers.put(
-        timerId,
+        getTimerDataKey(timerId, timerFamilyId),
         namespace,
         TimerData.of(timerId, timerFamilyId, namespace, timestamp, outputTimestamp, timeDomain));
-    timerStillPresent.put(timerId, namespace, true);
+    timerStillPresent.put(getTimerDataKey(timerId, timerFamilyId), namespace, true);
+  }
+
+  private String getTimerDataKey(String timerId, String timerFamilyId) {
+    // Identifies timer uniquely with timerFamilyId
+    return timerId + '+' + timerFamilyId;
   }
 
   @Override
   public void deleteTimer(TimerData timerKey) {
-    timers.put(timerKey.getTimerId(), timerKey.getNamespace(), timerKey);
-    timerStillPresent.put(timerKey.getTimerId(), timerKey.getNamespace(), false);
+    timers.put(
+        getTimerDataKey(timerKey.getTimerId(), timerKey.getTimerFamilyId()),
+        timerKey.getNamespace(),
+        timerKey);
+    timerStillPresent.put(
+        getTimerDataKey(timerKey.getTimerId(), timerKey.getTimerFamilyId()),
+        timerKey.getNamespace(),
+        false);
   }
 
   @Override
-  public void deleteTimer(StateNamespace namespace, String timerId) {
+  public void deleteTimer(StateNamespace namespace, String timerId, String timerFamilyId) {
     throw new UnsupportedOperationException("Canceling a timer by ID is not yet supported.");
   }
 
@@ -246,7 +263,7 @@
     // The tag is a path-structure string but cheaper to parse than a proper URI. It follows
     // this pattern, where no component but the ID can contain a slash
     //
-    //     prefix namespace '+' id
+    //     prefix namespace '+' id '+' familyId
     //
     //     prefix ::= '/' prefix_char
     //     namespace ::= '/' | '/' window '/'
@@ -269,13 +286,19 @@
         prefix.byteString());
     int namespaceStart = prefix.byteString().size(); // drop the prefix, leave the begin slash
     int namespaceEnd = tag.indexOf('+', namespaceStart); // keep the end slash, drop the +
-
     String namespaceString = tag.substring(namespaceStart, namespaceEnd);
-    String id = tag.substring(namespaceEnd + 1);
+    String timerIdPlusTimerFamilyId = tag.substring(namespaceEnd + 1); // timerId+timerFamilyId
+    int timerIdEnd = timerIdPlusTimerFamilyId.indexOf('+'); // end of timerId
+    // if no '+' found then timerFamilyId is empty string else they have a '+' separator
+    String familyId = timerIdEnd == -1 ? "" : timerIdPlusTimerFamilyId.substring(timerIdEnd + 1);
+    String id =
+        timerIdEnd == -1
+            ? timerIdPlusTimerFamilyId
+            : timerIdPlusTimerFamilyId.substring(0, timerIdEnd);
     StateNamespace namespace = StateNamespaces.fromString(namespaceString, windowCoder);
     Instant timestamp = WindmillTimeUtils.windmillToHarnessTimestamp(timer.getTimestamp());
 
-    return TimerData.of(id, namespace, timestamp, timerTypeToTimeDomain(timer.getType()));
+    return TimerData.of(id, familyId, namespace, timestamp, timerTypeToTimeDomain(timer.getType()));
   }
 
   /**
@@ -285,13 +308,27 @@
    * <p>This is necessary because Windmill will deduplicate based only on this tag.
    */
   public static ByteString timerTag(WindmillNamespacePrefix prefix, TimerData timerData) {
-    String tagString =
-        new StringBuilder()
-            .append(prefix.byteString().toStringUtf8()) // this never ends with a slash
-            .append(timerData.getNamespace().stringKey()) // this must begin and end with a slash
-            .append('+')
-            .append(timerData.getTimerId()) // this is arbitrary; currently unescaped
-            .toString();
+    String tagString;
+    // Timers without timerFamily would have timerFamily would be an empty string
+    if ("".equals(timerData.getTimerFamilyId())) {
+      tagString =
+          new StringBuilder()
+              .append(prefix.byteString().toStringUtf8()) // this never ends with a slash
+              .append(timerData.getNamespace().stringKey()) // this must begin and end with a slash
+              .append('+')
+              .append(timerData.getTimerId()) // this is arbitrary; currently unescaped
+              .toString();
+    } else {
+      tagString =
+          new StringBuilder()
+              .append(prefix.byteString().toStringUtf8()) // this never ends with a slash
+              .append(timerData.getNamespace().stringKey()) // this must begin and end with a slash
+              .append('+')
+              .append(timerData.getTimerId()) // this is arbitrary; currently unescaped
+              .append('+')
+              .append(timerData.getTimerFamilyId())
+              .toString();
+    }
     return ByteString.copyFromUtf8(tagString);
   }
 
@@ -300,14 +337,30 @@
    * hold that is only freed after the timer fires.
    */
   public static ByteString timerHoldTag(WindmillNamespacePrefix prefix, TimerData timerData) {
-    String tagString =
-        new StringBuilder()
-            .append(prefix.byteString().toStringUtf8()) // this never ends with a slash
-            .append(TIMER_HOLD_PREFIX) // this never ends with a slash
-            .append(timerData.getNamespace().stringKey()) // this must begin and end with a slash
-            .append('+')
-            .append(timerData.getTimerId()) // this is arbitrary; currently unescaped
-            .toString();
+    String tagString;
+    if ("".equals(timerData.getTimerFamilyId())) {
+      tagString =
+          new StringBuilder()
+              .append(prefix.byteString().toStringUtf8()) // this never ends with a slash
+              .append(TIMER_HOLD_PREFIX) // this never ends with a slash
+              .append(timerData.getNamespace().stringKey()) // this must begin and end with a slash
+              .append('+')
+              .append(timerData.getTimerId()) // this is arbitrary; currently unescaped
+              .toString();
+    } else {
+      tagString =
+          new StringBuilder()
+              .append(prefix.byteString().toStringUtf8()) // this never ends with a slash
+              .append(TIMER_HOLD_PREFIX) // this never ends with a slash
+              .append(timerData.getNamespace().stringKey()) // this must begin and end with a slash
+              .append('+')
+              .append(timerData.getTimerId()) // this is arbitrary; currently unescaped
+              .append('+')
+              .append(
+                  timerData.getTimerFamilyId()) // use to differentiate same timerId in different
+              // timerMap
+              .toString();
+    }
     return ByteString.copyFromUtf8(tagString);
   }
 
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 4964a89..344798e 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
@@ -61,7 +61,7 @@
 import org.apache.beam.sdk.util.FluentBackoff;
 import org.apache.beam.sdk.util.WindowedValue;
 import org.apache.beam.sdk.values.ValueWithRecordId;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
+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.collect.ImmutableList;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/fn/BeamFnControlService.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/fn/BeamFnControlService.java
index d701083..865d033 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/fn/BeamFnControlService.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/fn/BeamFnControlService.java
@@ -26,7 +26,7 @@
 import org.apache.beam.runners.dataflow.worker.fn.grpc.BeamFnService;
 import org.apache.beam.runners.fnexecution.HeaderAccessor;
 import org.apache.beam.runners.fnexecution.control.FnApiControlClient;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.StreamObserver;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.stub.StreamObserver;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
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 bf42c4d..2879c18 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
@@ -73,8 +73,8 @@
 import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
 import org.apache.beam.sdk.util.MoreFutures;
 import org.apache.beam.sdk.values.PCollectionView;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.TextFormat;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.TextFormat;
 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;
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/fn/data/BeamFnDataGrpcService.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/fn/data/BeamFnDataGrpcService.java
index dcde104..cad6a8e 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/fn/data/BeamFnDataGrpcService.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/fn/data/BeamFnDataGrpcService.java
@@ -40,7 +40,7 @@
 import org.apache.beam.sdk.fn.data.LogicalEndpoint;
 import org.apache.beam.sdk.fn.stream.OutboundObserverFactory;
 import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.StreamObserver;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.stub.StreamObserver;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
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 d1b62d1..2045317 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
@@ -28,8 +28,8 @@
 import org.apache.beam.runners.dataflow.worker.fn.grpc.BeamFnService;
 import org.apache.beam.runners.dataflow.worker.logging.DataflowWorkerLoggingMDC;
 import org.apache.beam.runners.fnexecution.HeaderAccessor;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.Server;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.StreamObserver;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.Server;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.stub.StreamObserver;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableSet;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
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 1fabd6d..e51c0ee 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
@@ -27,9 +27,9 @@
 import org.apache.beam.sdk.fn.stream.ForwardingClientResponseObserver;
 import org.apache.beam.sdk.fn.stream.OutboundObserverFactory;
 import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.CallStreamObserver;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.ServerCallStreamObserver;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.StreamObserver;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.stub.CallStreamObserver;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.stub.ServerCallStreamObserver;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.stub.StreamObserver;
 
 /**
  * A {@link StreamObserver} factory that wraps provided {@link CallStreamObserver}s making them flow
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 fbd10c0..1b888c6 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
@@ -17,7 +17,7 @@
  */
 package org.apache.beam.runners.dataflow.worker.graph;
 
-import static org.apache.beam.runners.core.construction.graph.ExecutableStage.DEFAULT_WIRE_CODER_SETTING;
+import static org.apache.beam.runners.core.construction.graph.ExecutableStage.DEFAULT_WIRE_CODER_SETTINGS;
 import static org.apache.beam.runners.dataflow.util.Structs.getBytes;
 import static org.apache.beam.runners.dataflow.util.Structs.getString;
 import static org.apache.beam.runners.dataflow.worker.graph.LengthPrefixUnknownCoders.forSideInputInfos;
@@ -78,8 +78,8 @@
 import org.apache.beam.sdk.util.WindowedValue.WindowedValueCoder;
 import org.apache.beam.sdk.values.PCollectionView;
 import org.apache.beam.sdk.values.WindowingStrategy;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.InvalidProtocolBufferException;
 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.Iterables;
@@ -486,7 +486,7 @@
             executableStageTimers,
             executableStageTransforms,
             executableStageOutputs,
-            DEFAULT_WIRE_CODER_SETTING);
+            DEFAULT_WIRE_CODER_SETTINGS);
     return ExecutableStageNode.create(
         executableStage,
         ptransformIdToNameContexts.build(),
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 0d96981..83ed105 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
@@ -37,7 +37,7 @@
 import org.apache.beam.sdk.transforms.ParDo;
 import org.apache.beam.sdk.values.PCollectionView;
 import org.apache.beam.sdk.values.WindowingStrategy;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.InvalidProtocolBufferException;
 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.Iterables;
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 7bd66e9..e1c3614 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
@@ -78,8 +78,8 @@
 import org.apache.beam.sdk.values.PCollectionView;
 import org.apache.beam.sdk.values.TupleTag;
 import org.apache.beam.sdk.values.WindowingStrategy;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.InvalidProtocolBufferException;
 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.Iterables;
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 f02ddda..c6f873f 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
@@ -30,6 +30,7 @@
 import java.io.PrintWriter;
 import java.io.StringWriter;
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.List;
 import java.util.concurrent.Executors;
 import java.util.concurrent.ScheduledExecutorService;
@@ -99,13 +100,13 @@
     private String project, job, host, region;
     private Dataflow client = null;
     private ScheduledExecutorService executor = null;
-    private List<Capturable> capturables;
+    private Collection<Capturable> capturables;
     private boolean enabled;
 
     private long lastCaptureUsec = 0;
     @VisibleForTesting Config captureConfig = new Config();
 
-    public Manager(DataflowWorkerHarnessOptions options, List<Capturable> capturables) {
+    public Manager(DataflowWorkerHarnessOptions options, Collection<Capturable> capturables) {
       try {
         client = options.getDataflowClient();
       } catch (Exception e) {
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/status/SdkWorkerStatusServlet.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/status/SdkWorkerStatusServlet.java
new file mode 100644
index 0000000..b304952
--- /dev/null
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/status/SdkWorkerStatusServlet.java
@@ -0,0 +1,99 @@
+/*
+ * 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.dataflow.worker.status;
+
+import com.google.common.base.Strings;
+import java.io.IOException;
+import java.io.OutputStreamWriter;
+import java.io.PrintWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+import javax.servlet.ServletException;
+import javax.servlet.ServletOutputStream;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import org.apache.beam.runners.dataflow.worker.status.DebugCapture.Capturable;
+import org.apache.beam.runners.fnexecution.status.BeamWorkerStatusGrpcService;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Servlet dedicated to provide live status info retrieved from SDK Harness. Note this is different
+ * from {@link WorkerStatusPages} which incorporates all info for Dataflow runner including this
+ * SDKWorkerStatus page.
+ */
+public class SdkWorkerStatusServlet extends BaseStatusServlet implements Capturable {
+
+  private static final Logger LOG = LoggerFactory.getLogger(SdkWorkerStatusServlet.class);
+  private final transient BeamWorkerStatusGrpcService statusGrpcService;
+
+  public SdkWorkerStatusServlet(BeamWorkerStatusGrpcService statusGrpcService) {
+    super("sdk_status");
+    this.statusGrpcService = statusGrpcService;
+  }
+
+  @Override
+  protected void doGet(HttpServletRequest request, HttpServletResponse response)
+      throws IOException, ServletException {
+    String id = request.getParameter("id");
+    if (Strings.isNullOrEmpty(id)) {
+      // return all connected sdk statuses if no id provided.
+      response.setContentType("text/html;charset=utf-8");
+      ServletOutputStream writer = response.getOutputStream();
+      try (PrintWriter out =
+          new PrintWriter(new OutputStreamWriter(writer, StandardCharsets.UTF_8))) {
+        captureData(out);
+      }
+    } else {
+      response.setContentType("text/plain;charset=utf-8");
+      ServletOutputStream writer = response.getOutputStream();
+      writer.println(statusGrpcService.getSingleWorkerStatus(id, 10, TimeUnit.SECONDS));
+    }
+    response.setStatus(HttpServletResponse.SC_OK);
+    response.flushBuffer();
+  }
+
+  @Override
+  public String pageName() {
+    return "/sdk_status";
+  }
+
+  @Override
+  public void captureData(PrintWriter writer) {
+    Map<String, String> allStatuses = statusGrpcService.getAllWorkerStatuses(10, TimeUnit.SECONDS);
+
+    writer.println("<html>");
+    writer.println("<h1>SDK harness</h1>");
+    // add links to each sdk section for easier navigation.
+    for (String sdkId : allStatuses.keySet()) {
+      writer.print(String.format("<a href=\"#%s\">%s</a> ", sdkId, sdkId));
+    }
+    writer.println();
+
+    for (Map.Entry<String, String> entry : allStatuses.entrySet()) {
+      writer.println(String.format("<h2 id=\"%s\">%s</h2>", entry.getKey(), entry.getKey()));
+      writer.println("<a href=\"#top\">return to top</a>");
+      writer.println("<div style=\"white-space:pre-wrap\">");
+      writer.println(entry.getValue());
+      writer.println("</div>");
+      writer.println("");
+    }
+    writer.println("</html>");
+  }
+}
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 cd3b9de..764a607 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
@@ -18,7 +18,8 @@
 package org.apache.beam.runners.dataflow.worker.status;
 
 import java.io.IOException;
-import java.util.Arrays;
+import java.util.ArrayList;
+import java.util.Collection;
 import java.util.List;
 import java.util.function.BooleanSupplier;
 import javax.servlet.ServletException;
@@ -39,6 +40,7 @@
   private static final Logger LOG = LoggerFactory.getLogger(WorkerStatusPages.class);
 
   private final Server statusServer;
+  private final List<Capturable> capturePages;
   private final StatuszServlet statuszServlet = new StatuszServlet();
   private final ThreadzServlet threadzServlet = new ThreadzServlet();
   private final ServletHandler servletHandler = new ServletHandler();
@@ -46,6 +48,7 @@
   @VisibleForTesting
   WorkerStatusPages(Server server, MemoryMonitor memoryMonitor, BooleanSupplier healthyIndicator) {
     this.statusServer = server;
+    this.capturePages = new ArrayList<>();
     this.statusServer.setHandler(servletHandler);
 
     // Install the default servlets (threadz, healthz, heapz, statusz)
@@ -54,6 +57,9 @@
     addServlet(new HeapzServlet(memoryMonitor));
     addServlet(statuszServlet);
 
+    // Add default capture pages (threadz, statusz)
+    this.capturePages.add(threadzServlet);
+    this.capturePages.add(statuszServlet);
     // Add some status pages
     addStatusDataProvider("resources", "Resources", memoryMonitor);
   }
@@ -107,8 +113,12 @@
   }
 
   /** Returns the set of pages than should be captured by DebugCapture. */
-  public List<Capturable> getDebugCapturePages() {
-    return Arrays.asList(threadzServlet, statuszServlet);
+  public Collection<Capturable> getDebugCapturePages() {
+    return this.capturePages;
+  }
+
+  public void addCapturePage(Capturable page) {
+    this.capturePages.add(page);
   }
 
   /** Redirect all invalid pages to /statusz. */
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/DirectStreamObserver.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/DirectStreamObserver.java
index a54733d..7565ba2 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/DirectStreamObserver.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/DirectStreamObserver.java
@@ -19,8 +19,8 @@
 
 import java.util.concurrent.Phaser;
 import javax.annotation.concurrent.ThreadSafe;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.CallStreamObserver;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.StreamObserver;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.stub.CallStreamObserver;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.stub.StreamObserver;
 
 /**
  * A {@link StreamObserver} which uses synchronization on the underlying {@link CallStreamObserver}
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/ForwardingClientResponseObserver.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/ForwardingClientResponseObserver.java
index 74d8e4d..d7eba1f 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/ForwardingClientResponseObserver.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/ForwardingClientResponseObserver.java
@@ -17,9 +17,9 @@
  */
 package org.apache.beam.runners.dataflow.worker.windmill;
 
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.ClientCallStreamObserver;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.ClientResponseObserver;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.StreamObserver;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.stub.ClientCallStreamObserver;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.stub.ClientResponseObserver;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.stub.StreamObserver;
 
 /**
  * A {@link ClientResponseObserver} which delegates all {@link StreamObserver} calls.
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 c64803d..632ab07 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
@@ -84,17 +84,17 @@
 import org.apache.beam.sdk.util.BackOffUtils;
 import org.apache.beam.sdk.util.FluentBackoff;
 import org.apache.beam.sdk.util.Sleeper;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.CallCredentials;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.Channel;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.Status;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.StatusRuntimeException;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.auth.MoreCallCredentials;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.inprocess.InProcessChannelBuilder;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.netty.GrpcSslContexts;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.netty.NegotiationType;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.netty.NettyChannelBuilder;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.StreamObserver;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.CallCredentials;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.Channel;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.Status;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.StatusRuntimeException;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.auth.MoreCallCredentials;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.inprocess.InProcessChannelBuilder;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.netty.GrpcSslContexts;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.netty.NegotiationType;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.netty.NettyChannelBuilder;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.stub.StreamObserver;
 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.Preconditions;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Splitter;
@@ -232,11 +232,11 @@
    */
   private static class VendoredRequestMetadataCallbackAdapter
       implements com.google.auth.RequestMetadataCallback {
-    private final org.apache.beam.vendor.grpc.v1p21p0.com.google.auth.RequestMetadataCallback
+    private final org.apache.beam.vendor.grpc.v1p26p0.com.google.auth.RequestMetadataCallback
         callback;
 
     private VendoredRequestMetadataCallbackAdapter(
-        org.apache.beam.vendor.grpc.v1p21p0.com.google.auth.RequestMetadataCallback callback) {
+        org.apache.beam.vendor.grpc.v1p26p0.com.google.auth.RequestMetadataCallback callback) {
       this.callback = callback;
     }
 
@@ -260,7 +260,7 @@
    * delegate to reduce maintenance burden.
    */
   private static class VendoredCredentialsAdapter
-      extends org.apache.beam.vendor.grpc.v1p21p0.com.google.auth.Credentials {
+      extends org.apache.beam.vendor.grpc.v1p26p0.com.google.auth.Credentials {
     private final com.google.auth.Credentials credentials;
 
     private VendoredCredentialsAdapter(com.google.auth.Credentials credentials) {
@@ -281,7 +281,7 @@
     public void getRequestMetadata(
         final URI uri,
         Executor executor,
-        final org.apache.beam.vendor.grpc.v1p21p0.com.google.auth.RequestMetadataCallback
+        final org.apache.beam.vendor.grpc.v1p26p0.com.google.auth.RequestMetadataCallback
             callback) {
       credentials.getRequestMetadata(
           uri, executor, new VendoredRequestMetadataCallbackAdapter(callback));
diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/StreamObserverFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/StreamObserverFactory.java
index 6731951..0216766 100644
--- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/StreamObserverFactory.java
+++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/StreamObserverFactory.java
@@ -20,8 +20,8 @@
 import java.util.function.Function;
 import org.apache.beam.sdk.fn.stream.AdvancingPhaser;
 import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.CallStreamObserver;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.StreamObserver;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.stub.CallStreamObserver;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.stub.StreamObserver;
 
 /**
  * Uses {@link PipelineOptions} to configure which underlying {@link StreamObserver} implementation
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/harness/test/TestStreams.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/harness/test/TestStreams.java
index 07ccdb1..e88054c 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/harness/test/TestStreams.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/harness/test/TestStreams.java
@@ -19,8 +19,8 @@
 
 import java.util.function.Consumer;
 import java.util.function.Supplier;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.CallStreamObserver;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.StreamObserver;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.stub.CallStreamObserver;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.stub.StreamObserver;
 
 /** Utility methods which enable testing of {@link StreamObserver}s. */
 public class TestStreams {
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DataflowMatchers.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DataflowMatchers.java
index 469add3..4b69f07 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DataflowMatchers.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DataflowMatchers.java
@@ -18,7 +18,7 @@
 package org.apache.beam.runners.dataflow.worker;
 
 import java.io.Serializable;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
 import org.hamcrest.Description;
 import org.hamcrest.TypeSafeMatcher;
 
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 66be62d..453dbea 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
@@ -32,7 +32,7 @@
 import org.apache.beam.runners.dataflow.worker.testing.RestoreDataflowLoggingMDC;
 import org.apache.beam.sdk.options.PipelineOptionsFactory;
 import org.apache.beam.sdk.testing.RestoreSystemProperties;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.TextFormat;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.TextFormat;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.TemporaryFolder;
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 fddfdc5..9cae865 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
@@ -31,7 +31,7 @@
 import org.apache.beam.sdk.coders.StringUtf8Coder;
 import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
 import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
 import org.joda.time.Instant;
 import org.junit.Before;
 import org.junit.Test;
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 9f45286..a01356e 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
@@ -29,7 +29,7 @@
 import org.apache.beam.sdk.coders.StringUtf8Coder;
 import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
 import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
 import org.joda.time.Instant;
 import org.junit.Before;
 import org.junit.Test;
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 51f20a4..1820ef1 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
@@ -26,7 +26,7 @@
 import java.io.IOException;
 import java.util.concurrent.TimeUnit;
 import org.apache.beam.sdk.io.UnboundedSource;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Stopwatch;
 import org.joda.time.Duration;
 import org.junit.Before;
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 5d90a85..6dc4f7b 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
@@ -43,7 +43,7 @@
 import org.apache.beam.sdk.transforms.View;
 import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
 import org.apache.beam.sdk.values.PCollectionView;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
+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.cache.Cache;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.cache.CacheBuilder;
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 c954bdb..7ce23a4 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
@@ -137,9 +137,9 @@
 import org.apache.beam.sdk.values.ValueWithRecordId;
 import org.apache.beam.sdk.values.WindowingStrategy;
 import org.apache.beam.sdk.values.WindowingStrategy.AccumulationMode;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString.Output;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.TextFormat;
+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.grpc.v1p26p0.com.google.protobuf.TextFormat;
 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.ImmutableList;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
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 0acb1dc..bf9e875 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
@@ -75,7 +75,7 @@
 import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.TupleTag;
 import org.apache.beam.sdk.values.WindowingStrategy;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
 import org.joda.time.Duration;
 import org.joda.time.Instant;
 import org.junit.Before;
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 11322a9..81a2bf5 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
@@ -48,7 +48,7 @@
 import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.TupleTag;
 import org.apache.beam.sdk.values.WindowingStrategy;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
 import org.hamcrest.Matchers;
 import org.joda.time.Duration;
 import org.joda.time.Instant;
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 03d4376..b0577e4 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
@@ -65,7 +65,7 @@
 import org.apache.beam.sdk.util.WindowedValue;
 import org.apache.beam.sdk.values.PCollectionView;
 import org.apache.beam.sdk.values.TupleTag;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Lists;
 import org.hamcrest.Matchers;
 import org.joda.time.Instant;
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 24d17ff..8d87a10 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
@@ -63,7 +63,7 @@
 import org.apache.beam.sdk.values.PCollectionView;
 import org.apache.beam.sdk.values.TupleTag;
 import org.apache.beam.sdk.values.WindowingStrategy;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables;
 import org.hamcrest.Matchers;
 import org.joda.time.Duration;
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 d70d7f6..c1a9945 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
@@ -49,7 +49,7 @@
 import org.apache.beam.sdk.values.PCollectionView;
 import org.apache.beam.sdk.values.TupleTag;
 import org.apache.beam.sdk.values.WindowingStrategy;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
 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.Sets;
 import org.hamcrest.Matchers;
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 4441f35..de62567 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
@@ -40,7 +40,7 @@
 import org.apache.beam.sdk.transforms.windowing.PaneInfo;
 import org.apache.beam.sdk.transforms.windowing.PaneInfo.Timing;
 import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
 import org.hamcrest.Matchers;
 import org.joda.time.Instant;
 import org.junit.Before;
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WindmillReaderIteratorBaseTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WindmillReaderIteratorBaseTest.java
index bff116f..0b89c99 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WindmillReaderIteratorBaseTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WindmillReaderIteratorBaseTest.java
@@ -26,7 +26,7 @@
 import java.util.List;
 import org.apache.beam.runners.dataflow.worker.windmill.Windmill;
 import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
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 64f425a..23fe22f 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
@@ -28,7 +28,7 @@
 import org.apache.beam.sdk.state.State;
 import org.apache.beam.sdk.state.StateSpec;
 import org.apache.beam.sdk.transforms.windowing.IntervalWindow;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
 import org.joda.time.Instant;
 import org.junit.Before;
 import org.junit.Test;
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 f708500..e693672 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
@@ -50,7 +50,7 @@
 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.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
+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.util.concurrent.Futures;
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 ef529df..f2628ff 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
@@ -29,8 +29,8 @@
 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.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString.Output;
+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.hamcrest.Matchers;
 import org.joda.time.Instant;
 import org.junit.Before;
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 181183d..15ff67f 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
@@ -102,7 +102,7 @@
 import org.apache.beam.sdk.util.WindowedValue;
 import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.ValueWithRecordId;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
 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.base.Preconditions;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
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 0cac04e..f986783 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
@@ -35,9 +35,9 @@
 import org.apache.beam.runners.fnexecution.control.FnApiControlClient;
 import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.options.PipelineOptionsFactory;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.ManagedChannelBuilder;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.Server;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.StreamObserver;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.ManagedChannelBuilder;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.Server;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.stub.StreamObserver;
 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.net.HostAndPort;
 import org.junit.Before;
@@ -88,7 +88,7 @@
     Server server = ServerFactory.createDefault().create(ImmutableList.of(service), descriptor);
     String url = service.getApiServiceDescriptor().getUrl();
     BeamFnControlGrpc.BeamFnControlStub clientStub =
-        BeamFnControlGrpc.newStub(ManagedChannelBuilder.forTarget(url).usePlaintext(true).build());
+        BeamFnControlGrpc.newStub(ManagedChannelBuilder.forTarget(url).usePlaintext().build());
 
     // Connect from the client.
     clientStub.control(requestObserver);
@@ -134,9 +134,9 @@
 
     String url = service.getApiServiceDescriptor().getUrl();
     BeamFnControlGrpc.BeamFnControlStub clientStub =
-        BeamFnControlGrpc.newStub(ManagedChannelBuilder.forTarget(url).usePlaintext(true).build());
+        BeamFnControlGrpc.newStub(ManagedChannelBuilder.forTarget(url).usePlaintext().build());
     BeamFnControlGrpc.BeamFnControlStub anotherClientStub =
-        BeamFnControlGrpc.newStub(ManagedChannelBuilder.forTarget(url).usePlaintext(true).build());
+        BeamFnControlGrpc.newStub(ManagedChannelBuilder.forTarget(url).usePlaintext().build());
 
     // Connect from the client.
     clientStub.control(requestObserver);
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 eb3d21d..a89dac6 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
@@ -80,7 +80,7 @@
 import org.apache.beam.sdk.values.PCollectionView;
 import org.apache.beam.sdk.values.TupleTag;
 import org.apache.beam.sdk.values.ValueInSingleWindow.Coder;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
 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;
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/fn/control/TimerReceiverTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/fn/control/TimerReceiverTest.java
index 2067a3a..68a54d5 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/fn/control/TimerReceiverTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/fn/control/TimerReceiverTest.java
@@ -70,7 +70,7 @@
 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.vendor.grpc.v1p21p0.com.google.protobuf.Struct;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.Struct;
 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.ImmutableMap;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables;
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 9c2b57a..ad514a8 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
@@ -51,22 +51,22 @@
 import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.options.PipelineOptionsFactory;
 import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.BindableService;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.CallOptions;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.Channel;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.ClientCall;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.ClientInterceptor;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.ForwardingClientCall.SimpleForwardingClientCall;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.ManagedChannel;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.Metadata;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.Metadata.Key;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.MethodDescriptor;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.Server;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.ServerInterceptors;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.inprocess.InProcessChannelBuilder;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.inprocess.InProcessServerBuilder;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.StreamObserver;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.BindableService;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.CallOptions;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.Channel;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.ClientCall;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.ClientInterceptor;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.ForwardingClientCall.SimpleForwardingClientCall;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.ManagedChannel;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.Metadata;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.Metadata.Key;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.MethodDescriptor;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.Server;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.ServerInterceptors;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.inprocess.InProcessChannelBuilder;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.inprocess.InProcessServerBuilder;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.stub.StreamObserver;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
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 55b81e0..114ded8 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
@@ -38,12 +38,12 @@
 import org.apache.beam.runners.dataflow.worker.fn.stream.ServerStreamObserverFactory;
 import org.apache.beam.runners.fnexecution.GrpcContextHeaderAccessorProvider;
 import org.apache.beam.sdk.options.PipelineOptionsFactory;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.BindableService;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.ManagedChannel;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.Server;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.inprocess.InProcessChannelBuilder;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.inprocess.InProcessServerBuilder;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.StreamObserver;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.BindableService;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.ManagedChannel;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.Server;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.inprocess.InProcessChannelBuilder;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.inprocess.InProcessServerBuilder;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.stub.StreamObserver;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.net.HostAndPort;
 import org.junit.After;
 import org.junit.Test;
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 43d6975..e41fd69 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
@@ -24,8 +24,8 @@
 import org.apache.beam.sdk.fn.stream.BufferingStreamObserver;
 import org.apache.beam.sdk.fn.stream.DirectStreamObserver;
 import org.apache.beam.sdk.options.PipelineOptionsFactory;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.CallStreamObserver;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.StreamObserver;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.stub.CallStreamObserver;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.stub.StreamObserver;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.runner.RunWith;
diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/graph/InsertFetchAndFilterStreamingSideInputNodesTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/graph/InsertFetchAndFilterStreamingSideInputNodesTest.java
index d1115c9..0e36473 100644
--- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/graph/InsertFetchAndFilterStreamingSideInputNodesTest.java
+++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/graph/InsertFetchAndFilterStreamingSideInputNodesTest.java
@@ -54,7 +54,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.vendor.grpc.v1p21p0.com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.InvalidProtocolBufferException;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Equivalence;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Equivalence.Wrapper;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
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 568fcff..84adfcd 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
@@ -35,7 +35,7 @@
 import org.apache.beam.runners.dataflow.worker.NameContextsForTests;
 import org.apache.beam.runners.dataflow.worker.TestOperationContext.TestDataflowExecutionState;
 import org.apache.beam.runners.dataflow.worker.testing.RestoreDataflowLoggingMDC;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.Timestamp;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.Timestamp;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Supplier;
 import org.junit.After;
 import org.junit.Before;
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 cac9fe3..146fb17 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
@@ -53,7 +53,9 @@
       assertThat(actual, is(jsonOf(expected)));
     } catch (AssertionError ex) {
       assertEquals(
-          "\nExpected: is {\"foo\":\"expected\"}\n     but: was <{foo=actual}>", ex.getMessage());
+          "\nExpected: is {\"foo\":\"expected\"}\n"
+              + "     but: was <GenericData{classInfo=[], {foo=actual}}>",
+          ex.getMessage());
 
       // pass
       return;
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 9adce9f..b889138 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
@@ -59,13 +59,13 @@
 import org.apache.beam.runners.dataflow.worker.windmill.WindmillServerStub.CommitWorkStream;
 import org.apache.beam.runners.dataflow.worker.windmill.WindmillServerStub.GetDataStream;
 import org.apache.beam.runners.dataflow.worker.windmill.WindmillServerStub.GetWorkStream;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.Server;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.Status;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.StatusRuntimeException;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.inprocess.InProcessServerBuilder;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.StreamObserver;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.util.MutableHandlerRegistry;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.Server;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.Status;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.StatusRuntimeException;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.inprocess.InProcessServerBuilder;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.stub.StreamObserver;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.util.MutableHandlerRegistry;
 import org.hamcrest.Matchers;
 import org.joda.time.Instant;
 import org.junit.After;
diff --git a/runners/java-fn-execution/build.gradle b/runners/java-fn-execution/build.gradle
index f032d8f..434ac02 100644
--- a/runners/java-fn-execution/build.gradle
+++ b/runners/java-fn-execution/build.gradle
@@ -30,7 +30,7 @@
   compile project(":sdks:java:fn-execution")
   compile project(":runners:core-construction-java")
   compile project(path: ":vendor:sdks-java-extensions-protobuf", configuration: "shadow")
-  compile library.java.vendored_grpc_1_21_0
+  compile library.java.vendored_grpc_1_26_0
   compile library.java.slf4j_api
   compile library.java.args4j
   testCompile project(":sdks:java:harness")
diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/FnService.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/FnService.java
index 3055b0b..634657a 100644
--- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/FnService.java
+++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/FnService.java
@@ -17,7 +17,7 @@
  */
 package org.apache.beam.runners.fnexecution;
 
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.BindableService;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.BindableService;
 
 /** An interface sharing common behavior with services used during execution of user Fns. */
 public interface FnService extends AutoCloseable, BindableService {
@@ -26,8 +26,8 @@
    *
    * <p>There should be no more calls to any service method by the time a call to {@link #close()}
    * begins. Specifically, this means that a {@link
-   * org.apache.beam.vendor.grpc.v1p21p0.io.grpc.Server} that this service is bound to should have
-   * completed a call to the {@link org.apache.beam.vendor.grpc.v1p21p0.io.grpc.Server#shutdown()}
+   * org.apache.beam.vendor.grpc.v1p26p0.io.grpc.Server} that this service is bound to should have
+   * completed a call to the {@link org.apache.beam.vendor.grpc.v1p26p0.io.grpc.Server#shutdown()}
    * method, and all future incoming calls will be rejected.
    */
   @Override
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 5d758a2..4c7899c 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
@@ -17,14 +17,14 @@
  */
 package org.apache.beam.runners.fnexecution;
 
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.Context;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.Contexts;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.Metadata;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.Metadata.Key;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.ServerCall;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.ServerCall.Listener;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.ServerCallHandler;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.ServerInterceptor;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.Context;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.Contexts;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.Metadata;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.Metadata.Key;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.ServerCall;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.ServerCall.Listener;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.ServerCallHandler;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.ServerInterceptor;
 
 /**
  * A HeaderAccessorProvider which intercept the header in a GRPC request and expose the relevant
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 f7a4a4b..fbb12af 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
@@ -20,7 +20,7 @@
 import java.io.IOException;
 import java.util.concurrent.TimeUnit;
 import org.apache.beam.model.pipeline.v1.Endpoints.ApiServiceDescriptor;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.Server;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.Server;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
 
 /**
diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/InProcessServerFactory.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/InProcessServerFactory.java
index a899cb2..e72b0bc 100644
--- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/InProcessServerFactory.java
+++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/InProcessServerFactory.java
@@ -21,10 +21,10 @@
 import java.util.List;
 import java.util.concurrent.atomic.AtomicInteger;
 import org.apache.beam.model.pipeline.v1.Endpoints.ApiServiceDescriptor;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.BindableService;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.Server;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.ServerInterceptors;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.inprocess.InProcessServerBuilder;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.BindableService;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.Server;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.ServerInterceptors;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.inprocess.InProcessServerBuilder;
 
 /**
  * A {@link ServerFactory} which creates {@link Server servers} with the {@link
diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/ServerFactory.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/ServerFactory.java
index ff0d5b4..30f6b8b 100644
--- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/ServerFactory.java
+++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/ServerFactory.java
@@ -29,16 +29,16 @@
 import java.util.function.Supplier;
 import org.apache.beam.model.pipeline.v1.Endpoints;
 import org.apache.beam.sdk.fn.channel.SocketAddressFactory;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.BindableService;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.Server;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.ServerBuilder;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.ServerInterceptors;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.netty.NettyServerBuilder;
-import org.apache.beam.vendor.grpc.v1p21p0.io.netty.channel.epoll.EpollEventLoopGroup;
-import org.apache.beam.vendor.grpc.v1p21p0.io.netty.channel.epoll.EpollServerDomainSocketChannel;
-import org.apache.beam.vendor.grpc.v1p21p0.io.netty.channel.epoll.EpollServerSocketChannel;
-import org.apache.beam.vendor.grpc.v1p21p0.io.netty.channel.unix.DomainSocketAddress;
-import org.apache.beam.vendor.grpc.v1p21p0.io.netty.util.internal.ThreadLocalRandom;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.BindableService;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.Server;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.ServerBuilder;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.ServerInterceptors;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.netty.NettyServerBuilder;
+import org.apache.beam.vendor.grpc.v1p26p0.io.netty.channel.epoll.EpollEventLoopGroup;
+import org.apache.beam.vendor.grpc.v1p26p0.io.netty.channel.epoll.EpollServerDomainSocketChannel;
+import org.apache.beam.vendor.grpc.v1p26p0.io.netty.channel.epoll.EpollServerSocketChannel;
+import org.apache.beam.vendor.grpc.v1p26p0.io.netty.channel.unix.DomainSocketAddress;
+import org.apache.beam.vendor.grpc.v1p26p0.io.netty.util.internal.ThreadLocalRandom;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.net.HostAndPort;
 
 /** A {@link Server gRPC server} factory. */
diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/artifact/AbstractArtifactRetrievalService.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/artifact/AbstractArtifactRetrievalService.java
index 72af9e81..a9b04b3 100644
--- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/artifact/AbstractArtifactRetrievalService.java
+++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/artifact/AbstractArtifactRetrievalService.java
@@ -29,11 +29,11 @@
 import org.apache.beam.model.jobmanagement.v1.ArtifactApi.ArtifactMetadata;
 import org.apache.beam.model.jobmanagement.v1.ArtifactApi.ProxyManifest;
 import org.apache.beam.model.jobmanagement.v1.ArtifactRetrievalServiceGrpc;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.util.JsonFormat;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.Status;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.StatusRuntimeException;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.StreamObserver;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.util.JsonFormat;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.Status;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.StatusRuntimeException;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.stub.StreamObserver;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Strings;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.cache.Cache;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.cache.CacheBuilder;
diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/artifact/AbstractArtifactStagingService.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/artifact/AbstractArtifactStagingService.java
index 0b0fadf..ae951d9 100644
--- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/artifact/AbstractArtifactStagingService.java
+++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/artifact/AbstractArtifactStagingService.java
@@ -35,11 +35,11 @@
 import org.apache.beam.model.jobmanagement.v1.ArtifactStagingServiceGrpc.ArtifactStagingServiceImplBase;
 import org.apache.beam.model.pipeline.v1.RunnerApi;
 import org.apache.beam.runners.fnexecution.FnService;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.util.JsonFormat;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.Status;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.StatusRuntimeException;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.StreamObserver;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.util.JsonFormat;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.Status;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.StatusRuntimeException;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.stub.StreamObserver;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.hash.Hasher;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.hash.Hashing;
 import org.slf4j.Logger;
diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/artifact/BeamFileSystemArtifactStagingService.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/artifact/BeamFileSystemArtifactStagingService.java
index c9baa17..74bce71 100644
--- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/artifact/BeamFileSystemArtifactStagingService.java
+++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/artifact/BeamFileSystemArtifactStagingService.java
@@ -31,8 +31,8 @@
 import org.apache.beam.sdk.io.fs.ResolveOptions.StandardResolveOptions;
 import org.apache.beam.sdk.io.fs.ResourceId;
 import org.apache.beam.sdk.util.MimeTypes;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.Status;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.StatusRuntimeException;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.Status;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.StatusRuntimeException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
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 9051051..e0e5d50 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
@@ -29,9 +29,9 @@
 import org.apache.beam.model.fnexecution.v1.BeamFnApi;
 import org.apache.beam.model.fnexecution.v1.BeamFnApi.InstructionRequest;
 import org.apache.beam.sdk.fn.stream.SynchronizedStreamObserver;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.Status;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.StatusRuntimeException;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.StreamObserver;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.Status;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.StatusRuntimeException;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.stub.StreamObserver;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/FnApiControlClientPoolService.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/FnApiControlClientPoolService.java
index 598e1db..82409ae 100644
--- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/FnApiControlClientPoolService.java
+++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/FnApiControlClientPoolService.java
@@ -26,7 +26,7 @@
 import org.apache.beam.model.fnexecution.v1.BeamFnControlGrpc;
 import org.apache.beam.runners.fnexecution.FnService;
 import org.apache.beam.runners.fnexecution.HeaderAccessor;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.StreamObserver;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.stub.StreamObserver;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Strings;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
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 cd81c0a..80b3f61 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
@@ -33,6 +33,7 @@
 import org.apache.beam.model.pipeline.v1.Endpoints.ApiServiceDescriptor;
 import org.apache.beam.model.pipeline.v1.RunnerApi;
 import org.apache.beam.model.pipeline.v1.RunnerApi.Components;
+import org.apache.beam.model.pipeline.v1.RunnerApi.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;
@@ -56,7 +57,7 @@
 import org.apache.beam.sdk.util.WindowedValue;
 import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder;
 import org.apache.beam.sdk.values.KV;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.beam.vendor.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.ImmutableMap;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableTable;
@@ -119,15 +120,19 @@
         ImmutableMap.builder();
     ImmutableMap.Builder<String, Coder> remoteOutputCodersBuilder = ImmutableMap.builder();
 
+    WireCoderSetting wireCoderSetting =
+        stage.getWireCoderSettings().stream()
+            .filter(ws -> ws.getInputOrOutputId().equals(stage.getInputPCollection().getId()))
+            .findAny()
+            .orElse(WireCoderSetting.getDefaultInstance());
     // The order of these does not matter.
     inputDestinationsBuilder.put(
         stage.getInputPCollection().getId(),
-        addStageInput(
-            dataEndpoint, stage.getInputPCollection(), components, stage.getWireCoderSetting()));
+        addStageInput(dataEndpoint, stage.getInputPCollection(), components, wireCoderSetting));
 
     remoteOutputCodersBuilder.putAll(
         addStageOutputs(
-            dataEndpoint, stage.getOutputPCollections(), components, stage.getWireCoderSetting()));
+            dataEndpoint, stage.getOutputPCollections(), components, stage.getWireCoderSettings()));
 
     Map<String, Map<String, SideInputSpec>> sideInputSpecs = addSideInputs(stage, components);
 
@@ -192,10 +197,15 @@
       ApiServiceDescriptor dataEndpoint,
       Collection<PCollectionNode> outputPCollections,
       Components.Builder components,
-      RunnerApi.WireCoderSetting wireCoderSetting)
+      Collection<WireCoderSetting> wireCoderSettings)
       throws IOException {
     Map<String, Coder<WindowedValue<?>>> remoteOutputCoders = new LinkedHashMap<>();
     for (PCollectionNode outputPCollection : outputPCollections) {
+      WireCoderSetting wireCoderSetting =
+          wireCoderSettings.stream()
+              .filter(ws -> ws.getInputOrOutputId().equals(outputPCollection.getId()))
+              .findAny()
+              .orElse(WireCoderSetting.getDefaultInstance());
       OutputEncoding outputEncoding =
           addStageOutput(dataEndpoint, components, outputPCollection, wireCoderSetting);
       remoteOutputCoders.put(outputEncoding.getPTransformId(), outputEncoding.getCoder());
@@ -207,7 +217,7 @@
       ApiServiceDescriptor dataEndpoint,
       PCollectionNode inputPCollection,
       Components.Builder components,
-      RunnerApi.WireCoderSetting wireCoderSetting)
+      WireCoderSetting wireCoderSetting)
       throws IOException {
     String inputWireCoderId =
         WireCoders.addSdkWireCoder(inputPCollection, components, wireCoderSetting);
@@ -235,7 +245,7 @@
       ApiServiceDescriptor dataEndpoint,
       Components.Builder components,
       PCollectionNode outputPCollection,
-      RunnerApi.WireCoderSetting wireCoderSetting)
+      WireCoderSetting wireCoderSetting)
       throws IOException {
     String outputWireCoderId =
         WireCoders.addSdkWireCoder(outputPCollection, components, wireCoderSetting);
@@ -385,6 +395,17 @@
               .setCoderId(timerCoderId)
               .build();
 
+      // The wire coder setting for both input and output of the timer. We haven't provided
+      // different settings for input and output now, because the timerCollectionSpec is same for
+      // both input and output of the timer.
+      WireCoderSetting wireCoderSetting =
+          stage.getWireCoderSettings().stream()
+              .filter(
+                  ws ->
+                      ws.getTimer().getTransformId().equals(timerReference.transform().getId())
+                          && ws.getTimer().getLocalName().equals(timerReference.localName()))
+              .findAny()
+              .orElse(WireCoderSetting.getDefaultInstance());
       // "Unroll" the timers into PCollections.
       String inputTimerPCollectionId =
           SyntheticComponents.uniqueId(
@@ -398,7 +419,7 @@
               dataEndpoint,
               PipelineNode.pCollection(inputTimerPCollectionId, timerCollectionSpec),
               components,
-              stage.getWireCoderSetting()));
+              wireCoderSetting));
       String outputTimerPCollectionId =
           SyntheticComponents.uniqueId(
               String.format(
@@ -411,7 +432,7 @@
               dataEndpoint,
               components,
               PipelineNode.pCollection(outputTimerPCollectionId, timerCollectionSpec),
-              stage.getWireCoderSetting());
+              wireCoderSetting);
       outputTransformCodersBuilder.put(outputEncoding.getPTransformId(), outputEncoding.getCoder());
       components.putTransforms(
           timerReference.transform().getId(),
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 5d58c5c..1c283bf 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
@@ -36,7 +36,7 @@
 import org.apache.beam.sdk.fn.data.LogicalEndpoint;
 import org.apache.beam.sdk.fn.stream.OutboundObserverFactory;
 import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.StreamObserver;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.stub.StreamObserver;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.util.concurrent.SettableFuture;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/jobsubmission/InMemoryJobService.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/jobsubmission/InMemoryJobService.java
index 2f457a5..abda968 100644
--- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/jobsubmission/InMemoryJobService.java
+++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/jobsubmission/InMemoryJobService.java
@@ -21,7 +21,7 @@
 import java.util.List;
 import java.util.UUID;
 import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ConcurrentLinkedDeque;
 import java.util.function.Consumer;
 import java.util.function.Function;
 import org.apache.beam.model.jobmanagement.v1.JobApi;
@@ -52,11 +52,12 @@
 import org.apache.beam.sdk.fn.stream.SynchronizedStreamObserver;
 import org.apache.beam.sdk.function.ThrowingConsumer;
 import org.apache.beam.sdk.options.PipelineOptionsFactory;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.Struct;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.Status;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.StatusException;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.StatusRuntimeException;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.StreamObserver;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.Struct;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.Status;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.StatusException;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.StatusRuntimeException;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.stub.StreamObserver;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -72,12 +73,16 @@
 public class InMemoryJobService extends JobServiceGrpc.JobServiceImplBase implements FnService {
   private static final Logger LOG = LoggerFactory.getLogger(InMemoryJobService.class);
 
+  /** The default maximum number of completed invocations to keep. */
+  public static final int DEFAULT_MAX_INVOCATION_HISTORY = 10;
+
   /**
    * Creates an InMemoryJobService.
    *
    * @param stagingServiceDescriptor Endpoint for the staging service.
    * @param stagingServiceTokenProvider Function mapping a preparationId to a staging service token.
-   * @param invoker A JobInvoker that will actually create the jobs.
+   * @param cleanupJobFn A cleanup function to run, parameterized with the staging token of a job.
+   * @param invoker A JobInvoker which creates the jobs.
    * @return A new InMemoryJobService.
    */
   public static InMemoryJobService create(
@@ -86,22 +91,60 @@
       ThrowingConsumer<Exception, String> cleanupJobFn,
       JobInvoker invoker) {
     return new InMemoryJobService(
-        stagingServiceDescriptor, stagingServiceTokenProvider, cleanupJobFn, invoker);
+        stagingServiceDescriptor,
+        stagingServiceTokenProvider,
+        cleanupJobFn,
+        invoker,
+        DEFAULT_MAX_INVOCATION_HISTORY);
   }
 
-  private final ConcurrentMap<String, JobPreparation> preparations;
-  private final ConcurrentMap<String, JobInvocation> invocations;
-  private final ConcurrentMap<String, String> stagingSessionTokens;
+  /**
+   * Creates an InMemoryJobService.
+   *
+   * @param stagingServiceDescriptor The endpoint for the staging service.
+   * @param stagingServiceTokenProvider Function mapping a preparationId to a staging service token.
+   * @param cleanupJobFn A cleanup function to run, parameterized with the staging token of a job.
+   * @param invoker A JobInvoker which creates the jobs.
+   * @param maxInvocationHistory The maximum number of completed invocations to keep.
+   * @return A new InMemoryJobService.
+   */
+  public static InMemoryJobService create(
+      Endpoints.ApiServiceDescriptor stagingServiceDescriptor,
+      Function<String, String> stagingServiceTokenProvider,
+      ThrowingConsumer<Exception, String> cleanupJobFn,
+      JobInvoker invoker,
+      int maxInvocationHistory) {
+    return new InMemoryJobService(
+        stagingServiceDescriptor,
+        stagingServiceTokenProvider,
+        cleanupJobFn,
+        invoker,
+        maxInvocationHistory);
+  }
+
+  /** Map of preparationId to preparation. */
+  private final ConcurrentHashMap<String, JobPreparation> preparations;
+  /** Map of preparationId to staging token. */
+  private final ConcurrentHashMap<String, String> stagingSessionTokens;
+  /** Map of invocationId to invocation. */
+  private final ConcurrentHashMap<String, JobInvocation> invocations;
+  /** InvocationIds of completed invocations in least-recently-completed order. */
+  private final ConcurrentLinkedDeque<String> completedInvocationsIds;
+
   private final Endpoints.ApiServiceDescriptor stagingServiceDescriptor;
   private final Function<String, String> stagingServiceTokenProvider;
   private final ThrowingConsumer<Exception, String> cleanupJobFn;
   private final JobInvoker invoker;
 
+  /** The maximum number of past invocations to keep. */
+  private final int maxInvocationHistory;
+
   private InMemoryJobService(
       Endpoints.ApiServiceDescriptor stagingServiceDescriptor,
       Function<String, String> stagingServiceTokenProvider,
       ThrowingConsumer<Exception, String> cleanupJobFn,
-      JobInvoker invoker) {
+      JobInvoker invoker,
+      int maxInvocationHistory) {
     this.stagingServiceDescriptor = stagingServiceDescriptor;
     this.stagingServiceTokenProvider = stagingServiceTokenProvider;
     this.cleanupJobFn = cleanupJobFn;
@@ -109,9 +152,10 @@
 
     this.preparations = new ConcurrentHashMap<>();
     this.invocations = new ConcurrentHashMap<>();
-
-    // Map "preparation ID" to staging token
     this.stagingSessionTokens = new ConcurrentHashMap<>();
+    this.completedInvocationsIds = new ConcurrentLinkedDeque<>();
+    Preconditions.checkArgument(maxInvocationHistory >= 0);
+    this.maxInvocationHistory = maxInvocationHistory;
   }
 
   @Override
@@ -196,20 +240,25 @@
             }
             String stagingSessionToken = stagingSessionTokens.get(preparationId);
             stagingSessionTokens.remove(preparationId);
-            if (cleanupJobFn != null) {
-              try {
+            try {
+              if (cleanupJobFn != null) {
                 cleanupJobFn.accept(stagingSessionToken);
-              } catch (Exception e) {
-                LOG.warn(
-                    "Failed to remove job staging directory for token {}: {}",
-                    stagingSessionToken,
-                    e);
               }
+            } catch (Exception e) {
+              LOG.warn(
+                  "Failed to remove job staging directory for token {}: {}",
+                  stagingSessionToken,
+                  e);
+            } finally {
+              onFinishedInvocationCleanup(invocationId);
             }
           });
 
       invocation.start();
       invocations.put(invocationId, invocation);
+      // Cleanup this preparation because we are running it now.
+      // If we fail, we need to prepare again.
+      preparations.remove(preparationId);
       RunJobResponse response = RunJobResponse.newBuilder().setJobId(invocationId).build();
       responseObserver.onNext(response);
       responseObserver.onCompleted();
@@ -344,18 +393,22 @@
           event -> {
             syncResponseObserver.onNext(
                 JobMessagesResponse.newBuilder().setStateResponse(event).build());
+            // The terminal state is always updated after the last message, that's
+            // why we can end the stream here.
+            if (JobInvocation.isTerminated(invocation.getStateEvent().getState())) {
+              responseObserver.onCompleted();
+            }
           };
       Consumer<JobMessage> messageListener =
           message ->
               syncResponseObserver.onNext(
                   JobMessagesResponse.newBuilder().setMessageResponse(message).build());
 
-      invocation.addStateListener(stateListener);
       invocation.addMessageListener(messageListener);
+      // The order matters here. Make sure to send all the message first because the stream
+      // will be ended by the terminal state request.
+      invocation.addStateListener(stateListener);
 
-      if (JobInvocation.isTerminated(invocation.getStateEvent().getState())) {
-        responseObserver.onCompleted();
-      }
     } catch (StatusRuntimeException | StatusException e) {
       responseObserver.onError(e);
     } catch (Exception e) {
@@ -422,4 +475,14 @@
     }
     return invocation;
   }
+
+  private void onFinishedInvocationCleanup(String invocationId) {
+    completedInvocationsIds.addLast(invocationId);
+    while (completedInvocationsIds.size() > maxInvocationHistory) {
+      // Clean up invocations
+      // "preparations" is cleaned up when adding to "invocations"
+      // "stagingTokens" is cleaned up when the invocation finishes
+      invocations.remove(completedInvocationsIds.removeFirst());
+    }
+  }
 }
diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/jobsubmission/JobInvocation.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/jobsubmission/JobInvocation.java
index 3623433..ddc379f 100644
--- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/jobsubmission/JobInvocation.java
+++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/jobsubmission/JobInvocation.java
@@ -36,7 +36,7 @@
 import org.apache.beam.model.pipeline.v1.RunnerApi.Pipeline;
 import org.apache.beam.runners.fnexecution.provisioning.JobInfo;
 import org.apache.beam.sdk.PipelineResult;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.util.Timestamps;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.util.Timestamps;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.util.concurrent.FutureCallback;
 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.ListenableFuture;
diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/jobsubmission/JobInvoker.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/jobsubmission/JobInvoker.java
index 7612d8b..0f66c38 100644
--- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/jobsubmission/JobInvoker.java
+++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/jobsubmission/JobInvoker.java
@@ -22,7 +22,7 @@
 import java.util.concurrent.ThreadFactory;
 import javax.annotation.Nullable;
 import org.apache.beam.model.pipeline.v1.RunnerApi;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.Struct;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.Struct;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.util.concurrent.ListeningExecutorService;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.util.concurrent.MoreExecutors;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.util.concurrent.ThreadFactoryBuilder;
diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/jobsubmission/JobPreparation.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/jobsubmission/JobPreparation.java
index a304093..30b0774 100644
--- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/jobsubmission/JobPreparation.java
+++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/jobsubmission/JobPreparation.java
@@ -19,7 +19,7 @@
 
 import com.google.auto.value.AutoValue;
 import org.apache.beam.model.pipeline.v1.RunnerApi.Pipeline;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.Struct;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.Struct;
 
 /** A job that has been prepared, but not invoked. */
 @AutoValue
diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/jobsubmission/JobServerDriver.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/jobsubmission/JobServerDriver.java
index d0061d2..d61162f 100644
--- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/jobsubmission/JobServerDriver.java
+++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/jobsubmission/JobServerDriver.java
@@ -63,7 +63,8 @@
             artifactStagingServer.getService().removeArtifacts(stagingSessionToken);
           }
         },
-        invoker);
+        invoker,
+        configuration.getMaxInvocationHistory());
   }
 
   /** Configuration for the jobServer. */
@@ -97,6 +98,9 @@
         handler = ExplicitBooleanOptionHandler.class)
     private boolean cleanArtifactsPerJob = true;
 
+    @Option(name = "--history-size", usage = "The maximum number of completed jobs to keep.")
+    private int maxInvocationHistory = 10;
+
     public String getHost() {
       return host;
     }
@@ -120,6 +124,10 @@
     public boolean isCleanArtifactsPerJob() {
       return cleanArtifactsPerJob;
     }
+
+    public int getMaxInvocationHistory() {
+      return maxInvocationHistory;
+    }
   }
 
   protected static ServerFactory createJobServerFactory(ServerConfiguration configuration) {
diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/jobsubmission/PortablePipelineJarCreator.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/jobsubmission/PortablePipelineJarCreator.java
index c14098a..bb70158 100644
--- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/jobsubmission/PortablePipelineJarCreator.java
+++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/jobsubmission/PortablePipelineJarCreator.java
@@ -58,9 +58,9 @@
 import org.apache.beam.sdk.metrics.MetricResults;
 import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.options.PortablePipelineOptions;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.MessageOrBuilder;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.util.JsonFormat;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.ManagedChannel;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.MessageOrBuilder;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.util.JsonFormat;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.ManagedChannel;
 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.Preconditions;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/jobsubmission/PortablePipelineJarUtils.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/jobsubmission/PortablePipelineJarUtils.java
index 291605a..d32f1e1 100644
--- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/jobsubmission/PortablePipelineJarUtils.java
+++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/jobsubmission/PortablePipelineJarUtils.java
@@ -27,9 +27,9 @@
 import java.util.jar.JarEntry;
 import java.util.jar.JarOutputStream;
 import org.apache.beam.model.pipeline.v1.RunnerApi.Pipeline;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.Message.Builder;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.Struct;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.util.JsonFormat;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.Message.Builder;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.Struct;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.util.JsonFormat;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.io.ByteStreams;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
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 a37a2f3..aa7117d 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
@@ -24,7 +24,7 @@
 import org.apache.beam.model.fnexecution.v1.BeamFnApi.LogControl;
 import org.apache.beam.model.fnexecution.v1.BeamFnLoggingGrpc;
 import org.apache.beam.runners.fnexecution.FnService;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.StreamObserver;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.stub.StreamObserver;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableSet;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/provisioning/JobInfo.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/provisioning/JobInfo.java
index aea6bb3..f9d9c66 100644
--- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/provisioning/JobInfo.java
+++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/provisioning/JobInfo.java
@@ -20,7 +20,7 @@
 import com.google.auto.value.AutoValue;
 import java.io.Serializable;
 import org.apache.beam.model.fnexecution.v1.ProvisionApi;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.Struct;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.Struct;
 
 /**
  * A subset of {@link org.apache.beam.model.fnexecution.v1.ProvisionApi.ProvisionInfo} that
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 4fec80c..aeece77 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
@@ -24,7 +24,7 @@
 import org.apache.beam.model.fnexecution.v1.ProvisionServiceGrpc;
 import org.apache.beam.model.fnexecution.v1.ProvisionServiceGrpc.ProvisionServiceImplBase;
 import org.apache.beam.runners.fnexecution.FnService;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.StreamObserver;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.stub.StreamObserver;
 
 /**
  * A {@link ProvisionServiceImplBase provision service} that returns a static response to all calls.
diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/splittabledofn/SDFFeederViaStateAndTimers.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/splittabledofn/SDFFeederViaStateAndTimers.java
index 920dae6..d4edbd2 100644
--- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/splittabledofn/SDFFeederViaStateAndTimers.java
+++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/splittabledofn/SDFFeederViaStateAndTimers.java
@@ -43,8 +43,8 @@
 import org.apache.beam.sdk.util.WindowedValue;
 import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder;
 import org.apache.beam.sdk.values.KV;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.util.Durations;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.util.Durations;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables;
 import org.joda.time.Instant;
 
diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/state/GrpcStateService.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/state/GrpcStateService.java
index 9c72d81..df10910 100644
--- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/state/GrpcStateService.java
+++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/state/GrpcStateService.java
@@ -28,8 +28,8 @@
 import org.apache.beam.model.fnexecution.v1.BeamFnApi.StateResponse;
 import org.apache.beam.model.fnexecution.v1.BeamFnStateGrpc;
 import org.apache.beam.runners.fnexecution.FnService;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.ServerCallStreamObserver;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.StreamObserver;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.stub.ServerCallStreamObserver;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.stub.StreamObserver;
 
 /** An implementation of the Beam Fn State service. */
 public class GrpcStateService extends BeamFnStateGrpc.BeamFnStateImplBase
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 f840864..988f63db 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
@@ -31,7 +31,7 @@
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.state.BagState;
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Charsets;
 
 /**
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 26dd6ac..b05776e 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
@@ -51,7 +51,7 @@
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
 import org.apache.beam.sdk.util.common.Reiterable;
 import org.apache.beam.sdk.values.KV;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
 import org.apache.beam.vendor.sdk.v2.sdk.extensions.protobuf.ByteStringCoder;
 
diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/status/BeamWorkerStatusGrpcService.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/status/BeamWorkerStatusGrpcService.java
index ce7498b..c3f9bba 100644
--- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/status/BeamWorkerStatusGrpcService.java
+++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/status/BeamWorkerStatusGrpcService.java
@@ -35,7 +35,7 @@
 import org.apache.beam.model.pipeline.v1.Endpoints.ApiServiceDescriptor;
 import org.apache.beam.runners.fnexecution.FnService;
 import org.apache.beam.runners.fnexecution.HeaderAccessor;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.StreamObserver;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.stub.StreamObserver;
 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.Strings;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableSet;
diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/status/WorkerStatusClient.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/status/WorkerStatusClient.java
index dbd7fa8..eacb3fc 100644
--- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/status/WorkerStatusClient.java
+++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/status/WorkerStatusClient.java
@@ -29,7 +29,7 @@
 import org.apache.beam.sdk.fn.IdGenerator;
 import org.apache.beam.sdk.fn.IdGenerators;
 import org.apache.beam.sdk.fn.stream.SynchronizedStreamObserver;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.StreamObserver;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.stub.StreamObserver;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
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 6e6c80a..f51072b 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
@@ -40,7 +40,7 @@
 import org.apache.beam.sdk.util.WindowedValue;
 import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.WindowingStrategy;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.InvalidProtocolBufferException;
+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.BiMap;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableBiMap;
diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/wire/WireCoders.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/wire/WireCoders.java
index f37901b..8cf7cd3 100644
--- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/wire/WireCoders.java
+++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/wire/WireCoders.java
@@ -18,11 +18,11 @@
 package org.apache.beam.runners.fnexecution.wire;
 
 import static org.apache.beam.runners.core.construction.BeamUrns.getUrn;
-import static org.apache.beam.runners.core.construction.graph.ExecutableStage.DEFAULT_WIRE_CODER_SETTING;
 import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument;
 
 import java.io.IOException;
 import org.apache.beam.model.pipeline.v1.RunnerApi;
+import org.apache.beam.model.pipeline.v1.RunnerApi.ExecutableStagePayload.WireCoderSetting;
 import org.apache.beam.runners.core.construction.ModelCoders;
 import org.apache.beam.runners.core.construction.RehydratedComponents;
 import org.apache.beam.runners.core.construction.SyntheticComponents;
@@ -45,7 +45,7 @@
   public static String addSdkWireCoder(
       PCollectionNode pCollectionNode,
       RunnerApi.Components.Builder components,
-      RunnerApi.WireCoderSetting wireCoderSetting) {
+      WireCoderSetting wireCoderSetting) {
     return addWireCoder(pCollectionNode, components, false, wireCoderSetting);
   }
 
@@ -60,7 +60,7 @@
   public static String addRunnerWireCoder(
       PCollectionNode pCollectionNode,
       RunnerApi.Components.Builder components,
-      RunnerApi.WireCoderSetting wireCoderSetting) {
+      WireCoderSetting wireCoderSetting) {
     return addWireCoder(pCollectionNode, components, true, wireCoderSetting);
   }
 
@@ -72,7 +72,8 @@
    */
   public static <T> Coder<WindowedValue<T>> instantiateRunnerWireCoder(
       PCollectionNode pCollectionNode, RunnerApi.Components components) throws IOException {
-    return instantiateRunnerWireCoder(pCollectionNode, components, DEFAULT_WIRE_CODER_SETTING);
+    return instantiateRunnerWireCoder(
+        pCollectionNode, components, WireCoderSetting.getDefaultInstance());
   }
 
   /**
@@ -84,7 +85,7 @@
   public static <T> Coder<WindowedValue<T>> instantiateRunnerWireCoder(
       PCollectionNode pCollectionNode,
       RunnerApi.Components components,
-      RunnerApi.WireCoderSetting wireCoderSetting)
+      WireCoderSetting wireCoderSetting)
       throws IOException {
     // NOTE: We discard the new set of components so we don't bother to ensure it's consistent with
     // the caller's view.
@@ -104,7 +105,7 @@
       PCollectionNode pCollectionNode,
       RunnerApi.Components.Builder components,
       boolean useByteArrayCoder,
-      RunnerApi.WireCoderSetting wireCoderSetting) {
+      WireCoderSetting wireCoderSetting) {
     String elementCoderId = pCollectionNode.getPCollection().getCoderId();
     String windowingStrategyId = pCollectionNode.getPCollection().getWindowingStrategyId();
     String windowCoderId =
diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/GrpcContextHeaderAccessorProviderTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/GrpcContextHeaderAccessorProviderTest.java
index 532e904..85763a7 100644
--- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/GrpcContextHeaderAccessorProviderTest.java
+++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/GrpcContextHeaderAccessorProviderTest.java
@@ -23,16 +23,16 @@
 import org.apache.beam.model.fnexecution.v1.BeamFnApi.Elements;
 import org.apache.beam.model.fnexecution.v1.BeamFnDataGrpc;
 import org.apache.beam.model.pipeline.v1.Endpoints.ApiServiceDescriptor;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.CallOptions;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.Channel;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.ClientCall;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.ClientInterceptor;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.ForwardingClientCall.SimpleForwardingClientCall;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.Metadata;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.MethodDescriptor;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.Server;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.inprocess.InProcessChannelBuilder;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.StreamObserver;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.CallOptions;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.Channel;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.ClientCall;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.ClientInterceptor;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.ForwardingClientCall.SimpleForwardingClientCall;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.Metadata;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.MethodDescriptor;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.Server;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.inprocess.InProcessChannelBuilder;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.stub.StreamObserver;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
 import org.junit.Assert;
 import org.junit.Test;
diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/ServerFactoryTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/ServerFactoryTest.java
index 0972d7b..2dbed76 100644
--- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/ServerFactoryTest.java
+++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/ServerFactoryTest.java
@@ -42,11 +42,11 @@
 import org.apache.beam.model.pipeline.v1.Endpoints.ApiServiceDescriptor;
 import org.apache.beam.sdk.fn.channel.ManagedChannelFactory;
 import org.apache.beam.sdk.fn.test.TestStreams;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.ManagedChannel;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.Server;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.CallStreamObserver;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.StreamObserver;
-import org.apache.beam.vendor.grpc.v1p21p0.io.netty.channel.epoll.Epoll;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.ManagedChannel;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.Server;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.stub.CallStreamObserver;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.stub.StreamObserver;
+import org.apache.beam.vendor.grpc.v1p26p0.io.netty.channel.epoll.Epoll;
 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.net.HostAndPort;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.util.concurrent.Uninterruptibles;
diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/artifact/BeamFileSystemArtifactServicesTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/artifact/BeamFileSystemArtifactServicesTest.java
index 9585530..2479da0 100644
--- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/artifact/BeamFileSystemArtifactServicesTest.java
+++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/artifact/BeamFileSystemArtifactServicesTest.java
@@ -58,10 +58,10 @@
 import org.apache.beam.runners.fnexecution.GrpcFnServer;
 import org.apache.beam.runners.fnexecution.InProcessServerFactory;
 import org.apache.beam.sdk.io.FileSystems;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.ManagedChannel;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.inprocess.InProcessChannelBuilder;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.StreamObserver;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.ManagedChannel;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.inprocess.InProcessChannelBuilder;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.stub.StreamObserver;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Strings;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Maps;
diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/artifact/ClassLoaderArtifactServiceTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/artifact/ClassLoaderArtifactServiceTest.java
index 65d54a9..849da70 100644
--- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/artifact/ClassLoaderArtifactServiceTest.java
+++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/artifact/ClassLoaderArtifactServiceTest.java
@@ -42,10 +42,10 @@
 import org.apache.beam.model.jobmanagement.v1.ArtifactStagingServiceGrpc;
 import org.apache.beam.runners.fnexecution.GrpcFnServer;
 import org.apache.beam.runners.fnexecution.InProcessServerFactory;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.ManagedChannel;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.inprocess.InProcessChannelBuilder;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.StreamObserver;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.ManagedChannel;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.inprocess.InProcessChannelBuilder;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.stub.StreamObserver;
 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.ImmutableMap;
 import org.junit.Assert;
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 b5ac3c6..1d53d35 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
@@ -62,8 +62,8 @@
 import org.apache.beam.sdk.options.ExperimentalOptions;
 import org.apache.beam.sdk.options.PipelineOptionsFactory;
 import org.apache.beam.sdk.options.PortablePipelineOptions;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.Struct;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.Struct;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
 import org.junit.Assert;
 import org.junit.Before;
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 cb65a0e..0af2602 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
@@ -35,8 +35,8 @@
 import org.apache.beam.runners.fnexecution.GrpcFnServer;
 import org.apache.beam.runners.fnexecution.InProcessServerFactory;
 import org.apache.beam.sdk.util.MoreFutures;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.inprocess.InProcessChannelBuilder;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.StreamObserver;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.inprocess.InProcessChannelBuilder;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.stub.StreamObserver;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
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 341b53c..63fbe55 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
@@ -31,7 +31,7 @@
 import org.apache.beam.model.fnexecution.v1.BeamFnApi.InstructionRequest;
 import org.apache.beam.model.fnexecution.v1.BeamFnApi.InstructionResponse;
 import org.apache.beam.sdk.util.MoreFutures;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.StreamObserver;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.stub.StreamObserver;
 import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
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 d9d51d4..6ba0b84 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
@@ -111,7 +111,7 @@
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PCollectionList;
 import org.apache.beam.sdk.values.PCollectionView;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
 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.Collections2;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/data/GrpcDataServiceTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/data/GrpcDataServiceTest.java
index be08b58..adf843d 100644
--- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/data/GrpcDataServiceTest.java
+++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/data/GrpcDataServiceTest.java
@@ -47,10 +47,10 @@
 import org.apache.beam.sdk.fn.test.TestStreams;
 import org.apache.beam.sdk.options.PipelineOptionsFactory;
 import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.ManagedChannel;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.inprocess.InProcessChannelBuilder;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.StreamObserver;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.ManagedChannel;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.inprocess.InProcessChannelBuilder;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.stub.StreamObserver;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/jobsubmission/InMemoryJobServiceTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/jobsubmission/InMemoryJobServiceTest.java
index e7b01af..2a9b366 100644
--- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/jobsubmission/InMemoryJobServiceTest.java
+++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/jobsubmission/InMemoryJobServiceTest.java
@@ -17,34 +17,39 @@
  */
 package org.apache.beam.runners.fnexecution.jobsubmission;
 
+import static org.hamcrest.MatcherAssert.assertThat;
 import static org.hamcrest.collection.IsCollectionWithSize.hasSize;
 import static org.hamcrest.core.Is.is;
 import static org.hamcrest.core.Is.isA;
 import static org.hamcrest.core.IsNull.notNullValue;
-import static org.junit.Assert.assertThat;
 import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.atLeastOnce;
 import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
 
 import java.util.ArrayList;
 import java.util.List;
+import java.util.UUID;
+import java.util.function.Consumer;
 import org.apache.beam.model.jobmanagement.v1.JobApi;
 import org.apache.beam.model.pipeline.v1.Endpoints;
 import org.apache.beam.model.pipeline.v1.RunnerApi;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.Struct;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.StatusException;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.StreamObserver;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.Struct;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.StatusException;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.stub.StreamObserver;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
+import org.mockito.ArgumentCaptor;
 import org.mockito.Mock;
 import org.mockito.MockitoAnnotations;
 
 /** Tests for {@link InMemoryJobService}. */
 @RunWith(JUnit4.class)
 public class InMemoryJobServiceTest {
+
   private static final String TEST_JOB_NAME = "test-job";
   private static final String TEST_JOB_ID = "test-job-id";
   private static final String TEST_RETRIEVAL_TOKEN = "test-staging-token";
@@ -57,6 +62,8 @@
           .setPipelineOptions(TEST_OPTIONS)
           .build();
 
+  private final int maxInvocationHistory = 3;
+
   Endpoints.ApiServiceDescriptor stagingServiceDescriptor;
   @Mock JobInvoker invoker;
   @Mock JobInvocation invocation;
@@ -186,7 +193,79 @@
     verify(invocation, times(1)).start();
   }
 
+  @Test
+  public void testInvocationCleanup() {
+    final int maxInvocationHistory = 3;
+    service =
+        InMemoryJobService.create(
+            stagingServiceDescriptor, session -> "token", null, invoker, maxInvocationHistory);
+
+    assertThat(getNumberOfInvocations(), is(0));
+
+    Job job1 = runJob();
+    assertThat(getNumberOfInvocations(), is(1));
+    Job job2 = runJob();
+    assertThat(getNumberOfInvocations(), is(2));
+    Job job3 = runJob();
+    assertThat(getNumberOfInvocations(), is(maxInvocationHistory));
+
+    // All running invocations must be available and never be discarded
+    // even if they exceed the max history size
+    Job job4 = runJob();
+    assertThat(getNumberOfInvocations(), is(maxInvocationHistory + 1));
+
+    // We need to have more than maxInvocationHistory completed jobs for the cleanup to trigger
+    job1.finish();
+    assertThat(getNumberOfInvocations(), is(maxInvocationHistory + 1));
+    job2.finish();
+    assertThat(getNumberOfInvocations(), is(maxInvocationHistory + 1));
+    job3.finish();
+    assertThat(getNumberOfInvocations(), is(maxInvocationHistory + 1));
+
+    // The fourth finished job exceeds maxInvocationHistory and triggers the cleanup
+    job4.finish();
+    assertThat(getNumberOfInvocations(), is(maxInvocationHistory));
+
+    // Run a new job after the cleanup
+    Job job5 = runJob();
+    assertThat(getNumberOfInvocations(), is(maxInvocationHistory + 1));
+    job5.finish();
+    assertThat(getNumberOfInvocations(), is(maxInvocationHistory));
+  }
+
+  private Job runJob() {
+    when(invocation.getId()).thenReturn(UUID.randomUUID().toString());
+    prepareAndRunJob();
+    // Retrieve the state listener for this invocation
+    ArgumentCaptor<Consumer<JobApi.JobStateEvent>> stateListener =
+        ArgumentCaptor.forClass(Consumer.class);
+    verify(invocation, atLeastOnce()).addStateListener(stateListener.capture());
+    return new Job(stateListener.getValue());
+  }
+
+  private int getNumberOfInvocations() {
+    RecordingObserver<JobApi.GetJobsResponse> recorder = new RecordingObserver<>();
+    final JobApi.GetJobsRequest getJobsRequest = JobApi.GetJobsRequest.newBuilder().build();
+    service.getJobs(getJobsRequest, recorder);
+    return recorder.getValue().getJobInfoCount();
+  }
+
+  private static class Job {
+    private Consumer<JobApi.JobStateEvent> stateListener;
+
+    private Job(Consumer<JobApi.JobStateEvent> stateListener) {
+      this.stateListener = stateListener;
+    }
+
+    void finish() {
+      JobApi.JobStateEvent terminalEvent =
+          JobApi.JobStateEvent.newBuilder().setState(JobApi.JobState.Enum.DONE).build();
+      stateListener.accept(terminalEvent);
+    }
+  }
+
   private static class RecordingObserver<T> implements StreamObserver<T> {
+
     ArrayList<T> values = new ArrayList<>();
     Throwable error = null;
     boolean isCompleted = false;
@@ -206,6 +285,11 @@
       isCompleted = true;
     }
 
+    T getValue() {
+      assert values.size() == 1;
+      return values.get(0);
+    }
+
     boolean isSuccessful() {
       return isCompleted && error == null;
     }
diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/jobsubmission/JobInvocationTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/jobsubmission/JobInvocationTest.java
index 30e34d4..f9c256f 100644
--- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/jobsubmission/JobInvocationTest.java
+++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/jobsubmission/JobInvocationTest.java
@@ -33,7 +33,7 @@
 import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.PipelineResult;
 import org.apache.beam.sdk.metrics.MetricResults;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.Struct;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.Struct;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.util.concurrent.ListeningExecutorService;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.util.concurrent.MoreExecutors;
 import org.joda.time.Duration;
diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/logging/GrpcLoggingServiceTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/logging/GrpcLoggingServiceTest.java
index 3bfda79..39a5e55 100644
--- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/logging/GrpcLoggingServiceTest.java
+++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/logging/GrpcLoggingServiceTest.java
@@ -37,9 +37,9 @@
 import org.apache.beam.runners.fnexecution.GrpcFnServer;
 import org.apache.beam.runners.fnexecution.InProcessServerFactory;
 import org.apache.beam.sdk.fn.test.TestStreams;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.ManagedChannel;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.inprocess.InProcessChannelBuilder;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.StreamObserver;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.ManagedChannel;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.inprocess.InProcessChannelBuilder;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.stub.StreamObserver;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/provisioning/StaticGrpcProvisionServiceTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/provisioning/StaticGrpcProvisionServiceTest.java
index 850a070..a0fd1b7 100644
--- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/provisioning/StaticGrpcProvisionServiceTest.java
+++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/provisioning/StaticGrpcProvisionServiceTest.java
@@ -31,11 +31,11 @@
 import org.apache.beam.model.fnexecution.v1.ProvisionServiceGrpc.ProvisionServiceBlockingStub;
 import org.apache.beam.runners.fnexecution.GrpcFnServer;
 import org.apache.beam.runners.fnexecution.InProcessServerFactory;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ListValue;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.NullValue;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.Struct;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.Value;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.inprocess.InProcessChannelBuilder;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ListValue;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.NullValue;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.Struct;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.Value;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.inprocess.InProcessChannelBuilder;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
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 f8b3f29..aa986c9 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
@@ -31,8 +31,8 @@
 import java.util.concurrent.TimeUnit;
 import org.apache.beam.model.fnexecution.v1.BeamFnApi;
 import org.apache.beam.sdk.fn.test.TestStreams;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.StreamObserver;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.stub.StreamObserver;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.runner.RunWith;
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 c9d6544..9b04914 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
@@ -39,10 +39,10 @@
 import org.apache.beam.runners.fnexecution.GrpcContextHeaderAccessorProvider;
 import org.apache.beam.runners.fnexecution.GrpcFnServer;
 import org.apache.beam.runners.fnexecution.InProcessServerFactory;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.ManagedChannel;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.inprocess.InProcessChannelBuilder;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.StreamObserver;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.testing.GrpcCleanupRule;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.ManagedChannel;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.inprocess.InProcessChannelBuilder;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.stub.StreamObserver;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.testing.GrpcCleanupRule;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Sets;
 import org.junit.After;
 import org.junit.Before;
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 0aca49a..5fa143e 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
@@ -26,7 +26,7 @@
 import org.apache.beam.model.fnexecution.v1.BeamFnApi;
 import org.apache.beam.model.fnexecution.v1.BeamFnApi.WorkerStatusRequest;
 import org.apache.beam.model.fnexecution.v1.BeamFnApi.WorkerStatusResponse;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.StreamObserver;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.stub.StreamObserver;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
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 e1ccad6..1f45945 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
@@ -17,7 +17,7 @@
  */
 package org.apache.beam.runners.fnexecution.translation;
 
-import static org.apache.beam.runners.core.construction.graph.ExecutableStage.DEFAULT_WIRE_CODER_SETTING;
+import static org.apache.beam.runners.core.construction.graph.ExecutableStage.DEFAULT_WIRE_CODER_SETTINGS;
 import static org.hamcrest.MatcherAssert.assertThat;
 import static org.hamcrest.Matchers.contains;
 import static org.hamcrest.Matchers.containsInAnyOrder;
@@ -236,6 +236,6 @@
         Collections.emptyList(),
         Collections.emptyList(),
         Collections.emptyList(),
-        DEFAULT_WIRE_CODER_SETTING);
+        DEFAULT_WIRE_CODER_SETTINGS);
   }
 }
diff --git a/runners/portability/java/build.gradle b/runners/portability/java/build.gradle
index 5425b8f..9c529a6 100644
--- a/runners/portability/java/build.gradle
+++ b/runners/portability/java/build.gradle
@@ -33,7 +33,7 @@
   compile library.java.hamcrest_library
   compile project(":runners:java-fn-execution")
   compile project(path: ":sdks:java:harness", configuration: "shadow")
-  compile library.java.vendored_grpc_1_21_0
+  compile library.java.vendored_grpc_1_26_0
   compile library.java.slf4j_api
   testCompile project(path: ":runners:core-construction-java", configuration: "testRuntime")
   testCompile library.java.hamcrest_core
diff --git a/runners/portability/java/src/main/java/org/apache/beam/runners/portability/ExternalWorkerService.java b/runners/portability/java/src/main/java/org/apache/beam/runners/portability/ExternalWorkerService.java
index 028a934..363b013 100644
--- a/runners/portability/java/src/main/java/org/apache/beam/runners/portability/ExternalWorkerService.java
+++ b/runners/portability/java/src/main/java/org/apache/beam/runners/portability/ExternalWorkerService.java
@@ -25,7 +25,7 @@
 import org.apache.beam.runners.fnexecution.GrpcFnServer;
 import org.apache.beam.runners.fnexecution.ServerFactory;
 import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.StreamObserver;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.stub.StreamObserver;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
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 d4f2aa7..820e93b 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
@@ -34,7 +34,7 @@
 import org.apache.beam.model.jobmanagement.v1.JobServiceGrpc.JobServiceBlockingStub;
 import org.apache.beam.sdk.PipelineResult;
 import org.apache.beam.sdk.metrics.MetricResults;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
 import org.joda.time.Duration;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
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 72c3cbc..31c7be0 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
@@ -53,8 +53,8 @@
 import org.apache.beam.sdk.options.PipelineOptionsValidator;
 import org.apache.beam.sdk.options.PortablePipelineOptions;
 import org.apache.beam.sdk.util.ZipFiles;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.ManagedChannel;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.ManagedChannel;
 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;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Sets;
diff --git a/runners/portability/java/src/main/java/org/apache/beam/runners/portability/testing/TestJobService.java b/runners/portability/java/src/main/java/org/apache/beam/runners/portability/testing/TestJobService.java
index 76b8974..53578b7 100644
--- a/runners/portability/java/src/main/java/org/apache/beam/runners/portability/testing/TestJobService.java
+++ b/runners/portability/java/src/main/java/org/apache/beam/runners/portability/testing/TestJobService.java
@@ -27,7 +27,7 @@
 import org.apache.beam.model.jobmanagement.v1.JobApi.RunJobResponse;
 import org.apache.beam.model.jobmanagement.v1.JobServiceGrpc.JobServiceImplBase;
 import org.apache.beam.model.pipeline.v1.Endpoints.ApiServiceDescriptor;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.StreamObserver;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.stub.StreamObserver;
 
 /**
  * A JobService for tests.
diff --git a/runners/portability/java/src/test/java/org/apache/beam/runners/portability/PortableRunnerTest.java b/runners/portability/java/src/test/java/org/apache/beam/runners/portability/PortableRunnerTest.java
index 40f7900..9f011f3 100644
--- a/runners/portability/java/src/test/java/org/apache/beam/runners/portability/PortableRunnerTest.java
+++ b/runners/portability/java/src/test/java/org/apache/beam/runners/portability/PortableRunnerTest.java
@@ -38,9 +38,9 @@
 import org.apache.beam.sdk.options.PipelineOptionsFactory;
 import org.apache.beam.sdk.options.PortablePipelineOptions;
 import org.apache.beam.sdk.testing.TestPipeline;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.Timestamp;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.Server;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.inprocess.InProcessServerBuilder;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.Timestamp;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.Server;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.inprocess.InProcessServerBuilder;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.runner.RunWith;
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 f21d666..b820687 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
@@ -32,7 +32,7 @@
 import org.apache.beam.runners.fnexecution.jobsubmission.JobInvoker;
 import org.apache.beam.runners.fnexecution.provisioning.JobInfo;
 import org.apache.beam.sdk.options.PipelineOptionsFactory;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.Struct;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.Struct;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.util.concurrent.ListeningExecutorService;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -101,7 +101,8 @@
           }
         },
         stagingSessionToken -> {},
-        jobInvoker);
+        jobInvoker,
+        InMemoryJobService.DEFAULT_MAX_INVOCATION_HISTORY);
   }
 
   public void run() throws Exception {
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 330fb24..7673bd3 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
@@ -139,8 +139,8 @@
     }
 
     @Override
-    public void deleteTimer(StateNamespace namespace, String timerId) {
-      getInternals().deleteTimer(namespace, timerId);
+    public void deleteTimer(StateNamespace namespace, String timerId, String timerFamilyId) {
+      getInternals().deleteTimer(namespace, timerId, timerFamilyId);
     }
 
     @Override
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 9ac082b..22c2ac9 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
@@ -248,7 +248,7 @@
     }
 
     @Override
-    public void deleteTimer(StateNamespace namespace, String timerId) {
+    public void deleteTimer(StateNamespace namespace, String timerId, String timerFamilyId) {
       deleteTimer(TimerData.of(timerId, namespace, Instant.now(), TimeDomain.EVENT_TIME));
     }
 
diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/WindowAssignTranslator.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/WindowAssignTranslator.java
index 95c7328..114a256 100644
--- a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/WindowAssignTranslator.java
+++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/WindowAssignTranslator.java
@@ -28,7 +28,7 @@
 import org.apache.beam.sdk.transforms.windowing.Window;
 import org.apache.beam.sdk.transforms.windowing.WindowFn;
 import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.InvalidProtocolBufferException;
 import org.apache.samza.operators.MessageStream;
 
 /**
diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/util/SamzaPipelineTranslatorUtils.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/util/SamzaPipelineTranslatorUtils.java
index 758515a4..b1bfc83 100644
--- a/runners/samza/src/main/java/org/apache/beam/runners/samza/util/SamzaPipelineTranslatorUtils.java
+++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/util/SamzaPipelineTranslatorUtils.java
@@ -28,7 +28,7 @@
 import org.apache.beam.sdk.util.WindowedValue;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.WindowingStrategy;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.InvalidProtocolBufferException;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables;
 
 /** Utilities for pipeline translation. */
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 2ea261f..956e9bc 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
@@ -27,7 +27,7 @@
 import org.apache.beam.runners.fnexecution.jobsubmission.PortablePipelineRunner;
 import org.apache.beam.runners.fnexecution.provisioning.JobInfo;
 import org.apache.beam.sdk.options.PortablePipelineOptions;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.Struct;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.Struct;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Strings;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.util.concurrent.ListeningExecutorService;
 import org.slf4j.Logger;
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 d0c1c0f..e5166a3 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
@@ -49,7 +49,7 @@
 import org.apache.beam.sdk.options.PipelineOptionsFactory;
 import org.apache.beam.sdk.options.PortablePipelineOptions;
 import org.apache.beam.sdk.options.PortablePipelineOptions.RetrievalServiceType;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.Struct;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.Struct;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions;
 import org.apache.spark.api.java.JavaSparkContext;
 import org.kohsuke.args4j.CmdLineException;
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 1d5b36b..58fbee9 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
@@ -165,7 +165,7 @@
   }
 
   @Override
-  public void deleteTimer(StateNamespace namespace, String timerId) {
+  public void deleteTimer(StateNamespace namespace, String timerId, String timerFamilyId) {
     throw new UnsupportedOperationException("Deleting a timer by ID is not yet supported.");
   }
 
diff --git a/sdks/java/container/build.gradle b/sdks/java/container/build.gradle
index ca5b3cf..5c182c9 100644
--- a/sdks/java/container/build.gradle
+++ b/sdks/java/container/build.gradle
@@ -74,7 +74,7 @@
           root: project.rootProject.hasProperty(["docker-repository-root"]) ?
                   project.rootProject["docker-repository-root"] : "apachebeam",
           tag: project.rootProject.hasProperty(["docker-tag"]) ?
-                  project.rootProject["docker-tag"] : project.version)
+                  project.rootProject["docker-tag"] : project.sdk_version)
   dockerfile project.file("./${dockerfileName}")
   files "./build/"
 }
diff --git a/sdks/java/core/build.gradle b/sdks/java/core/build.gradle
index a14305d..ea0f5c9 100644
--- a/sdks/java/core/build.gradle
+++ b/sdks/java/core/build.gradle
@@ -46,6 +46,7 @@
 processResources {
   filter org.apache.tools.ant.filters.ReplaceTokens, tokens: [
     'pom.version': version,
+    'pom.sdk_version': sdk_version,
     'timestamp': new Date().format("yyyy-MM-dd HH:mm")
   ]
 }
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 27863ce..a125444 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
@@ -33,8 +33,9 @@
 import java.util.SortedSet;
 import javax.annotation.Nullable;
 import org.apache.avro.AvroRuntimeException;
+import org.apache.avro.Conversion;
+import org.apache.avro.LogicalType;
 import org.apache.avro.Schema;
-import org.apache.avro.data.TimeConversions.TimestampConversion;
 import org.apache.avro.generic.GenericDatumReader;
 import org.apache.avro.generic.GenericDatumWriter;
 import org.apache.avro.generic.GenericRecord;
@@ -59,6 +60,8 @@
 import org.apache.beam.sdk.values.TypeDescriptor;
 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.base.Suppliers;
+import org.joda.time.DateTime;
+import org.joda.time.DateTimeZone;
 
 /**
  * A {@link Coder} using Avro binary format.
@@ -239,7 +242,7 @@
     @Override
     public ReflectData get() {
       ReflectData reflectData = new ReflectData(clazz.getClassLoader());
-      reflectData.addLogicalTypeConversion(new TimestampConversion());
+      reflectData.addLogicalTypeConversion(new JodaTimestampConversion());
       return reflectData;
     }
   }
@@ -715,4 +718,35 @@
   public int hashCode() {
     return Objects.hash(schemaSupplier.get(), typeDescriptor);
   }
+
+  /**
+   * Conversion for DateTime.
+   *
+   * <p>This is a copy from Avro 1.8's TimestampConversion, which is renamed in Avro 1.9. Defining
+   * own copy gives flexibility for Beam Java SDK to work with Avro 1.8 and 1.9 at runtime.
+   *
+   * @see <a href="https://issues.apache.org/jira/browse/BEAM-9144">BEAM-9144: Beam's own Avro
+   *     TimeConversion class in beam-sdk-java-core</a>
+   */
+  public static class JodaTimestampConversion extends Conversion<DateTime> {
+    @Override
+    public Class<DateTime> getConvertedType() {
+      return DateTime.class;
+    }
+
+    @Override
+    public String getLogicalTypeName() {
+      return "timestamp-millis";
+    }
+
+    @Override
+    public DateTime fromLong(Long millisFromEpoch, Schema schema, LogicalType type) {
+      return new DateTime(millisFromEpoch, DateTimeZone.UTC);
+    }
+
+    @Override
+    public Long toLong(DateTime timestamp, Schema schema, LogicalType type) {
+      return timestamp.getMillis();
+    }
+  }
 }
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 6784712..380e324 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
@@ -31,7 +31,7 @@
 import org.apache.beam.sdk.schemas.Schema.TypeName;
 import org.apache.beam.sdk.util.SerializableUtils;
 import org.apache.beam.sdk.values.Row;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
 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.Maps;
 
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 3740de6..b9b9119 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
@@ -38,7 +38,6 @@
 import org.apache.avro.LogicalType;
 import org.apache.avro.LogicalTypes;
 import org.apache.avro.Schema.Type;
-import org.apache.avro.data.TimeConversions;
 import org.apache.avro.generic.GenericData;
 import org.apache.avro.generic.GenericFixed;
 import org.apache.avro.generic.GenericRecord;
@@ -51,6 +50,7 @@
 import org.apache.avro.util.Utf8;
 import org.apache.beam.sdk.annotations.Experimental;
 import org.apache.beam.sdk.coders.AvroCoder;
+import org.apache.beam.sdk.coders.AvroCoder.JodaTimestampConversion;
 import org.apache.beam.sdk.schemas.AvroRecordSchema;
 import org.apache.beam.sdk.schemas.FieldValueGetter;
 import org.apache.beam.sdk.schemas.FieldValueTypeInformation;
@@ -94,8 +94,8 @@
   static {
     // This works around a bug in the Avro library (AVRO-1891) around SpecificRecord's handling
     // of DateTime types.
-    SpecificData.get().addLogicalTypeConversion(new TimeConversions.TimestampConversion());
-    GenericData.get().addLogicalTypeConversion(new TimeConversions.TimestampConversion());
+    SpecificData.get().addLogicalTypeConversion(new JodaTimestampConversion());
+    GenericData.get().addLogicalTypeConversion(new JodaTimestampConversion());
   }
 
   // Unwrap an AVRO schema into the base type an whether it is nullable.
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/state/TimerMap.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/state/TimerMap.java
new file mode 100644
index 0000000..92c6d59
--- /dev/null
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/state/TimerMap.java
@@ -0,0 +1,29 @@
+/*
+ * 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.state;
+
+import org.apache.beam.sdk.annotations.Experimental;
+import org.joda.time.Instant;
+
+@Experimental(Experimental.Kind.TIMERS)
+public interface TimerMap {
+
+  void set(String timerId, Instant absoluteTime);
+
+  Timer get(String timerId);
+}
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/state/TimerSpecs.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/state/TimerSpecs.java
index 32bc54c..d5a2159 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/state/TimerSpecs.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/state/TimerSpecs.java
@@ -29,6 +29,10 @@
     return new AutoValue_TimerSpecs_SimpleTimerSpec(timeDomain);
   }
 
+  public static TimerSpec timerMap(TimeDomain timeDomain) {
+    return new AutoValue_TimerSpecs_SimpleTimerSpec(timeDomain);
+  }
+
   /** A straightforward POJO {@link TimerSpec}. Package-level access for AutoValue. */
   @AutoValue
   abstract static class SimpleTimerSpec implements TimerSpec {
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/UsesTimerMap.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/UsesTimerMap.java
new file mode 100644
index 0000000..2b7bb67
--- /dev/null
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/UsesTimerMap.java
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.testing;
+/**
+ * Category tag for validation tests which use timerMap. Tests tagged with {@link UsesTimerMap}
+ * should be run for runners which support timerMap.
+ */
+public interface UsesTimerMap {}
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java
index 3b2b24e..53ce058 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java
@@ -433,6 +433,16 @@
   @Experimental(Kind.TIMERS)
   public @interface TimerId {
     /** The timer ID. */
+    String value() default "";
+  }
+
+  /** Parameter annotation for the TimerMap for a {@link ProcessElement} method. */
+  @Documented
+  @Retention(RetentionPolicy.RUNTIME)
+  @Target({ElementType.FIELD, ElementType.PARAMETER})
+  @Experimental(Kind.TIMERS)
+  public @interface TimerFamily {
+    /** The TimerMap tag ID. */
     String value();
   }
 
@@ -464,6 +474,25 @@
   }
 
   /**
+   * Annotation for registering a callback for a timerFamily.
+   *
+   * <p>See the javadoc for {@link TimerFamily} for use in a full example.
+   *
+   * <p>The method annotated with {@code @OnTimerFamily} may have parameters according to the same
+   * logic as {@link ProcessElement}, but limited to the {@link BoundedWindow}, {@link State}
+   * subclasses, and {@link org.apache.beam.sdk.state.TimerMap}. State and timer parameters must be
+   * annotated with their {@link StateId} and {@link TimerId} respectively.
+   */
+  @Documented
+  @Retention(RetentionPolicy.RUNTIME)
+  @Target(ElementType.METHOD)
+  @Experimental(Kind.TIMERS)
+  public @interface OnTimerFamily {
+    /** The timer ID. */
+    String value();
+  }
+
+  /**
    * Annotation for the method to use for performing actions on window expiration. For example,
    * users can use this annotation to write a method that extracts a value saved in a state before
    * it gets garbage collected on window expiration.
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java
index 3bec152..5fb574d 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnTester.java
@@ -33,6 +33,7 @@
 import org.apache.beam.sdk.options.PipelineOptionsFactory;
 import org.apache.beam.sdk.state.TimeDomain;
 import org.apache.beam.sdk.state.Timer;
+import org.apache.beam.sdk.state.TimerMap;
 import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.transforms.DoFn.MultiOutputReceiver;
 import org.apache.beam.sdk.transforms.DoFn.OnTimerContext;
@@ -267,6 +268,12 @@
             }
 
             @Override
+            public String timerId(DoFn<InputT, OutputT> doFn) {
+              throw new UnsupportedOperationException(
+                  "Cannot access timerId as parameter outside of @OnTimer method.");
+            }
+
+            @Override
             public TimeDomain timeDomain(DoFn<InputT, OutputT> doFn) {
               throw new UnsupportedOperationException(
                   "Not expected to access TimeDomain from @ProcessElement");
@@ -307,6 +314,11 @@
             public Timer timer(String timerId) {
               throw new UnsupportedOperationException("DoFnTester doesn't support timers yet");
             }
+
+            @Override
+            public TimerMap timerFamily(String tagId) {
+              throw new UnsupportedOperationException("DoFnTester doesn't support timerFamily yet");
+            }
           });
     } catch (UserCodeException e) {
       unwrapUserCodeException(e);
@@ -688,6 +700,11 @@
             }
 
             @Override
+            public Void dispatch(DoFnSignature.Parameter.TimerIdParameter p) {
+              return null;
+            }
+
+            @Override
             protected Void dispatchDefault(DoFnSignature.Parameter p) {
               throw new UnsupportedOperationException(
                   "Parameter " + p + " not supported by DoFnTester");
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 aec80c4..bca2eb4 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
@@ -548,6 +548,9 @@
     for (OnTimerMethod method : signature.onTimerMethods().values()) {
       validateWindowTypeForMethod(actualWindowT, method);
     }
+    for (DoFnSignature.OnTimerFamilyMethod method : signature.onTimerFamilyMethods().values()) {
+      validateWindowTypeForMethod(actualWindowT, method);
+    }
   }
 
   private static void validateWindowTypeForMethod(
@@ -586,6 +589,15 @@
               "%s is splittable and uses timers, but these are not compatible",
               fn.getClass().getName()));
     }
+
+    // TimerFamily is semantically incompatible with splitting
+    if (!signature.timerFamilyDeclarations().isEmpty()
+        && signature.processElement().isSplittable()) {
+      throw new UnsupportedOperationException(
+          String.format(
+              "%s is splittable and uses timer family, but these are not compatible",
+              fn.getClass().getName()));
+    }
   }
   /**
    * Extract information on how the DoFn uses schemas. In particular, if the schema of an element
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 2f54c27..33013fe 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
@@ -33,6 +33,7 @@
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.coders.CoderRegistry;
 import org.apache.beam.sdk.state.Timer;
+import org.apache.beam.sdk.state.TimerMap;
 import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.transforms.DoFn.ProcessElement;
 import org.apache.beam.sdk.transforms.reflect.DoFnSignature.OnTimerMethod;
@@ -50,6 +51,7 @@
 import org.apache.beam.sdk.transforms.reflect.DoFnSignature.Parameter.StateParameter;
 import org.apache.beam.sdk.transforms.reflect.DoFnSignature.Parameter.TaggedOutputReceiverParameter;
 import org.apache.beam.sdk.transforms.reflect.DoFnSignature.Parameter.TimeDomainParameter;
+import org.apache.beam.sdk.transforms.reflect.DoFnSignature.Parameter.TimerFamilyParameter;
 import org.apache.beam.sdk.transforms.reflect.DoFnSignature.Parameter.TimerParameter;
 import org.apache.beam.sdk.transforms.reflect.DoFnSignature.Parameter.TimestampParameter;
 import org.apache.beam.sdk.transforms.reflect.DoFnSignature.Parameter.WindowParameter;
@@ -114,6 +116,8 @@
   public static final String STATE_PARAMETER_METHOD = "state";
   public static final String TIMER_PARAMETER_METHOD = "timer";
   public static final String SIDE_INPUT_PARAMETER_METHOD = "sideInput";
+  public static final String TIMER_FAMILY_PARAMETER_METHOD = "timerFamily";
+  public static final String TIMER_ID_PARAMETER_METHOD = "timerId";
 
   /**
    * Returns a {@link ByteBuddyDoFnInvokerFactory} shared with all other invocations, so that its
@@ -185,7 +189,9 @@
 
     @Override
     public void invokeOnTimer(
-        String timerId, DoFnInvoker.ArgumentProvider<InputT, OutputT> arguments) {
+        String timerId,
+        String timerFamilyId,
+        DoFnInvoker.ArgumentProvider<InputT, OutputT> arguments) {
       @Nullable OnTimerInvoker onTimerInvoker = onTimerInvokers.get(timerId);
 
       if (onTimerInvoker != null) {
@@ -206,6 +212,59 @@
     }
   }
 
+  /**
+   * Internal base class for generated {@link DoFnInvoker} instances.
+   *
+   * <p>This class should <i>not</i> be extended directly, or by Beam users. It must be public for
+   * generated instances to have adequate access, as they are generated "inside" the invoked {@link
+   * DoFn} class.
+   */
+  public abstract static class DoFnInvokerTimerFamily<
+          InputT, OutputT, DoFnT extends DoFn<InputT, OutputT>>
+      implements DoFnInvoker<InputT, OutputT> {
+    protected DoFnT delegate;
+
+    private Map<String, OnTimerInvoker> onTimerInvokers = new HashMap<>();
+
+    public DoFnInvokerTimerFamily(DoFnT delegate) {
+      this.delegate = delegate;
+    }
+
+    /**
+     * Associates the given timerFamily ID with the given {@link OnTimerInvoker}.
+     *
+     * <p>ByteBuddy does not like to generate conditional code, so we use a map + lookup of the
+     * timer ID rather than a generated conditional branch to choose which OnTimerInvoker to invoke.
+     */
+    void addOnTimerFamilyInvoker(String timerFamilyId, OnTimerInvoker onTimerInvoker) {
+      this.onTimerInvokers.put(timerFamilyId, onTimerInvoker);
+    }
+
+    @Override
+    public void invokeOnTimer(
+        String timerId,
+        String timerFamilyId,
+        DoFnInvoker.ArgumentProvider<InputT, OutputT> arguments) {
+      @Nullable OnTimerInvoker onTimerInvoker = onTimerInvokers.get(timerFamilyId);
+
+      if (onTimerInvoker != null) {
+        onTimerInvoker.invokeOnTimer(arguments);
+      } else {
+        throw new IllegalArgumentException(
+            String.format(
+                "Attempted to invoke timerFamily %s on %s, but that timerFamily is not registered."
+                    + " This is the responsibility of the runner, which must only deliver"
+                    + " registered timers.",
+                timerFamilyId, delegate.getClass().getName()));
+      }
+    }
+
+    @Override
+    public DoFn<InputT, OutputT> getFn() {
+      return delegate;
+    }
+  }
+
   /** @return the {@link DoFnInvoker} for the given {@link DoFn}. */
   public <InputT, OutputT> DoFnInvoker<InputT, OutputT> newByteBuddyInvoker(
       DoFnSignature signature, DoFn<InputT, OutputT> fn) {
@@ -216,17 +275,33 @@
         fn.getClass());
 
     try {
-      @SuppressWarnings("unchecked")
-      DoFnInvokerBase<InputT, OutputT, DoFn<InputT, OutputT>> invoker =
-          (DoFnInvokerBase<InputT, OutputT, DoFn<InputT, OutputT>>)
-              getByteBuddyInvokerConstructor(signature).newInstance(fn);
+      if (signature.timerFamilyDeclarations().size() > 0) {
+        @SuppressWarnings("unchecked")
+        DoFnInvokerTimerFamily<InputT, OutputT, DoFn<InputT, OutputT>> invoker =
+            (DoFnInvokerTimerFamily<InputT, OutputT, DoFn<InputT, OutputT>>)
+                getByteBuddyInvokerConstructor(signature).newInstance(fn);
 
-      for (OnTimerMethod onTimerMethod : signature.onTimerMethods().values()) {
-        invoker.addOnTimerInvoker(
-            onTimerMethod.id(), OnTimerInvokers.forTimer(fn, onTimerMethod.id()));
+        for (DoFnSignature.OnTimerFamilyMethod onTimerFamilyMethod :
+            signature.onTimerFamilyMethods().values()) {
+          invoker.addOnTimerFamilyInvoker(
+              onTimerFamilyMethod.id(),
+              OnTimerInvokers.forTimerFamily(fn, onTimerFamilyMethod.id()));
+        }
+        return invoker;
+      } else {
+
+        @SuppressWarnings("unchecked")
+        DoFnInvokerBase<InputT, OutputT, DoFn<InputT, OutputT>> invoker =
+            (DoFnInvokerBase<InputT, OutputT, DoFn<InputT, OutputT>>)
+                getByteBuddyInvokerConstructor(signature).newInstance(fn);
+
+        for (OnTimerMethod onTimerMethod : signature.onTimerMethods().values()) {
+          invoker.addOnTimerInvoker(
+              onTimerMethod.id(), OnTimerInvokers.forTimer(fn, onTimerMethod.id()));
+        }
+        return invoker;
       }
 
-      return invoker;
     } catch (InstantiationException
         | IllegalAccessException
         | IllegalArgumentException
@@ -246,7 +321,12 @@
     Class<? extends DoFn<?, ?>> fnClass = signature.fnClass();
     Constructor<?> constructor = byteBuddyInvokerConstructorCache.get(fnClass);
     if (constructor == null) {
-      Class<? extends DoFnInvoker<?, ?>> invokerClass = generateInvokerClass(signature);
+      Class<? extends DoFnInvoker<?, ?>> invokerClass =
+          generateInvokerClass(
+              signature,
+              signature.timerFamilyDeclarations().size() > 0
+                  ? DoFnInvokerTimerFamily.class
+                  : DoFnInvokerBase.class);
       try {
         constructor = invokerClass.getConstructor(fnClass);
       } catch (IllegalArgumentException | NoSuchMethodException | SecurityException e) {
@@ -293,7 +373,8 @@
   }
 
   /** Generates a {@link DoFnInvoker} class for the given {@link DoFnSignature}. */
-  private static Class<? extends DoFnInvoker<?, ?>> generateInvokerClass(DoFnSignature signature) {
+  private static Class<? extends DoFnInvoker<?, ?>> generateInvokerClass(
+      DoFnSignature signature, Class<? extends DoFnInvoker> clazz) {
     Class<? extends DoFn<?, ?>> fnClass = signature.fnClass();
 
     final TypeDescription clazzDescription = new TypeDescription.ForLoadedType(fnClass);
@@ -307,12 +388,12 @@
                     .withSuffix(DoFnInvoker.class.getSimpleName()))
 
             // class <invoker class> extends DoFnInvokerBase {
-            .subclass(DoFnInvokerBase.class, ConstructorStrategy.Default.NO_CONSTRUCTORS)
+            .subclass(clazz, ConstructorStrategy.Default.NO_CONSTRUCTORS)
 
             //   public <invoker class>(<fn class> delegate) { this.delegate = delegate; }
             .defineConstructor(Visibility.PUBLIC)
             .withParameter(fnClass)
-            .intercept(new InvokerConstructor())
+            .intercept(new InvokerConstructor(clazz))
 
             //   public invokeProcessElement(ProcessContext, ExtraContextFactory) {
             //     delegate.<@ProcessElement>(... pass just the right args ...);
@@ -786,6 +867,16 @@
           }
 
           @Override
+          public StackManipulation dispatch(TimerFamilyParameter p) {
+            return new StackManipulation.Compound(
+                new TextConstant(p.referent().id()),
+                MethodInvocation.invoke(
+                    getExtraContextFactoryMethodDescription(
+                        TIMER_FAMILY_PARAMETER_METHOD, String.class)),
+                TypeCasting.to(new TypeDescription.ForLoadedType(TimerMap.class)));
+          }
+
+          @Override
           public StackManipulation dispatch(DoFnSignature.Parameter.PipelineOptionsParameter p) {
             return simpleExtraContextParameter(PIPELINE_OPTIONS_PARAMETER_METHOD);
           }
@@ -799,6 +890,15 @@
                         SIDE_INPUT_PARAMETER_METHOD, String.class)),
                 TypeCasting.to(new TypeDescription.ForLoadedType(p.elementT().getRawType())));
           }
+
+          @Override
+          public StackManipulation dispatch(DoFnSignature.Parameter.TimerIdParameter p) {
+            return new StackManipulation.Compound(
+                pushDelegate,
+                MethodInvocation.invoke(
+                    getExtraContextFactoryMethodDescription(
+                        TIMER_ID_PARAMETER_METHOD, DoFn.class)));
+          }
         });
   }
 
@@ -984,6 +1084,12 @@
    * for a constructor that takes a single argument and assigns it to the delegate field.
    */
   private static final class InvokerConstructor implements Implementation {
+    Class<? extends DoFnInvoker> clazz;
+
+    InvokerConstructor(Class<? extends DoFnInvoker> clazz) {
+      this.clazz = clazz;
+    }
+
     @Override
     public InstrumentedType prepare(InstrumentedType instrumentedType) {
       return instrumentedType;
@@ -1000,7 +1106,7 @@
                     MethodVariableAccess.REFERENCE.loadFrom(1),
                     // Invoke the super constructor (default constructor of Object)
                     MethodInvocation.invoke(
-                        new TypeDescription.ForLoadedType(DoFnInvokerBase.class)
+                        new TypeDescription.ForLoadedType(clazz)
                             .getDeclaredMethods()
                             .filter(
                                 ElementMatchers.isConstructor()
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 b997653..b305d78 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
@@ -81,6 +81,31 @@
     }
   }
 
+  public <InputT, OutputT> OnTimerInvoker<InputT, OutputT> forTimerFamily(
+      DoFn<InputT, OutputT> fn, String timerId) {
+
+    @SuppressWarnings("unchecked")
+    Class<? extends DoFn<?, ?>> fnClass = (Class<? extends DoFn<?, ?>>) fn.getClass();
+    try {
+      OnTimerMethodSpecifier onTimerMethodSpecifier =
+          OnTimerMethodSpecifier.forClassAndTimerId(fnClass, timerId);
+      Constructor<?> constructor = constructorTimerFamilyCache.get(onTimerMethodSpecifier);
+
+      return (OnTimerInvoker<InputT, OutputT>) constructor.newInstance(fn);
+    } catch (InstantiationException
+        | IllegalAccessException
+        | IllegalArgumentException
+        | InvocationTargetException
+        | SecurityException
+        | ExecutionException e) {
+      throw new RuntimeException(
+          String.format(
+              "Unable to construct @%s invoker for %s",
+              DoFn.OnTimerFamily.class.getSimpleName(), fn.getClass().getName()),
+          e);
+    }
+  }
+
   public static ByteBuddyOnTimerInvokerFactory only() {
     return INSTANCE;
   }
@@ -120,6 +145,34 @@
                   }
                 }
               });
+
+  /**
+   * A cache of constructors of generated {@link OnTimerInvoker} classes, keyed by {@link
+   * OnTimerMethodSpecifier}.
+   *
+   * <p>Needed because generating an invoker class is expensive, and to avoid generating an
+   * excessive number of classes consuming PermGen memory in Java's that still have PermGen.
+   */
+  private final LoadingCache<OnTimerMethodSpecifier, Constructor<?>> constructorTimerFamilyCache =
+      CacheBuilder.newBuilder()
+          .build(
+              new CacheLoader<OnTimerMethodSpecifier, Constructor<?>>() {
+                @Override
+                public Constructor<?> load(final OnTimerMethodSpecifier onTimerMethodSpecifier)
+                    throws Exception {
+                  DoFnSignature signature =
+                      DoFnSignatures.getSignature(onTimerMethodSpecifier.fnClass());
+                  Class<? extends OnTimerInvoker<?, ?>> invokerClass =
+                      generateOnTimerFamilyInvokerClass(
+                          signature, onTimerMethodSpecifier.timerId());
+                  try {
+                    return invokerClass.getConstructor(signature.fnClass());
+                  } catch (IllegalArgumentException | NoSuchMethodException | SecurityException e) {
+                    throw new RuntimeException(e);
+                  }
+                }
+              });
+
   /**
    * Generates a {@link OnTimerInvoker} class for the given {@link DoFnSignature} and {@link
    * TimerId}.
@@ -176,6 +229,89 @@
     return res;
   }
 
+  private static Class<? extends OnTimerInvoker<?, ?>> generateOnTimerFamilyInvokerClass(
+      DoFnSignature signature, String timerId) {
+    Class<? extends DoFn<?, ?>> fnClass = signature.fnClass();
+
+    final TypeDescription clazzDescription = new TypeDescription.ForLoadedType(fnClass);
+
+    final String suffix =
+        String.format(
+            "%s$%s$%s",
+            OnTimerInvoker.class.getSimpleName(),
+            CharMatcher.javaLetterOrDigit().retainFrom(timerId),
+            BaseEncoding.base64().omitPadding().encode(timerId.getBytes(Charsets.UTF_8)));
+
+    DynamicType.Builder<?> builder =
+        new ByteBuddy()
+            // Create subclasses inside the target class, to have access to
+            // private and package-private bits
+            .with(StableInvokerNamingStrategy.forDoFnClass(fnClass).withSuffix(suffix))
+
+            // class <invoker class> implements OnTimerInvoker {
+            .subclass(OnTimerInvoker.class, ConstructorStrategy.Default.NO_CONSTRUCTORS)
+
+            //   private final <fn class> delegate;
+            .defineField(
+                FN_DELEGATE_FIELD_NAME, fnClass, Visibility.PRIVATE, FieldManifestation.FINAL)
+
+            //   <invoker class>(<fn class> delegate) { this.delegate = delegate; }
+            .defineConstructor(Visibility.PUBLIC)
+            .withParameter(fnClass)
+            .intercept(new InvokerConstructor())
+
+            //   public invokeOnTimer(DoFn.ArgumentProvider) {
+            //     this.delegate.<@OnTimer method>(... pass the right args ...)
+            //   }
+            .method(ElementMatchers.named("invokeOnTimer"))
+            .intercept(
+                new InvokeOnTimerFamilyDelegation(
+                    clazzDescription, signature.onTimerFamilyMethods().get(timerId)));
+
+    DynamicType.Unloaded<?> unloaded = builder.make();
+
+    @SuppressWarnings("unchecked")
+    Class<? extends OnTimerInvoker<?, ?>> res =
+        (Class<? extends OnTimerInvoker<?, ?>>)
+            unloaded
+                .load(
+                    findClassLoader(fnClass.getClassLoader()),
+                    ClassLoadingStrategy.Default.INJECTION)
+                .getLoaded();
+    return res;
+  }
+
+  /**
+   * An "invokeOnTimer" method implementation akin to @ProcessElement, but simpler because no
+   * splitting-related parameters need to be handled.
+   */
+  private static class InvokeOnTimerFamilyDelegation
+      extends DoFnMethodWithExtraParametersDelegation {
+
+    private final DoFnSignature.OnTimerFamilyMethod signature;
+
+    public InvokeOnTimerFamilyDelegation(
+        TypeDescription clazzDescription, DoFnSignature.OnTimerFamilyMethod signature) {
+      super(clazzDescription, signature);
+      this.signature = signature;
+    }
+
+    @Override
+    public InstrumentedType prepare(InstrumentedType instrumentedType) {
+      // Remember the field description of the instrumented type.
+      // Kind of a hack to set the protected value, because the instrumentedType
+      // is only available to prepare, while we need this information in
+      // beforeDelegation
+      delegateField =
+          instrumentedType
+              .getDeclaredFields() // the delegate is declared on the OnTimerInvoker
+              .filter(ElementMatchers.named(FN_DELEGATE_FIELD_NAME))
+              .getOnly();
+      // Delegating the method call doesn't require any changes to the instrumented type.
+      return instrumentedType;
+    }
+  }
+
   /**
    * An "invokeOnTimer" method implementation akin to @ProcessElement, but simpler because no
    * splitting-related parameters need to be handled.
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnInvoker.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnInvoker.java
index fe31c64..5aaea28 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnInvoker.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnInvoker.java
@@ -23,6 +23,7 @@
 import org.apache.beam.sdk.state.State;
 import org.apache.beam.sdk.state.TimeDomain;
 import org.apache.beam.sdk.state.Timer;
+import org.apache.beam.sdk.state.TimerMap;
 import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.transforms.DoFn.FinishBundle;
 import org.apache.beam.sdk.transforms.DoFn.MultiOutputReceiver;
@@ -69,7 +70,8 @@
   DoFn.ProcessContinuation invokeProcessElement(ArgumentProvider<InputT, OutputT> extra);
 
   /** Invoke the appropriate {@link DoFn.OnTimer} method on the bound {@link DoFn}. */
-  void invokeOnTimer(String timerId, ArgumentProvider<InputT, OutputT> arguments);
+  void invokeOnTimer(
+      String timerId, String timerFamilyId, ArgumentProvider<InputT, OutputT> arguments);
 
   /** Invoke the {@link DoFn.GetInitialRestriction} method on the bound {@link DoFn}. */
   @SuppressWarnings("TypeParameterUnusedInFormals")
@@ -170,6 +172,13 @@
 
     /** Returns the timer for the given {@link TimerId}. */
     Timer timer(String timerId);
+
+    /**
+     * Returns the timerMap for the given {@link org.apache.beam.sdk.transforms.DoFn.TimerFamily}.
+     */
+    TimerMap timerFamily(String tagId);
+
+    String timerId(DoFn<InputT, OutputT> doFn);
   }
 
   /**
@@ -202,6 +211,14 @@
     }
 
     @Override
+    public TimerMap timerFamily(String tagId) {
+      throw new UnsupportedOperationException(
+          String.format(
+              "Should never call non-overridden methods of %s",
+              FakeArgumentProvider.class.getSimpleName()));
+    }
+
+    @Override
     public InputT schemaElement(int index) {
       throw new UnsupportedOperationException(
           String.format(
@@ -218,6 +235,14 @@
     }
 
     @Override
+    public String timerId(DoFn<InputT, OutputT> doFn) {
+      throw new UnsupportedOperationException(
+          String.format(
+              "Should never call non-overridden methods of %s",
+              FakeArgumentProvider.class.getSimpleName()));
+    }
+
+    @Override
     public TimeDomain timeDomain(DoFn<InputT, OutputT> doFn) {
       throw new UnsupportedOperationException(
           String.format(
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 2606359..1ea3547 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
@@ -93,6 +93,9 @@
   /** Timer declarations present on the {@link DoFn} class. Immutable. */
   public abstract Map<String, TimerDeclaration> timerDeclarations();
 
+  /** TimerMap declarations present on the {@link DoFn} class. Immutable. */
+  public abstract Map<String, TimerFamilyDeclaration> timerFamilyDeclarations();
+
   /** Field access declaration. */
   @Nullable
   public abstract Map<String, FieldAccessDeclaration> fieldAccessDeclarations();
@@ -117,6 +120,10 @@
   @Nullable
   public abstract Map<String, OnTimerMethod> onTimerMethods();
 
+  /** Details about this {@link DoFn}'s {@link DoFn.OnTimerFamily} methods. */
+  @Nullable
+  public abstract Map<String, OnTimerFamilyMethod> onTimerFamilyMethods();
+
   /** @deprecated use {@link #usesState()}, it's cleaner */
   @Deprecated
   public boolean isStateful() {
@@ -130,7 +137,7 @@
 
   /** Whether the {@link DoFn} described by this signature uses timers. */
   public boolean usesTimers() {
-    return timerDeclarations().size() > 0;
+    return timerDeclarations().size() > 0 || timerFamilyDeclarations().size() > 0;
   }
 
   static Builder builder() {
@@ -167,11 +174,16 @@
 
     abstract Builder setTimerDeclarations(Map<String, TimerDeclaration> timerDeclarations);
 
+    abstract Builder setTimerFamilyDeclarations(
+        Map<String, TimerFamilyDeclaration> timerFamilyDeclarations);
+
     abstract Builder setFieldAccessDeclarations(
         Map<String, FieldAccessDeclaration> fieldAccessDeclaration);
 
     abstract Builder setOnTimerMethods(Map<String, OnTimerMethod> onTimerMethods);
 
+    abstract Builder setOnTimerFamilyMethods(Map<String, OnTimerFamilyMethod> onTimerFamilyMethods);
+
     abstract DoFnSignature build();
   }
 
@@ -245,6 +257,10 @@
         return cases.dispatch((TimeDomainParameter) this);
       } else if (this instanceof SideInputParameter) {
         return cases.dispatch((SideInputParameter) this);
+      } else if (this instanceof TimerFamilyParameter) {
+        return cases.dispatch((TimerFamilyParameter) this);
+      } else if (this instanceof TimerIdParameter) {
+        return cases.dispatch((TimerIdParameter) this);
       } else {
         throw new IllegalStateException(
             String.format(
@@ -289,6 +305,10 @@
 
       ResultT dispatch(SideInputParameter p);
 
+      ResultT dispatch(TimerFamilyParameter p);
+
+      ResultT dispatch(TimerIdParameter p);
+
       /** A base class for a visitor with a default method for cases it is not interested in. */
       abstract class WithDefault<ResultT> implements Cases<ResultT> {
 
@@ -335,6 +355,11 @@
         }
 
         @Override
+        public ResultT dispatch(TimerIdParameter p) {
+          return dispatchDefault(p);
+        }
+
+        @Override
         public ResultT dispatch(TimeDomainParameter p) {
           return dispatchDefault(p);
         }
@@ -378,6 +403,11 @@
         public ResultT dispatch(SideInputParameter p) {
           return dispatchDefault(p);
         }
+
+        @Override
+        public ResultT dispatch(TimerFamilyParameter p) {
+          return dispatchDefault(p);
+        }
       }
     }
 
@@ -392,6 +422,8 @@
         new AutoValue_DoFnSignature_Parameter_OnTimerContextParameter();
     private static final TimestampParameter TIMESTAMP_PARAMETER =
         new AutoValue_DoFnSignature_Parameter_TimestampParameter();
+    private static final TimerIdParameter TIMER_ID_PARAMETER =
+        new AutoValue_DoFnSignature_Parameter_TimerIdParameter();
     private static final PaneInfoParameter PANE_INFO_PARAMETER =
         new AutoValue_DoFnSignature_Parameter_PaneInfoParameter();
     private static final TimeDomainParameter TIME_DOMAIN_PARAMETER =
@@ -423,6 +455,10 @@
       return TIMESTAMP_PARAMETER;
     }
 
+    public static TimerIdParameter timerIdParameter() {
+      return TIMER_ID_PARAMETER;
+    }
+
     public static SideInputParameter sideInputParameter(
         TypeDescriptor<?> elementT, String sideInputId) {
       return new AutoValue_DoFnSignature_Parameter_SideInputParameter.Builder()
@@ -476,6 +512,10 @@
       return new AutoValue_DoFnSignature_Parameter_TimerParameter(decl);
     }
 
+    public static TimerFamilyParameter timerFamilyParameter(TimerFamilyDeclaration decl) {
+      return new AutoValue_DoFnSignature_Parameter_TimerFamilyParameter(decl);
+    }
+
     /** Descriptor for a {@link Parameter} of a subtype of {@link PipelineOptions}. */
     @AutoValue
     public abstract static class PipelineOptionsParameter extends Parameter {
@@ -564,6 +604,11 @@
       TimestampParameter() {}
     }
 
+    @AutoValue
+    public abstract static class TimerIdParameter extends Parameter {
+      TimerIdParameter() {}
+    }
+
     /**
      * Descriptor for a {@link Parameter} representing the time domain of a timer.
      *
@@ -689,6 +734,15 @@
 
       public abstract TimerDeclaration referent();
     }
+
+    /** Descriptor for a {@link Parameter} of type {@link DoFn.TimerFamily}. */
+    @AutoValue
+    public abstract static class TimerFamilyParameter extends Parameter {
+      // Package visible for AutoValue
+      TimerFamilyParameter() {}
+
+      public abstract TimerFamilyDeclaration referent();
+    }
   }
 
   /** Describes a {@link DoFn.ProcessElement} method. */
@@ -830,6 +884,48 @@
     }
   }
 
+  /** Describes a {@link DoFn.OnTimerFamily} method. */
+  @AutoValue
+  public abstract static class OnTimerFamilyMethod implements MethodWithExtraParameters {
+
+    /** The id on the method's {@link DoFn.TimerId} annotation. */
+    public abstract String id();
+
+    /** The annotated method itself. */
+    @Override
+    public abstract Method targetMethod();
+
+    /**
+     * Whether this method requires stable input, expressed via {@link
+     * org.apache.beam.sdk.transforms.DoFn.RequiresStableInput}. For timers, this means that any
+     * state must be stably persisted prior to calling it.
+     */
+    public abstract boolean requiresStableInput();
+
+    /** The window type used by this method, if any. */
+    @Nullable
+    @Override
+    public abstract TypeDescriptor<? extends BoundedWindow> windowT();
+
+    /** Types of optional parameters of the annotated method, in the order they appear. */
+    @Override
+    public abstract List<Parameter> extraParameters();
+
+    static OnTimerFamilyMethod create(
+        Method targetMethod,
+        String id,
+        boolean requiresStableInput,
+        TypeDescriptor<? extends BoundedWindow> windowT,
+        List<Parameter> extraParameters) {
+      return new AutoValue_DoFnSignature_OnTimerFamilyMethod(
+          id,
+          targetMethod,
+          requiresStableInput,
+          windowT,
+          Collections.unmodifiableList(extraParameters));
+    }
+  }
+
   /** Describes a {@link DoFn.OnWindowExpiration} method. */
   @AutoValue
   public abstract static class OnWindowExpirationMethod implements MethodWithExtraParameters {
@@ -883,6 +979,21 @@
     }
   }
 
+  /**
+   * Describes a timer family declaration; a field of type {@link TimerSpec} annotated with {@link
+   * DoFn.TimerFamily}.
+   */
+  @AutoValue
+  public abstract static class TimerFamilyDeclaration {
+    public abstract String id();
+
+    public abstract Field field();
+
+    static TimerFamilyDeclaration create(String id, Field field) {
+      return new AutoValue_DoFnSignature_TimerFamilyDeclaration(id, field);
+    }
+  }
+
   /** Describes a {@link DoFn.StartBundle} or {@link DoFn.FinishBundle} method. */
   @AutoValue
   public abstract static class BundleMethod implements DoFnMethod {
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 04bf2a5..3c5faf4 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
@@ -45,6 +45,7 @@
 import org.apache.beam.sdk.state.StateSpec;
 import org.apache.beam.sdk.state.TimeDomain;
 import org.apache.beam.sdk.state.Timer;
+import org.apache.beam.sdk.state.TimerMap;
 import org.apache.beam.sdk.state.TimerSpec;
 import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.transforms.DoFn.MultiOutputReceiver;
@@ -57,10 +58,12 @@
 import org.apache.beam.sdk.transforms.reflect.DoFnSignature.Parameter.RestrictionTrackerParameter;
 import org.apache.beam.sdk.transforms.reflect.DoFnSignature.Parameter.SchemaElementParameter;
 import org.apache.beam.sdk.transforms.reflect.DoFnSignature.Parameter.StateParameter;
+import org.apache.beam.sdk.transforms.reflect.DoFnSignature.Parameter.TimerFamilyParameter;
 import org.apache.beam.sdk.transforms.reflect.DoFnSignature.Parameter.TimerParameter;
 import org.apache.beam.sdk.transforms.reflect.DoFnSignature.Parameter.WindowParameter;
 import org.apache.beam.sdk.transforms.reflect.DoFnSignature.StateDeclaration;
 import org.apache.beam.sdk.transforms.reflect.DoFnSignature.TimerDeclaration;
+import org.apache.beam.sdk.transforms.reflect.DoFnSignature.TimerFamilyDeclaration;
 import org.apache.beam.sdk.transforms.splittabledofn.HasDefaultTracker;
 import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker;
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
@@ -98,7 +101,8 @@
               Parameter.PipelineOptionsParameter.class,
               Parameter.TimerParameter.class,
               Parameter.StateParameter.class,
-              Parameter.SideInputParameter.class);
+              Parameter.SideInputParameter.class,
+              Parameter.TimerFamilyParameter.class);
 
   private static final ImmutableList<Class<? extends Parameter>>
       ALLOWED_SPLITTABLE_PROCESS_ELEMENT_PARAMETERS =
@@ -124,7 +128,24 @@
           Parameter.OutputReceiverParameter.class,
           Parameter.TaggedOutputReceiverParameter.class,
           Parameter.TimerParameter.class,
-          Parameter.StateParameter.class);
+          Parameter.StateParameter.class,
+          Parameter.TimerFamilyParameter.class,
+          Parameter.TimerIdParameter.class);
+
+  private static final ImmutableList<Class<? extends Parameter>>
+      ALLOWED_ON_TIMER_FAMILY_PARAMETERS =
+          ImmutableList.of(
+              Parameter.OnTimerContextParameter.class,
+              Parameter.TimestampParameter.class,
+              Parameter.TimeDomainParameter.class,
+              Parameter.WindowParameter.class,
+              Parameter.PipelineOptionsParameter.class,
+              Parameter.OutputReceiverParameter.class,
+              Parameter.TaggedOutputReceiverParameter.class,
+              Parameter.TimerParameter.class,
+              Parameter.StateParameter.class,
+              Parameter.TimerFamilyParameter.class,
+              Parameter.TimerIdParameter.class);
 
   private static final Collection<Class<? extends Parameter>>
       ALLOWED_ON_WINDOW_EXPIRATION_PARAMETERS =
@@ -178,6 +199,7 @@
 
     private final Map<String, StateDeclaration> stateDeclarations = new HashMap<>();
     private final Map<String, TimerDeclaration> timerDeclarations = new HashMap<>();
+    private final Map<String, TimerFamilyDeclaration> timerFamilyDeclarations = new HashMap<>();
     private final Map<String, FieldAccessDeclaration> fieldAccessDeclarations = new HashMap<>();
 
     private FnAnalysisContext() {}
@@ -197,6 +219,14 @@
       return Collections.unmodifiableMap(timerDeclarations);
     }
 
+    /**
+     * TimerMap parameters declared in this context, keyed by {@link
+     * org.apache.beam.sdk.transforms.DoFn.TimerFamily}. Unmodifiable.
+     */
+    public Map<String, TimerFamilyDeclaration> getTimerFamilyDeclarations() {
+      return Collections.unmodifiableMap(timerFamilyDeclarations);
+    }
+
     /** Field access declaration declared in this context. */
     @Nullable
     public Map<String, FieldAccessDeclaration> getFieldAccessDeclarations() {
@@ -217,12 +247,22 @@
       timerDeclarations.put(decl.id(), decl);
     }
 
+    public void addTimerFamilyDeclaration(TimerFamilyDeclaration decl) {
+      timerFamilyDeclarations.put(decl.id(), decl);
+    }
+
     public void addTimerDeclarations(Iterable<TimerDeclaration> decls) {
       for (TimerDeclaration decl : decls) {
         addTimerDeclaration(decl);
       }
     }
 
+    public void addTimerFamilyDeclarations(Iterable<TimerFamilyDeclaration> decls) {
+      for (TimerFamilyDeclaration decl : decls) {
+        addTimerFamilyDeclaration(decl);
+      }
+    }
+
     public void addFieldAccessDeclaration(FieldAccessDeclaration decl) {
       fieldAccessDeclarations.put(decl.id(), decl);
     }
@@ -244,6 +284,7 @@
 
     private final Map<String, StateParameter> stateParameters = new HashMap<>();
     private final Map<String, TimerParameter> timerParameters = new HashMap<>();
+    private final Map<String, TimerFamilyParameter> timerFamilyParameters = new HashMap<>();
     private final List<Parameter> extraParameters = new ArrayList<>();
 
     @Nullable private TypeDescriptor<? extends BoundedWindow> windowT;
@@ -282,6 +323,13 @@
     public Map<String, TimerParameter> getTimerParameters() {
       return Collections.unmodifiableMap(timerParameters);
     }
+    /**
+     * TimerMap parameters declared in this context, keyed by {@link
+     * org.apache.beam.sdk.transforms.DoFn.TimerFamily}.
+     */
+    public Map<String, TimerFamilyParameter> getTimerFamilyParameters() {
+      return Collections.unmodifiableMap(timerFamilyParameters);
+    }
     /** Extra parameters in their entirety. Unmodifiable. */
     public List<Parameter> getExtraParameters() {
       return Collections.unmodifiableList(extraParameters);
@@ -306,6 +354,10 @@
         TimerParameter timerParameter = (TimerParameter) param;
         timerParameters.put(timerParameter.referent().id(), timerParameter);
       }
+      if (param instanceof TimerFamilyParameter) {
+        TimerFamilyParameter timerFamilyParameter = (TimerFamilyParameter) param;
+        timerFamilyParameters.put(timerFamilyParameter.referent().id(), timerFamilyParameter);
+      }
     }
 
     /** Create an empty context, with no declarations. */
@@ -364,6 +416,7 @@
     FnAnalysisContext fnContext = FnAnalysisContext.create();
     fnContext.addStateDeclarations(analyzeStateDeclarations(errors, fnClass).values());
     fnContext.addTimerDeclarations(analyzeTimerDeclarations(errors, fnClass).values());
+    fnContext.addTimerFamilyDeclarations(analyzeTimerFamilyDeclarations(errors, fnClass).values());
     fnContext.addFieldAccessDeclarations(analyzeFieldAccessDeclaration(errors, fnClass).values());
 
     Method processElementMethod =
@@ -409,6 +462,36 @@
     }
     signatureBuilder.setOnTimerMethods(onTimerMethodMap);
 
+    // Check for TimerFamily
+    Collection<Method> onTimerFamilyMethods =
+        declaredMethodsWithAnnotation(DoFn.OnTimerFamily.class, fnClass, DoFn.class);
+    HashMap<String, DoFnSignature.OnTimerFamilyMethod> onTimerFamilyMethodMap =
+        Maps.newHashMapWithExpectedSize(onTimerFamilyMethods.size());
+
+    for (Method onTimerFamilyMethod : onTimerFamilyMethods) {
+      String id = onTimerFamilyMethod.getAnnotation(DoFn.OnTimerFamily.class).value();
+      errors.checkArgument(
+          fnContext.getTimerFamilyDeclarations().containsKey(id),
+          "Callback %s is for undeclared timerFamily %s",
+          onTimerFamilyMethod,
+          id);
+
+      TimerFamilyDeclaration timerDecl = fnContext.getTimerFamilyDeclarations().get(id);
+      errors.checkArgument(
+          timerDecl.field().getDeclaringClass().equals(getDeclaringClass(onTimerFamilyMethod)),
+          "Callback %s is for timerFamily %s declared in a different class %s."
+              + " TimerFamily callbacks must be declared in the same lexical scope as their timer",
+          onTimerFamilyMethod,
+          id,
+          timerDecl.field().getDeclaringClass().getCanonicalName());
+
+      onTimerFamilyMethodMap.put(
+          id,
+          analyzeOnTimerFamilyMethod(
+              errors, fnT, onTimerFamilyMethod, id, inputT, outputT, fnContext));
+    }
+    signatureBuilder.setOnTimerFamilyMethods(onTimerFamilyMethodMap);
+
     // Check the converse - that all timers have a callback. This could be relaxed to only
     // those timers used in methods, once method parameter lists support timers.
     for (TimerDeclaration decl : fnContext.getTimerDeclarations().values()) {
@@ -419,6 +502,16 @@
           decl.id());
     }
 
+    // Check the converse - that all timer family have a callback.
+
+    for (TimerFamilyDeclaration decl : fnContext.getTimerFamilyDeclarations().values()) {
+      errors.checkArgument(
+          onTimerFamilyMethodMap.containsKey(decl.id()),
+          "No callback registered via %s for timerFamily %s",
+          DoFn.OnTimerFamily.class.getSimpleName(),
+          decl.id());
+    }
+
     ErrorReporter processElementErrors =
         errors.forMethod(DoFn.ProcessElement.class, processElementMethod);
     DoFnSignature.ProcessElementMethod processElement =
@@ -497,6 +590,7 @@
 
     signatureBuilder.setStateDeclarations(fnContext.getStateDeclarations());
     signatureBuilder.setTimerDeclarations(fnContext.getTimerDeclarations());
+    signatureBuilder.setTimerFamilyDeclarations(fnContext.getTimerFamilyDeclarations());
     signatureBuilder.setFieldAccessDeclarations(fnContext.getFieldAccessDeclarations());
 
     DoFnSignature signature = signatureBuilder.build();
@@ -777,6 +871,51 @@
   }
 
   @VisibleForTesting
+  static DoFnSignature.OnTimerFamilyMethod analyzeOnTimerFamilyMethod(
+      ErrorReporter errors,
+      TypeDescriptor<? extends DoFn<?, ?>> fnClass,
+      Method m,
+      String timerFamilyId,
+      TypeDescriptor<?> inputT,
+      TypeDescriptor<?> outputT,
+      FnAnalysisContext fnContext) {
+    errors.checkArgument(void.class.equals(m.getReturnType()), "Must return void");
+
+    Type[] params = m.getGenericParameterTypes();
+
+    MethodAnalysisContext methodContext = MethodAnalysisContext.create();
+
+    boolean requiresStableInput = m.isAnnotationPresent(DoFn.RequiresStableInput.class);
+
+    @Nullable TypeDescriptor<? extends BoundedWindow> windowT = getWindowType(fnClass, m);
+
+    List<DoFnSignature.Parameter> extraParameters = new ArrayList<>();
+    ErrorReporter onTimerErrors = errors.forMethod(DoFn.OnTimerFamily.class, m);
+    for (int i = 0; i < params.length; ++i) {
+      Parameter parameter =
+          analyzeExtraParameter(
+              onTimerErrors,
+              fnContext,
+              methodContext,
+              fnClass,
+              ParameterDescription.of(
+                  m,
+                  i,
+                  fnClass.resolveType(params[i]),
+                  Arrays.asList(m.getParameterAnnotations()[i])),
+              inputT,
+              outputT);
+
+      checkParameterOneOf(errors, parameter, ALLOWED_ON_TIMER_FAMILY_PARAMETERS);
+
+      extraParameters.add(parameter);
+    }
+
+    return DoFnSignature.OnTimerFamilyMethod.create(
+        m, timerFamilyId, requiresStableInput, windowT, extraParameters);
+  }
+
+  @VisibleForTesting
   static DoFnSignature.OnWindowExpirationMethod analyzeOnWindowExpirationMethod(
       ErrorReporter errors,
       TypeDescriptor<? extends DoFn<?, ?>> fnClass,
@@ -1024,6 +1163,43 @@
 
       return Parameter.timerParameter(timerDecl);
 
+    } else if (hasTimerIdAnnotation(param.getAnnotations())) {
+      boolean isValidTimerIdForTimerFamily =
+          fnContext.getTimerFamilyDeclarations().size() > 0 && rawType.equals(String.class);
+      paramErrors.checkArgument(
+          isValidTimerIdForTimerFamily, "%s not allowed here", DoFn.TimerId.class.getSimpleName());
+      return Parameter.timerIdParameter();
+    } else if (rawType.equals(TimerMap.class)) {
+      String id = getTimerFamilyId(param.getAnnotations());
+
+      paramErrors.checkArgument(
+          id != null,
+          "%s missing %s annotation",
+          TimerMap.class.getSimpleName(),
+          DoFn.TimerFamily.class.getSimpleName());
+
+      paramErrors.checkArgument(
+          !methodContext.getTimerFamilyParameters().containsKey(id),
+          "duplicate %s: \"%s\"",
+          DoFn.TimerFamily.class.getSimpleName(),
+          id);
+
+      TimerFamilyDeclaration timerDecl = fnContext.getTimerFamilyDeclarations().get(id);
+      paramErrors.checkArgument(
+          timerDecl != null,
+          "reference to undeclared %s: \"%s\"",
+          DoFn.TimerFamily.class.getSimpleName(),
+          id);
+
+      paramErrors.checkArgument(
+          timerDecl.field().getDeclaringClass().equals(getDeclaringClass(param.getMethod())),
+          "%s %s declared in a different class %s."
+              + " Timers may be referenced only in the lexical scope where they are declared.",
+          DoFn.TimerFamily.class.getSimpleName(),
+          id,
+          timerDecl.field().getDeclaringClass().getName());
+
+      return Parameter.timerFamilyParameter(timerDecl);
     } else if (State.class.isAssignableFrom(rawType)) {
       // m.getParameters() is not available until Java 8
       String id = getStateId(param.getAnnotations());
@@ -1076,6 +1252,12 @@
   }
 
   @Nullable
+  private static String getTimerFamilyId(List<Annotation> annotations) {
+    DoFn.TimerFamily timerFamilyId = findFirstOfType(annotations, DoFn.TimerFamily.class);
+    return timerFamilyId != null ? timerFamilyId.value() : null;
+  }
+
+  @Nullable
   private static String getStateId(List<Annotation> annotations) {
     DoFn.StateId stateId = findFirstOfType(annotations, DoFn.StateId.class);
     return stateId != null ? stateId.value() : null;
@@ -1112,6 +1294,10 @@
     return annotations.stream().anyMatch(a -> a.annotationType().equals(DoFn.SideInput.class));
   }
 
+  private static boolean hasTimerIdAnnotation(List<Annotation> annotations) {
+    return annotations.stream().anyMatch(a -> a.annotationType().equals(DoFn.TimerId.class));
+  }
+
   @Nullable
   private static TypeDescriptor<?> getTrackerType(TypeDescriptor<?> fnClass, Method method) {
     Type[] params = method.getGenericParameterTypes();
@@ -1288,6 +1474,20 @@
         m, restrictionT, windowT, methodContext.getExtraParameters());
   }
 
+  private static ImmutableMap<String, TimerFamilyDeclaration> analyzeTimerFamilyDeclarations(
+      ErrorReporter errors, Class<?> fnClazz) {
+    Map<String, TimerFamilyDeclaration> declarations = new HashMap<>();
+    for (Field field : declaredFieldsWithAnnotation(DoFn.TimerFamily.class, fnClazz, DoFn.class)) {
+      // TimerSpec fields may generally be private, but will be accessed via the signature
+      field.setAccessible(true);
+      String id = field.getAnnotation(DoFn.TimerFamily.class).value();
+      validateTimerFamilyField(errors, declarations, id, field);
+      declarations.put(id, TimerFamilyDeclaration.create(id, field));
+    }
+
+    return ImmutableMap.copyOf(declarations);
+  }
+
   private static ImmutableMap<String, TimerDeclaration> analyzeTimerDeclarations(
       ErrorReporter errors, Class<?> fnClazz) {
     Map<String, DoFnSignature.TimerDeclaration> declarations = new HashMap<>();
@@ -1331,6 +1531,41 @@
     }
   }
 
+  /**
+   * Returns successfully if the field is valid, otherwise throws an exception via its {@link
+   * ErrorReporter} parameter describing validation failures for the timer family declaration.
+   */
+  private static void validateTimerFamilyField(
+      ErrorReporter errors,
+      Map<String, TimerFamilyDeclaration> declarations,
+      String id,
+      Field field) {
+
+    if (declarations.containsKey(id)) {
+      errors.throwIllegalArgument(
+          "Duplicate %s \"%s\", used on both of [%s] and [%s]",
+          DoFn.TimerFamily.class.getSimpleName(),
+          id,
+          field.toString(),
+          declarations.get(id).field().toString());
+    }
+
+    Class<?> timerSpecRawType = field.getType();
+    if (!(timerSpecRawType.equals(TimerSpec.class))) {
+      errors.throwIllegalArgument(
+          "%s annotation on non-%s field [%s]",
+          DoFn.TimerFamily.class.getSimpleName(),
+          TimerSpec.class.getSimpleName(),
+          field.toString());
+    }
+
+    if (!Modifier.isFinal(field.getModifiers())) {
+      errors.throwIllegalArgument(
+          "Non-final field %s annotated with %s. TimerMap declarations must be final.",
+          field.toString(), DoFn.TimerFamily.class.getSimpleName());
+    }
+  }
+
   /** Generates a {@link TypeDescriptor} for {@code Coder<T>} given {@code T}. */
   private static <T> TypeDescriptor<Coder<T>> coderTypeOf(TypeDescriptor<T> elementT) {
     return new TypeDescriptor<Coder<T>>() {}.where(new TypeParameter<T>() {}, elementT);
@@ -1681,4 +1916,27 @@
               timerDeclaration.field().getName()));
     }
   }
+
+  public static TimerSpec getTimerFamilySpecOrThrow(
+      TimerFamilyDeclaration timerFamilyDeclaration, DoFn<?, ?> target) {
+    try {
+      Object fieldValue = timerFamilyDeclaration.field().get(target);
+      checkState(
+          fieldValue instanceof TimerSpec,
+          "Malformed %s class %s: timer declaration field %s does not have type %s.",
+          DoFn.class.getSimpleName(),
+          target.getClass().getName(),
+          timerFamilyDeclaration.field().getName(),
+          TimerSpec.class);
+
+      return (TimerSpec) timerFamilyDeclaration.field().get(target);
+    } catch (IllegalAccessException exc) {
+      throw new RuntimeException(
+          String.format(
+              "Malformed %s class %s: timer declaration field %s is not accessible.",
+              DoFn.class.getSimpleName(),
+              target.getClass().getName(),
+              timerFamilyDeclaration.field().getName()));
+    }
+  }
 }
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/OnTimerInvokers.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/OnTimerInvokers.java
index 287828a..366e904 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/OnTimerInvokers.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/OnTimerInvokers.java
@@ -39,4 +39,9 @@
       DoFn<InputT, OutputT> fn, String timerId) {
     return ByteBuddyOnTimerInvokerFactory.only().forTimer(fn, timerId);
   }
+
+  public static <InputT, OutputT> OnTimerInvoker<InputT, OutputT> forTimerFamily(
+      DoFn<InputT, OutputT> fn, String timerFamilyId) {
+    return ByteBuddyOnTimerInvokerFactory.only().forTimerFamily(fn, timerFamilyId);
+  }
 }
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 741f373..08a962a 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
@@ -49,11 +49,16 @@
     return getProperties().get("name");
   }
 
-  /** Provides the SDK version. */
+  /** Provides the BEAM version. ie: 2.18.0-SNAPSHOT */
   public String getVersion() {
     return getProperties().get("version");
   }
 
+  /** Provides the SDK version. ie: 2.18.0 or 2.18.0.dev */
+  public String getSdkVersion() {
+    return getProperties().get("sdk_version");
+  }
+
   /////////////////////////////////////////////////////////////////////////
   private static final Logger LOG = LoggerFactory.getLogger(ReleaseInfo.class);
   private static final String DEFAULT_NAME = "Apache Beam SDK for Java";
@@ -79,6 +84,9 @@
       if (!properties.containsKey("version")) {
         properties.setProperty("version", DEFAULT_VERSION);
       }
+      if (!properties.containsKey("sdk_version")) {
+        properties.setProperty("sdk_version", DEFAULT_VERSION);
+      }
       INSTANCE = new AutoValue_ReleaseInfo(ImmutableMap.copyOf((Map) properties));
     }
   }
diff --git a/sdks/java/core/src/main/resources/org/apache/beam/sdk/sdk.properties b/sdks/java/core/src/main/resources/org/apache/beam/sdk/sdk.properties
index 38181c4..3320a4c 100644
--- a/sdks/java/core/src/main/resources/org/apache/beam/sdk/sdk.properties
+++ b/sdks/java/core/src/main/resources/org/apache/beam/sdk/sdk.properties
@@ -17,6 +17,7 @@
 # SDK source version
 
 version=@pom.version@
+sdk_version=@pom.sdk_version@
 
 build.date=@timestamp@
 
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 bad9d55..841a75c 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
@@ -191,7 +191,7 @@
                 TimestampedValue.of("firstPane", new Instant(100)),
                 TimestampedValue.of("alsoFirstPane", new Instant(200)))
             .addElements(TimestampedValue.of("onTimePane", new Instant(500)))
-            .advanceWatermarkTo(new Instant(1001L))
+            .advanceWatermarkTo(new Instant(1000L))
             .addElements(
                 TimestampedValue.of("finalLatePane", new Instant(750)),
                 TimestampedValue.of("alsoFinalLatePane", new Instant(250)))
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 7cd03a7..57d8c5c 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
@@ -79,6 +79,7 @@
 import org.apache.beam.sdk.state.StateSpecs;
 import org.apache.beam.sdk.state.TimeDomain;
 import org.apache.beam.sdk.state.Timer;
+import org.apache.beam.sdk.state.TimerMap;
 import org.apache.beam.sdk.state.TimerSpec;
 import org.apache.beam.sdk.state.TimerSpecs;
 import org.apache.beam.sdk.state.ValueState;
@@ -96,6 +97,7 @@
 import org.apache.beam.sdk.testing.UsesTestStream;
 import org.apache.beam.sdk.testing.UsesTestStreamWithOutputTimestamp;
 import org.apache.beam.sdk.testing.UsesTestStreamWithProcessingTime;
+import org.apache.beam.sdk.testing.UsesTimerMap;
 import org.apache.beam.sdk.testing.UsesTimersInParDo;
 import org.apache.beam.sdk.testing.ValidatesRunner;
 import org.apache.beam.sdk.transforms.DoFn.OnTimer;
@@ -4289,4 +4291,153 @@
       r.output("It works");
     }
   }
+
+  /** Tests to validate ParDo timerFamily. */
+  @RunWith(JUnit4.class)
+  public static class TimerFamilyTests extends SharedTestBase implements Serializable {
+
+    @Test
+    @Category({
+      NeedsRunner.class,
+      UsesTimersInParDo.class,
+      UsesTestStream.class,
+      UsesTimerMap.class
+    })
+    public void testTimerFamilyEventTime() throws Exception {
+      final String timerFamilyId = "foo";
+
+      DoFn<KV<String, Integer>, String> fn =
+          new DoFn<KV<String, Integer>, String>() {
+
+            @TimerFamily(timerFamilyId)
+            private final TimerSpec spec = TimerSpecs.timerMap(TimeDomain.EVENT_TIME);
+
+            @ProcessElement
+            public void processElement(
+                @TimerFamily(timerFamilyId) TimerMap timers, OutputReceiver<String> r) {
+              timers.set("timer1", new Instant(1));
+              timers.set("timer2", new Instant(2));
+              r.output("process");
+            }
+
+            @OnTimerFamily(timerFamilyId)
+            public void onTimer(
+                @TimerId String timerId,
+                @Timestamp Instant ts,
+                @TimerFamily(timerFamilyId) TimerMap timerMap,
+                OutputReceiver<String> r) {
+              System.out.println("timer Id : " + timerId);
+              System.out.println("timerMap : " + timerMap.toString());
+              r.output(timerId);
+            }
+          };
+
+      TestStream<KV<String, Integer>> stream =
+          TestStream.create(KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of()))
+              .advanceWatermarkTo(new Instant(0))
+              .addElements(KV.of("hello", 37))
+              .advanceWatermarkToInfinity();
+
+      PCollection<String> output = pipeline.apply(stream).apply(ParDo.of(fn));
+      PAssert.that(output).containsInAnyOrder("process", "timer1", "timer2");
+      pipeline.run();
+    }
+
+    @Test
+    @Category({
+      NeedsRunner.class,
+      UsesTimersInParDo.class,
+      UsesTestStream.class,
+      UsesTimerMap.class
+    })
+    public void testTimerWithMultipleTimerFamily() throws Exception {
+      final String timerFamilyId1 = "foo";
+      final String timerFamilyId2 = "bar";
+
+      DoFn<KV<String, Integer>, String> fn =
+          new DoFn<KV<String, Integer>, String>() {
+
+            @TimerFamily(timerFamilyId1)
+            private final TimerSpec spec1 = TimerSpecs.timerMap(TimeDomain.EVENT_TIME);
+
+            @TimerFamily(timerFamilyId2)
+            private final TimerSpec spec2 = TimerSpecs.timerMap(TimeDomain.EVENT_TIME);
+
+            @ProcessElement
+            public void processElement(
+                @TimerFamily(timerFamilyId1) TimerMap timerMap1,
+                @TimerFamily(timerFamilyId2) TimerMap timerMap2,
+                OutputReceiver<String> r) {
+              timerMap1.set("timer", new Instant(1));
+              timerMap2.set("timer", new Instant(2));
+              r.output("process");
+            }
+
+            @OnTimerFamily(timerFamilyId1)
+            public void onTimer1(
+                @TimerId String timerId, @Timestamp Instant ts, OutputReceiver<String> r) {
+              r.output(timerId);
+            }
+
+            @OnTimerFamily(timerFamilyId2)
+            public void onTimer2(
+                @TimerId String timerId, @Timestamp Instant ts, OutputReceiver<String> r) {
+              r.output(timerId);
+            }
+          };
+
+      TestStream<KV<String, Integer>> stream =
+          TestStream.create(KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of()))
+              .advanceWatermarkTo(new Instant(0))
+              .addElements(KV.of("hello", 37))
+              .advanceWatermarkToInfinity();
+
+      PCollection<String> output = pipeline.apply(stream).apply(ParDo.of(fn));
+      PAssert.that(output).containsInAnyOrder("process", "timer", "timer");
+      pipeline.run();
+    }
+
+    @Test
+    @Category({
+      NeedsRunner.class,
+      UsesTimersInParDo.class,
+      UsesTestStreamWithProcessingTime.class,
+      UsesTimerMap.class
+    })
+    public void testTimerFamilyProcessingTime() throws Exception {
+      final String timerId = "foo";
+
+      DoFn<KV<String, Integer>, Integer> fn =
+          new DoFn<KV<String, Integer>, Integer>() {
+
+            @TimerFamily(timerId)
+            private final TimerSpec spec = TimerSpecs.timerMap(TimeDomain.PROCESSING_TIME);
+
+            @ProcessElement
+            public void processElement(
+                @TimerFamily(timerId) TimerMap timerMap, OutputReceiver<Integer> r) {
+              Timer timer = timerMap.get("timerId1");
+              timer.offset(Duration.standardSeconds(1)).setRelative();
+              r.output(3);
+            }
+
+            @OnTimerFamily(timerId)
+            public void onTimer(TimeDomain timeDomain, OutputReceiver<Integer> r) {
+              if (timeDomain.equals(TimeDomain.PROCESSING_TIME)) {
+                r.output(42);
+              }
+            }
+          };
+
+      TestStream<KV<String, Integer>> stream =
+          TestStream.create(KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of()))
+              .addElements(KV.of("hello", 37))
+              .advanceProcessingTime(Duration.standardSeconds(2))
+              .advanceWatermarkToInfinity();
+
+      PCollection<Integer> output = pipeline.apply(stream).apply(ParDo.of(fn));
+      PAssert.that(output).containsInAnyOrder(3, 42);
+      pipeline.run();
+    }
+  }
 }
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 44e943e..18223e5 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
@@ -113,7 +113,7 @@
   }
 
   private void invokeOnTimer(String timerId, DoFn<String, String> fn) {
-    DoFnInvokers.invokerFor(fn).invokeOnTimer(timerId, mockArgumentProvider);
+    DoFnInvokers.invokerFor(fn).invokeOnTimer(timerId, timerId, mockArgumentProvider);
   }
 
   @Test
@@ -831,7 +831,7 @@
     SimpleTimerDoFn fn = new SimpleTimerDoFn();
 
     DoFnInvoker<String, String> invoker = DoFnInvokers.invokerFor(fn);
-    invoker.invokeOnTimer(timerId, mockArgumentProvider);
+    invoker.invokeOnTimer(timerId, timerId, mockArgumentProvider);
     assertThat(fn.status, equalTo("OK now"));
   }
 
@@ -860,7 +860,7 @@
     SimpleTimerDoFn fn = new SimpleTimerDoFn();
 
     DoFnInvoker<String, String> invoker = DoFnInvokers.invokerFor(fn);
-    invoker.invokeOnTimer(timerId, mockArgumentProvider);
+    invoker.invokeOnTimer(timerId, timerId, mockArgumentProvider);
     assertThat(fn.window, equalTo(testWindow));
   }
 
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 e0427f2..fedccf4 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
@@ -1026,6 +1026,11 @@
                 assertThat(stateParam.referent(), equalTo(decl));
                 return null;
               }
+
+              @Override
+              public Void dispatch(Parameter.TimerIdParameter p) {
+                return null;
+              }
             });
   }
 
diff --git a/sdks/java/extensions/sql/perf-tests/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BigQueryIOPushDownIT.java b/sdks/java/extensions/sql/perf-tests/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BigQueryIOPushDownIT.java
index 05ad30e..caa5497 100644
--- a/sdks/java/extensions/sql/perf-tests/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BigQueryIOPushDownIT.java
+++ b/sdks/java/extensions/sql/perf-tests/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BigQueryIOPushDownIT.java
@@ -56,7 +56,8 @@
 
 @RunWith(JUnit4.class)
 public class BigQueryIOPushDownIT {
-  private static final String READ_FROM_TABLE = "bigquery-public-data:hacker_news.full";
+  private static final String READ_FROM_TABLE =
+      "apache-beam-testing:beam_performance.hacker_news_full";
   private static final String NAMESPACE = BigQueryIOPushDownIT.class.getName();
   private static final String FIELDS_READ_METRIC = "fields_read";
   private static final String READ_TIME_METRIC = "read_time";
diff --git a/sdks/java/fn-execution/build.gradle b/sdks/java/fn-execution/build.gradle
index ea46cff..d099333 100644
--- a/sdks/java/fn-execution/build.gradle
+++ b/sdks/java/fn-execution/build.gradle
@@ -27,7 +27,7 @@
   compile project(path: ":model:pipeline", configuration: "shadow")
   compile project(path: ":model:fn-execution", configuration: "shadow")
   compile project(path: ":sdks:java:core", configuration: "shadow")
-  compile library.java.vendored_grpc_1_21_0
+  compile library.java.vendored_grpc_1_26_0
   compile library.java.vendored_guava_26_0_jre
   compile library.java.slf4j_api
   compile library.java.joda_time
diff --git a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/channel/ManagedChannelFactory.java b/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/channel/ManagedChannelFactory.java
index bc6da0e..c6180a3 100644
--- a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/channel/ManagedChannelFactory.java
+++ b/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/channel/ManagedChannelFactory.java
@@ -20,14 +20,14 @@
 import java.net.SocketAddress;
 import java.util.List;
 import org.apache.beam.model.pipeline.v1.Endpoints.ApiServiceDescriptor;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.ClientInterceptor;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.ManagedChannel;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.ManagedChannelBuilder;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.netty.NettyChannelBuilder;
-import org.apache.beam.vendor.grpc.v1p21p0.io.netty.channel.epoll.EpollDomainSocketChannel;
-import org.apache.beam.vendor.grpc.v1p21p0.io.netty.channel.epoll.EpollEventLoopGroup;
-import org.apache.beam.vendor.grpc.v1p21p0.io.netty.channel.epoll.EpollSocketChannel;
-import org.apache.beam.vendor.grpc.v1p21p0.io.netty.channel.unix.DomainSocketAddress;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.ClientInterceptor;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.ManagedChannel;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.ManagedChannelBuilder;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.netty.NettyChannelBuilder;
+import org.apache.beam.vendor.grpc.v1p26p0.io.netty.channel.epoll.EpollDomainSocketChannel;
+import org.apache.beam.vendor.grpc.v1p26p0.io.netty.channel.epoll.EpollEventLoopGroup;
+import org.apache.beam.vendor.grpc.v1p26p0.io.netty.channel.epoll.EpollSocketChannel;
+import org.apache.beam.vendor.grpc.v1p26p0.io.netty.channel.unix.DomainSocketAddress;
 
 /** A Factory which creates an underlying {@link ManagedChannel} implementation. */
 public abstract class ManagedChannelFactory {
@@ -36,7 +36,7 @@
   }
 
   public static ManagedChannelFactory createEpoll() {
-    org.apache.beam.vendor.grpc.v1p21p0.io.netty.channel.epoll.Epoll.ensureAvailability();
+    org.apache.beam.vendor.grpc.v1p26p0.io.netty.channel.epoll.Epoll.ensureAvailability();
     return new Epoll();
   }
 
diff --git a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/channel/SocketAddressFactory.java b/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/channel/SocketAddressFactory.java
index c77e1bc..b7d9d76 100644
--- a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/channel/SocketAddressFactory.java
+++ b/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/channel/SocketAddressFactory.java
@@ -23,7 +23,7 @@
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.net.SocketAddress;
-import org.apache.beam.vendor.grpc.v1p21p0.io.netty.channel.unix.DomainSocketAddress;
+import org.apache.beam.vendor.grpc.v1p26p0.io.netty.channel.unix.DomainSocketAddress;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.net.HostAndPort;
 
 /** Creates a {@link SocketAddress} based upon a supplied string. */
diff --git a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/data/BeamFnDataBufferingOutboundObserver.java b/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/data/BeamFnDataBufferingOutboundObserver.java
index bbc2916..e741c7e 100644
--- a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/data/BeamFnDataBufferingOutboundObserver.java
+++ b/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/data/BeamFnDataBufferingOutboundObserver.java
@@ -23,7 +23,7 @@
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.options.ExperimentalOptions;
 import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.StreamObserver;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.stub.StreamObserver;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
 
 /**
diff --git a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/data/BeamFnDataGrpcMultiplexer.java b/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/data/BeamFnDataGrpcMultiplexer.java
index 7ed83df..3140616 100644
--- a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/data/BeamFnDataGrpcMultiplexer.java
+++ b/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/data/BeamFnDataGrpcMultiplexer.java
@@ -28,8 +28,8 @@
 import org.apache.beam.model.fnexecution.v1.BeamFnApi.Elements.Data;
 import org.apache.beam.model.pipeline.v1.Endpoints;
 import org.apache.beam.sdk.fn.stream.OutboundObserverFactory;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.Status;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.StreamObserver;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.Status;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.stub.StreamObserver;
 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.ImmutableList;
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 c0215ae..51d9595 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
@@ -20,8 +20,8 @@
 import java.io.IOException;
 import org.apache.beam.model.fnexecution.v1.BeamFnApi;
 import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.StreamObserver;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.stub.StreamObserver;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
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 3595fbd..b339e33 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
@@ -25,7 +25,7 @@
 import java.util.concurrent.TimeUnit;
 import org.apache.beam.model.fnexecution.v1.BeamFnApi;
 import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.StreamObserver;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.stub.StreamObserver;
 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.util.concurrent.ThreadFactoryBuilder;
 
diff --git a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/data/RemoteGrpcPortRead.java b/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/data/RemoteGrpcPortRead.java
index 9568b90..f06599e 100644
--- a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/data/RemoteGrpcPortRead.java
+++ b/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/data/RemoteGrpcPortRead.java
@@ -23,7 +23,7 @@
 import org.apache.beam.model.fnexecution.v1.BeamFnApi.RemoteGrpcPort;
 import org.apache.beam.model.pipeline.v1.RunnerApi.FunctionSpec;
 import org.apache.beam.model.pipeline.v1.RunnerApi.PTransform;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.InvalidProtocolBufferException;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables;
 
 /**
diff --git a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/data/RemoteGrpcPortWrite.java b/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/data/RemoteGrpcPortWrite.java
index b1c7604..42fd798 100644
--- a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/data/RemoteGrpcPortWrite.java
+++ b/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/data/RemoteGrpcPortWrite.java
@@ -24,7 +24,7 @@
 import org.apache.beam.model.pipeline.v1.RunnerApi.FunctionSpec;
 import org.apache.beam.model.pipeline.v1.RunnerApi.PCollection;
 import org.apache.beam.model.pipeline.v1.RunnerApi.PTransform;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.InvalidProtocolBufferException;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables;
 
 /**
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 da7505d..12f042d 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
@@ -27,8 +27,8 @@
 import java.util.concurrent.Phaser;
 import java.util.concurrent.TimeUnit;
 import javax.annotation.concurrent.ThreadSafe;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.CallStreamObserver;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.StreamObserver;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.stub.CallStreamObserver;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.stub.StreamObserver;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
 
 /**
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 3134ea4..140f508 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
@@ -27,7 +27,7 @@
 import java.util.NoSuchElementException;
 import java.util.concurrent.BlockingQueue;
 import org.apache.beam.sdk.coders.Coder;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.io.ByteStreams;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.io.CountingInputStream;
 
diff --git a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/stream/DirectStreamObserver.java b/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/stream/DirectStreamObserver.java
index dce3452..3906318 100644
--- a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/stream/DirectStreamObserver.java
+++ b/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/stream/DirectStreamObserver.java
@@ -21,8 +21,8 @@
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 import javax.annotation.concurrent.ThreadSafe;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.CallStreamObserver;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.StreamObserver;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.stub.CallStreamObserver;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.stub.StreamObserver;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
diff --git a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/stream/ForwardingClientResponseObserver.java b/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/stream/ForwardingClientResponseObserver.java
index a25985d..016cb11 100644
--- a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/stream/ForwardingClientResponseObserver.java
+++ b/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/stream/ForwardingClientResponseObserver.java
@@ -17,9 +17,9 @@
  */
 package org.apache.beam.sdk.fn.stream;
 
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.ClientCallStreamObserver;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.ClientResponseObserver;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.StreamObserver;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.stub.ClientCallStreamObserver;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.stub.ClientResponseObserver;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.stub.StreamObserver;
 
 /**
  * A {@link ClientResponseObserver} which delegates all {@link StreamObserver} calls.
diff --git a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/stream/OutboundObserverFactory.java b/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/stream/OutboundObserverFactory.java
index 83f94f9..6693fee 100644
--- a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/stream/OutboundObserverFactory.java
+++ b/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/stream/OutboundObserverFactory.java
@@ -18,8 +18,8 @@
 package org.apache.beam.sdk.fn.stream;
 
 import java.util.concurrent.ExecutorService;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.CallStreamObserver;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.StreamObserver;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.stub.CallStreamObserver;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.stub.StreamObserver;
 
 /**
  * Creates factories which determine an underlying {@link StreamObserver} implementation to use in
diff --git a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/stream/SynchronizedStreamObserver.java b/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/stream/SynchronizedStreamObserver.java
index c960d96..31e9af2 100644
--- a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/stream/SynchronizedStreamObserver.java
+++ b/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/stream/SynchronizedStreamObserver.java
@@ -17,7 +17,7 @@
  */
 package org.apache.beam.sdk.fn.stream;
 
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.StreamObserver;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.stub.StreamObserver;
 
 /**
  * A {@link StreamObserver} which provides synchronous access access to an underlying {@link
diff --git a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/test/InProcessManagedChannelFactory.java b/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/test/InProcessManagedChannelFactory.java
index a4c99a1..aad1fd1 100644
--- a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/test/InProcessManagedChannelFactory.java
+++ b/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/test/InProcessManagedChannelFactory.java
@@ -19,8 +19,8 @@
 
 import org.apache.beam.model.pipeline.v1.Endpoints.ApiServiceDescriptor;
 import org.apache.beam.sdk.fn.channel.ManagedChannelFactory;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.ManagedChannelBuilder;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.inprocess.InProcessChannelBuilder;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.ManagedChannelBuilder;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.inprocess.InProcessChannelBuilder;
 
 /**
  * A {@link ManagedChannelFactory} that uses in-process channels.
diff --git a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/test/TestStreams.java b/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/test/TestStreams.java
index b76997e..cd8b977 100644
--- a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/test/TestStreams.java
+++ b/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/test/TestStreams.java
@@ -19,8 +19,8 @@
 
 import java.util.function.Consumer;
 import java.util.function.Supplier;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.CallStreamObserver;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.StreamObserver;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.stub.CallStreamObserver;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.stub.StreamObserver;
 
 /** Utility methods which enable testing of {@link StreamObserver}s. */
 public class TestStreams {
diff --git a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/windowing/EncodedBoundedWindow.java b/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/windowing/EncodedBoundedWindow.java
index 94d3400..3e9b21c 100644
--- a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/windowing/EncodedBoundedWindow.java
+++ b/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/windowing/EncodedBoundedWindow.java
@@ -25,7 +25,7 @@
 import org.apache.beam.sdk.coders.CoderException;
 import org.apache.beam.sdk.transforms.windowing.BoundedWindow;
 import org.apache.beam.sdk.util.VarInt;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.io.ByteStreams;
 import org.joda.time.Instant;
 
diff --git a/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/channel/ManagedChannelFactoryTest.java b/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/channel/ManagedChannelFactoryTest.java
index 3e60697..fc0c813 100644
--- a/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/channel/ManagedChannelFactoryTest.java
+++ b/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/channel/ManagedChannelFactoryTest.java
@@ -21,7 +21,7 @@
 import static org.junit.Assume.assumeTrue;
 
 import org.apache.beam.model.pipeline.v1.Endpoints;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.ManagedChannel;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.ManagedChannel;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.TemporaryFolder;
@@ -45,7 +45,7 @@
 
   @Test
   public void testEpollHostPortChannel() {
-    assumeTrue(org.apache.beam.vendor.grpc.v1p21p0.io.netty.channel.epoll.Epoll.isAvailable());
+    assumeTrue(org.apache.beam.vendor.grpc.v1p26p0.io.netty.channel.epoll.Epoll.isAvailable());
     Endpoints.ApiServiceDescriptor apiServiceDescriptor =
         Endpoints.ApiServiceDescriptor.newBuilder().setUrl("localhost:123").build();
     ManagedChannel channel =
@@ -56,7 +56,7 @@
 
   @Test
   public void testEpollDomainSocketChannel() throws Exception {
-    assumeTrue(org.apache.beam.vendor.grpc.v1p21p0.io.netty.channel.epoll.Epoll.isAvailable());
+    assumeTrue(org.apache.beam.vendor.grpc.v1p26p0.io.netty.channel.epoll.Epoll.isAvailable());
     Endpoints.ApiServiceDescriptor apiServiceDescriptor =
         Endpoints.ApiServiceDescriptor.newBuilder()
             .setUrl("unix://" + tmpFolder.newFile().getAbsolutePath())
diff --git a/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/channel/SocketAddressFactoryTest.java b/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/channel/SocketAddressFactoryTest.java
index 0107a7b..91c1e17 100644
--- a/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/channel/SocketAddressFactoryTest.java
+++ b/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/channel/SocketAddressFactoryTest.java
@@ -23,7 +23,7 @@
 import java.io.File;
 import java.net.InetSocketAddress;
 import java.net.SocketAddress;
-import org.apache.beam.vendor.grpc.v1p21p0.io.netty.channel.unix.DomainSocketAddress;
+import org.apache.beam.vendor.grpc.v1p26p0.io.netty.channel.unix.DomainSocketAddress;
 import org.hamcrest.Matchers;
 import org.junit.Rule;
 import org.junit.Test;
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 bf1b1d3..51301e4 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
@@ -31,7 +31,7 @@
 import org.apache.beam.model.pipeline.v1.Endpoints;
 import org.apache.beam.sdk.fn.stream.OutboundObserverFactory;
 import org.apache.beam.sdk.fn.test.TestStreams;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.util.concurrent.Uninterruptibles;
 import org.junit.Test;
 
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 ed2f700..a75b9fa 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
@@ -40,7 +40,7 @@
 import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.options.PipelineOptionsFactory;
 import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables;
 import org.junit.Test;
 import org.junit.runner.RunWith;
diff --git a/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/data/RemoteGrpcPortReadTest.java b/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/data/RemoteGrpcPortReadTest.java
index c1b2175..97aebaf 100644
--- a/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/data/RemoteGrpcPortReadTest.java
+++ b/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/data/RemoteGrpcPortReadTest.java
@@ -24,7 +24,7 @@
 import org.apache.beam.model.pipeline.v1.Endpoints.ApiServiceDescriptor;
 import org.apache.beam.model.pipeline.v1.Endpoints.OAuth2ClientCredentialsGrant;
 import org.apache.beam.model.pipeline.v1.RunnerApi.PTransform;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.InvalidProtocolBufferException;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
diff --git a/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/data/RemoteGrpcPortWriteTest.java b/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/data/RemoteGrpcPortWriteTest.java
index c4be16b..1775728 100644
--- a/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/data/RemoteGrpcPortWriteTest.java
+++ b/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/data/RemoteGrpcPortWriteTest.java
@@ -24,7 +24,7 @@
 import org.apache.beam.model.pipeline.v1.Endpoints.ApiServiceDescriptor;
 import org.apache.beam.model.pipeline.v1.Endpoints.OAuth2ClientCredentialsGrant;
 import org.apache.beam.model.pipeline.v1.RunnerApi.PTransform;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.InvalidProtocolBufferException;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
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 3e66d50..8dd5819 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
@@ -41,7 +41,7 @@
 import org.apache.beam.sdk.fn.stream.DataStreams.DataStreamDecoder;
 import org.apache.beam.sdk.fn.stream.DataStreams.ElementDelimitedOutputStream;
 import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
 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.io.ByteStreams;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.io.CountingOutputStream;
diff --git a/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/stream/ForwardingClientResponseObserverTest.java b/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/stream/ForwardingClientResponseObserverTest.java
index 97fc2da..5e0e4b5 100644
--- a/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/stream/ForwardingClientResponseObserverTest.java
+++ b/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/stream/ForwardingClientResponseObserverTest.java
@@ -21,9 +21,9 @@
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.verifyNoMoreInteractions;
 
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.ClientCallStreamObserver;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.ClientResponseObserver;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.StreamObserver;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.stub.ClientCallStreamObserver;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.stub.ClientResponseObserver;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.stub.StreamObserver;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
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 60cd8b0..de56d01 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
@@ -22,8 +22,8 @@
 import static org.junit.Assert.assertThat;
 
 import java.util.concurrent.Executors;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.CallStreamObserver;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.StreamObserver;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.stub.CallStreamObserver;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.stub.StreamObserver;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.runner.RunWith;
diff --git a/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/windowing/EncodedBoundedWindowTest.java b/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/windowing/EncodedBoundedWindowTest.java
index 18d6896..ecf5bc1 100644
--- a/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/windowing/EncodedBoundedWindowTest.java
+++ b/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/windowing/EncodedBoundedWindowTest.java
@@ -19,7 +19,7 @@
 
 import org.apache.beam.sdk.fn.windowing.EncodedBoundedWindow.Coder;
 import org.apache.beam.sdk.testing.CoderProperties;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
diff --git a/sdks/java/harness/build.gradle b/sdks/java/harness/build.gradle
index 42378ea..bfc9112 100644
--- a/sdks/java/harness/build.gradle
+++ b/sdks/java/harness/build.gradle
@@ -62,7 +62,7 @@
   shadowTest library.java.powermock_mockito
   compile library.java.joda_time
   compile library.java.slf4j_api
-  compile library.java.vendored_grpc_1_21_0
+  compile library.java.vendored_grpc_1_26_0
   provided library.java.error_prone_annotations
   testCompile library.java.hamcrest_core
   testCompile library.java.hamcrest_library
diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/BoundedSourceRunner.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/BoundedSourceRunner.java
index a632aa2..a06c002 100644
--- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/BoundedSourceRunner.java
+++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/BoundedSourceRunner.java
@@ -43,7 +43,7 @@
 import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.util.SerializableUtils;
 import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.InvalidProtocolBufferException;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.InvalidProtocolBufferException;
 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;
 
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 deba809..f0d3ac3 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
@@ -60,6 +60,7 @@
 import org.apache.beam.sdk.state.State;
 import org.apache.beam.sdk.state.StateSpec;
 import org.apache.beam.sdk.state.TimeDomain;
+import org.apache.beam.sdk.state.TimerMap;
 import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.transforms.DoFn.MultiOutputReceiver;
 import org.apache.beam.sdk.transforms.DoFn.OutputReceiver;
@@ -86,8 +87,8 @@
 import org.apache.beam.sdk.values.Row;
 import org.apache.beam.sdk.values.TupleTag;
 import org.apache.beam.sdk.values.WindowingStrategy;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.util.Durations;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.util.Durations;
 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.ImmutableList;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableListMultimap;
@@ -765,7 +766,7 @@
           (Iterator<BoundedWindow>) timer.getWindows().iterator();
       while (windowIterator.hasNext()) {
         currentWindow = windowIterator.next();
-        doFnInvoker.invokeOnTimer(timerId, onTimerContext);
+        doFnInvoker.invokeOnTimer(timerId, timerId, onTimerContext);
       }
     } finally {
       currentTimer = null;
@@ -922,6 +923,18 @@
     }
   }
 
+  private static class FnApiTimerMap implements TimerMap {
+    FnApiTimerMap() {}
+
+    @Override
+    public void set(String timerId, Instant absoluteTime) {}
+
+    @Override
+    public org.apache.beam.sdk.state.Timer get(String timerId) {
+      return null;
+    }
+  }
+
   /**
    * Provides arguments for a {@link DoFnInvoker} for {@link DoFn.ProcessElement @ProcessElement}.
    */
@@ -982,6 +995,12 @@
     }
 
     @Override
+    public String timerId(DoFn<InputT, OutputT> doFn) {
+      throw new UnsupportedOperationException(
+          "Cannot access timerId as parameter outside of @OnTimer method.");
+    }
+
+    @Override
     public TimeDomain timeDomain(DoFn<InputT, OutputT> doFn) {
       throw new UnsupportedOperationException(
           "Cannot access time domain outside of @ProcessTimer method.");
@@ -1037,6 +1056,12 @@
     }
 
     @Override
+    public TimerMap timerFamily(String tagId) {
+      // TODO: implement timerFamily
+      return null;
+    }
+
+    @Override
     public PipelineOptions getPipelineOptions() {
       return context.pipelineOptions;
     }
@@ -1170,6 +1195,11 @@
     }
 
     @Override
+    public String timerId(DoFn<InputT, OutputT> doFn) {
+      throw new UnsupportedOperationException("TimerId parameters are not supported.");
+    }
+
+    @Override
     public TimeDomain timeDomain(DoFn<InputT, OutputT> doFn) {
       return timeDomain();
     }
@@ -1223,6 +1253,12 @@
     }
 
     @Override
+    public TimerMap timerFamily(String tagId) {
+      // TODO: implement timerFamily
+      throw new UnsupportedOperationException("TimerFamily parameters are not supported.");
+    }
+
+    @Override
     public PipelineOptions getPipelineOptions() {
       return context.pipelineOptions;
     }
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 6ec1673..579d447 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
@@ -44,7 +44,7 @@
 import org.apache.beam.sdk.io.FileSystems;
 import org.apache.beam.sdk.options.ExperimentalOptions;
 import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.TextFormat;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.TextFormat;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
 import org.slf4j.Logger;
diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/AddHarnessIdInterceptor.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/AddHarnessIdInterceptor.java
index 7fec44e..f949716 100644
--- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/AddHarnessIdInterceptor.java
+++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/AddHarnessIdInterceptor.java
@@ -19,10 +19,10 @@
 
 import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument;
 
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.ClientInterceptor;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.Metadata;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.Metadata.Key;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.MetadataUtils;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.ClientInterceptor;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.Metadata;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.Metadata.Key;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.stub.MetadataUtils;
 
 /** A {@link ClientInterceptor} that attaches a provided SDK Harness ID to outgoing messages. */
 public class AddHarnessIdInterceptor {
diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/BeamFnControlClient.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/BeamFnControlClient.java
index a6a0211..bfaddda 100644
--- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/BeamFnControlClient.java
+++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/BeamFnControlClient.java
@@ -32,8 +32,8 @@
 import org.apache.beam.sdk.fn.channel.ManagedChannelFactory;
 import org.apache.beam.sdk.fn.stream.OutboundObserverFactory;
 import org.apache.beam.sdk.function.ThrowingFunction;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.Status;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.StreamObserver;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.Status;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.stub.StreamObserver;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.util.concurrent.Uninterruptibles;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
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 a258e01..00882b5 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
@@ -64,8 +64,8 @@
 import org.apache.beam.sdk.function.ThrowingRunnable;
 import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.util.common.ReflectHelpers;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.Message;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.TextFormat;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.Message;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.TextFormat;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ArrayListMultimap;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.HashMultimap;
diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/RegisterHandler.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/RegisterHandler.java
index 6a02c7d..bfa0980 100644
--- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/RegisterHandler.java
+++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/RegisterHandler.java
@@ -25,7 +25,7 @@
 import org.apache.beam.model.fnexecution.v1.BeamFnApi;
 import org.apache.beam.model.fnexecution.v1.BeamFnApi.RegisterResponse;
 import org.apache.beam.model.pipeline.v1.RunnerApi;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.Message;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.Message;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/data/BeamFnDataGrpcClient.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/data/BeamFnDataGrpcClient.java
index 61c4580..e21fee9 100644
--- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/data/BeamFnDataGrpcClient.java
+++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/data/BeamFnDataGrpcClient.java
@@ -33,7 +33,7 @@
 import org.apache.beam.sdk.fn.data.LogicalEndpoint;
 import org.apache.beam.sdk.fn.stream.OutboundObserverFactory;
 import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.ManagedChannel;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.ManagedChannel;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
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 1941a10..e17f5eb 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
@@ -46,12 +46,12 @@
 import org.apache.beam.sdk.extensions.gcp.options.GcsOptions;
 import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.options.SdkHarnessOptions;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.Timestamp;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.ManagedChannel;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.Status;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.CallStreamObserver;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.ClientCallStreamObserver;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.ClientResponseObserver;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.Timestamp;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.ManagedChannel;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.Status;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.stub.CallStreamObserver;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.stub.ClientCallStreamObserver;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.stub.ClientResponseObserver;
 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;
 
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 b3e6f64..38e2027 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
@@ -27,7 +27,7 @@
 import org.apache.beam.model.fnexecution.v1.BeamFnApi.StateRequest;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.fn.stream.DataStreams;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables;
 
 /**
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 f85b3c8..bb4a661 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
@@ -31,8 +31,8 @@
 import org.apache.beam.model.pipeline.v1.Endpoints.ApiServiceDescriptor;
 import org.apache.beam.sdk.fn.IdGenerator;
 import org.apache.beam.sdk.fn.stream.OutboundObserverFactory;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.ManagedChannel;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.StreamObserver;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.ManagedChannel;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.stub.StreamObserver;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
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 26b0dfa..1d2dc1f 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
@@ -54,7 +54,7 @@
 import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.PCollectionView;
 import org.apache.beam.sdk.values.TupleTag;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
 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.Maps;
 
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 996b87e..d795d44 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
@@ -22,7 +22,7 @@
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.fn.stream.DataStreams;
 import org.apache.beam.sdk.transforms.Materializations.MultimapView;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
 
 /**
  * An implementation of a multimap side input that utilizes the Beam Fn State API to fetch values.
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 1ebadb5..a7cb38d 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
@@ -24,7 +24,7 @@
 import org.apache.beam.model.fnexecution.v1.BeamFnApi.StateGetRequest;
 import org.apache.beam.model.fnexecution.v1.BeamFnApi.StateRequest;
 import org.apache.beam.model.fnexecution.v1.BeamFnApi.StateResponse;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Throwables;
 
 /**
diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/stream/HarnessStreamObserverFactories.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/stream/HarnessStreamObserverFactories.java
index 7f21991..8707c3b 100644
--- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/stream/HarnessStreamObserverFactories.java
+++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/stream/HarnessStreamObserverFactories.java
@@ -22,7 +22,7 @@
 import org.apache.beam.sdk.fn.stream.OutboundObserverFactory;
 import org.apache.beam.sdk.options.ExperimentalOptions;
 import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.StreamObserver;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.stub.StreamObserver;
 
 /**
  * Uses {@link PipelineOptions} to configure which underlying {@link StreamObserver} implementation
diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/BoundedSourceRunnerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/BoundedSourceRunnerTest.java
index bc31e9c..0f461d2 100644
--- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/BoundedSourceRunnerTest.java
+++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/BoundedSourceRunnerTest.java
@@ -44,7 +44,7 @@
 import org.apache.beam.sdk.options.PipelineOptionsFactory;
 import org.apache.beam.sdk.util.SerializableUtils;
 import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Suppliers;
 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;
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 0f4b375..8bdce50 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
@@ -100,7 +100,7 @@
 import org.apache.beam.sdk.values.PCollectionView;
 import org.apache.beam.sdk.values.TupleTag;
 import org.apache.beam.sdk.values.TupleTagList;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Suppliers;
 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;
diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FnHarnessTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FnHarnessTest.java
index eaf0b07..90d6848 100644
--- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FnHarnessTest.java
+++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FnHarnessTest.java
@@ -42,10 +42,10 @@
 import org.apache.beam.sdk.harness.JvmInitializer;
 import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.options.PipelineOptionsFactory;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.TextFormat;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.Server;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.ServerBuilder;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.StreamObserver;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.TextFormat;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.Server;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.ServerBuilder;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.stub.StreamObserver;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.util.concurrent.Uninterruptibles;
 import org.junit.Test;
 import org.junit.runner.RunWith;
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 36a4779..5adb001 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
@@ -43,10 +43,10 @@
 import org.apache.beam.sdk.fn.test.InProcessManagedChannelFactory;
 import org.apache.beam.sdk.fn.test.TestStreams;
 import org.apache.beam.sdk.function.ThrowingFunction;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.Server;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.inprocess.InProcessServerBuilder;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.CallStreamObserver;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.StreamObserver;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.Server;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.inprocess.InProcessServerBuilder;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.stub.CallStreamObserver;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.stub.StreamObserver;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.util.concurrent.Uninterruptibles;
 import org.junit.Test;
 import org.junit.runner.RunWith;
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 a7a233d..a3e959b 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
@@ -77,8 +77,8 @@
 import org.apache.beam.sdk.util.SerializableUtils;
 import org.apache.beam.sdk.util.WindowedValue;
 import org.apache.beam.sdk.values.TupleTag;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.Message;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.Message;
 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.Maps;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Multimap;
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 deb6218..7e5dda9 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
@@ -46,13 +46,13 @@
 import org.apache.beam.sdk.options.PipelineOptionsFactory;
 import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
 import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.ManagedChannel;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.Server;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.inprocess.InProcessChannelBuilder;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.inprocess.InProcessServerBuilder;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.CallStreamObserver;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.StreamObserver;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.ManagedChannel;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.Server;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.inprocess.InProcessChannelBuilder;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.inprocess.InProcessServerBuilder;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.stub.CallStreamObserver;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.stub.StreamObserver;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/data/BeamFnDataInboundObserverTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/data/BeamFnDataInboundObserverTest.java
index aa45df6..f51f006 100644
--- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/data/BeamFnDataInboundObserverTest.java
+++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/data/BeamFnDataInboundObserverTest.java
@@ -36,7 +36,7 @@
 import org.apache.beam.sdk.fn.data.InboundDataClient;
 import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
 import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.ExpectedException;
diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/data/QueueingBeamFnDataClientTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/data/QueueingBeamFnDataClientTest.java
index 8bcacfa..094d9f7 100644
--- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/data/QueueingBeamFnDataClientTest.java
+++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/data/QueueingBeamFnDataClientTest.java
@@ -47,13 +47,13 @@
 import org.apache.beam.sdk.options.PipelineOptionsFactory;
 import org.apache.beam.sdk.transforms.windowing.GlobalWindow;
 import org.apache.beam.sdk.util.WindowedValue;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.ManagedChannel;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.Server;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.inprocess.InProcessChannelBuilder;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.inprocess.InProcessServerBuilder;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.CallStreamObserver;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.StreamObserver;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.ManagedChannel;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.Server;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.inprocess.InProcessChannelBuilder;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.inprocess.InProcessServerBuilder;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.stub.CallStreamObserver;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.stub.StreamObserver;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.runner.RunWith;
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 e3a4266..dc49275 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
@@ -37,14 +37,14 @@
 import org.apache.beam.model.pipeline.v1.Endpoints;
 import org.apache.beam.sdk.fn.test.TestStreams;
 import org.apache.beam.sdk.options.PipelineOptionsFactory;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.Timestamp;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.ManagedChannel;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.Server;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.Status;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.inprocess.InProcessChannelBuilder;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.inprocess.InProcessServerBuilder;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.CallStreamObserver;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.StreamObserver;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.Timestamp;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.ManagedChannel;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.Server;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.Status;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.inprocess.InProcessChannelBuilder;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.inprocess.InProcessServerBuilder;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.stub.CallStreamObserver;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.stub.StreamObserver;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.ExpectedException;
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 5b01c0f..dbf9885 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
@@ -25,7 +25,7 @@
 import java.io.IOException;
 import org.apache.beam.model.fnexecution.v1.BeamFnApi.StateKey;
 import org.apache.beam.sdk.coders.StringUtf8Coder;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
 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.junit.Rule;
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 e1feac1..e8f1780 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
@@ -36,14 +36,14 @@
 import org.apache.beam.sdk.fn.IdGenerators;
 import org.apache.beam.sdk.fn.stream.OutboundObserverFactory;
 import org.apache.beam.sdk.fn.test.TestStreams;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.ManagedChannel;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.Server;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.Status;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.StatusRuntimeException;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.inprocess.InProcessChannelBuilder;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.inprocess.InProcessServerBuilder;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.CallStreamObserver;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.StreamObserver;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.ManagedChannel;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.Server;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.Status;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.StatusRuntimeException;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.inprocess.InProcessChannelBuilder;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.inprocess.InProcessServerBuilder;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.stub.CallStreamObserver;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.stub.StreamObserver;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.util.concurrent.Uninterruptibles;
 import org.junit.After;
 import org.junit.Before;
diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/state/FakeBeamFnStateClient.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/state/FakeBeamFnStateClient.java
index 7762e66..e634652 100644
--- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/state/FakeBeamFnStateClient.java
+++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/state/FakeBeamFnStateClient.java
@@ -32,7 +32,7 @@
 import org.apache.beam.model.fnexecution.v1.BeamFnApi.StateRequest;
 import org.apache.beam.model.fnexecution.v1.BeamFnApi.StateRequest.RequestCase;
 import org.apache.beam.model.fnexecution.v1.BeamFnApi.StateResponse;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
 
 /** A fake implementation of a {@link BeamFnStateClient} to aid with testing. */
 public class FakeBeamFnStateClient implements BeamFnStateClient {
diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/state/MultimapSideInputTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/state/MultimapSideInputTest.java
index 9705267..635c111 100644
--- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/state/MultimapSideInputTest.java
+++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/state/MultimapSideInputTest.java
@@ -22,7 +22,7 @@
 import java.io.IOException;
 import org.apache.beam.model.fnexecution.v1.BeamFnApi.StateKey;
 import org.apache.beam.sdk.coders.StringUtf8Coder;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
 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.junit.Test;
diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/state/StateFetchingIteratorsTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/state/StateFetchingIteratorsTest.java
index 630627d..d184ca2 100644
--- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/state/StateFetchingIteratorsTest.java
+++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/state/StateFetchingIteratorsTest.java
@@ -24,7 +24,7 @@
 import org.apache.beam.model.fnexecution.v1.BeamFnApi.StateGetResponse;
 import org.apache.beam.model.fnexecution.v1.BeamFnApi.StateRequest;
 import org.apache.beam.model.fnexecution.v1.BeamFnApi.StateResponse;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterators;
 import org.junit.Test;
 import org.junit.runner.RunWith;
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 d8f5872..76294ee 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
@@ -25,8 +25,8 @@
 import org.apache.beam.sdk.fn.stream.DirectStreamObserver;
 import org.apache.beam.sdk.fn.stream.ForwardingClientResponseObserver;
 import org.apache.beam.sdk.options.PipelineOptionsFactory;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.CallStreamObserver;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.StreamObserver;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.stub.CallStreamObserver;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.stub.StreamObserver;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.runner.RunWith;
diff --git a/sdks/java/io/google-cloud-platform/build.gradle b/sdks/java/io/google-cloud-platform/build.gradle
index 0c1befd..559488f 100644
--- a/sdks/java/io/google-cloud-platform/build.gradle
+++ b/sdks/java/io/google-cloud-platform/build.gradle
@@ -33,6 +33,7 @@
   compile project(":sdks:java:extensions:protobuf")
   compile library.java.avro
   compile library.java.bigdataoss_util
+  compile library.java.gax
   compile library.java.gax_grpc
   compile library.java.google_api_client
   compile library.java.google_api_services_bigquery
@@ -50,6 +51,7 @@
   compile library.java.grpc_all
   compile library.java.grpc_auth
   compile library.java.grpc_core
+  compile library.java.grpc_context
   compile library.java.grpc_netty
   compile library.java.grpc_stub
   compile library.java.grpc_google_cloud_pubsub_v1
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 f097e47..d477b08 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
@@ -136,6 +136,7 @@
   private final Coder<ElementT> elementCoder;
   private final RowWriterFactory<ElementT, DestinationT> rowWriterFactory;
   private String kmsKey;
+  private boolean clusteringEnabled;
 
   // The maximum number of times to retry failed load or copy jobs.
   private int maxRetryJobs = DEFAULT_MAX_RETRY_JOBS;
@@ -151,7 +152,8 @@
       boolean ignoreUnknownValues,
       Coder<ElementT> elementCoder,
       RowWriterFactory<ElementT, DestinationT> rowWriterFactory,
-      @Nullable String kmsKey) {
+      @Nullable String kmsKey,
+      boolean clusteringEnabled) {
     bigQueryServices = new BigQueryServicesImpl();
     this.writeDisposition = writeDisposition;
     this.createDisposition = createDisposition;
@@ -170,6 +172,7 @@
     this.elementCoder = elementCoder;
     this.kmsKey = kmsKey;
     this.rowWriterFactory = rowWriterFactory;
+    this.clusteringEnabled = clusteringEnabled;
     schemaUpdateOptions = Collections.emptySet();
   }
 
@@ -319,6 +322,9 @@
                     .withOutputTags(multiPartitionsTag, TupleTagList.of(singlePartitionTag)));
     PCollection<KV<TableDestination, String>> tempTables =
         writeTempTables(partitions.get(multiPartitionsTag), loadJobIdPrefixView);
+
+    Coder<TableDestination> tableDestinationCoder =
+        clusteringEnabled ? TableDestinationCoderV3.of() : TableDestinationCoderV2.of();
     tempTables
         // Now that the load job has happened, we want the rename to happen immediately.
         .apply(
@@ -326,8 +332,7 @@
                 .triggering(Repeatedly.forever(AfterPane.elementCountAtLeast(1))))
         .apply(WithKeys.of((Void) null))
         .setCoder(
-            KvCoder.of(
-                VoidCoder.of(), KvCoder.of(TableDestinationCoderV2.of(), StringUtf8Coder.of())))
+            KvCoder.of(VoidCoder.of(), KvCoder.of(tableDestinationCoder, StringUtf8Coder.of())))
         .apply(GroupByKey.create())
         .apply(Values.create())
         .apply(
@@ -391,9 +396,11 @@
     PCollection<KV<TableDestination, String>> tempTables =
         writeTempTables(partitions.get(multiPartitionsTag), loadJobIdPrefixView);
 
+    Coder<TableDestination> tableDestinationCoder =
+        clusteringEnabled ? TableDestinationCoderV3.of() : TableDestinationCoderV2.of();
     tempTables
         .apply("ReifyRenameInput", new ReifyAsIterable<>())
-        .setCoder(IterableCoder.of(KvCoder.of(TableDestinationCoderV2.of(), StringUtf8Coder.of())))
+        .setCoder(IterableCoder.of(KvCoder.of(tableDestinationCoder, StringUtf8Coder.of())))
         .apply(
             "WriteRenameUntriggered",
             ParDo.of(
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 0462269..1fa8408 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
@@ -2645,7 +2645,8 @@
                 getIgnoreUnknownValues(),
                 elementCoder,
                 rowWriterFactory,
-                getKmsKey());
+                getKmsKey(),
+                getClustering() != null);
         batchLoads.setTestServices(getBigQueryServices());
         if (getSchemaUpdateOptions() != null) {
           batchLoads.setSchemaUpdateOptions(getSchemaUpdateOptions());
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 50f7528..abecf89 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
@@ -36,8 +36,8 @@
 import org.apache.beam.sdk.transforms.Create;
 import org.apache.beam.sdk.transforms.DoFn;
 import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.StreamObserver;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.stub.StreamObserver;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables;
 import org.hamcrest.Matchers;
 import org.junit.Test;
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 4e5f6e2..b68839b 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
@@ -65,6 +65,10 @@
 
   @Override
   public String getFilename() {
+    if (isDirectory()) {
+      Path parentPath = new Path(uri).getParent();
+      return parentPath == null ? null : parentPath.getName();
+    }
     return new Path(uri).getName();
   }
 
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 4d7fb8d..1726a3e 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
@@ -17,6 +17,9 @@
  */
 package org.apache.beam.sdk.io.hdfs;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+
 import java.net.URI;
 import java.util.Collections;
 import org.apache.beam.sdk.io.FileSystems;
@@ -65,4 +68,16 @@
             "hdfs://" + hdfsClusterBaseUri.getPath(), true /* isDirectory */);
     ResourceIdTester.runResourceIdBattery(baseDirectory);
   }
+
+  @Test
+  public void testGetFilename() {
+    assertNull(toResourceIdentifier("").getFilename());
+    assertEquals("abc", toResourceIdentifier("/dirA/abc").getFilename());
+    assertEquals("abc", toResourceIdentifier("/dirA/abc/").getFilename());
+    assertEquals("xyz.txt", toResourceIdentifier("/dirA/abc/xyz.txt").getFilename());
+  }
+
+  private ResourceId toResourceIdentifier(String path) {
+    return new HadoopResourceId(hdfsClusterBaseUri.resolve(path));
+  }
 }
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 a7b7f8a..500673c 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
@@ -39,8 +39,8 @@
 import org.apache.beam.sdk.transforms.Impulse;
 import org.apache.beam.sdk.transforms.WithKeys;
 import org.apache.beam.sdk.values.KV;
-import org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf.ByteString;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.stub.StreamObserver;
+import org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf.ByteString;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.stub.StreamObserver;
 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.Iterables;
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 9612487..57617c0 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
@@ -114,6 +114,7 @@
         .setConnectionConfiguration(RedisConnectionConfiguration.create())
         .setKeyPattern("*")
         .setBatchSize(1000)
+        .setOutputParallelization(true)
         .build();
   }
 
@@ -125,6 +126,7 @@
     return new AutoValue_RedisIO_ReadAll.Builder()
         .setConnectionConfiguration(RedisConnectionConfiguration.create())
         .setBatchSize(1000)
+        .setOutputParallelization(true)
         .build();
   }
 
@@ -150,7 +152,9 @@
 
     abstract int batchSize();
 
-    abstract Builder builder();
+    abstract boolean outputParallelization();
+
+    abstract Builder toBuilder();
 
     @AutoValue.Builder
     abstract static class Builder {
@@ -162,41 +166,53 @@
 
       abstract Builder setBatchSize(int batchSize);
 
+      abstract Builder setOutputParallelization(boolean outputParallelization);
+
       abstract Read build();
     }
 
     public Read withEndpoint(String host, int port) {
       checkArgument(host != null, "host can not be null");
       checkArgument(0 < port && port < 65536, "port must be a positive integer less than 65536");
-      return builder()
+      return toBuilder()
           .setConnectionConfiguration(connectionConfiguration().withHost(host).withPort(port))
           .build();
     }
 
     public Read withAuth(String auth) {
       checkArgument(auth != null, "auth can not be null");
-      return builder().setConnectionConfiguration(connectionConfiguration().withAuth(auth)).build();
+      return toBuilder()
+          .setConnectionConfiguration(connectionConfiguration().withAuth(auth))
+          .build();
     }
 
     public Read withTimeout(int timeout) {
       checkArgument(timeout >= 0, "timeout can not be negative");
-      return builder()
+      return toBuilder()
           .setConnectionConfiguration(connectionConfiguration().withTimeout(timeout))
           .build();
     }
 
     public Read withKeyPattern(String keyPattern) {
       checkArgument(keyPattern != null, "keyPattern can not be null");
-      return builder().setKeyPattern(keyPattern).build();
+      return toBuilder().setKeyPattern(keyPattern).build();
     }
 
     public Read withConnectionConfiguration(RedisConnectionConfiguration connection) {
       checkArgument(connection != null, "connection can not be null");
-      return builder().setConnectionConfiguration(connection).build();
+      return toBuilder().setConnectionConfiguration(connection).build();
     }
 
     public Read withBatchSize(int batchSize) {
-      return builder().setBatchSize(batchSize).build();
+      return toBuilder().setBatchSize(batchSize).build();
+    }
+
+    /**
+     * Whether to reshuffle the resulting PCollection so results are distributed to all workers. The
+     * default is to parallelize and should only be changed if this is known to be unnecessary.
+     */
+    public Read withOutputParallelization(boolean outputParallelization) {
+      return toBuilder().setOutputParallelization(outputParallelization).build();
     }
 
     @Override
@@ -214,7 +230,8 @@
           .apply(
               RedisIO.readAll()
                   .withConnectionConfiguration(connectionConfiguration())
-                  .withBatchSize(batchSize()));
+                  .withBatchSize(batchSize())
+                  .withOutputParallelization(outputParallelization()));
     }
   }
 
@@ -228,14 +245,18 @@
 
     abstract int batchSize();
 
-    abstract ReadAll.Builder builder();
+    abstract boolean outputParallelization();
+
+    abstract Builder toBuilder();
 
     @AutoValue.Builder
     abstract static class Builder {
       @Nullable
-      abstract ReadAll.Builder setConnectionConfiguration(RedisConnectionConfiguration connection);
+      abstract Builder setConnectionConfiguration(RedisConnectionConfiguration connection);
 
-      abstract ReadAll.Builder setBatchSize(int batchSize);
+      abstract Builder setBatchSize(int batchSize);
+
+      abstract Builder setOutputParallelization(boolean outputParallelization);
 
       abstract ReadAll build();
     }
@@ -243,44 +264,57 @@
     public ReadAll withEndpoint(String host, int port) {
       checkArgument(host != null, "host can not be null");
       checkArgument(port > 0, "port can not be negative or 0");
-      return builder()
+      return toBuilder()
           .setConnectionConfiguration(connectionConfiguration().withHost(host).withPort(port))
           .build();
     }
 
     public ReadAll withAuth(String auth) {
       checkArgument(auth != null, "auth can not be null");
-      return builder().setConnectionConfiguration(connectionConfiguration().withAuth(auth)).build();
+      return toBuilder()
+          .setConnectionConfiguration(connectionConfiguration().withAuth(auth))
+          .build();
     }
 
     public ReadAll withTimeout(int timeout) {
       checkArgument(timeout >= 0, "timeout can not be negative");
-      return builder()
+      return toBuilder()
           .setConnectionConfiguration(connectionConfiguration().withTimeout(timeout))
           .build();
     }
 
     public ReadAll withConnectionConfiguration(RedisConnectionConfiguration connection) {
       checkArgument(connection != null, "connection can not be null");
-      return builder().setConnectionConfiguration(connection).build();
+      return toBuilder().setConnectionConfiguration(connection).build();
     }
 
     public ReadAll withBatchSize(int batchSize) {
-      return builder().setBatchSize(batchSize).build();
+      return toBuilder().setBatchSize(batchSize).build();
+    }
+
+    /**
+     * Whether to reshuffle the resulting PCollection so results are distributed to all workers. The
+     * default is to parallelize and should only be changed if this is known to be unnecessary.
+     */
+    public ReadAll withOutputParallelization(boolean outputParallelization) {
+      return toBuilder().setOutputParallelization(outputParallelization).build();
     }
 
     @Override
     public PCollection<KV<String, String>> expand(PCollection<String> input) {
       checkArgument(connectionConfiguration() != null, "withConnectionConfiguration() is required");
-
-      return input
-          .apply(ParDo.of(new ReadFn(connectionConfiguration(), batchSize())))
-          .setCoder(KvCoder.of(StringUtf8Coder.of(), StringUtf8Coder.of()))
-          .apply(new Reparallelize());
+      PCollection<KV<String, String>> output =
+          input
+              .apply(ParDo.of(new ReadFn(connectionConfiguration(), batchSize())))
+              .setCoder(KvCoder.of(StringUtf8Coder.of(), StringUtf8Coder.of()));
+      if (outputParallelization()) {
+        output = output.apply(new Reparallelize());
+      }
+      return output;
     }
   }
 
-  abstract static class BaseReadFn<T> extends DoFn<String, T> {
+  private abstract static class BaseReadFn<T> extends DoFn<String, T> {
     protected final RedisConnectionConfiguration connectionConfiguration;
 
     transient Jedis jedis;
@@ -307,9 +341,9 @@
     }
 
     @ProcessElement
-    public void processElement(ProcessContext processContext) throws Exception {
+    public void processElement(ProcessContext c) {
       ScanParams scanParams = new ScanParams();
-      scanParams.match(processContext.element());
+      scanParams.match(c.element());
 
       String cursor = ScanParams.SCAN_POINTER_START;
       boolean finished = false;
@@ -317,7 +351,7 @@
         ScanResult<String> scanResult = jedis.scan(cursor, scanParams);
         List<String> keys = scanResult.getResult();
         for (String k : keys) {
-          processContext.output(k);
+          c.output(k);
         }
         cursor = scanResult.getCursor();
         if (cursor.equals(ScanParams.SCAN_POINTER_START)) {
@@ -326,42 +360,52 @@
       }
     }
   }
+
   /** A {@link DoFn} requesting Redis server to get key/value pairs. */
   private static class ReadFn extends BaseReadFn<KV<String, String>> {
     @Nullable transient Multimap<BoundedWindow, String> bundles = null;
     @Nullable AtomicInteger batchCount = null;
     private final int batchSize;
 
-    @StartBundle
-    public void startBundle(StartBundleContext context) {
-      bundles = ArrayListMultimap.create();
-      batchCount = new AtomicInteger();
-    }
-
     ReadFn(RedisConnectionConfiguration connectionConfiguration, int batchSize) {
       super(connectionConfiguration);
       this.batchSize = batchSize;
     }
 
-    private int getBatchSize() {
-      return batchSize;
+    @StartBundle
+    public void startBundle() {
+      bundles = ArrayListMultimap.create();
+      batchCount = new AtomicInteger();
     }
 
     @ProcessElement
-    public void processElement(ProcessContext processContext, BoundedWindow window)
-        throws Exception {
-      String key = processContext.element();
+    public void processElement(ProcessContext c, BoundedWindow window) {
+      String key = c.element();
       bundles.put(window, key);
       if (batchCount.incrementAndGet() > getBatchSize()) {
         Multimap<BoundedWindow, KV<String, String>> kvs = fetchAndFlush();
         for (BoundedWindow w : kvs.keySet()) {
           for (KV<String, String> kv : kvs.get(w)) {
-            processContext.output(kv);
+            c.output(kv);
           }
         }
       }
     }
 
+    @FinishBundle
+    public void finishBundle(FinishBundleContext context) {
+      Multimap<BoundedWindow, KV<String, String>> kvs = fetchAndFlush();
+      for (BoundedWindow w : kvs.keySet()) {
+        for (KV<String, String> kv : kvs.get(w)) {
+          context.output(kv, w.maxTimestamp(), w);
+        }
+      }
+    }
+
+    private int getBatchSize() {
+      return batchSize;
+    }
+
     private Multimap<BoundedWindow, KV<String, String>> fetchAndFlush() {
       Multimap<BoundedWindow, KV<String, String>> kvs = ArrayListMultimap.create();
       for (BoundedWindow w : bundles.keySet()) {
@@ -378,16 +422,6 @@
       batchCount.set(0);
       return kvs;
     }
-
-    @FinishBundle
-    public void finishBundle(FinishBundleContext context) throws Exception {
-      Multimap<BoundedWindow, KV<String, String>> kvs = fetchAndFlush();
-      for (BoundedWindow w : kvs.keySet()) {
-        for (KV<String, String> kv : kvs.get(w)) {
-          context.output(kv, w.maxTimestamp(), w);
-        }
-      }
-    }
   }
 
   private static class Reparallelize
@@ -395,8 +429,7 @@
 
     @Override
     public PCollection<KV<String, String>> expand(PCollection<KV<String, String>> input) {
-      // reparallelize mimics the same behavior as in JdbcIO
-      // breaking fusion
+      // reparallelize mimics the same behavior as in JdbcIO, used to break fusion
       PCollectionView<Iterable<KV<String, String>>> empty =
           input
               .apply("Consume", Filter.by(SerializableFunctions.constant(false)))
@@ -407,8 +440,8 @@
               ParDo.of(
                       new DoFn<KV<String, String>, KV<String, String>>() {
                         @ProcessElement
-                        public void processElement(ProcessContext context) {
-                          context.output(context.element());
+                        public void processElement(ProcessContext c) {
+                          c.output(c.element());
                         }
                       })
                   .withSideInputs(empty));
@@ -468,7 +501,7 @@
     @Nullable
     abstract Long expireTime();
 
-    abstract Builder builder();
+    abstract Builder toBuilder();
 
     @AutoValue.Builder
     abstract static class Builder {
@@ -486,37 +519,39 @@
     public Write withEndpoint(String host, int port) {
       checkArgument(host != null, "host can not be null");
       checkArgument(port > 0, "port can not be negative or 0");
-      return builder()
+      return toBuilder()
           .setConnectionConfiguration(connectionConfiguration().withHost(host).withPort(port))
           .build();
     }
 
     public Write withAuth(String auth) {
       checkArgument(auth != null, "auth can not be null");
-      return builder().setConnectionConfiguration(connectionConfiguration().withAuth(auth)).build();
+      return toBuilder()
+          .setConnectionConfiguration(connectionConfiguration().withAuth(auth))
+          .build();
     }
 
     public Write withTimeout(int timeout) {
       checkArgument(timeout >= 0, "timeout can not be negative");
-      return builder()
+      return toBuilder()
           .setConnectionConfiguration(connectionConfiguration().withTimeout(timeout))
           .build();
     }
 
     public Write withConnectionConfiguration(RedisConnectionConfiguration connection) {
       checkArgument(connection != null, "connection can not be null");
-      return builder().setConnectionConfiguration(connection).build();
+      return toBuilder().setConnectionConfiguration(connection).build();
     }
 
     public Write withMethod(Method method) {
       checkArgument(method != null, "method can not be null");
-      return builder().setMethod(method).build();
+      return toBuilder().setMethod(method).build();
     }
 
     public Write withExpireTime(Long expireTimeMillis) {
       checkArgument(expireTimeMillis != null, "expireTimeMillis can not be null");
       checkArgument(expireTimeMillis > 0, "expireTimeMillis can not be negative or 0");
-      return builder().setExpireTime(expireTimeMillis).build();
+      return toBuilder().setExpireTime(expireTimeMillis).build();
     }
 
     @Override
@@ -555,8 +590,8 @@
       }
 
       @ProcessElement
-      public void processElement(ProcessContext processContext) {
-        KV<String, String> record = processContext.element();
+      public void processElement(ProcessContext c) {
+        KV<String, String> record = c.element();
 
         writeRecord(record);
 
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 bcb3fca..badf039 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
@@ -175,7 +175,7 @@
   }
 
   @Test
-  public void testWriteUsingINCRBY() throws Exception {
+  public void testWriteUsingINCRBY() {
     String key = "key_incr";
     List<String> values = Arrays.asList("0", "1", "2", "-3", "2", "4", "0", "5");
     List<KV<String, String>> data = buildConstantKeyList(key, values);
@@ -190,7 +190,7 @@
   }
 
   @Test
-  public void testWriteUsingDECRBY() throws Exception {
+  public void testWriteUsingDECRBY() {
     String key = "key_decr";
 
     List<String> values = Arrays.asList("-10", "1", "2", "-3", "2", "4", "0", "5");
diff --git a/sdks/java/testing/expansion-service/src/test/java/org/apache/beam/sdk/expansion/TestExpansionService.java b/sdks/java/testing/expansion-service/src/test/java/org/apache/beam/sdk/expansion/TestExpansionService.java
index 5978416..d1a832f 100644
--- a/sdks/java/testing/expansion-service/src/test/java/org/apache/beam/sdk/expansion/TestExpansionService.java
+++ b/sdks/java/testing/expansion-service/src/test/java/org/apache/beam/sdk/expansion/TestExpansionService.java
@@ -31,8 +31,8 @@
 import org.apache.beam.sdk.transforms.Values;
 import org.apache.beam.sdk.values.PBegin;
 import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.Server;
-import org.apache.beam.vendor.grpc.v1p21p0.io.grpc.ServerBuilder;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.Server;
+import org.apache.beam.vendor.grpc.v1p26p0.io.grpc.ServerBuilder;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
 
 /**
diff --git a/sdks/python/apache_beam/examples/cookbook/filters_test.py b/sdks/python/apache_beam/examples/cookbook/filters_test.py
index c0d8e12..eebbc10 100644
--- a/sdks/python/apache_beam/examples/cookbook/filters_test.py
+++ b/sdks/python/apache_beam/examples/cookbook/filters_test.py
@@ -41,33 +41,31 @@
       {'year': 2011, 'month': 3, 'day': 3, 'mean_temp': 5, 'removed': 'a'},
       ]
 
-  def _get_result_for_month(self, month):
-    p = TestPipeline()
-    rows = (p | 'create' >> beam.Create(self.input_data))
-
+  def _get_result_for_month(self, pipeline, month):
+    rows = (pipeline | 'create' >> beam.Create(self.input_data))
     results = filters.filter_cold_days(rows, month)
     return results
 
   def test_basic(self):
     """Test that the correct result is returned for a simple dataset."""
-    results = self._get_result_for_month(1)
-    assert_that(
-        results,
-        equal_to([{'year': 2010, 'month': 1, 'day': 1, 'mean_temp': 3},
-                  {'year': 2012, 'month': 1, 'day': 2, 'mean_temp': 3}]))
-    results.pipeline.run()
+    with TestPipeline() as p:
+      results = self._get_result_for_month(p, 1)
+      assert_that(
+          results,
+          equal_to([{'year': 2010, 'month': 1, 'day': 1, 'mean_temp': 3},
+                    {'year': 2012, 'month': 1, 'day': 2, 'mean_temp': 3}]))
 
   def test_basic_empty(self):
     """Test that the correct empty result is returned for a simple dataset."""
-    results = self._get_result_for_month(3)
-    assert_that(results, equal_to([]))
-    results.pipeline.run()
+    with TestPipeline() as p:
+      results = self._get_result_for_month(p, 3)
+      assert_that(results, equal_to([]))
 
   def test_basic_empty_missing(self):
     """Test that the correct empty result is returned for a missing month."""
-    results = self._get_result_for_month(4)
-    assert_that(results, equal_to([]))
-    results.pipeline.run()
+    with TestPipeline() as p:
+      results = self._get_result_for_month(p, 4)
+      assert_that(results, equal_to([]))
 
 
 if __name__ == '__main__':
diff --git a/sdks/python/apache_beam/examples/fastavro_it_test.py b/sdks/python/apache_beam/examples/fastavro_it_test.py
index 5b1a3c5..35afa52 100644
--- a/sdks/python/apache_beam/examples/fastavro_it_test.py
+++ b/sdks/python/apache_beam/examples/fastavro_it_test.py
@@ -163,43 +163,42 @@
     result.wait_until_finish()
     assert result.state == PipelineState.DONE
 
-    fastavro_read_pipeline = TestPipeline(is_integration_test=True)
+    with TestPipeline(is_integration_test=True) as fastavro_read_pipeline:
 
-    fastavro_records = \
-        fastavro_read_pipeline \
-        | 'create-fastavro' >> Create(['%s*' % fastavro_output]) \
-        | 'read-fastavro' >> ReadAllFromAvro(use_fastavro=True) \
-        | Map(lambda rec: (rec['number'], rec))
+      fastavro_records = \
+          fastavro_read_pipeline \
+          | 'create-fastavro' >> Create(['%s*' % fastavro_output]) \
+          | 'read-fastavro' >> ReadAllFromAvro(use_fastavro=True) \
+          | Map(lambda rec: (rec['number'], rec))
 
-    avro_records = \
-        fastavro_read_pipeline \
-        | 'create-avro' >> Create(['%s*' % avro_output]) \
-        | 'read-avro' >> ReadAllFromAvro(use_fastavro=False) \
-        | Map(lambda rec: (rec['number'], rec))
+      avro_records = \
+          fastavro_read_pipeline \
+          | 'create-avro' >> Create(['%s*' % avro_output]) \
+          | 'read-avro' >> ReadAllFromAvro(use_fastavro=False) \
+          | Map(lambda rec: (rec['number'], rec))
 
-    def check(elem):
-      v = elem[1]
+      def check(elem):
+        v = elem[1]
 
-      def assertEqual(l, r):
-        if l != r:
-          raise BeamAssertException('Assertion failed: %s == %s' % (l, r))
+        def assertEqual(l, r):
+          if l != r:
+            raise BeamAssertException('Assertion failed: %s == %s' % (l, r))
 
-      assertEqual(v.keys(), ['avro', 'fastavro'])
-      avro_values = v['avro']
-      fastavro_values = v['fastavro']
-      assertEqual(avro_values, fastavro_values)
-      assertEqual(len(avro_values), 1)
+        assertEqual(v.keys(), ['avro', 'fastavro'])
+        avro_values = v['avro']
+        fastavro_values = v['fastavro']
+        assertEqual(avro_values, fastavro_values)
+        assertEqual(len(avro_values), 1)
 
-    # pylint: disable=expression-not-assigned
-    {
-        'avro': avro_records,
-        'fastavro': fastavro_records
-    } \
-    | CoGroupByKey() \
-    | Map(check)
+      # pylint: disable=expression-not-assigned
+      {
+          'avro': avro_records,
+          'fastavro': fastavro_records
+      } \
+      | CoGroupByKey() \
+      | Map(check)
 
-    self.addCleanup(delete_files, [self.output])
-    fastavro_read_pipeline.run().wait_until_finish()
+      self.addCleanup(delete_files, [self.output])
     assert result.state == PipelineState.DONE
 
 
diff --git a/sdks/python/apache_beam/examples/flink/flink_streaming_impulse.py b/sdks/python/apache_beam/examples/flink/flink_streaming_impulse.py
index 24ca510..47c3836 100644
--- a/sdks/python/apache_beam/examples/flink/flink_streaming_impulse.py
+++ b/sdks/python/apache_beam/examples/flink/flink_streaming_impulse.py
@@ -75,24 +75,22 @@
 
   pipeline_options = PipelineOptions(pipeline_args)
 
-  p = beam.Pipeline(options=pipeline_options)
+  with beam.Pipeline(options=pipeline_options) as p:
 
-  messages = (p | FlinkStreamingImpulseSource()
-              .set_message_count(known_args.count)
-              .set_interval_ms(known_args.interval_ms))
+    messages = (p | FlinkStreamingImpulseSource()
+                .set_message_count(known_args.count)
+                .set_interval_ms(known_args.interval_ms))
 
-  _ = (messages | 'decode' >> beam.Map(lambda x: ('', 1))
-       | 'window' >> beam.WindowInto(window.GlobalWindows(),
-                                     trigger=Repeatedly(
-                                         AfterProcessingTime(5 * 1000)),
-                                     accumulation_mode=
-                                     AccumulationMode.DISCARDING)
-       | 'group' >> beam.GroupByKey()
-       | 'count' >> beam.Map(count)
-       | 'log' >> beam.Map(lambda x: logging.info("%d" % x[1])))
+    _ = (messages | 'decode' >> beam.Map(lambda x: ('', 1))
+         | 'window' >> beam.WindowInto(window.GlobalWindows(),
+                                       trigger=Repeatedly(
+                                           AfterProcessingTime(5 * 1000)),
+                                       accumulation_mode=
+                                       AccumulationMode.DISCARDING)
+         | 'group' >> beam.GroupByKey()
+         | 'count' >> beam.Map(count)
+         | 'log' >> beam.Map(lambda x: logging.info("%d" % x[1])))
 
-  result = p.run()
-  result.wait_until_finish()
 
 
 if __name__ == '__main__':
diff --git a/sdks/python/apache_beam/examples/snippets/snippets.py b/sdks/python/apache_beam/examples/snippets/snippets.py
index 76521c5de..aaa99e9 100644
--- a/sdks/python/apache_beam/examples/snippets/snippets.py
+++ b/sdks/python/apache_beam/examples/snippets/snippets.py
@@ -111,31 +111,29 @@
   import apache_beam as beam
   from apache_beam.options.pipeline_options import PipelineOptions
 
-  p = beam.Pipeline(options=PipelineOptions())
-  # [END pipelines_constructing_creating]
+  with beam.Pipeline(options=PipelineOptions()) as p:
+    pass  # build your pipeline here
+    # [END pipelines_constructing_creating]
 
-  p = TestPipeline() # Use TestPipeline for testing.
+    with TestPipeline() as p:  # Use TestPipeline for testing.
+      # pylint: disable=line-too-long
 
-  # [START pipelines_constructing_reading]
-  lines = p | 'ReadMyFile' >> beam.io.ReadFromText('gs://some/inputData.txt')
-  # [END pipelines_constructing_reading]
+      # [START pipelines_constructing_reading]
+      lines = p | 'ReadMyFile' >> beam.io.ReadFromText('gs://some/inputData.txt')
+      # [END pipelines_constructing_reading]
 
-  # [START pipelines_constructing_applying]
-  words = lines | beam.FlatMap(lambda x: re.findall(r'[A-Za-z\']+', x))
-  reversed_words = words | ReverseWords()
-  # [END pipelines_constructing_applying]
+      # [START pipelines_constructing_applying]
+      words = lines | beam.FlatMap(lambda x: re.findall(r'[A-Za-z\']+', x))
+      reversed_words = words | ReverseWords()
+      # [END pipelines_constructing_applying]
 
-  # [START pipelines_constructing_writing]
-  filtered_words = reversed_words | 'FilterWords' >> beam.Filter(filter_words)
-  filtered_words | 'WriteMyFile' >> beam.io.WriteToText(
-      'gs://some/outputData.txt')
-  # [END pipelines_constructing_writing]
+      # [START pipelines_constructing_writing]
+      filtered_words = reversed_words | 'FilterWords' >> beam.Filter(filter_words)
+      filtered_words | 'WriteMyFile' >> beam.io.WriteToText(
+          'gs://some/outputData.txt')
+      # [END pipelines_constructing_writing]
 
-  p.visit(SnippetUtils.RenameFiles(renames))
-
-  # [START pipelines_constructing_running]
-  p.run()
-  # [END pipelines_constructing_running]
+      p.visit(SnippetUtils.RenameFiles(renames))
 
 
 def model_pipelines(argv):
@@ -249,12 +247,10 @@
   my_input = my_options.input
   my_output = my_options.output
 
-  p = TestPipeline()  # Use TestPipeline for testing.
+  with TestPipeline() as p:  # Use TestPipeline for testing.
 
-  lines = p | beam.io.ReadFromText(my_input)
-  lines | beam.io.WriteToText(my_output)
-
-  p.run()
+    lines = p | beam.io.ReadFromText(my_input)
+    lines | beam.io.WriteToText(my_output)
 
 
 def pipeline_options_local(argv):
@@ -286,13 +282,12 @@
   # [START pipeline_options_local]
   # Create and set your Pipeline Options.
   options = PipelineOptions()
-  p = Pipeline(options=options)
-  # [END pipeline_options_local]
+  with Pipeline(options=options) as p:
+    # [END pipeline_options_local]
 
-  p = TestPipeline()  # Use TestPipeline for testing.
-  lines = p | beam.io.ReadFromText(my_input)
-  lines | beam.io.WriteToText(my_output)
-  p.run()
+    with TestPipeline() as p:  # Use TestPipeline for testing.
+      lines = p | beam.io.ReadFromText(my_input)
+      lines | beam.io.WriteToText(my_output)
 
 
 def pipeline_options_command_line(argv):
@@ -541,30 +536,28 @@
           required=True,
           help='Output file to write results to.')
   pipeline_options = PipelineOptions(['--output', 'some/output_path'])
-  p = beam.Pipeline(options=pipeline_options)
+  with beam.Pipeline(options=pipeline_options) as p:
 
-  wordcount_options = pipeline_options.view_as(WordcountTemplatedOptions)
-  lines = p | 'Read' >> ReadFromText(wordcount_options.input)
-  # [END example_wordcount_templated]
+    wordcount_options = pipeline_options.view_as(WordcountTemplatedOptions)
+    lines = p | 'Read' >> ReadFromText(wordcount_options.input)
+    # [END example_wordcount_templated]
 
-  def format_result(word_count):
-    (word, count) = word_count
-    return '%s: %s' % (word, count)
+    def format_result(word_count):
+      (word, count) = word_count
+      return '%s: %s' % (word, count)
 
-  (
-      lines
-      | 'ExtractWords' >> beam.FlatMap(
-          lambda x: re.findall(r'[A-Za-z\']+', x))
-      | 'PairWithOnes' >> beam.Map(lambda x: (x, 1))
-      | 'Group' >> beam.GroupByKey()
-      | 'Sum' >> beam.Map(lambda word_ones: (word_ones[0], sum(word_ones[1])))
-      | 'Format' >> beam.Map(format_result)
-      | 'Write' >> WriteToText(wordcount_options.output)
-  )
+    (
+        lines
+        | 'ExtractWords' >> beam.FlatMap(
+            lambda x: re.findall(r'[A-Za-z\']+', x))
+        | 'PairWithOnes' >> beam.Map(lambda x: (x, 1))
+        | 'Group' >> beam.GroupByKey()
+        | 'Sum' >> beam.Map(lambda word_ones: (word_ones[0], sum(word_ones[1])))
+        | 'Format' >> beam.Map(format_result)
+        | 'Write' >> WriteToText(wordcount_options.output)
+    )
 
-  p.visit(SnippetUtils.RenameFiles(renames))
-  result = p.run()
-  result.wait_until_finish()
+    p.visit(SnippetUtils.RenameFiles(renames))
 
 
 def examples_wordcount_debugging(renames):
@@ -713,25 +706,23 @@
       yield self.templated_int.get() + an_int
 
   pipeline_options = PipelineOptions()
-  p = beam.Pipeline(options=pipeline_options)
+  with beam.Pipeline(options=pipeline_options) as p:
 
-  user_options = pipeline_options.view_as(TemplatedUserOptions)
-  my_sum_fn = MySumFn(user_options.templated_int)
-  sum = (p
-         | 'ReadCollection' >> beam.io.ReadFromText(
-             'gs://some/integer_collection')
-         | 'StringToInt' >> beam.Map(lambda w: int(w))
-         | 'AddGivenInt' >> beam.ParDo(my_sum_fn)
-         | 'WriteResultingCollection' >> WriteToText('some/output_path'))
-  # [END examples_ptransforms_templated]
+    user_options = pipeline_options.view_as(TemplatedUserOptions)
+    my_sum_fn = MySumFn(user_options.templated_int)
+    sum = (p
+           | 'ReadCollection' >> beam.io.ReadFromText(
+               'gs://some/integer_collection')
+           | 'StringToInt' >> beam.Map(lambda w: int(w))
+           | 'AddGivenInt' >> beam.ParDo(my_sum_fn)
+           | 'WriteResultingCollection' >> WriteToText('some/output_path'))
+    # [END examples_ptransforms_templated]
 
-  # Templates are not supported by DirectRunner (only by DataflowRunner)
-  # so a value must be provided at graph-construction time
-  my_sum_fn.templated_int = StaticValueProvider(int, 10)
+    # Templates are not supported by DirectRunner (only by DataflowRunner)
+    # so a value must be provided at graph-construction time
+    my_sum_fn.templated_int = StaticValueProvider(int, 10)
 
-  p.visit(SnippetUtils.RenameFiles(renames))
-  result = p.run()
-  result.wait_until_finish()
+    p.visit(SnippetUtils.RenameFiles(renames))
 
 
 # Defining a new source.
@@ -835,16 +826,15 @@
             ['line ' + str(number) for number in range(0, count)]))
 
   # [START model_custom_source_use_ptransform]
-  p = beam.Pipeline(options=PipelineOptions())
-  numbers = p | 'ProduceNumbers' >> ReadFromCountingSource(count)
-  # [END model_custom_source_use_ptransform]
+  with beam.Pipeline(options=PipelineOptions()) as p:
+    numbers = p | 'ProduceNumbers' >> ReadFromCountingSource(count)
+    # [END model_custom_source_use_ptransform]
 
-  lines = numbers | beam.core.Map(lambda number: 'line %d' % number)
-  assert_that(
-      lines, equal_to(
-          ['line ' + str(number) for number in range(0, count)]))
+    lines = numbers | beam.core.Map(lambda number: 'line %d' % number)
+    assert_that(
+        lines, equal_to(
+            ['line ' + str(number) for number in range(0, count)]))
 
-  p.run().wait_until_finish()
 
 
 # Defining the new sink.
@@ -1402,20 +1392,19 @@
 
   pipeline_options = PipelineOptions()
   # Create pipeline.
-  p = beam.Pipeline(options=pipeline_options)
+  with beam.Pipeline(options=pipeline_options) as p:
 
-  my_options = pipeline_options.view_as(MyOptions)
-  # Add a branch for logging the ValueProvider value.
-  _ = (p
-       | beam.Create([None])
-       | 'LogValueProvs' >> beam.ParDo(
-           LogValueProvidersFn(my_options.string_value)))
+    my_options = pipeline_options.view_as(MyOptions)
+    # Add a branch for logging the ValueProvider value.
+    _ = (p
+         | beam.Create([None])
+         | 'LogValueProvs' >> beam.ParDo(
+             LogValueProvidersFn(my_options.string_value)))
 
-  # The main pipeline.
-  result_pc = (p
-               | "main_pc" >> beam.Create([1, 2, 3])
-               | beam.combiners.Sum.Globally())
+    # The main pipeline.
+    result_pc = (p
+                 | "main_pc" >> beam.Create([1, 2, 3])
+                 | beam.combiners.Sum.Globally())
 
-  p.run().wait_until_finish()
 
   # [END AccessingValueProviderInfoAfterRunSnip1]
diff --git a/sdks/python/apache_beam/examples/streaming_wordcount.py b/sdks/python/apache_beam/examples/streaming_wordcount.py
index 461e073..cdfb6a1 100644
--- a/sdks/python/apache_beam/examples/streaming_wordcount.py
+++ b/sdks/python/apache_beam/examples/streaming_wordcount.py
@@ -58,50 +58,48 @@
   pipeline_options = PipelineOptions(pipeline_args)
   pipeline_options.view_as(SetupOptions).save_main_session = save_main_session
   pipeline_options.view_as(StandardOptions).streaming = True
-  p = beam.Pipeline(options=pipeline_options)
+  with beam.Pipeline(options=pipeline_options) as p:
 
-  # Read from PubSub into a PCollection.
-  if known_args.input_subscription:
-    messages = (p
-                | beam.io.ReadFromPubSub(
-                    subscription=known_args.input_subscription)
-                .with_output_types(bytes))
-  else:
-    messages = (p
-                | beam.io.ReadFromPubSub(topic=known_args.input_topic)
-                .with_output_types(bytes))
+    # Read from PubSub into a PCollection.
+    if known_args.input_subscription:
+      messages = (p
+                  | beam.io.ReadFromPubSub(
+                      subscription=known_args.input_subscription)
+                  .with_output_types(bytes))
+    else:
+      messages = (p
+                  | beam.io.ReadFromPubSub(topic=known_args.input_topic)
+                  .with_output_types(bytes))
 
-  lines = messages | 'decode' >> beam.Map(lambda x: x.decode('utf-8'))
+    lines = messages | 'decode' >> beam.Map(lambda x: x.decode('utf-8'))
 
-  # Count the occurrences of each word.
-  def count_ones(word_ones):
-    (word, ones) = word_ones
-    return (word, sum(ones))
+    # Count the occurrences of each word.
+    def count_ones(word_ones):
+      (word, ones) = word_ones
+      return (word, sum(ones))
 
-  counts = (lines
-            | 'split' >> (beam.ParDo(WordExtractingDoFn())
-                          .with_output_types(unicode))
-            | 'pair_with_one' >> beam.Map(lambda x: (x, 1))
-            | beam.WindowInto(window.FixedWindows(15, 0))
-            | 'group' >> beam.GroupByKey()
-            | 'count' >> beam.Map(count_ones))
+    counts = (lines
+              | 'split' >> (beam.ParDo(WordExtractingDoFn())
+                            .with_output_types(unicode))
+              | 'pair_with_one' >> beam.Map(lambda x: (x, 1))
+              | beam.WindowInto(window.FixedWindows(15, 0))
+              | 'group' >> beam.GroupByKey()
+              | 'count' >> beam.Map(count_ones))
 
-  # Format the counts into a PCollection of strings.
-  def format_result(word_count):
-    (word, count) = word_count
-    return '%s: %d' % (word, count)
+    # Format the counts into a PCollection of strings.
+    def format_result(word_count):
+      (word, count) = word_count
+      return '%s: %d' % (word, count)
 
-  output = (counts
-            | 'format' >> beam.Map(format_result)
-            | 'encode' >> beam.Map(lambda x: x.encode('utf-8'))
-            .with_output_types(bytes))
+    output = (counts
+              | 'format' >> beam.Map(format_result)
+              | 'encode' >> beam.Map(lambda x: x.encode('utf-8'))
+              .with_output_types(bytes))
 
-  # Write to PubSub.
-  # pylint: disable=expression-not-assigned
-  output | beam.io.WriteToPubSub(known_args.output_topic)
+    # Write to PubSub.
+    # pylint: disable=expression-not-assigned
+    output | beam.io.WriteToPubSub(known_args.output_topic)
 
-  result = p.run()
-  result.wait_until_finish()
 
 
 if __name__ == '__main__':
diff --git a/sdks/python/apache_beam/examples/streaming_wordcount_debugging.py b/sdks/python/apache_beam/examples/streaming_wordcount_debugging.py
index db5304d..79eecea 100644
--- a/sdks/python/apache_beam/examples/streaming_wordcount_debugging.py
+++ b/sdks/python/apache_beam/examples/streaming_wordcount_debugging.py
@@ -103,82 +103,80 @@
   pipeline_options = PipelineOptions(pipeline_args)
   pipeline_options.view_as(SetupOptions).save_main_session = True
   pipeline_options.view_as(StandardOptions).streaming = True
-  p = beam.Pipeline(options=pipeline_options)
+  with beam.Pipeline(options=pipeline_options) as p:
 
-  # Read from PubSub into a PCollection.
-  if known_args.input_subscription:
-    lines = p | beam.io.ReadFromPubSub(
-        subscription=known_args.input_subscription)
-  else:
-    lines = p | beam.io.ReadFromPubSub(topic=known_args.input_topic)
+    # Read from PubSub into a PCollection.
+    if known_args.input_subscription:
+      lines = p | beam.io.ReadFromPubSub(
+          subscription=known_args.input_subscription)
+    else:
+      lines = p | beam.io.ReadFromPubSub(topic=known_args.input_topic)
 
-  # Count the occurrences of each word.
-  def count_ones(word_ones):
-    (word, ones) = word_ones
-    return (word, sum(ones))
+    # Count the occurrences of each word.
+    def count_ones(word_ones):
+      (word, ones) = word_ones
+      return (word, sum(ones))
 
-  counts = (lines
-            | 'AddTimestampFn' >> beam.ParDo(AddTimestampFn())
-            | 'After AddTimestampFn' >> ParDo(PrintFn('After AddTimestampFn'))
-            | 'Split' >> (beam.ParDo(WordExtractingDoFn())
-                          .with_output_types(unicode))
-            | 'PairWithOne' >> beam.Map(lambda x: (x, 1))
-            | beam.WindowInto(window.FixedWindows(5, 0))
-            | 'GroupByKey' >> beam.GroupByKey()
-            | 'CountOnes' >> beam.Map(count_ones))
+    counts = (lines
+              | 'AddTimestampFn' >> beam.ParDo(AddTimestampFn())
+              | 'After AddTimestampFn' >> ParDo(PrintFn('After AddTimestampFn'))
+              | 'Split' >> (beam.ParDo(WordExtractingDoFn())
+                            .with_output_types(unicode))
+              | 'PairWithOne' >> beam.Map(lambda x: (x, 1))
+              | beam.WindowInto(window.FixedWindows(5, 0))
+              | 'GroupByKey' >> beam.GroupByKey()
+              | 'CountOnes' >> beam.Map(count_ones))
 
-  # Format the counts into a PCollection of strings.
-  def format_result(word_count):
-    (word, count) = word_count
-    return '%s: %d' % (word, count)
+    # Format the counts into a PCollection of strings.
+    def format_result(word_count):
+      (word, count) = word_count
+      return '%s: %d' % (word, count)
 
-  output = counts | 'format' >> beam.Map(format_result)
+    output = counts | 'format' >> beam.Map(format_result)
 
-  # Write to PubSub.
-  # pylint: disable=expression-not-assigned
-  output | beam.io.WriteStringsToPubSub(known_args.output_topic)
+    # Write to PubSub.
+    # pylint: disable=expression-not-assigned
+    output | beam.io.WriteStringsToPubSub(known_args.output_topic)
 
-  def check_gbk_format():
-    # A matcher that checks that the output of GBK is of the form word: count.
-    def matcher(elements):
-      # pylint: disable=unused-variable
-      actual_elements_in_window, window = elements
-      for elm in actual_elements_in_window:
-        assert re.match(r'\S+:\s+\d+', elm) is not None
-    return matcher
+    def check_gbk_format():
+      # A matcher that checks that the output of GBK is of the form word: count.
+      def matcher(elements):
+        # pylint: disable=unused-variable
+        actual_elements_in_window, window = elements
+        for elm in actual_elements_in_window:
+          assert re.match(r'\S+:\s+\d+', elm) is not None
+      return matcher
 
-  # Check that the format of the output is correct.
-  assert_that(
-      output,
-      check_gbk_format(),
-      use_global_window=False,
-      label='Assert word:count format.')
+    # Check that the format of the output is correct.
+    assert_that(
+        output,
+        check_gbk_format(),
+        use_global_window=False,
+        label='Assert word:count format.')
 
-  # Check also that elements are ouput in the right window.
-  # This expects exactly 1 occurrence of any subset of the elements
-  # 150, 151, 152, 153, 154 in the window [150, 155)
-  # or exactly 1 occurrence of any subset of the elements
-  # 210, 211, 212, 213, 214 in the window [210, 215).
-  expected_window_to_elements = {
-      window.IntervalWindow(150, 155): [
-          ('150: 1'), ('151: 1'), ('152: 1'), ('153: 1'), ('154: 1'),
-      ],
-      window.IntervalWindow(210, 215): [
-          ('210: 1'), ('211: 1'), ('212: 1'), ('213: 1'), ('214: 1'),
-      ],
-  }
+    # Check also that elements are ouput in the right window.
+    # This expects exactly 1 occurrence of any subset of the elements
+    # 150, 151, 152, 153, 154 in the window [150, 155)
+    # or exactly 1 occurrence of any subset of the elements
+    # 210, 211, 212, 213, 214 in the window [210, 215).
+    expected_window_to_elements = {
+        window.IntervalWindow(150, 155): [
+            ('150: 1'), ('151: 1'), ('152: 1'), ('153: 1'), ('154: 1'),
+        ],
+        window.IntervalWindow(210, 215): [
+            ('210: 1'), ('211: 1'), ('212: 1'), ('213: 1'), ('214: 1'),
+        ],
+    }
 
-  # To make it pass, publish numbers in [150-155) or [210-215) with no repeats.
-  # To make it fail, publish a repeated number in the range above range.
-  # For example: '210 213 151 213'
-  assert_that(
-      output,
-      equal_to_per_window(expected_window_to_elements),
-      use_global_window=False,
-      label='Assert correct streaming windowing.')
+    # To pass, publish numbers in [150-155) or [210-215) with no repeats.
+    # To fail, publish a repeated number in the range above range.
+    # For example: '210 213 151 213'
+    assert_that(
+        output,
+        equal_to_per_window(expected_window_to_elements),
+        use_global_window=False,
+        label='Assert correct streaming windowing.')
 
-  result = p.run()
-  result.wait_until_finish()
 
 
 if __name__ == '__main__':
diff --git a/sdks/python/apache_beam/examples/wordcount_xlang.py b/sdks/python/apache_beam/examples/wordcount_xlang.py
index 12e2b6b..b8353bb 100644
--- a/sdks/python/apache_beam/examples/wordcount_xlang.py
+++ b/sdks/python/apache_beam/examples/wordcount_xlang.py
@@ -27,6 +27,7 @@
 import subprocess
 
 import grpc
+from past.builtins import unicode
 
 import apache_beam as beam
 from apache_beam.io import ReadFromText
@@ -58,13 +59,13 @@
     return re.findall(r'[\w\']+', text_line)
 
 
-def run(p, input_file, output_file):
+def build_pipeline(p, input_file, output_file):
   # Read the text file[pattern] into a PCollection.
   lines = p | 'read' >> ReadFromText(input_file)
 
   counts = (lines
             | 'split' >> (beam.ParDo(WordExtractingDoFn())
-                          .with_output_types(bytes))
+                          .with_output_types(unicode))
             | 'count' >> beam.ExternalTransform(
                 'beam:transforms:xlang:count', None, EXPANSION_SERVICE_ADDR))
 
@@ -79,9 +80,6 @@
   # pylint: disable=expression-not-assigned
   output | 'write' >> WriteToText(output_file)
 
-  result = p.run()
-  result.wait_until_finish()
-
 
 def main():
   logging.getLogger().setLevel(logging.INFO)
@@ -111,10 +109,6 @@
   # workflow rely on global context (e.g., a module imported at module level).
   pipeline_options.view_as(SetupOptions).save_main_session = True
 
-  p = beam.Pipeline(options=pipeline_options)
-  # Preemptively start due to BEAM-6666.
-  p.runner.create_job_service(pipeline_options)
-
   try:
     server = subprocess.Popen([
         'java', '-jar', known_args.expansion_service_jar,
@@ -123,7 +117,11 @@
     with grpc.insecure_channel(EXPANSION_SERVICE_ADDR) as channel:
       grpc.channel_ready_future(channel).result()
 
-    run(p, known_args.input, known_args.output)
+    with beam.Pipeline(options=pipeline_options) as p:
+      # Preemptively start due to BEAM-6666.
+      p.runner.create_job_service(pipeline_options)
+
+      build_pipeline(p, known_args.input, known_args.output)
 
   finally:
     server.kill()
diff --git a/sdks/python/apache_beam/internal/gcp/auth.py b/sdks/python/apache_beam/internal/gcp/auth.py
index 5f248b3..3921f73 100644
--- a/sdks/python/apache_beam/internal/gcp/auth.py
+++ b/sdks/python/apache_beam/internal/gcp/auth.py
@@ -128,7 +128,8 @@
           'https://www.googleapis.com/auth/cloud-platform',
           'https://www.googleapis.com/auth/devstorage.full_control',
           'https://www.googleapis.com/auth/userinfo.email',
-          'https://www.googleapis.com/auth/datastore'
+          'https://www.googleapis.com/auth/datastore',
+          'https://www.googleapis.com/auth/spanner'
       ]
       try:
         credentials = GoogleCredentials.get_application_default()
diff --git a/sdks/python/apache_beam/io/concat_source_test.py b/sdks/python/apache_beam/io/concat_source_test.py
index eea44e0..41e4e63 100644
--- a/sdks/python/apache_beam/io/concat_source_test.py
+++ b/sdks/python/apache_beam/io/concat_source_test.py
@@ -225,11 +225,10 @@
                            RangeSource(10, 100),
                            RangeSource(100, 1000),
                           ])
-    pipeline = TestPipeline()
-    pcoll = pipeline | beam.io.Read(source)
-    assert_that(pcoll, equal_to(list(range(1000))))
+    with TestPipeline() as pipeline:
+      pcoll = pipeline | beam.io.Read(source)
+      assert_that(pcoll, equal_to(list(range(1000))))
 
-    pipeline.run()
 
   def test_conact_source_exhaustive(self):
     source = ConcatSource([RangeSource(0, 10),
diff --git a/sdks/python/apache_beam/io/external/generate_sequence_test.py b/sdks/python/apache_beam/io/external/generate_sequence_test.py
index 652e47b..060ce28 100644
--- a/sdks/python/apache_beam/io/external/generate_sequence_test.py
+++ b/sdks/python/apache_beam/io/external/generate_sequence_test.py
@@ -41,12 +41,11 @@
     "EXPANSION_PORT environment var is not provided.")
 class XlangGenerateSequenceTest(unittest.TestCase):
   def test_generate_sequence(self):
-    test_pipeline = TestPipeline()
     port = os.environ.get('EXPANSION_PORT')
     address = 'localhost:%s' % port
 
     try:
-      with test_pipeline as p:
+      with TestPipeline() as p:
         res = (
             p
             | GenerateSequence(start=1, stop=10,
diff --git a/sdks/python/apache_beam/io/external/xlang_parquetio_test.py b/sdks/python/apache_beam/io/external/xlang_parquetio_test.py
index ed49a58..aee35a0 100644
--- a/sdks/python/apache_beam/io/external/xlang_parquetio_test.py
+++ b/sdks/python/apache_beam/io/external/xlang_parquetio_test.py
@@ -53,11 +53,10 @@
     port = os.environ.get('EXPANSION_PORT')
     address = 'localhost:%s' % port
     try:
-      test_pipeline = TestPipeline()
-      test_pipeline.get_pipeline_options().view_as(
-          DebugOptions).experiments.append('jar_packages='+expansion_jar)
-      test_pipeline.not_use_test_runner_api = True
-      with test_pipeline as p:
+      with TestPipeline() as p:
+        p.get_pipeline_options().view_as(
+            DebugOptions).experiments.append('jar_packages='+expansion_jar)
+        p.not_use_test_runner_api = True
         _ = p \
           | beam.Create([
               AvroRecord({"name": "abc"}), AvroRecord({"name": "def"}),
diff --git a/sdks/python/apache_beam/io/filebasedsource_test.py b/sdks/python/apache_beam/io/filebasedsource_test.py
index 2c5bd98..3c9adbd 100644
--- a/sdks/python/apache_beam/io/filebasedsource_test.py
+++ b/sdks/python/apache_beam/io/filebasedsource_test.py
@@ -436,11 +436,10 @@
     self.assertCountEqual(expected_data, read_data)
 
   def _run_source_test(self, pattern, expected_data, splittable=True):
-    pipeline = TestPipeline()
-    pcoll = pipeline | 'Read' >> beam.io.Read(LineSource(
-        pattern, splittable=splittable))
-    assert_that(pcoll, equal_to(expected_data))
-    pipeline.run()
+    with TestPipeline() as pipeline:
+      pcoll = pipeline | 'Read' >> beam.io.Read(LineSource(
+          pattern, splittable=splittable))
+      assert_that(pcoll, equal_to(expected_data))
 
   def test_source_file(self):
     file_name, expected_data = write_data(100)
@@ -476,13 +475,12 @@
     with bz2.BZ2File(filename, 'wb') as f:
       f.write(b'\n'.join(lines))
 
-    pipeline = TestPipeline()
-    pcoll = pipeline | 'Read' >> beam.io.Read(LineSource(
-        filename,
-        splittable=False,
-        compression_type=CompressionTypes.BZIP2))
-    assert_that(pcoll, equal_to(lines))
-    pipeline.run()
+    with TestPipeline() as pipeline:
+      pcoll = pipeline | 'Read' >> beam.io.Read(LineSource(
+          filename,
+          splittable=False,
+          compression_type=CompressionTypes.BZIP2))
+      assert_that(pcoll, equal_to(lines))
 
   def test_read_file_gzip(self):
     _, lines = write_data(10)
@@ -491,13 +489,12 @@
     with gzip.GzipFile(filename, 'wb') as f:
       f.write(b'\n'.join(lines))
 
-    pipeline = TestPipeline()
-    pcoll = pipeline | 'Read' >> beam.io.Read(LineSource(
-        filename,
-        splittable=False,
-        compression_type=CompressionTypes.GZIP))
-    assert_that(pcoll, equal_to(lines))
-    pipeline.run()
+    with TestPipeline() as pipeline:
+      pcoll = pipeline | 'Read' >> beam.io.Read(LineSource(
+          filename,
+          splittable=False,
+          compression_type=CompressionTypes.GZIP))
+      assert_that(pcoll, equal_to(lines))
 
   def test_read_pattern_bzip2(self):
     _, lines = write_data(200)
@@ -509,13 +506,12 @@
       compressed_chunks.append(
           compressobj.compress(b'\n'.join(c)) + compressobj.flush())
     file_pattern = write_prepared_pattern(compressed_chunks)
-    pipeline = TestPipeline()
-    pcoll = pipeline | 'Read' >> beam.io.Read(LineSource(
-        file_pattern,
-        splittable=False,
-        compression_type=CompressionTypes.BZIP2))
-    assert_that(pcoll, equal_to(lines))
-    pipeline.run()
+    with TestPipeline() as pipeline:
+      pcoll = pipeline | 'Read' >> beam.io.Read(LineSource(
+          file_pattern,
+          splittable=False,
+          compression_type=CompressionTypes.BZIP2))
+      assert_that(pcoll, equal_to(lines))
 
   def test_read_pattern_gzip(self):
     _, lines = write_data(200)
@@ -528,13 +524,12 @@
         f.write(b'\n'.join(c))
       compressed_chunks.append(out.getvalue())
     file_pattern = write_prepared_pattern(compressed_chunks)
-    pipeline = TestPipeline()
-    pcoll = pipeline | 'Read' >> beam.io.Read(LineSource(
-        file_pattern,
-        splittable=False,
-        compression_type=CompressionTypes.GZIP))
-    assert_that(pcoll, equal_to(lines))
-    pipeline.run()
+    with TestPipeline() as pipeline:
+      pcoll = pipeline | 'Read' >> beam.io.Read(LineSource(
+          file_pattern,
+          splittable=False,
+          compression_type=CompressionTypes.GZIP))
+      assert_that(pcoll, equal_to(lines))
 
   def test_read_auto_single_file_bzip2(self):
     _, lines = write_data(10)
@@ -543,12 +538,11 @@
     with bz2.BZ2File(filename, 'wb') as f:
       f.write(b'\n'.join(lines))
 
-    pipeline = TestPipeline()
-    pcoll = pipeline | 'Read' >> beam.io.Read(LineSource(
-        filename,
-        compression_type=CompressionTypes.AUTO))
-    assert_that(pcoll, equal_to(lines))
-    pipeline.run()
+    with TestPipeline() as pipeline:
+      pcoll = pipeline | 'Read' >> beam.io.Read(LineSource(
+          filename,
+          compression_type=CompressionTypes.AUTO))
+      assert_that(pcoll, equal_to(lines))
 
   def test_read_auto_single_file_gzip(self):
     _, lines = write_data(10)
@@ -557,12 +551,11 @@
     with gzip.GzipFile(filename, 'wb') as f:
       f.write(b'\n'.join(lines))
 
-    pipeline = TestPipeline()
-    pcoll = pipeline | 'Read' >> beam.io.Read(LineSource(
-        filename,
-        compression_type=CompressionTypes.AUTO))
-    assert_that(pcoll, equal_to(lines))
-    pipeline.run()
+    with TestPipeline() as pipeline:
+      pcoll = pipeline | 'Read' >> beam.io.Read(LineSource(
+          filename,
+          compression_type=CompressionTypes.AUTO))
+      assert_that(pcoll, equal_to(lines))
 
   def test_read_auto_pattern(self):
     _, lines = write_data(200)
@@ -576,12 +569,11 @@
       compressed_chunks.append(out.getvalue())
     file_pattern = write_prepared_pattern(
         compressed_chunks, suffixes=['.gz']*len(chunks))
-    pipeline = TestPipeline()
-    pcoll = pipeline | 'Read' >> beam.io.Read(LineSource(
-        file_pattern,
-        compression_type=CompressionTypes.AUTO))
-    assert_that(pcoll, equal_to(lines))
-    pipeline.run()
+    with TestPipeline() as pipeline:
+      pcoll = pipeline | 'Read' >> beam.io.Read(LineSource(
+          file_pattern,
+          compression_type=CompressionTypes.AUTO))
+      assert_that(pcoll, equal_to(lines))
 
   def test_read_auto_pattern_compressed_and_uncompressed(self):
     _, lines = write_data(200)
@@ -598,12 +590,11 @@
         chunks_to_write.append(b'\n'.join(c))
     file_pattern = write_prepared_pattern(chunks_to_write,
                                           suffixes=(['.gz', '']*3))
-    pipeline = TestPipeline()
-    pcoll = pipeline | 'Read' >> beam.io.Read(LineSource(
-        file_pattern,
-        compression_type=CompressionTypes.AUTO))
-    assert_that(pcoll, equal_to(lines))
-    pipeline.run()
+    with TestPipeline() as pipeline:
+      pcoll = pipeline | 'Read' >> beam.io.Read(LineSource(
+          file_pattern,
+          compression_type=CompressionTypes.AUTO))
+      assert_that(pcoll, equal_to(lines))
 
   def test_splits_get_coder_from_fbs(self):
     class DummyCoder(object):
diff --git a/sdks/python/apache_beam/io/gcp/bigquery_io_read_it_test.py b/sdks/python/apache_beam/io/gcp/bigquery_io_read_it_test.py
index b9a5cf4..386d5d7 100644
--- a/sdks/python/apache_beam/io/gcp/bigquery_io_read_it_test.py
+++ b/sdks/python/apache_beam/io/gcp/bigquery_io_read_it_test.py
@@ -45,17 +45,22 @@
                  "1G": 11110839,
                  "1T": 11110839000,}
 
-  def run_bigquery_io_read_pipeline(self, input_size):
+  def run_bigquery_io_read_pipeline(self, input_size, beam_bq_source=False):
     test_pipeline = TestPipeline(is_integration_test=True)
     pipeline_verifiers = [PipelineStateMatcher(),]
     extra_opts = {'input_table': self.DEFAULT_DATASET + "." +
                                  self.DEFAULT_TABLE_PREFIX + input_size,
                   'num_records': self.NUM_RECORDS[input_size],
+                  'beam_bq_source': str(beam_bq_source),
                   'on_success_matcher': all_of(*pipeline_verifiers)}
     bigquery_io_read_pipeline.run(test_pipeline.get_full_options_as_args(
         **extra_opts))
 
   @attr('IT')
+  def test_bigquery_read_custom_1M_python(self):
+    self.run_bigquery_io_read_pipeline('1M', True)
+
+  @attr('IT')
   def test_bigquery_read_1M_python(self):
     self.run_bigquery_io_read_pipeline('1M')
 
diff --git a/sdks/python/apache_beam/io/gcp/bigquery_io_read_pipeline.py b/sdks/python/apache_beam/io/gcp/bigquery_io_read_pipeline.py
index 81cbab4..7e1dc94 100644
--- a/sdks/python/apache_beam/io/gcp/bigquery_io_read_pipeline.py
+++ b/sdks/python/apache_beam/io/gcp/bigquery_io_read_pipeline.py
@@ -30,6 +30,8 @@
 import time
 
 import apache_beam as beam
+from apache_beam.io.gcp.bigquery import _ReadFromBigQuery
+from apache_beam.options.pipeline_options import GoogleCloudOptions
 from apache_beam.options.pipeline_options import PipelineOptions
 from apache_beam.testing.test_pipeline import TestPipeline
 from apache_beam.testing.util import assert_that
@@ -60,20 +62,27 @@
   parser.add_argument('--num_slow', default=0,
                       help=('Percentage of rows that will be slow. '
                             'Must be in the range [0, 100)'))
+  parser.add_argument('--beam_bq_source', default=False, type=bool,
+                      help=('Whether to use the new _ReadFromBigQuery'
+                            ' transform, or the BigQuerySource.'))
   known_args, pipeline_args = parser.parse_known_args(argv)
 
-  p = TestPipeline(options=PipelineOptions(pipeline_args))
+  options = PipelineOptions(pipeline_args)
+  with TestPipeline(options=options) as p:
+    if known_args.beam_bq_source:
+      reader = _ReadFromBigQuery(
+          table='%s:%s' % (options.view_as(GoogleCloudOptions).project,
+                           known_args.input_table))
+    else:
+      reader = beam.io.Read(beam.io.BigQuerySource(known_args.input_table))
 
-  # pylint: disable=expression-not-assigned
-  count = (p | 'read' >> beam.io.Read(beam.io.BigQuerySource(
-      known_args.input_table))
-           | 'row to string' >> beam.ParDo(RowToStringWithSlowDown(),
-                                           num_slow=known_args.num_slow)
-           | 'count' >> beam.combiners.Count.Globally())
+    # pylint: disable=expression-not-assigned
+    count = (p | 'read' >> reader
+             | 'row to string' >> beam.ParDo(RowToStringWithSlowDown(),
+                                             num_slow=known_args.num_slow)
+             | 'count' >> beam.combiners.Count.Globally())
 
-  assert_that(count, equal_to([known_args.num_records]))
-
-  p.run()
+    assert_that(count, equal_to([known_args.num_records]))
 
 
 if __name__ == '__main__':
diff --git a/sdks/python/apache_beam/io/gcp/bigquery_read_perf_test.py b/sdks/python/apache_beam/io/gcp/bigquery_read_perf_test.py
index 5d1fc98..d93cd88 100644
--- a/sdks/python/apache_beam/io/gcp/bigquery_read_perf_test.py
+++ b/sdks/python/apache_beam/io/gcp/bigquery_read_perf_test.py
@@ -117,17 +117,17 @@
       # of the part
       return {'data': base64.b64encode(record[1])}
 
-    p = TestPipeline()
-    # pylint: disable=expression-not-assigned
-    (p
-     | 'Produce rows' >> Read(SyntheticSource(self.parseTestPipelineOptions()))
-     | 'Format' >> Map(format_record)
-     | 'Write to BigQuery' >> WriteToBigQuery(
-         dataset=self.input_dataset, table=self.input_table,
-         schema=SCHEMA,
-         create_disposition=BigQueryDisposition.CREATE_IF_NEEDED,
-         write_disposition=BigQueryDisposition.WRITE_EMPTY))
-    p.run().wait_until_finish()
+    with TestPipeline() as p:
+      # pylint: disable=expression-not-assigned
+      (p
+       | 'Produce rows' >> Read(SyntheticSource(
+           self.parseTestPipelineOptions()))
+       | 'Format' >> Map(format_record)
+       | 'Write to BigQuery' >> WriteToBigQuery(
+           dataset=self.input_dataset, table=self.input_table,
+           schema=SCHEMA,
+           create_disposition=BigQueryDisposition.CREATE_IF_NEEDED,
+           write_disposition=BigQueryDisposition.WRITE_EMPTY))
 
   def test(self):
     output = (self.pipeline
diff --git a/sdks/python/apache_beam/io/gcp/experimental/__init__.py b/sdks/python/apache_beam/io/gcp/experimental/__init__.py
new file mode 100644
index 0000000..f4f43cb
--- /dev/null
+++ b/sdks/python/apache_beam/io/gcp/experimental/__init__.py
@@ -0,0 +1,17 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+from __future__ import absolute_import
diff --git a/sdks/python/apache_beam/io/gcp/experimental/spannerio.py b/sdks/python/apache_beam/io/gcp/experimental/spannerio.py
new file mode 100644
index 0000000..21a2f8f
--- /dev/null
+++ b/sdks/python/apache_beam/io/gcp/experimental/spannerio.py
@@ -0,0 +1,583 @@
+#
+# 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.
+#
+
+"""Google Cloud Spanner IO
+
+Experimental; no backwards-compatibility guarantees.
+
+This is an experimental module for reading and writing data from Google Cloud
+Spanner. Visit: https://cloud.google.com/spanner for more details.
+
+To read from Cloud Spanner apply ReadFromSpanner transformation. It will
+return a PCollection, where each element represents an individual row returned
+from the read operation. Both Query and Read APIs are supported.
+
+ReadFromSpanner relies on the ReadOperation objects which is exposed by the
+SpannerIO API. ReadOperation holds the immutable data which is responsible to
+execute batch and naive reads on Cloud Spanner. This is done for more
+convenient programming.
+
+ReadFromSpanner reads from Cloud Spanner by providing either an 'sql' param
+in the constructor or 'table' name with 'columns' as list. For example:::
+
+  records = (pipeline
+            | ReadFromSpanner(PROJECT_ID, INSTANCE_ID, DB_NAME,
+            sql='Select * from users'))
+
+  records = (pipeline
+            | ReadFromSpanner(PROJECT_ID, INSTANCE_ID, DB_NAME,
+            table='users', columns=['id', 'name', 'email']))
+
+You can also perform multiple reads by providing a list of ReadOperations
+to the ReadFromSpanner transform constructor. ReadOperation exposes two static
+methods. Use 'query' to perform sql based reads, 'table' to perform read from
+table name. For example:::
+
+  read_operations = [
+                      ReadOperation.table(table='customers', columns=['name',
+                      'email']),
+                      ReadOperation.table(table='vendors', columns=['name',
+                      'email']),
+                    ]
+  all_users = pipeline | ReadFromSpanner(PROJECT_ID, INSTANCE_ID, DB_NAME,
+        read_operations=read_operations)
+
+  ...OR...
+
+  read_operations = [
+                      ReadOperation.query(sql='Select name, email from
+                      customers'),
+                      ReadOperation.query(
+                        sql='Select * from users where id <= @user_id',
+                        params={'user_id': 100},
+                        params_type={'user_id': param_types.INT64}
+                      ),
+                    ]
+  # `params_types` are instance of `google.cloud.spanner.param_types`
+  all_users = pipeline | ReadFromSpanner(PROJECT_ID, INSTANCE_ID, DB_NAME,
+        read_operations=read_operations)
+
+For more information, please review the docs on class ReadOperation.
+
+User can also able to provide the ReadOperation in form of PCollection via
+pipeline. For example:::
+
+  users = (pipeline
+           | beam.Create([ReadOperation...])
+           | ReadFromSpanner(PROJECT_ID, INSTANCE_ID, DB_NAME))
+
+User may also create cloud spanner transaction from the transform called
+`create_transaction` which is available in the SpannerIO API.
+
+The transform is guaranteed to be executed on a consistent snapshot of data,
+utilizing the power of read only transactions. Staleness of data can be
+controlled by providing the `read_timestamp` or `exact_staleness` param values
+in the constructor.
+
+This transform requires root of the pipeline (PBegin) and returns PTransform
+which is passed later to the `ReadFromSpanner` constructor. `ReadFromSpanner`
+pass this transaction PTransform as a singleton side input to the
+`_NaiveSpannerReadDoFn` containing 'session_id' and 'transaction_id'.
+For example:::
+
+  transaction = (pipeline | create_transaction(TEST_PROJECT_ID,
+                                              TEST_INSTANCE_ID,
+                                              DB_NAME))
+
+  users = pipeline | ReadFromSpanner(PROJECT_ID, INSTANCE_ID, DB_NAME,
+        sql='Select * from users', transaction=transaction)
+
+  tweets = pipeline | ReadFromSpanner(PROJECT_ID, INSTANCE_ID, DB_NAME,
+        sql='Select * from tweets', transaction=transaction)
+
+For further details of this transform, please review the docs on the
+:meth:`create_transaction` method available in the SpannerIO API.
+
+ReadFromSpanner takes this transform in the constructor and pass this to the
+read pipeline as the singleton side input.
+"""
+from __future__ import absolute_import
+
+import typing
+from collections import namedtuple
+
+from apache_beam import Create
+from apache_beam import DoFn
+from apache_beam import ParDo
+from apache_beam import Reshuffle
+from apache_beam.pvalue import AsSingleton
+from apache_beam.pvalue import PBegin
+from apache_beam.transforms import PTransform
+from apache_beam.transforms import ptransform_fn
+from apache_beam.transforms.display import DisplayDataItem
+from apache_beam.typehints import with_input_types
+from apache_beam.typehints import with_output_types
+from apache_beam.utils.annotations import experimental
+
+try:
+  from google.cloud.spanner import Client
+  from google.cloud.spanner import KeySet
+  from google.cloud.spanner_v1.database import BatchSnapshot
+except ImportError:
+  Client = None
+  KeySet = None
+  BatchSnapshot = None
+
+__all__ = ['create_transaction', 'ReadFromSpanner', 'ReadOperation']
+
+
+class _SPANNER_TRANSACTION(namedtuple("SPANNER_TRANSACTION", ["transaction"])):
+  """
+  Holds the spanner transaction details.
+  """
+
+  __slots__ = ()
+
+
+class ReadOperation(namedtuple("ReadOperation", ["is_sql", "is_table",
+                                                 "read_operation", "kwargs"])):
+  """
+  Encapsulates a spanner read operation.
+  """
+
+  __slots__ = ()
+
+  @classmethod
+  def query(cls, sql, params=None, param_types=None):
+    """
+    A convenient method to construct ReadOperation from sql query.
+
+    Args:
+      sql: SQL query statement
+      params: (optional) values for parameter replacement. Keys must match the
+        names used in sql
+      param_types: (optional) maps explicit types for one or more param values;
+        required if parameters are passed.
+    """
+
+    if params:
+      assert param_types is not None
+
+    return cls(
+        is_sql=True,
+        is_table=False,
+        read_operation="process_query_batch",
+        kwargs={'sql': sql, 'params': params, 'param_types': param_types}
+    )
+
+  @classmethod
+  def table(cls, table, columns, index="", keyset=None):
+    """
+    A convenient method to construct ReadOperation from table.
+
+    Args:
+      table: name of the table from which to fetch data.
+      columns: names of columns to be retrieved.
+      index: (optional) name of index to use, rather than the table's primary
+        key.
+      keyset: (optional) `KeySet` keys / ranges identifying rows to be
+        retrieved.
+    """
+    keyset = keyset or KeySet(all_=True)
+    if not isinstance(keyset, KeySet):
+      raise ValueError("keyset must be an instance of class "
+                       "google.cloud.spanner.KeySet")
+    return cls(
+        is_sql=False,
+        is_table=True,
+        read_operation="process_read_batch",
+        kwargs={'table': table, 'columns': columns, 'index': index,
+                'keyset': keyset}
+    )
+
+
+class _BeamSpannerConfiguration(namedtuple(
+    "_BeamSpannerConfiguration", ["project", "instance", "database",
+                                  "credentials", "pool",
+                                  "snapshot_read_timestamp",
+                                  "snapshot_exact_staleness"])):
+  """
+  A namedtuple holds the immutable data of the connection string to the cloud
+  spanner.
+  """
+
+  @property
+  def snapshot_options(self):
+    snapshot_options = {}
+    if self.snapshot_exact_staleness:
+      snapshot_options['exact_staleness'] = self.snapshot_exact_staleness
+    if self.snapshot_read_timestamp:
+      snapshot_options['read_timestamp'] = self.snapshot_read_timestamp
+    return snapshot_options
+
+@with_input_types(ReadOperation, typing.Dict[typing.Any, typing.Any])
+@with_output_types(typing.List[typing.Any])
+class _NaiveSpannerReadDoFn(DoFn):
+
+  def __init__(self, spanner_configuration):
+    """
+    A naive version of Spanner read which uses the transaction API of the
+    cloud spanner.
+    https://googleapis.dev/python/spanner/latest/transaction-api.html
+    In Naive reads, this transform performs single reads, where as the
+    Batch reads use the spanner partitioning query to create batches.
+
+    Args:
+      spanner_configuration: (_BeamSpannerConfiguration) Connection details to
+        connect with cloud spanner.
+    """
+    self._spanner_configuration = spanner_configuration
+    self._snapshot = None
+    self._session = None
+
+  def _get_session(self):
+    if self._session is None:
+      session = self._session = self._database.session()
+      session.create()
+    return self._session
+
+  def _close_session(self):
+    if self._session is not None:
+      self._session.delete()
+
+  def setup(self):
+    # setting up client to connect with cloud spanner
+    spanner_client = Client(self._spanner_configuration.project)
+    instance = spanner_client.instance(self._spanner_configuration.instance)
+    self._database = instance.database(self._spanner_configuration.database,
+                                       pool=self._spanner_configuration.pool)
+
+  def process(self, element, spanner_transaction):
+    # `spanner_transaction` should be the instance of the _SPANNER_TRANSACTION
+    # object.
+    if not isinstance(spanner_transaction, _SPANNER_TRANSACTION):
+      raise ValueError("Invalid transaction object: %s. It should be instance "
+                       "of SPANNER_TRANSACTION object created by "
+                       "spannerio.create_transaction transform."
+                       % type(spanner_transaction))
+
+    transaction_info = spanner_transaction.transaction
+
+    # We used batch snapshot to reuse the same transaction passed through the
+    # side input
+    self._snapshot = BatchSnapshot.from_dict(self._database, transaction_info)
+
+    # getting the transaction from the snapshot's session to run read operation.
+    # with self._snapshot.session().transaction() as transaction:
+    with self._get_session().transaction() as transaction:
+      if element.is_sql is True:
+        transaction_read = transaction.execute_sql
+      elif element.is_table is True:
+        transaction_read = transaction.read
+      else:
+        raise ValueError("ReadOperation is improperly configure: %s" % str(
+            element))
+
+      for row in transaction_read(**element.kwargs):
+        yield row
+
+
+@with_input_types(ReadOperation)
+@with_output_types(typing.Dict[typing.Any, typing.Any])
+class _CreateReadPartitions(DoFn):
+  """
+  A DoFn to create partitions. Uses the Partitioning API (PartitionRead /
+  PartitionQuery) request to start a partitioned query operation. Returns a
+  list of batch information needed to perform the actual queries.
+
+  If the element is the instance of :class:`ReadOperation` is to perform sql
+  query, `PartitionQuery` API is used the create partitions and returns mappings
+  of information used perform actual partitioned reads via
+  :meth:`process_query_batch`.
+
+  If the element is the instance of :class:`ReadOperation` is to perform read
+  from table, `PartitionRead` API is used the create partitions and returns
+  mappings of information used perform actual partitioned reads via
+  :meth:`process_read_batch`.
+  """
+
+  def __init__(self, spanner_configuration):
+    self._spanner_configuration = spanner_configuration
+
+  def setup(self):
+    spanner_client = Client(project=self._spanner_configuration.project,
+                            credentials=self._spanner_configuration.credentials)
+    instance = spanner_client.instance(self._spanner_configuration.instance)
+    self._database = instance.database(self._spanner_configuration.database,
+                                       pool=self._spanner_configuration.pool)
+    self._snapshot = self._database.batch_snapshot(**self._spanner_configuration
+                                                   .snapshot_options)
+    self._snapshot_dict = self._snapshot.to_dict()
+
+  def process(self, element):
+    if element.is_sql is True:
+      partitioning_action = self._snapshot.generate_query_batches
+    elif element.is_table is True:
+      partitioning_action = self._snapshot.generate_read_batches
+    else:
+      raise ValueError("ReadOperation is improperly configure: %s" % str(
+          element))
+
+    for p in partitioning_action(**element.kwargs):
+      yield {"is_sql": element.is_sql, "is_table": element.is_table,
+             "read_operation": element.read_operation, "partitions": p,
+             "transaction_info": self._snapshot_dict}
+
+
+@with_input_types(int)
+@with_output_types(typing.Dict[typing.Any, typing.Any])
+class _CreateTransactionFn(DoFn):
+  """
+  A DoFn to create the transaction of cloud spanner.
+  It connects to the database and and returns the transaction_id and session_id
+  by using the batch_snapshot.to_dict() method available in the google cloud
+  spanner sdk.
+
+  https://googleapis.dev/python/spanner/latest/database-api.html?highlight=
+  batch_snapshot#google.cloud.spanner_v1.database.BatchSnapshot.to_dict
+  """
+
+  def __init__(self, project_id, instance_id, database_id, credentials,
+               pool, read_timestamp,
+               exact_staleness):
+    self._project_id = project_id
+    self._instance_id = instance_id
+    self._database_id = database_id
+    self._credentials = credentials
+    self._pool = pool
+
+    self._snapshot_options = {}
+    if read_timestamp:
+      self._snapshot_options['read_timestamp'] = read_timestamp
+    if exact_staleness:
+      self._snapshot_options['exact_staleness'] = exact_staleness
+    self._snapshot = None
+
+  def setup(self):
+    self._spanner_client = Client(project=self._project_id,
+                                  credentials=self._credentials)
+    self._instance = self._spanner_client.instance(self._instance_id)
+    self._database = self._instance.database(self._database_id, pool=self._pool)
+
+  def process(self, element, *args, **kwargs):
+    self._snapshot = self._database.batch_snapshot(**self._snapshot_options)
+    return [_SPANNER_TRANSACTION(self._snapshot.to_dict())]
+
+
+@ptransform_fn
+def create_transaction(pbegin, project_id, instance_id, database_id,
+                       credentials=None, pool=None, read_timestamp=None,
+                       exact_staleness=None):
+  """
+  A PTransform method to create a batch transaction.
+
+  Args:
+    pbegin: Root of the pipeline
+    project_id: Cloud spanner project id. Be sure to use the Project ID,
+      not the Project Number.
+    instance_id: Cloud spanner instance id.
+    database_id: Cloud spanner database id.
+    credentials: (optional) The authorization credentials to attach to requests.
+      These credentials identify this application to the service.
+      If none are specified, the client will attempt to ascertain
+      the credentials from the environment.
+    pool: (optional) session pool to be used by database. If not passed,
+      Spanner Cloud SDK uses the BurstyPool by default.
+      `google.cloud.spanner.BurstyPool`. Ref:
+      https://googleapis.dev/python/spanner/latest/database-api.html?#google.
+      cloud.spanner_v1.database.Database
+    read_timestamp: (optional) An instance of the `datetime.datetime` object to
+      execute all reads at the given timestamp.
+    exact_staleness: (optional) An instance of the `datetime.timedelta`
+      object. These timestamp bounds execute reads at a user-specified
+      timestamp.
+  """
+
+  assert isinstance(pbegin, PBegin)
+
+  return (pbegin | Create([1]) | ParDo(_CreateTransactionFn(
+      project_id, instance_id, database_id, credentials,
+      pool, read_timestamp,
+      exact_staleness)))
+
+@with_input_types(typing.Dict[typing.Any, typing.Any])
+@with_output_types(typing.List[typing.Any])
+class _ReadFromPartitionFn(DoFn):
+  """
+  A DoFn to perform reads from the partition.
+  """
+
+  def __init__(self, spanner_configuration):
+    self._spanner_configuration = spanner_configuration
+
+  def setup(self):
+    spanner_client = Client(self._spanner_configuration.project)
+    instance = spanner_client.instance(self._spanner_configuration.instance)
+    self._database = instance.database(self._spanner_configuration.database,
+                                       pool=self._spanner_configuration.pool)
+    self._snapshot = self._database.batch_snapshot(**self._spanner_configuration
+                                                   .snapshot_options)
+
+  def process(self, element):
+    self._snapshot = BatchSnapshot.from_dict(
+        self._database,
+        element['transaction_info']
+    )
+
+    if element['is_sql'] is True:
+      read_action = self._snapshot.process_query_batch
+    elif element['is_table'] is True:
+      read_action = self._snapshot.process_read_batch
+    else:
+      raise ValueError("ReadOperation is improperly configure: %s" % str(
+          element))
+
+    for row in read_action(element['partitions']):
+      yield row
+
+  def teardown(self):
+    if self._snapshot:
+      self._snapshot.close()
+
+
+@experimental(extra_message="No backwards-compatibility guarantees.")
+class ReadFromSpanner(PTransform):
+  """
+  A PTransform to perform reads from cloud spanner.
+  ReadFromSpanner uses BatchAPI to perform all read operations.
+  """
+
+  def __init__(self, project_id, instance_id, database_id, pool=None,
+               read_timestamp=None, exact_staleness=None, credentials=None,
+               sql=None, params=None, param_types=None,  # with_query
+               table=None, columns=None, index="", keyset=None,  # with_table
+               read_operations=None,  # for read all
+               transaction=None
+              ):
+    """
+    A PTransform that uses Spanner Batch API to perform reads.
+
+    Args:
+      project_id: Cloud spanner project id. Be sure to use the Project ID,
+        not the Project Number.
+      instance_id: Cloud spanner instance id.
+      database_id: Cloud spanner database id.
+      pool: (optional) session pool to be used by database. If not passed,
+        Spanner Cloud SDK uses the BurstyPool by default.
+        `google.cloud.spanner.BurstyPool`. Ref:
+        https://googleapis.dev/python/spanner/latest/database-api.html?#google.
+        cloud.spanner_v1.database.Database
+      read_timestamp: (optional) An instance of the `datetime.datetime` object
+        to execute all reads at the given timestamp. By default, set to `None`.
+      exact_staleness: (optional) An instance of the `datetime.timedelta`
+        object. These timestamp bounds execute reads at a user-specified
+        timestamp. By default, set to `None`.
+      credentials: (optional) The authorization credentials to attach to
+        requests. These credentials identify this application to the service.
+        If none are specified, the client will attempt to ascertain
+        the credentials from the environment. By default, set to `None`.
+      sql: (optional) SQL query statement.
+      params: (optional) Values for parameter replacement. Keys must match the
+        names used in sql. By default, set to `None`.
+      param_types: (optional) maps explicit types for one or more param values;
+        required if params are passed. By default, set to `None`.
+      table: (optional) Name of the table from which to fetch data. By
+        default, set to `None`.
+      columns: (optional) List of names of columns to be retrieved; required if
+        the table is passed. By default, set to `None`.
+      index: (optional) name of index to use, rather than the table's primary
+        key. By default, set to `None`.
+      keyset: (optional) keys / ranges identifying rows to be retrieved. By
+        default, set to `None`.
+      read_operations: (optional) List of the objects of :class:`ReadOperation`
+        to perform read all. By default, set to `None`.
+      transaction: (optional) PTransform of the :meth:`create_transaction` to
+        perform naive read on cloud spanner. By default, set to `None`.
+    """
+    self._configuration = _BeamSpannerConfiguration(
+        project=project_id, instance=instance_id, database=database_id,
+        credentials=credentials, pool=pool,
+        snapshot_read_timestamp=read_timestamp,
+        snapshot_exact_staleness=exact_staleness
+    )
+
+    self._read_operations = read_operations
+    self._transaction = transaction
+
+    if self._read_operations is None:
+      if table is not None:
+        if columns is None:
+          raise ValueError("Columns are required with the table name.")
+        self._read_operations = [ReadOperation.table(
+            table=table, columns=columns, index=index, keyset=keyset)]
+      elif sql is not None:
+        self._read_operations = [ReadOperation.query(
+            sql=sql, params=params, param_types=param_types)]
+
+  def expand(self, pbegin):
+    if self._read_operations is not None and isinstance(pbegin,
+                                                        PBegin):
+      pcoll = pbegin.pipeline | Create(self._read_operations)
+    elif not isinstance(pbegin, PBegin):
+      if self._read_operations is not None:
+        raise ValueError("Read operation in the constructor only works with "
+                         "the root of the pipeline.")
+      pcoll = pbegin
+    else:
+      raise ValueError("Spanner required read operation, sql or table "
+                       "with columns.")
+
+    if self._transaction is None:
+      # reading as batch read using the spanner partitioning query to create
+      # batches.
+      p = (pcoll
+           | 'Generate Partitions' >> ParDo(_CreateReadPartitions(
+               spanner_configuration=self._configuration))
+           | 'Reshuffle' >> Reshuffle()
+           | 'Read From Partitions' >> ParDo(_ReadFromPartitionFn(
+               spanner_configuration=self._configuration)))
+    else:
+      # reading as naive read, in which we don't make batches and execute the
+      # queries as a single read.
+      p = (pcoll
+           | 'Reshuffle' >> Reshuffle().with_input_types(ReadOperation)
+           | 'Perform Read' >> ParDo(
+               _NaiveSpannerReadDoFn(spanner_configuration=self._configuration),
+               AsSingleton(self._transaction)))
+    return p
+
+  def display_data(self):
+    res = dict()
+    sql = []
+    table = []
+    if self._read_operations is not None:
+      for ro in self._read_operations:
+        if ro.is_sql is True:
+          sql.append(ro.kwargs)
+        elif ro.is_table is True:
+          table.append(ro.kwargs)
+
+      if sql:
+        res['sql'] = DisplayDataItem(str(sql), label='Sql')
+      if table:
+        res['table'] = DisplayDataItem(str(table), label='Table')
+
+    if self._transaction:
+      res['transaction'] = DisplayDataItem(str(self._transaction),
+                                           label='transaction')
+
+    return res
diff --git a/sdks/python/apache_beam/io/gcp/experimental/spannerio_test.py b/sdks/python/apache_beam/io/gcp/experimental/spannerio_test.py
new file mode 100644
index 0000000..be838f4
--- /dev/null
+++ b/sdks/python/apache_beam/io/gcp/experimental/spannerio_test.py
@@ -0,0 +1,338 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+from __future__ import absolute_import
+
+import datetime
+import logging
+import random
+import string
+import unittest
+
+import mock
+
+import apache_beam as beam
+from apache_beam.testing.test_pipeline import TestPipeline
+from apache_beam.testing.util import assert_that
+from apache_beam.testing.util import equal_to
+
+# Protect against environments where spanner library is not available.
+# pylint: disable=wrong-import-order, wrong-import-position, ungrouped-imports
+try:
+  from google.cloud import spanner
+  from apache_beam.io.gcp.experimental.spannerio import (create_transaction,
+                                                         ReadOperation,
+                                                         ReadFromSpanner) # pylint: disable=unused-import
+  # disable=unused-import
+except ImportError:
+  spanner = None
+# pylint: enable=wrong-import-order, wrong-import-position, ungrouped-imports
+
+
+MAX_DB_NAME_LENGTH = 30
+TEST_PROJECT_ID = 'apache-beam-testing'
+TEST_INSTANCE_ID = 'beam-test'
+TEST_DATABASE_PREFIX = 'spanner-testdb-'
+FAKE_TRANSACTION_INFO = {"session_id": "qwerty", "transaction_id": "qwerty"}
+FAKE_ROWS = [[1, 'Alice'], [2, 'Bob'], [3, 'Carl'], [4, 'Dan'], [5, 'Evan'],
+             [6, 'Floyd']]
+
+
+def _generate_database_name():
+  mask = string.ascii_lowercase + string.digits
+  length = MAX_DB_NAME_LENGTH - 1 - len(TEST_DATABASE_PREFIX)
+  return TEST_DATABASE_PREFIX + ''.join(random.choice(mask) for i in range(
+      length))
+
+
+def _generate_test_data():
+  mask = string.ascii_lowercase + string.digits
+  length = 100
+  return [('users', ['Key', 'Value'], [(x, ''.join(
+      random.choice(mask) for _ in range(length))) for x in range(1, 5)])]
+
+
+@unittest.skipIf(spanner is None, 'GCP dependencies are not installed.')
+@mock.patch('apache_beam.io.gcp.experimental.spannerio.Client')
+@mock.patch('apache_beam.io.gcp.experimental.spannerio.BatchSnapshot')
+class SpannerReadTest(unittest.TestCase):
+
+  def test_read_with_query_batch(self, mock_batch_snapshot_class,
+                                 mock_client_class):
+    mock_snapshot = mock.MagicMock()
+
+    mock_snapshot.generate_query_batches.return_value = [
+        {'query': {'sql': 'SELECT * FROM users'},
+         'partition': 'test_partition'} for _ in range(3)]
+    mock_snapshot.process_query_batch.side_effect = [
+        FAKE_ROWS[0:2], FAKE_ROWS[2:4], FAKE_ROWS[4:]]
+
+    ro = [ReadOperation.query("Select * from users")]
+    pipeline = TestPipeline()
+
+    read = (pipeline
+            | 'read' >> ReadFromSpanner(TEST_PROJECT_ID, TEST_INSTANCE_ID,
+                                        _generate_database_name(),
+                                        sql="SELECT * FROM users"))
+
+    readall = (pipeline
+               | 'read all' >> ReadFromSpanner(TEST_PROJECT_ID,
+                                               TEST_INSTANCE_ID,
+                                               _generate_database_name(),
+                                               read_operations=ro))
+
+    readpipeline = (pipeline
+                    | 'create reads' >> beam.Create(ro)
+                    | 'reads' >> ReadFromSpanner(TEST_PROJECT_ID,
+                                                 TEST_INSTANCE_ID,
+                                                 _generate_database_name()))
+
+    pipeline.run()
+    assert_that(read, equal_to(FAKE_ROWS), label='checkRead')
+    assert_that(readall, equal_to(FAKE_ROWS), label='checkReadAll')
+    assert_that(readpipeline, equal_to(FAKE_ROWS), label='checkReadPipeline')
+
+  def test_read_with_table_batch(self, mock_batch_snapshot_class,
+                                 mock_client_class):
+    mock_snapshot = mock.MagicMock()
+    mock_snapshot.generate_read_batches.return_value = [{
+        'read': {'table': 'users', 'keyset': {'all': True},
+                 'columns': ['Key', 'Value'], 'index': ''},
+        'partition': 'test_partition'} for _ in range(3)]
+    mock_snapshot.process_read_batch.side_effect = [
+        FAKE_ROWS[0:2], FAKE_ROWS[2:4], FAKE_ROWS[4:]]
+
+    ro = [ReadOperation.table("users", ["Key", "Value"])]
+    pipeline = TestPipeline()
+
+    read = (pipeline
+            | 'read' >> ReadFromSpanner(TEST_PROJECT_ID, TEST_INSTANCE_ID,
+                                        _generate_database_name(),
+                                        table="users",
+                                        columns=["Key", "Value"]))
+
+    readall = (pipeline
+               | 'read all' >> ReadFromSpanner(TEST_PROJECT_ID,
+                                               TEST_INSTANCE_ID,
+                                               _generate_database_name(),
+                                               read_operations=ro))
+
+    readpipeline = (pipeline
+                    | 'create reads' >> beam.Create(ro)
+                    | 'reads' >> ReadFromSpanner(TEST_PROJECT_ID,
+                                                 TEST_INSTANCE_ID,
+                                                 _generate_database_name()))
+
+    pipeline.run()
+    assert_that(read, equal_to(FAKE_ROWS), label='checkRead')
+    assert_that(readall, equal_to(FAKE_ROWS), label='checkReadAll')
+    assert_that(readpipeline, equal_to(FAKE_ROWS), label='checkReadPipeline')
+
+    with self.assertRaises(ValueError):
+      # Test the exception raised when user passes the read operations in the
+      # constructor and also in the pipeline.
+      _ = (pipeline | 'reads error' >> ReadFromSpanner(
+          project_id=TEST_PROJECT_ID,
+          instance_id=TEST_INSTANCE_ID,
+          database_id=_generate_database_name(),
+          table="users"
+      ))
+      pipeline.run()
+
+  def test_read_with_index(self, mock_batch_snapshot_class,
+                           mock_client_class):
+    mock_snapshot = mock.MagicMock()
+    mock_snapshot.generate_read_batches.return_value = [{
+        'read': {'table': 'users', 'keyset': {'all': True},
+                 'columns': ['Key', 'Value'], 'index': ''},
+        'partition': 'test_partition'} for _ in range(3)]
+    mock_snapshot.process_read_batch.side_effect = [
+        FAKE_ROWS[0:2], FAKE_ROWS[2:4], FAKE_ROWS[4:]]
+    ro = [ReadOperation.table("users", ["Key", "Value"], index="Key")]
+    pipeline = TestPipeline()
+    read = (pipeline
+            | 'read' >> ReadFromSpanner(TEST_PROJECT_ID, TEST_INSTANCE_ID,
+                                        _generate_database_name(),
+                                        table="users",
+                                        columns=["Key", "Value"]))
+    readall = (pipeline
+               | 'read all' >> ReadFromSpanner(TEST_PROJECT_ID,
+                                               TEST_INSTANCE_ID,
+                                               _generate_database_name(),
+                                               read_operations=ro))
+    readpipeline = (pipeline
+                    | 'create reads' >> beam.Create(ro)
+                    | 'reads' >> ReadFromSpanner(TEST_PROJECT_ID,
+                                                 TEST_INSTANCE_ID,
+                                                 _generate_database_name()))
+    pipeline.run()
+    assert_that(read, equal_to(FAKE_ROWS), label='checkRead')
+    assert_that(readall, equal_to(FAKE_ROWS), label='checkReadAll')
+    assert_that(readpipeline, equal_to(FAKE_ROWS), label='checkReadPipeline')
+    with self.assertRaises(ValueError):
+      # Test the exception raised when user passes the read operations in the
+      # constructor and also in the pipeline.
+      _ = (pipeline | 'reads error' >> ReadFromSpanner(
+          project_id=TEST_PROJECT_ID,
+          instance_id=TEST_INSTANCE_ID,
+          database_id=_generate_database_name(),
+          table="users"
+      ))
+      pipeline.run()
+
+  def test_read_with_transaction(self, mock_batch_snapshot_class,
+                                 mock_client_class):
+    mock_client = mock.MagicMock()
+    mock_instance = mock.MagicMock()
+    mock_database = mock.MagicMock()
+    mock_snapshot = mock.MagicMock()
+
+    mock_client_class.return_value = mock_client
+    mock_client.instance.return_value = mock_instance
+    mock_instance.database.return_value = mock_database
+    mock_database.batch_snapshot.return_value = mock_snapshot
+    mock_batch_snapshot_class.return_value = mock_snapshot
+    mock_batch_snapshot_class.from_dict.return_value = mock_snapshot
+    mock_snapshot.to_dict.return_value = FAKE_TRANSACTION_INFO
+
+    mock_session = mock.MagicMock()
+    mock_transaction_ctx = mock.MagicMock()
+    mock_transaction = mock.MagicMock()
+
+    mock_snapshot._get_session.return_value = mock_session
+    mock_session.transaction.return_value = mock_transaction
+    mock_transaction.__enter__.return_value = mock_transaction_ctx
+    mock_transaction_ctx.execute_sql.return_value = FAKE_ROWS
+
+    ro = [ReadOperation.query("Select * from users")]
+    p = TestPipeline()
+
+    transaction = (p | create_transaction(
+        project_id=TEST_PROJECT_ID, instance_id=TEST_INSTANCE_ID,
+        database_id=_generate_database_name(),
+        exact_staleness=datetime.timedelta(seconds=10)))
+
+    read_query = (p | 'with query' >> ReadFromSpanner(
+        project_id=TEST_PROJECT_ID,
+        instance_id=TEST_INSTANCE_ID,
+        database_id=_generate_database_name(),
+        transaction=transaction,
+        sql="Select * from users"
+    ))
+
+    read_table = (p | 'with table' >> ReadFromSpanner(
+        project_id=TEST_PROJECT_ID,
+        instance_id=TEST_INSTANCE_ID,
+        database_id=_generate_database_name(),
+        transaction=transaction,
+        table="users",
+        columns=["Key", "Value"]
+    ))
+
+    read_indexed_table = (p | 'with index' >> ReadFromSpanner(
+        project_id=TEST_PROJECT_ID,
+        instance_id=TEST_INSTANCE_ID,
+        database_id=_generate_database_name(),
+        transaction=transaction,
+        table="users",
+        index="Key",
+        columns=["Key", "Value"]
+    ))
+
+    read = (p | 'read all' >> ReadFromSpanner(TEST_PROJECT_ID,
+                                              TEST_INSTANCE_ID,
+                                              _generate_database_name(),
+                                              transaction=transaction,
+                                              read_operations=ro))
+
+    read_pipeline = (p
+                     | 'create read operations' >> beam.Create(ro)
+                     | 'reads' >> ReadFromSpanner(TEST_PROJECT_ID,
+                                                  TEST_INSTANCE_ID,
+                                                  _generate_database_name(),
+                                                  transaction=transaction))
+
+    p.run()
+
+    assert_that(read_query, equal_to(FAKE_ROWS), label='checkQuery')
+    assert_that(read_table, equal_to(FAKE_ROWS), label='checkTable')
+    assert_that(read_indexed_table, equal_to(FAKE_ROWS),
+                label='checkTableIndex')
+    assert_that(read, equal_to(FAKE_ROWS), label='checkReadAll')
+    assert_that(read_pipeline, equal_to(FAKE_ROWS), label='checkReadPipeline')
+
+    with self.assertRaises(ValueError):
+      # Test the exception raised when user passes the read operations in the
+      # constructor and also in the pipeline.
+      _ = (p
+           | 'create read operations2' >> beam.Create(ro)
+           | 'reads with error' >> ReadFromSpanner(TEST_PROJECT_ID,
+                                                   TEST_INSTANCE_ID,
+                                                   _generate_database_name(),
+                                                   transaction=transaction,
+                                                   read_operations=ro))
+      p.run()
+
+  def test_invalid_transaction(self, mock_batch_snapshot_class,
+                               mock_client_class):
+    with self.assertRaises(ValueError):
+      p = TestPipeline()
+      transaction = (p | beam.Create([{"invalid": "transaction"}]))
+      _ = (p | 'with query' >> ReadFromSpanner(
+          project_id=TEST_PROJECT_ID,
+          instance_id=TEST_INSTANCE_ID,
+          database_id=_generate_database_name(),
+          transaction=transaction,
+          sql="Select * from users"
+      ))
+      p.run()
+
+
+  def test_display_data(self, *args):
+    dd_sql = ReadFromSpanner(
+        project_id=TEST_PROJECT_ID,
+        instance_id=TEST_INSTANCE_ID,
+        database_id=_generate_database_name(),
+        sql="Select * from users"
+    ).display_data()
+
+    dd_table = ReadFromSpanner(
+        project_id=TEST_PROJECT_ID,
+        instance_id=TEST_INSTANCE_ID,
+        database_id=_generate_database_name(),
+        table="users",
+        columns=['id', 'name']
+    ).display_data()
+
+    dd_transaction = ReadFromSpanner(
+        project_id=TEST_PROJECT_ID,
+        instance_id=TEST_INSTANCE_ID,
+        database_id=_generate_database_name(),
+        table="users",
+        columns=['id', 'name'],
+        transaction={"transaction_id": "test123", "session_id": "test456"}
+    ).display_data()
+
+    self.assertTrue("sql" in dd_sql)
+    self.assertTrue("table" in dd_table)
+    self.assertTrue("table" in dd_transaction)
+    self.assertTrue("transaction" in dd_transaction)
+
+
+if __name__ == '__main__':
+  logging.getLogger().setLevel(logging.INFO)
+  unittest.main()
diff --git a/sdks/python/apache_beam/io/gcp/pubsub_test.py b/sdks/python/apache_beam/io/gcp/pubsub_test.py
index 8912a3f..a4b9e62 100644
--- a/sdks/python/apache_beam/io/gcp/pubsub_test.py
+++ b/sdks/python/apache_beam/io/gcp/pubsub_test.py
@@ -354,12 +354,11 @@
 
     options = PipelineOptions([])
     options.view_as(StandardOptions).streaming = True
-    p = TestPipeline(options=options)
-    pcoll = (p
-             | ReadFromPubSub('projects/fakeprj/topics/a_topic',
-                              None, None, with_attributes=True))
-    assert_that(pcoll, equal_to(expected_elements), reify_windows=True)
-    p.run()
+    with TestPipeline(options=options) as p:
+      pcoll = (p
+               | ReadFromPubSub('projects/fakeprj/topics/a_topic',
+                                None, None, with_attributes=True))
+      assert_that(pcoll, equal_to(expected_elements), reify_windows=True)
     mock_pubsub.return_value.acknowledge.assert_has_calls([
         mock.call(mock.ANY, [ack_id])])
 
@@ -378,12 +377,11 @@
 
     options = PipelineOptions([])
     options.view_as(StandardOptions).streaming = True
-    p = TestPipeline(options=options)
-    pcoll = (p
-             | ReadStringsFromPubSub('projects/fakeprj/topics/a_topic',
-                                     None, None))
-    assert_that(pcoll, equal_to(expected_elements))
-    p.run()
+    with TestPipeline(options=options) as p:
+      pcoll = (p
+               | ReadStringsFromPubSub('projects/fakeprj/topics/a_topic',
+                                       None, None))
+      assert_that(pcoll, equal_to(expected_elements))
     mock_pubsub.return_value.acknowledge.assert_has_calls([
         mock.call(mock.ANY, [ack_id])])
 
@@ -400,11 +398,10 @@
 
     options = PipelineOptions([])
     options.view_as(StandardOptions).streaming = True
-    p = TestPipeline(options=options)
-    pcoll = (p
-             | ReadFromPubSub('projects/fakeprj/topics/a_topic', None, None))
-    assert_that(pcoll, equal_to(expected_elements))
-    p.run()
+    with TestPipeline(options=options) as p:
+      pcoll = (p
+               | ReadFromPubSub('projects/fakeprj/topics/a_topic', None, None))
+      assert_that(pcoll, equal_to(expected_elements))
     mock_pubsub.return_value.acknowledge.assert_has_calls([
         mock.call(mock.ANY, [ack_id])])
 
@@ -431,13 +428,12 @@
 
     options = PipelineOptions([])
     options.view_as(StandardOptions).streaming = True
-    p = TestPipeline(options=options)
-    pcoll = (p
-             | ReadFromPubSub(
-                 'projects/fakeprj/topics/a_topic', None, None,
-                 with_attributes=True, timestamp_attribute='time'))
-    assert_that(pcoll, equal_to(expected_elements), reify_windows=True)
-    p.run()
+    with TestPipeline(options=options) as p:
+      pcoll = (p
+               | ReadFromPubSub(
+                   'projects/fakeprj/topics/a_topic', None, None,
+                   with_attributes=True, timestamp_attribute='time'))
+      assert_that(pcoll, equal_to(expected_elements), reify_windows=True)
     mock_pubsub.return_value.acknowledge.assert_has_calls([
         mock.call(mock.ANY, [ack_id])])
 
@@ -464,13 +460,12 @@
 
     options = PipelineOptions([])
     options.view_as(StandardOptions).streaming = True
-    p = TestPipeline(options=options)
-    pcoll = (p
-             | ReadFromPubSub(
-                 'projects/fakeprj/topics/a_topic', None, None,
-                 with_attributes=True, timestamp_attribute='time'))
-    assert_that(pcoll, equal_to(expected_elements), reify_windows=True)
-    p.run()
+    with TestPipeline(options=options) as p:
+      pcoll = (p
+               | ReadFromPubSub(
+                   'projects/fakeprj/topics/a_topic', None, None,
+                   with_attributes=True, timestamp_attribute='time'))
+      assert_that(pcoll, equal_to(expected_elements), reify_windows=True)
     mock_pubsub.return_value.acknowledge.assert_has_calls([
         mock.call(mock.ANY, [ack_id])])
 
@@ -498,13 +493,12 @@
 
     options = PipelineOptions([])
     options.view_as(StandardOptions).streaming = True
-    p = TestPipeline(options=options)
-    pcoll = (p
-             | ReadFromPubSub(
-                 'projects/fakeprj/topics/a_topic', None, None,
-                 with_attributes=True, timestamp_attribute='nonexistent'))
-    assert_that(pcoll, equal_to(expected_elements), reify_windows=True)
-    p.run()
+    with TestPipeline(options=options) as p:
+      pcoll = (p
+               | ReadFromPubSub(
+                   'projects/fakeprj/topics/a_topic', None, None,
+                   with_attributes=True, timestamp_attribute='nonexistent'))
+      assert_that(pcoll, equal_to(expected_elements), reify_windows=True)
     mock_pubsub.return_value.acknowledge.assert_has_calls([
         mock.call(mock.ANY, [ack_id])])
 
@@ -541,11 +535,11 @@
     # id_label is unsupported in DirectRunner.
     options = PipelineOptions([])
     options.view_as(StandardOptions).streaming = True
-    p = TestPipeline(options=options)
-    _ = (p | ReadFromPubSub('projects/fakeprj/topics/a_topic', None, 'a_label'))
     with self.assertRaisesRegex(NotImplementedError,
                                 r'id_label is not supported'):
-      p.run()
+      with TestPipeline(options=options) as p:
+        _ = (p | ReadFromPubSub(
+            'projects/fakeprj/topics/a_topic', None, 'a_label'))
 
 
 @unittest.skipIf(pubsub is None, 'GCP dependencies are not installed')
@@ -558,12 +552,11 @@
 
     options = PipelineOptions([])
     options.view_as(StandardOptions).streaming = True
-    p = TestPipeline(options=options)
-    _ = (p
-         | Create(payloads)
-         | WriteToPubSub('projects/fakeprj/topics/a_topic',
-                         with_attributes=False))
-    p.run()
+    with TestPipeline(options=options) as p:
+      _ = (p
+           | Create(payloads)
+           | WriteToPubSub('projects/fakeprj/topics/a_topic',
+                           with_attributes=False))
     mock_pubsub.return_value.publish.assert_has_calls([
         mock.call(mock.ANY, data)])
 
@@ -573,11 +566,10 @@
 
     options = PipelineOptions([])
     options.view_as(StandardOptions).streaming = True
-    p = TestPipeline(options=options)
-    _ = (p
-         | Create(payloads)
-         | WriteStringsToPubSub('projects/fakeprj/topics/a_topic'))
-    p.run()
+    with TestPipeline(options=options) as p:
+      _ = (p
+           | Create(payloads)
+           | WriteStringsToPubSub('projects/fakeprj/topics/a_topic'))
     mock_pubsub.return_value.publish.assert_has_calls([
         mock.call(mock.ANY, data)])
 
@@ -588,12 +580,11 @@
 
     options = PipelineOptions([])
     options.view_as(StandardOptions).streaming = True
-    p = TestPipeline(options=options)
-    _ = (p
-         | Create(payloads)
-         | WriteToPubSub('projects/fakeprj/topics/a_topic',
-                         with_attributes=True))
-    p.run()
+    with TestPipeline(options=options) as p:
+      _ = (p
+           | Create(payloads)
+           | WriteToPubSub('projects/fakeprj/topics/a_topic',
+                           with_attributes=True))
     mock_pubsub.return_value.publish.assert_has_calls([
         mock.call(mock.ANY, data, **attributes)])
 
@@ -604,14 +595,13 @@
 
     options = PipelineOptions([])
     options.view_as(StandardOptions).streaming = True
-    p = TestPipeline(options=options)
-    _ = (p
-         | Create(payloads)
-         | WriteToPubSub('projects/fakeprj/topics/a_topic',
-                         with_attributes=True))
     with self.assertRaisesRegex(AttributeError,
                                 r'str.*has no attribute.*data'):
-      p.run()
+      with TestPipeline(options=options) as p:
+        _ = (p
+             | Create(payloads)
+             | WriteToPubSub('projects/fakeprj/topics/a_topic',
+                             with_attributes=True))
 
   def test_write_messages_unsupported_features(self, mock_pubsub):
     data = b'data'
@@ -620,24 +610,23 @@
 
     options = PipelineOptions([])
     options.view_as(StandardOptions).streaming = True
-    p = TestPipeline(options=options)
-    _ = (p
-         | Create(payloads)
-         | WriteToPubSub('projects/fakeprj/topics/a_topic',
-                         id_label='a_label'))
     with self.assertRaisesRegex(NotImplementedError,
                                 r'id_label is not supported'):
-      p.run()
+      with TestPipeline(options=options) as p:
+        _ = (p
+             | Create(payloads)
+             | WriteToPubSub('projects/fakeprj/topics/a_topic',
+                             id_label='a_label'))
+
     options = PipelineOptions([])
     options.view_as(StandardOptions).streaming = True
-    p = TestPipeline(options=options)
-    _ = (p
-         | Create(payloads)
-         | WriteToPubSub('projects/fakeprj/topics/a_topic',
-                         timestamp_attribute='timestamp'))
     with self.assertRaisesRegex(NotImplementedError,
                                 r'timestamp_attribute is not supported'):
-      p.run()
+      with TestPipeline(options=options) as p:
+        _ = (p
+             | Create(payloads)
+             | WriteToPubSub('projects/fakeprj/topics/a_topic',
+                             timestamp_attribute='timestamp'))
 
 
 if __name__ == '__main__':
diff --git a/sdks/python/apache_beam/io/hdfs_integration_test/Dockerfile b/sdks/python/apache_beam/io/hdfs_integration_test/Dockerfile
index 788b8d2..1c78181 100644
--- a/sdks/python/apache_beam/io/hdfs_integration_test/Dockerfile
+++ b/sdks/python/apache_beam/io/hdfs_integration_test/Dockerfile
@@ -24,22 +24,13 @@
 
 WORKDIR /app
 ENV HDFSCLI_CONFIG /app/sdks/python/apache_beam/io/hdfs_integration_test/hdfscli.cfg
-RUN pip install --no-cache-dir holdup gsutil
-RUN gsutil cp gs://dataflow-samples/shakespeare/kinglear.txt .
 
-# Install Beam and dependencies.
-ADD sdks/python /app/sdks/python
-ADD model /app/model
-RUN cd sdks/python && \
-    python setup.py sdist && \
-    pip install --no-cache-dir $(ls dist/apache-beam-*.tar.gz | tail -n1)[gcp]
+# Add Beam SDK sources.
+COPY sdks/python /app/sdks/python
+COPY model /app/model
+
+# This step should look like setupVirtualenv minus virtualenv creation.
+RUN pip install --no-cache-dir tox==3.11.1 -r sdks/python/build-requirements.txt
 
 # Run wordcount, and write results to HDFS.
-CMD holdup -t 45 http://namenode:50070 http://datanode:50075 && \
-    echo "Waiting for safe mode to end." && \
-    sleep 45 && \
-    hdfscli -v -v -v upload -f kinglear.txt / && \
-    python -m apache_beam.examples.wordcount \
-        --input hdfs://kinglear* \
-        --output hdfs://py-wordcount-integration \
-        --hdfs_host namenode --hdfs_port 50070 --hdfs_user root
+CMD cd sdks/python && tox -e hdfs_integration_test
diff --git a/sdks/python/apache_beam/io/parquetio_it_test.py b/sdks/python/apache_beam/io/parquetio_it_test.py
index cc85cd2..ecc1957 100644
--- a/sdks/python/apache_beam/io/parquetio_it_test.py
+++ b/sdks/python/apache_beam/io/parquetio_it_test.py
@@ -68,12 +68,10 @@
     file_prefix = "parquet_it_test"
     init_size = 10
     data_size = 20000
-    p = TestPipeline(is_integration_test=True)
-    pcol = self._generate_data(
-        p, file_prefix, init_size, data_size)
-    self._verify_data(pcol, init_size, data_size)
-    result = p.run()
-    result.wait_until_finish()
+    with TestPipeline(is_integration_test=True) as p:
+      pcol = self._generate_data(
+          p, file_prefix, init_size, data_size)
+      self._verify_data(pcol, init_size, data_size)
 
   @staticmethod
   def _sum_verifier(init_size, data_size, x):
diff --git a/sdks/python/apache_beam/io/sources_test.py b/sdks/python/apache_beam/io/sources_test.py
index e210a5b..fb34a98 100644
--- a/sdks/python/apache_beam/io/sources_test.py
+++ b/sdks/python/apache_beam/io/sources_test.py
@@ -125,11 +125,10 @@
 
   def test_run_direct(self):
     file_name = self._create_temp_file(b'aaaa\nbbbb\ncccc\ndddd')
-    pipeline = TestPipeline()
-    pcoll = pipeline | beam.io.Read(LineSource(file_name))
-    assert_that(pcoll, equal_to([b'aaaa', b'bbbb', b'cccc', b'dddd']))
+    with TestPipeline() as pipeline:
+      pcoll = pipeline | beam.io.Read(LineSource(file_name))
+      assert_that(pcoll, equal_to([b'aaaa', b'bbbb', b'cccc', b'dddd']))
 
-    pipeline.run()
 
 
 if __name__ == '__main__':
diff --git a/sdks/python/apache_beam/io/textio_test.py b/sdks/python/apache_beam/io/textio_test.py
index 0761e13..32765f0 100644
--- a/sdks/python/apache_beam/io/textio_test.py
+++ b/sdks/python/apache_beam/io/textio_test.py
@@ -424,28 +424,25 @@
   def test_read_from_text_single_file(self):
     file_name, expected_data = write_data(5)
     assert len(expected_data) == 5
-    pipeline = TestPipeline()
-    pcoll = pipeline | 'Read' >> ReadFromText(file_name)
-    assert_that(pcoll, equal_to(expected_data))
-    pipeline.run()
+    with TestPipeline() as pipeline:
+      pcoll = pipeline | 'Read' >> ReadFromText(file_name)
+      assert_that(pcoll, equal_to(expected_data))
 
   def test_read_from_text_with_file_name_single_file(self):
     file_name, data = write_data(5)
     expected_data = [(file_name, el) for el in data]
     assert len(expected_data) == 5
-    pipeline = TestPipeline()
-    pcoll = pipeline | 'Read' >> ReadFromTextWithFilename(file_name)
-    assert_that(pcoll, equal_to(expected_data))
-    pipeline.run()
+    with TestPipeline() as pipeline:
+      pcoll = pipeline | 'Read' >> ReadFromTextWithFilename(file_name)
+      assert_that(pcoll, equal_to(expected_data))
 
   def test_read_all_single_file(self):
     file_name, expected_data = write_data(5)
     assert len(expected_data) == 5
-    pipeline = TestPipeline()
-    pcoll = pipeline | 'Create' >> Create(
-        [file_name]) |'ReadAll' >> ReadAllFromText()
-    assert_that(pcoll, equal_to(expected_data))
-    pipeline.run()
+    with TestPipeline() as pipeline:
+      pcoll = pipeline | 'Create' >> Create(
+          [file_name]) |'ReadAll' >> ReadAllFromText()
+      assert_that(pcoll, equal_to(expected_data))
 
   def test_read_all_many_single_files(self):
     file_name1, expected_data1 = write_data(5)
@@ -458,11 +455,10 @@
     expected_data.extend(expected_data1)
     expected_data.extend(expected_data2)
     expected_data.extend(expected_data3)
-    pipeline = TestPipeline()
-    pcoll = pipeline | 'Create' >> Create(
-        [file_name1, file_name2, file_name3]) |'ReadAll' >> ReadAllFromText()
-    assert_that(pcoll, equal_to(expected_data))
-    pipeline.run()
+    with TestPipeline() as pipeline:
+      pcoll = pipeline | 'Create' >> Create(
+          [file_name1, file_name2, file_name3]) |'ReadAll' >> ReadAllFromText()
+      assert_that(pcoll, equal_to(expected_data))
 
   def test_read_all_unavailable_files_ignored(self):
     file_name1, expected_data1 = write_data(5)
@@ -476,13 +472,12 @@
     expected_data.extend(expected_data1)
     expected_data.extend(expected_data2)
     expected_data.extend(expected_data3)
-    pipeline = TestPipeline()
-    pcoll = (pipeline
-             | 'Create' >> Create(
-                 [file_name1, file_name2, file_name3, file_name4])
-             |'ReadAll' >> ReadAllFromText())
-    assert_that(pcoll, equal_to(expected_data))
-    pipeline.run()
+    with TestPipeline() as pipeline:
+      pcoll = (pipeline
+               | 'Create' >> Create(
+                   [file_name1, file_name2, file_name3, file_name4])
+               |'ReadAll' >> ReadAllFromText())
+      assert_that(pcoll, equal_to(expected_data))
 
   def test_read_from_text_single_file_with_coder(self):
     class DummyCoder(coders.Coder):
@@ -494,37 +489,33 @@
 
     file_name, expected_data = write_data(5)
     assert len(expected_data) == 5
-    pipeline = TestPipeline()
-    pcoll = pipeline | 'Read' >> ReadFromText(file_name, coder=DummyCoder())
-    assert_that(pcoll, equal_to([record * 2 for record in expected_data]))
-    pipeline.run()
+    with TestPipeline() as pipeline:
+      pcoll = pipeline | 'Read' >> ReadFromText(file_name, coder=DummyCoder())
+      assert_that(pcoll, equal_to([record * 2 for record in expected_data]))
 
   def test_read_from_text_file_pattern(self):
     pattern, expected_data = write_pattern([5, 3, 12, 8, 8, 4])
     assert len(expected_data) == 40
-    pipeline = TestPipeline()
-    pcoll = pipeline | 'Read' >> ReadFromText(pattern)
-    assert_that(pcoll, equal_to(expected_data))
-    pipeline.run()
+    with TestPipeline() as pipeline:
+      pcoll = pipeline | 'Read' >> ReadFromText(pattern)
+      assert_that(pcoll, equal_to(expected_data))
 
   def test_read_from_text_with_file_name_file_pattern(self):
     pattern, expected_data = write_pattern(
         lines_per_file=[5, 5], return_filenames=True)
     assert len(expected_data) == 10
-    pipeline = TestPipeline()
-    pcoll = pipeline | 'Read' >> ReadFromTextWithFilename(pattern)
-    assert_that(pcoll, equal_to(expected_data))
-    pipeline.run()
+    with TestPipeline() as pipeline:
+      pcoll = pipeline | 'Read' >> ReadFromTextWithFilename(pattern)
+      assert_that(pcoll, equal_to(expected_data))
 
   def test_read_all_file_pattern(self):
     pattern, expected_data = write_pattern([5, 3, 12, 8, 8, 4])
     assert len(expected_data) == 40
-    pipeline = TestPipeline()
-    pcoll = (pipeline
-             | 'Create' >> Create([pattern])
-             |'ReadAll' >> ReadAllFromText())
-    assert_that(pcoll, equal_to(expected_data))
-    pipeline.run()
+    with TestPipeline() as pipeline:
+      pcoll = (pipeline
+               | 'Create' >> Create([pattern])
+               |'ReadAll' >> ReadAllFromText())
+      assert_that(pcoll, equal_to(expected_data))
 
   def test_read_all_many_file_patterns(self):
     pattern1, expected_data1 = write_pattern([5, 3, 12, 8, 8, 4])
@@ -537,11 +528,10 @@
     expected_data.extend(expected_data1)
     expected_data.extend(expected_data2)
     expected_data.extend(expected_data3)
-    pipeline = TestPipeline()
-    pcoll = pipeline | 'Create' >> Create(
-        [pattern1, pattern2, pattern3]) |'ReadAll' >> ReadAllFromText()
-    assert_that(pcoll, equal_to(expected_data))
-    pipeline.run()
+    with TestPipeline() as pipeline:
+      pcoll = pipeline | 'Create' >> Create(
+          [pattern1, pattern2, pattern3]) |'ReadAll' >> ReadAllFromText()
+      assert_that(pcoll, equal_to(expected_data))
 
   def test_read_auto_bzip2(self):
     _, lines = write_data(15)
@@ -550,10 +540,9 @@
       with bz2.BZ2File(file_name, 'wb') as f:
         f.write('\n'.join(lines).encode('utf-8'))
 
-      pipeline = TestPipeline()
-      pcoll = pipeline | 'Read' >> ReadFromText(file_name)
-      assert_that(pcoll, equal_to(lines))
-      pipeline.run()
+      with TestPipeline() as pipeline:
+        pcoll = pipeline | 'Read' >> ReadFromText(file_name)
+        assert_that(pcoll, equal_to(lines))
 
   def test_read_auto_deflate(self):
     _, lines = write_data(15)
@@ -562,10 +551,9 @@
       with open(file_name, 'wb') as f:
         f.write(zlib.compress('\n'.join(lines).encode('utf-8')))
 
-      pipeline = TestPipeline()
-      pcoll = pipeline | 'Read' >> ReadFromText(file_name)
-      assert_that(pcoll, equal_to(lines))
-      pipeline.run()
+      with TestPipeline() as pipeline:
+        pcoll = pipeline | 'Read' >> ReadFromText(file_name)
+        assert_that(pcoll, equal_to(lines))
 
   def test_read_auto_gzip(self):
     _, lines = write_data(15)
@@ -575,10 +563,9 @@
       with gzip.GzipFile(file_name, 'wb') as f:
         f.write('\n'.join(lines).encode('utf-8'))
 
-      pipeline = TestPipeline()
-      pcoll = pipeline | 'Read' >> ReadFromText(file_name)
-      assert_that(pcoll, equal_to(lines))
-      pipeline.run()
+      with TestPipeline() as pipeline:
+        pcoll = pipeline | 'Read' >> ReadFromText(file_name)
+        assert_that(pcoll, equal_to(lines))
 
   def test_read_bzip2(self):
     _, lines = write_data(15)
@@ -587,12 +574,11 @@
       with bz2.BZ2File(file_name, 'wb') as f:
         f.write('\n'.join(lines).encode('utf-8'))
 
-      pipeline = TestPipeline()
-      pcoll = pipeline | 'Read' >> ReadFromText(
-          file_name,
-          compression_type=CompressionTypes.BZIP2)
-      assert_that(pcoll, equal_to(lines))
-      pipeline.run()
+      with TestPipeline() as pipeline:
+        pcoll = pipeline | 'Read' >> ReadFromText(
+            file_name,
+            compression_type=CompressionTypes.BZIP2)
+        assert_that(pcoll, equal_to(lines))
 
   def test_read_corrupted_bzip2_fails(self):
     _, lines = write_data(15)
@@ -604,13 +590,12 @@
       with open(file_name, 'wb') as f:
         f.write(b'corrupt')
 
-      pipeline = TestPipeline()
-      pcoll = pipeline | 'Read' >> ReadFromText(
-          file_name,
-          compression_type=CompressionTypes.BZIP2)
-      assert_that(pcoll, equal_to(lines))
       with self.assertRaises(Exception):
-        pipeline.run()
+        with TestPipeline() as pipeline:
+          pcoll = pipeline | 'Read' >> ReadFromText(
+              file_name,
+              compression_type=CompressionTypes.BZIP2)
+          assert_that(pcoll, equal_to(lines))
 
   def test_read_bzip2_concat(self):
     with TempDir() as tempdir:
@@ -645,14 +630,13 @@
           final_bzip2_file, 'ab') as dst:
         dst.writelines(src.readlines())
 
-      pipeline = TestPipeline()
-      lines = pipeline | 'ReadFromText' >> beam.io.ReadFromText(
-          final_bzip2_file,
-          compression_type=beam.io.filesystem.CompressionTypes.BZIP2)
+      with TestPipeline() as pipeline:
+        lines = pipeline | 'ReadFromText' >> beam.io.ReadFromText(
+            final_bzip2_file,
+            compression_type=beam.io.filesystem.CompressionTypes.BZIP2)
 
-      expected = ['a', 'b', 'c', 'p', 'q', 'r', 'x', 'y', 'z']
-      assert_that(lines, equal_to(expected))
-      pipeline.run()
+        expected = ['a', 'b', 'c', 'p', 'q', 'r', 'x', 'y', 'z']
+        assert_that(lines, equal_to(expected))
 
   def test_read_deflate(self):
     _, lines = write_data(15)
@@ -661,13 +645,12 @@
       with open(file_name, 'wb') as f:
         f.write(zlib.compress('\n'.join(lines).encode('utf-8')))
 
-      pipeline = TestPipeline()
-      pcoll = pipeline | 'Read' >> ReadFromText(
-          file_name,
-          0, CompressionTypes.DEFLATE,
-          True, coders.StrUtf8Coder())
-      assert_that(pcoll, equal_to(lines))
-      pipeline.run()
+      with TestPipeline() as pipeline:
+        pcoll = pipeline | 'Read' >> ReadFromText(
+            file_name,
+            0, CompressionTypes.DEFLATE,
+            True, coders.StrUtf8Coder())
+        assert_that(pcoll, equal_to(lines))
 
   def test_read_corrupted_deflate_fails(self):
     _, lines = write_data(15)
@@ -679,15 +662,13 @@
       with open(file_name, 'wb') as f:
         f.write(b'corrupt')
 
-      pipeline = TestPipeline()
-      pcoll = pipeline | 'Read' >> ReadFromText(
-          file_name,
-          0, CompressionTypes.DEFLATE,
-          True, coders.StrUtf8Coder())
-      assert_that(pcoll, equal_to(lines))
-
       with self.assertRaises(Exception):
-        pipeline.run()
+        with TestPipeline() as pipeline:
+          pcoll = pipeline | 'Read' >> ReadFromText(
+              file_name,
+              0, CompressionTypes.DEFLATE,
+              True, coders.StrUtf8Coder())
+          assert_that(pcoll, equal_to(lines))
 
   def test_read_deflate_concat(self):
     with TempDir() as tempdir:
@@ -722,13 +703,13 @@
               open(final_deflate_file, 'ab') as dst:
         dst.writelines(src.readlines())
 
-      pipeline = TestPipeline()
-      lines = pipeline | 'ReadFromText' >> beam.io.ReadFromText(
-          final_deflate_file,
-          compression_type=beam.io.filesystem.CompressionTypes.DEFLATE)
+      with TestPipeline() as pipeline:
+        lines = pipeline | 'ReadFromText' >> beam.io.ReadFromText(
+            final_deflate_file,
+            compression_type=beam.io.filesystem.CompressionTypes.DEFLATE)
 
-      expected = ['a', 'b', 'c', 'p', 'q', 'r', 'x', 'y', 'z']
-      assert_that(lines, equal_to(expected))
+        expected = ['a', 'b', 'c', 'p', 'q', 'r', 'x', 'y', 'z']
+        assert_that(lines, equal_to(expected))
 
   def test_read_gzip(self):
     _, lines = write_data(15)
@@ -737,13 +718,12 @@
       with gzip.GzipFile(file_name, 'wb') as f:
         f.write('\n'.join(lines).encode('utf-8'))
 
-      pipeline = TestPipeline()
-      pcoll = pipeline | 'Read' >> ReadFromText(
-          file_name,
-          0, CompressionTypes.GZIP,
-          True, coders.StrUtf8Coder())
-      assert_that(pcoll, equal_to(lines))
-      pipeline.run()
+      with TestPipeline() as pipeline:
+        pcoll = pipeline | 'Read' >> ReadFromText(
+            file_name,
+            0, CompressionTypes.GZIP,
+            True, coders.StrUtf8Coder())
+        assert_that(pcoll, equal_to(lines))
 
   def test_read_corrupted_gzip_fails(self):
     _, lines = write_data(15)
@@ -755,15 +735,13 @@
       with open(file_name, 'wb') as f:
         f.write(b'corrupt')
 
-      pipeline = TestPipeline()
-      pcoll = pipeline | 'Read' >> ReadFromText(
-          file_name,
-          0, CompressionTypes.GZIP,
-          True, coders.StrUtf8Coder())
-      assert_that(pcoll, equal_to(lines))
-
       with self.assertRaises(Exception):
-        pipeline.run()
+        with TestPipeline() as pipeline:
+          pcoll = pipeline | 'Read' >> ReadFromText(
+              file_name,
+              0, CompressionTypes.GZIP,
+              True, coders.StrUtf8Coder())
+          assert_that(pcoll, equal_to(lines))
 
   def test_read_gzip_concat(self):
     with TempDir() as tempdir:
@@ -798,13 +776,13 @@
            open(final_gzip_file, 'ab') as dst:
         dst.writelines(src.readlines())
 
-      pipeline = TestPipeline()
-      lines = pipeline | 'ReadFromText' >> beam.io.ReadFromText(
-          final_gzip_file,
-          compression_type=beam.io.filesystem.CompressionTypes.GZIP)
+      with TestPipeline() as pipeline:
+        lines = pipeline | 'ReadFromText' >> beam.io.ReadFromText(
+            final_gzip_file,
+            compression_type=beam.io.filesystem.CompressionTypes.GZIP)
 
-      expected = ['a', 'b', 'c', 'p', 'q', 'r', 'x', 'y', 'z']
-      assert_that(lines, equal_to(expected))
+        expected = ['a', 'b', 'c', 'p', 'q', 'r', 'x', 'y', 'z']
+        assert_that(lines, equal_to(expected))
 
   def test_read_all_gzip(self):
     _, lines = write_data(100)
@@ -812,13 +790,12 @@
       file_name = tempdir.create_temp_file()
       with gzip.GzipFile(file_name, 'wb') as f:
         f.write('\n'.join(lines).encode('utf-8'))
-      pipeline = TestPipeline()
-      pcoll = (pipeline
-               | Create([file_name])
-               | 'ReadAll' >> ReadAllFromText(
-                   compression_type=CompressionTypes.GZIP))
-      assert_that(pcoll, equal_to(lines))
-      pipeline.run()
+      with TestPipeline() as pipeline:
+        pcoll = (pipeline
+                 | Create([file_name])
+                 | 'ReadAll' >> ReadAllFromText(
+                     compression_type=CompressionTypes.GZIP))
+        assert_that(pcoll, equal_to(lines))
 
   def test_read_gzip_large(self):
     _, lines = write_data(10000)
@@ -828,13 +805,12 @@
       with gzip.GzipFile(file_name, 'wb') as f:
         f.write('\n'.join(lines).encode('utf-8'))
 
-      pipeline = TestPipeline()
-      pcoll = pipeline | 'Read' >> ReadFromText(
-          file_name,
-          0, CompressionTypes.GZIP,
-          True, coders.StrUtf8Coder())
-      assert_that(pcoll, equal_to(lines))
-      pipeline.run()
+      with TestPipeline() as pipeline:
+        pcoll = pipeline | 'Read' >> ReadFromText(
+            file_name,
+            0, CompressionTypes.GZIP,
+            True, coders.StrUtf8Coder())
+        assert_that(pcoll, equal_to(lines))
 
   def test_read_gzip_large_after_splitting(self):
     _, lines = write_data(10000)
@@ -861,13 +837,12 @@
   def test_read_gzip_empty_file(self):
     with TempDir() as tempdir:
       file_name = tempdir.create_temp_file()
-      pipeline = TestPipeline()
-      pcoll = pipeline | 'Read' >> ReadFromText(
-          file_name,
-          0, CompressionTypes.GZIP,
-          True, coders.StrUtf8Coder())
-      assert_that(pcoll, equal_to([]))
-      pipeline.run()
+      with TestPipeline() as pipeline:
+        pcoll = pipeline | 'Read' >> ReadFromText(
+            file_name,
+            0, CompressionTypes.GZIP,
+            True, coders.StrUtf8Coder())
+        assert_that(pcoll, equal_to([]))
 
   def _remove_lines(self, lines, sublist_lengths, num_to_remove):
     """Utility function to remove num_to_remove lines from each sublist.
@@ -950,12 +925,11 @@
       with gzip.GzipFile(file_name, 'wb') as f:
         f.write('\n'.join(lines).encode('utf-8'))
 
-      pipeline = TestPipeline()
-      pcoll = pipeline | 'Read' >> ReadFromText(
-          file_name, 0, CompressionTypes.GZIP,
-          True, coders.StrUtf8Coder(), skip_header_lines=2)
-      assert_that(pcoll, equal_to(lines[2:]))
-      pipeline.run()
+      with TestPipeline() as pipeline:
+        pcoll = pipeline | 'Read' >> ReadFromText(
+            file_name, 0, CompressionTypes.GZIP,
+            True, coders.StrUtf8Coder(), skip_header_lines=2)
+        assert_that(pcoll, equal_to(lines[2:]))
 
   def test_read_after_splitting_skip_header(self):
     file_name, expected_data = write_data(100)
@@ -1105,10 +1079,9 @@
       self.assertEqual(f.read().splitlines(), header.splitlines())
 
   def test_write_dataflow(self):
-    pipeline = TestPipeline()
-    pcoll = pipeline | beam.core.Create(self.lines)
-    pcoll | 'Write' >> WriteToText(self.path)  # pylint: disable=expression-not-assigned
-    pipeline.run()
+    with TestPipeline() as pipeline:
+      pcoll = pipeline | beam.core.Create(self.lines)
+      pcoll | 'Write' >> WriteToText(self.path)  # pylint: disable=expression-not-assigned
 
     read_result = []
     for file_name in glob.glob(self.path + '*'):
@@ -1118,10 +1091,9 @@
     self.assertEqual(sorted(read_result), sorted(self.lines))
 
   def test_write_dataflow_auto_compression(self):
-    pipeline = TestPipeline()
-    pcoll = pipeline | beam.core.Create(self.lines)
-    pcoll | 'Write' >> WriteToText(self.path, file_name_suffix='.gz')  # pylint: disable=expression-not-assigned
-    pipeline.run()
+    with TestPipeline() as pipeline:
+      pcoll = pipeline | beam.core.Create(self.lines)
+      pcoll | 'Write' >> WriteToText(self.path, file_name_suffix='.gz')  # pylint: disable=expression-not-assigned
 
     read_result = []
     for file_name in glob.glob(self.path + '*'):
@@ -1131,13 +1103,12 @@
     self.assertEqual(sorted(read_result), sorted(self.lines))
 
   def test_write_dataflow_auto_compression_unsharded(self):
-    pipeline = TestPipeline()
-    pcoll = pipeline | 'Create' >> beam.core.Create(self.lines)
-    pcoll | 'Write' >> WriteToText(  # pylint: disable=expression-not-assigned
-        self.path + '.gz',
-        shard_name_template='')
+    with TestPipeline() as pipeline:
+      pcoll = pipeline | 'Create' >> beam.core.Create(self.lines)
+      pcoll | 'Write' >> WriteToText(  # pylint: disable=expression-not-assigned
+          self.path + '.gz',
+          shard_name_template='')
 
-    pipeline.run()
 
     read_result = []
     for file_name in glob.glob(self.path + '*'):
@@ -1147,14 +1118,13 @@
     self.assertEqual(sorted(read_result), sorted(self.lines))
 
   def test_write_dataflow_header(self):
-    pipeline = TestPipeline()
-    pcoll = pipeline | 'Create' >> beam.core.Create(self.lines)
-    header_text = 'foo'
-    pcoll | 'Write' >> WriteToText(  # pylint: disable=expression-not-assigned
-        self.path + '.gz',
-        shard_name_template='',
-        header=header_text)
-    pipeline.run()
+    with TestPipeline() as pipeline:
+      pcoll = pipeline | 'Create' >> beam.core.Create(self.lines)
+      header_text = 'foo'
+      pcoll | 'Write' >> WriteToText(  # pylint: disable=expression-not-assigned
+          self.path + '.gz',
+          shard_name_template='',
+          header=header_text)
 
     read_result = []
     for file_name in glob.glob(self.path + '*'):
diff --git a/sdks/python/apache_beam/io/vcfio_test.py b/sdks/python/apache_beam/io/vcfio_test.py
index 0c820ab..8091ab6 100644
--- a/sdks/python/apache_beam/io/vcfio_test.py
+++ b/sdks/python/apache_beam/io/vcfio_test.py
@@ -496,26 +496,23 @@
     with TempDir() as tempdir:
       file_name = self._create_temp_vcf_file(_SAMPLE_HEADER_LINES +
                                              _SAMPLE_TEXT_LINES, tempdir)
-      pipeline = TestPipeline()
-      pcoll = pipeline | 'Read' >> ReadFromVcf(file_name)
-      assert_that(pcoll, _count_equals_to(len(_SAMPLE_TEXT_LINES)))
-      pipeline.run()
+      with TestPipeline() as pipeline:
+        pcoll = pipeline | 'Read' >> ReadFromVcf(file_name)
+        assert_that(pcoll, _count_equals_to(len(_SAMPLE_TEXT_LINES)))
 
   @unittest.skipIf(VCF_FILE_DIR_MISSING, 'VCF test file directory is missing')
   def test_pipeline_read_single_file_large(self):
-    pipeline = TestPipeline()
-    pcoll = pipeline | 'Read' >> ReadFromVcf(
-        get_full_file_path('valid-4.0.vcf'))
-    assert_that(pcoll, _count_equals_to(5))
-    pipeline.run()
+    with TestPipeline() as pipeline:
+      pcoll = pipeline | 'Read' >> ReadFromVcf(
+          get_full_file_path('valid-4.0.vcf'))
+      assert_that(pcoll, _count_equals_to(5))
 
   @unittest.skipIf(VCF_FILE_DIR_MISSING, 'VCF test file directory is missing')
   def test_pipeline_read_file_pattern_large(self):
-    pipeline = TestPipeline()
-    pcoll = pipeline | 'Read' >> ReadFromVcf(
-        os.path.join(get_full_dir(), 'valid-*.vcf'))
-    assert_that(pcoll, _count_equals_to(9900))
-    pipeline.run()
+    with TestPipeline() as pipeline:
+      pcoll = pipeline | 'Read' >> ReadFromVcf(
+          os.path.join(get_full_dir(), 'valid-*.vcf'))
+      assert_that(pcoll, _count_equals_to(9900))
 
   def test_read_reentrant_without_splitting(self):
     with TempDir() as tempdir:
diff --git a/sdks/python/apache_beam/pipeline_test.py b/sdks/python/apache_beam/pipeline_test.py
index 48d3b0d..7415baf 100644
--- a/sdks/python/apache_beam/pipeline_test.py
+++ b/sdks/python/apache_beam/pipeline_test.py
@@ -154,86 +154,83 @@
       self.leave_composite.append(transform_node)
 
   def test_create(self):
-    pipeline = TestPipeline()
-    pcoll = pipeline | 'label1' >> Create([1, 2, 3])
-    assert_that(pcoll, equal_to([1, 2, 3]))
+    with TestPipeline() as pipeline:
+      pcoll = pipeline | 'label1' >> Create([1, 2, 3])
+      assert_that(pcoll, equal_to([1, 2, 3]))
 
-    # Test if initial value is an iterator object.
-    pcoll2 = pipeline | 'label2' >> Create(iter((4, 5, 6)))
-    pcoll3 = pcoll2 | 'do' >> FlatMap(lambda x: [x + 10])
-    assert_that(pcoll3, equal_to([14, 15, 16]), label='pcoll3')
-    pipeline.run()
+      # Test if initial value is an iterator object.
+      pcoll2 = pipeline | 'label2' >> Create(iter((4, 5, 6)))
+      pcoll3 = pcoll2 | 'do' >> FlatMap(lambda x: [x + 10])
+      assert_that(pcoll3, equal_to([14, 15, 16]), label='pcoll3')
 
   def test_flatmap_builtin(self):
-    pipeline = TestPipeline()
-    pcoll = pipeline | 'label1' >> Create([1, 2, 3])
-    assert_that(pcoll, equal_to([1, 2, 3]))
+    with TestPipeline() as pipeline:
+      pcoll = pipeline | 'label1' >> Create([1, 2, 3])
+      assert_that(pcoll, equal_to([1, 2, 3]))
 
-    pcoll2 = pcoll | 'do' >> FlatMap(lambda x: [x + 10])
-    assert_that(pcoll2, equal_to([11, 12, 13]), label='pcoll2')
+      pcoll2 = pcoll | 'do' >> FlatMap(lambda x: [x + 10])
+      assert_that(pcoll2, equal_to([11, 12, 13]), label='pcoll2')
 
-    pcoll3 = pcoll2 | 'm1' >> Map(lambda x: [x, 12])
-    assert_that(pcoll3,
-                equal_to([[11, 12], [12, 12], [13, 12]]), label='pcoll3')
+      pcoll3 = pcoll2 | 'm1' >> Map(lambda x: [x, 12])
+      assert_that(pcoll3,
+                  equal_to([[11, 12], [12, 12], [13, 12]]), label='pcoll3')
 
-    pcoll4 = pcoll3 | 'do2' >> FlatMap(set)
-    assert_that(pcoll4, equal_to([11, 12, 12, 12, 13]), label='pcoll4')
-    pipeline.run()
+      pcoll4 = pcoll3 | 'do2' >> FlatMap(set)
+      assert_that(pcoll4, equal_to([11, 12, 12, 12, 13]), label='pcoll4')
 
   def test_maptuple_builtin(self):
-    pipeline = TestPipeline()
-    pcoll = pipeline | Create([('e1', 'e2')])
-    side1 = beam.pvalue.AsSingleton(pipeline | 'side1' >> Create(['s1']))
-    side2 = beam.pvalue.AsSingleton(pipeline | 'side2' >> Create(['s2']))
+    with TestPipeline() as pipeline:
+      pcoll = pipeline | Create([('e1', 'e2')])
+      side1 = beam.pvalue.AsSingleton(pipeline | 'side1' >> Create(['s1']))
+      side2 = beam.pvalue.AsSingleton(pipeline | 'side2' >> Create(['s2']))
 
-    # A test function with a tuple input, an auxiliary parameter,
-    # and some side inputs.
-    fn = lambda e1, e2, t=DoFn.TimestampParam, s1=None, s2=None: (
-        e1, e2, t, s1, s2)
-    assert_that(pcoll | 'NoSides' >> beam.core.MapTuple(fn),
-                equal_to([('e1', 'e2', MIN_TIMESTAMP, None, None)]),
-                label='NoSidesCheck')
-    assert_that(pcoll | 'StaticSides' >> beam.core.MapTuple(fn, 's1', 's2'),
-                equal_to([('e1', 'e2', MIN_TIMESTAMP, 's1', 's2')]),
-                label='StaticSidesCheck')
-    assert_that(pcoll | 'DynamicSides' >> beam.core.MapTuple(fn, side1, side2),
-                equal_to([('e1', 'e2', MIN_TIMESTAMP, 's1', 's2')]),
-                label='DynamicSidesCheck')
-    assert_that(pcoll | 'MixedSides' >> beam.core.MapTuple(fn, s2=side2),
-                equal_to([('e1', 'e2', MIN_TIMESTAMP, None, 's2')]),
-                label='MixedSidesCheck')
-    pipeline.run()
+      # A test function with a tuple input, an auxiliary parameter,
+      # and some side inputs.
+      fn = lambda e1, e2, t=DoFn.TimestampParam, s1=None, s2=None: (
+          e1, e2, t, s1, s2)
+      assert_that(pcoll | 'NoSides' >> beam.core.MapTuple(fn),
+                  equal_to([('e1', 'e2', MIN_TIMESTAMP, None, None)]),
+                  label='NoSidesCheck')
+      assert_that(pcoll | 'StaticSides' >> beam.core.MapTuple(fn, 's1', 's2'),
+                  equal_to([('e1', 'e2', MIN_TIMESTAMP, 's1', 's2')]),
+                  label='StaticSidesCheck')
+      assert_that(pcoll | 'DynamicSides' >> beam.core.MapTuple(
+          fn, side1, side2),
+                  equal_to([('e1', 'e2', MIN_TIMESTAMP, 's1', 's2')]),
+                  label='DynamicSidesCheck')
+      assert_that(pcoll | 'MixedSides' >> beam.core.MapTuple(fn, s2=side2),
+                  equal_to([('e1', 'e2', MIN_TIMESTAMP, None, 's2')]),
+                  label='MixedSidesCheck')
 
   def test_flatmaptuple_builtin(self):
-    pipeline = TestPipeline()
-    pcoll = pipeline | Create([('e1', 'e2')])
-    side1 = beam.pvalue.AsSingleton(pipeline | 'side1' >> Create(['s1']))
-    side2 = beam.pvalue.AsSingleton(pipeline | 'side2' >> Create(['s2']))
+    with TestPipeline() as pipeline:
+      pcoll = pipeline | Create([('e1', 'e2')])
+      side1 = beam.pvalue.AsSingleton(pipeline | 'side1' >> Create(['s1']))
+      side2 = beam.pvalue.AsSingleton(pipeline | 'side2' >> Create(['s2']))
 
-    # A test function with a tuple input, an auxiliary parameter,
-    # and some side inputs.
-    fn = lambda e1, e2, t=DoFn.TimestampParam, s1=None, s2=None: (
-        e1, e2, t, s1, s2)
-    assert_that(pcoll | 'NoSides' >> beam.core.FlatMapTuple(fn),
-                equal_to(['e1', 'e2', MIN_TIMESTAMP, None, None]),
-                label='NoSidesCheck')
-    assert_that(pcoll | 'StaticSides' >> beam.core.FlatMapTuple(fn, 's1', 's2'),
-                equal_to(['e1', 'e2', MIN_TIMESTAMP, 's1', 's2']),
-                label='StaticSidesCheck')
-    assert_that(pcoll
-                | 'DynamicSides' >> beam.core.FlatMapTuple(fn, side1, side2),
-                equal_to(['e1', 'e2', MIN_TIMESTAMP, 's1', 's2']),
-                label='DynamicSidesCheck')
-    assert_that(pcoll | 'MixedSides' >> beam.core.FlatMapTuple(fn, s2=side2),
-                equal_to(['e1', 'e2', MIN_TIMESTAMP, None, 's2']),
-                label='MixedSidesCheck')
-    pipeline.run()
+      # A test function with a tuple input, an auxiliary parameter,
+      # and some side inputs.
+      fn = lambda e1, e2, t=DoFn.TimestampParam, s1=None, s2=None: (
+          e1, e2, t, s1, s2)
+      assert_that(pcoll | 'NoSides' >> beam.core.FlatMapTuple(fn),
+                  equal_to(['e1', 'e2', MIN_TIMESTAMP, None, None]),
+                  label='NoSidesCheck')
+      assert_that(pcoll | 'StaticSides' >> beam.core.FlatMapTuple(
+          fn, 's1', 's2'),
+                  equal_to(['e1', 'e2', MIN_TIMESTAMP, 's1', 's2']),
+                  label='StaticSidesCheck')
+      assert_that(pcoll
+                  | 'DynamicSides' >> beam.core.FlatMapTuple(fn, side1, side2),
+                  equal_to(['e1', 'e2', MIN_TIMESTAMP, 's1', 's2']),
+                  label='DynamicSidesCheck')
+      assert_that(pcoll | 'MixedSides' >> beam.core.FlatMapTuple(fn, s2=side2),
+                  equal_to(['e1', 'e2', MIN_TIMESTAMP, None, 's2']),
+                  label='MixedSidesCheck')
 
   def test_create_singleton_pcollection(self):
-    pipeline = TestPipeline()
-    pcoll = pipeline | 'label' >> Create([[1, 2, 3]])
-    assert_that(pcoll, equal_to([[1, 2, 3]]))
-    pipeline.run()
+    with TestPipeline() as pipeline:
+      pcoll = pipeline | 'label' >> Create([[1, 2, 3]])
+      assert_that(pcoll, equal_to([[1, 2, 3]]))
 
   # TODO(BEAM-1555): Test is failing on the service, with FakeSource.
   # @attr('ValidatesRunner')
@@ -249,10 +246,9 @@
     self.assertEqual(outputs_counter.committed, 6)
 
   def test_fake_read(self):
-    pipeline = TestPipeline()
-    pcoll = pipeline | 'read' >> Read(FakeSource([1, 2, 3]))
-    assert_that(pcoll, equal_to([1, 2, 3]))
-    pipeline.run()
+    with TestPipeline() as pipeline:
+      pcoll = pipeline | 'read' >> Read(FakeSource([1, 2, 3]))
+      assert_that(pcoll, equal_to([1, 2, 3]))
 
   def test_visit_entire_graph(self):
     pipeline = Pipeline()
@@ -274,11 +270,10 @@
     self.assertEqual(visitor.leave_composite[0].transform, transform)
 
   def test_apply_custom_transform(self):
-    pipeline = TestPipeline()
-    pcoll = pipeline | 'pcoll' >> Create([1, 2, 3])
-    result = pcoll | PipelineTest.CustomTransform()
-    assert_that(result, equal_to([2, 3, 4]))
-    pipeline.run()
+    with TestPipeline() as pipeline:
+      pcoll = pipeline | 'pcoll' >> Create([1, 2, 3])
+      result = pcoll | PipelineTest.CustomTransform()
+      assert_that(result, equal_to([2, 3, 4]))
 
   def test_reuse_custom_transform_instance(self):
     pipeline = Pipeline()
@@ -295,15 +290,14 @@
         'pvalue | "label" >> transform')
 
   def test_reuse_cloned_custom_transform_instance(self):
-    pipeline = TestPipeline()
-    pcoll1 = pipeline | 'pc1' >> Create([1, 2, 3])
-    pcoll2 = pipeline | 'pc2' >> Create([4, 5, 6])
-    transform = PipelineTest.CustomTransform()
-    result1 = pcoll1 | transform
-    result2 = pcoll2 | 'new_label' >> transform
-    assert_that(result1, equal_to([2, 3, 4]), label='r1')
-    assert_that(result2, equal_to([5, 6, 7]), label='r2')
-    pipeline.run()
+    with TestPipeline() as pipeline:
+      pcoll1 = pipeline | 'pc1' >> Create([1, 2, 3])
+      pcoll2 = pipeline | 'pc2' >> Create([4, 5, 6])
+      transform = PipelineTest.CustomTransform()
+      result1 = pcoll1 | transform
+      result2 = pcoll2 | 'new_label' >> transform
+      assert_that(result1, equal_to([2, 3, 4]), label='r1')
+      assert_that(result2, equal_to([5, 6, 7]), label='r2')
 
   def test_transform_no_super_init(self):
     class AddSuffix(PTransform):
@@ -347,24 +341,23 @@
 
     # TODO(robertwb): reduce memory usage of FnApiRunner so that this test
     # passes.
-    pipeline = TestPipeline(runner='BundleBasedDirectRunner')
+    with TestPipeline(runner='BundleBasedDirectRunner') as pipeline:
 
-    # Consumed memory should not be proportional to the number of maps.
-    memory_threshold = (
-        get_memory_usage_in_bytes() + (5 * len_elements * num_elements))
+      # Consumed memory should not be proportional to the number of maps.
+      memory_threshold = (
+          get_memory_usage_in_bytes() + (5 * len_elements * num_elements))
 
-    # Plus small additional slack for memory fluctuations during the test.
-    memory_threshold += 10 * (2 ** 20)
+      # Plus small additional slack for memory fluctuations during the test.
+      memory_threshold += 10 * (2 ** 20)
 
-    biglist = pipeline | 'oom:create' >> Create(
-        ['x' * len_elements] * num_elements)
-    for i in range(num_maps):
-      biglist = biglist | ('oom:addone-%d' % i) >> Map(lambda x: x + 'y')
-    result = biglist | 'oom:check' >> Map(check_memory, memory_threshold)
-    assert_that(result, equal_to(
-        ['x' * len_elements + 'y' * num_maps] * num_elements))
+      biglist = pipeline | 'oom:create' >> Create(
+          ['x' * len_elements] * num_elements)
+      for i in range(num_maps):
+        biglist = biglist | ('oom:addone-%d' % i) >> Map(lambda x: x + 'y')
+      result = biglist | 'oom:check' >> Map(check_memory, memory_threshold)
+      assert_that(result, equal_to(
+          ['x' * len_elements + 'y' * num_maps] * num_elements))
 
-    pipeline.run()
 
   def test_aggregator_empty_input(self):
     actual = [] | CombineGlobally(max).without_defaults()
@@ -473,27 +466,27 @@
       else:
         yield TaggedOutput('letters', x)
 
-    p = TestPipeline()
-    multi = (p
-             | beam.Create([1, 2, 3, 'a', 'b', 'c'])
-             | 'MyMultiOutput' >> beam.ParDo(mux_input).with_outputs())
-    letters = multi.letters | 'MyLetters' >> beam.Map(lambda x: x)
-    numbers = multi.numbers | 'MyNumbers' >> beam.Map(lambda x: x)
+    with TestPipeline() as p:
+      multi = (p
+               | beam.Create([1, 2, 3, 'a', 'b', 'c'])
+               | 'MyMultiOutput' >> beam.ParDo(mux_input).with_outputs())
+      letters = multi.letters | 'MyLetters' >> beam.Map(lambda x: x)
+      numbers = multi.numbers | 'MyNumbers' >> beam.Map(lambda x: x)
 
-    # Assert that the PCollection replacement worked correctly and that elements
-    # are flowing through. The replacement transform first multiples by 2 then
-    # the leaf nodes inside the composite multiply by an additional 3 and 5. Use
-    # prime numbers to ensure that each transform is getting executed once.
-    assert_that(letters,
-                equal_to(['a'*2*3, 'b'*2*3, 'c'*2*3]),
-                label='assert letters')
-    assert_that(numbers,
-                equal_to([1*2*5, 2*2*5, 3*2*5]),
-                label='assert numbers')
+      # Assert that the PCollection replacement worked correctly and that
+      # elements are flowing through. The replacement transform first
+      # multiples by 2 then the leaf nodes inside the composite multiply by
+      # an additional 3 and 5. Use prime numbers to ensure that each
+      # transform is getting executed once.
+      assert_that(letters,
+                  equal_to(['a'*2*3, 'b'*2*3, 'c'*2*3]),
+                  label='assert letters')
+      assert_that(numbers,
+                  equal_to([1*2*5, 2*2*5, 3*2*5]),
+                  label='assert numbers')
 
-    # Do the replacement and run the element assertions.
-    p.replace_all([MultiOutputOverride()])
-    p.run()
+      # Do the replacement and run the element assertions.
+      p.replace_all([MultiOutputOverride()])
 
     # The following checks the graph to make sure the replacement occurred.
     visitor = PipelineTest.Visitor(visited=[])
@@ -535,20 +528,18 @@
       def process(self, element, counter=DoFn.StateParam(BYTES_STATE)):
         return self.return_recursive(1)
 
-    p = TestPipeline()
-    pcoll = (p
-             | beam.Create([(1, 1), (2, 2), (3, 3)])
-             | beam.GroupByKey()
-             | beam.ParDo(StatefulDoFn()))
-    p.run()
+    with TestPipeline() as p:
+      pcoll = (p
+               | beam.Create([(1, 1), (2, 2), (3, 3)])
+               | beam.GroupByKey()
+               | beam.ParDo(StatefulDoFn()))
     self.assertEqual(pcoll.element_type, typehints.Any)
 
-    p = TestPipeline()
-    pcoll = (p
-             | beam.Create([(1, 1), (2, 2), (3, 3)])
-             | beam.GroupByKey()
-             | beam.ParDo(StatefulDoFn()).with_output_types(str))
-    p.run()
+    with TestPipeline() as p:
+      pcoll = (p
+               | beam.Create([(1, 1), (2, 2), (3, 3)])
+               | beam.GroupByKey()
+               | beam.ParDo(StatefulDoFn()).with_output_types(str))
     self.assertEqual(pcoll.element_type, str)
 
   def test_track_pcoll_unbounded(self):
@@ -609,40 +600,37 @@
       def process(self, element):
         yield element + 10
 
-    pipeline = TestPipeline()
-    pcoll = pipeline | 'Create' >> Create([1, 2]) | 'Do' >> ParDo(TestDoFn())
-    assert_that(pcoll, equal_to([11, 12]))
-    pipeline.run()
+    with TestPipeline() as pipeline:
+      pcoll = pipeline | 'Create' >> Create([1, 2]) | 'Do' >> ParDo(TestDoFn())
+      assert_that(pcoll, equal_to([11, 12]))
 
   def test_side_input_no_tag(self):
     class TestDoFn(DoFn):
       def process(self, element, prefix, suffix):
         return ['%s-%s-%s' % (prefix, element, suffix)]
 
-    pipeline = TestPipeline()
-    words_list = ['aa', 'bb', 'cc']
-    words = pipeline | 'SomeWords' >> Create(words_list)
-    prefix = 'zyx'
-    suffix = pipeline | 'SomeString' >> Create(['xyz'])  # side in
-    result = words | 'DecorateWordsDoFnNoTag' >> ParDo(
-        TestDoFn(), prefix, suffix=AsSingleton(suffix))
-    assert_that(result, equal_to(['zyx-%s-xyz' % x for x in words_list]))
-    pipeline.run()
+    with TestPipeline() as pipeline:
+      words_list = ['aa', 'bb', 'cc']
+      words = pipeline | 'SomeWords' >> Create(words_list)
+      prefix = 'zyx'
+      suffix = pipeline | 'SomeString' >> Create(['xyz'])  # side in
+      result = words | 'DecorateWordsDoFnNoTag' >> ParDo(
+          TestDoFn(), prefix, suffix=AsSingleton(suffix))
+      assert_that(result, equal_to(['zyx-%s-xyz' % x for x in words_list]))
 
   def test_side_input_tagged(self):
     class TestDoFn(DoFn):
       def process(self, element, prefix, suffix=DoFn.SideInputParam):
         return ['%s-%s-%s' % (prefix, element, suffix)]
 
-    pipeline = TestPipeline()
-    words_list = ['aa', 'bb', 'cc']
-    words = pipeline | 'SomeWords' >> Create(words_list)
-    prefix = 'zyx'
-    suffix = pipeline | 'SomeString' >> Create(['xyz'])  # side in
-    result = words | 'DecorateWordsDoFnNoTag' >> ParDo(
-        TestDoFn(), prefix, suffix=AsSingleton(suffix))
-    assert_that(result, equal_to(['zyx-%s-xyz' % x for x in words_list]))
-    pipeline.run()
+    with TestPipeline() as pipeline:
+      words_list = ['aa', 'bb', 'cc']
+      words = pipeline | 'SomeWords' >> Create(words_list)
+      prefix = 'zyx'
+      suffix = pipeline | 'SomeString' >> Create(['xyz'])  # side in
+      result = words | 'DecorateWordsDoFnNoTag' >> ParDo(
+          TestDoFn(), prefix, suffix=AsSingleton(suffix))
+      assert_that(result, equal_to(['zyx-%s-xyz' % x for x in words_list]))
 
   @attr('ValidatesRunner')
   def test_element_param(self):
@@ -668,32 +656,30 @@
       def process(self, element, window=DoFn.WindowParam):
         yield (element, (float(window.start), float(window.end)))
 
-    pipeline = TestPipeline()
-    pcoll = (pipeline
-             | Create([1, 7])
-             | Map(lambda x: TimestampedValue(x, x))
-             | WindowInto(windowfn=SlidingWindows(10, 5))
-             | ParDo(TestDoFn()))
-    assert_that(pcoll, equal_to([(1, (-5, 5)), (1, (0, 10)),
-                                 (7, (0, 10)), (7, (5, 15))]))
-    pcoll2 = pcoll | 'Again' >> ParDo(TestDoFn())
-    assert_that(
-        pcoll2,
-        equal_to([
-            ((1, (-5, 5)), (-5, 5)), ((1, (0, 10)), (0, 10)),
-            ((7, (0, 10)), (0, 10)), ((7, (5, 15)), (5, 15))]),
-        label='doubled windows')
-    pipeline.run()
+    with TestPipeline() as pipeline:
+      pcoll = (pipeline
+               | Create([1, 7])
+               | Map(lambda x: TimestampedValue(x, x))
+               | WindowInto(windowfn=SlidingWindows(10, 5))
+               | ParDo(TestDoFn()))
+      assert_that(pcoll, equal_to([(1, (-5, 5)), (1, (0, 10)),
+                                   (7, (0, 10)), (7, (5, 15))]))
+      pcoll2 = pcoll | 'Again' >> ParDo(TestDoFn())
+      assert_that(
+          pcoll2,
+          equal_to([
+              ((1, (-5, 5)), (-5, 5)), ((1, (0, 10)), (0, 10)),
+              ((7, (0, 10)), (0, 10)), ((7, (5, 15)), (5, 15))]),
+          label='doubled windows')
 
   def test_timestamp_param(self):
     class TestDoFn(DoFn):
       def process(self, element, timestamp=DoFn.TimestampParam):
         yield timestamp
 
-    pipeline = TestPipeline()
-    pcoll = pipeline | 'Create' >> Create([1, 2]) | 'Do' >> ParDo(TestDoFn())
-    assert_that(pcoll, equal_to([MIN_TIMESTAMP, MIN_TIMESTAMP]))
-    pipeline.run()
+    with TestPipeline() as pipeline:
+      pcoll = pipeline | 'Create' >> Create([1, 2]) | 'Do' >> ParDo(TestDoFn())
+      assert_that(pcoll, equal_to([MIN_TIMESTAMP, MIN_TIMESTAMP]))
 
   def test_timestamp_param_map(self):
     with TestPipeline() as p:
@@ -733,12 +719,11 @@
 
     # Ensure that we don't use default values in a context where they must be
     # comparable (see BEAM-8301).
-    pipeline = TestPipeline()
-    pcoll = (pipeline
-             | beam.Create([None])
-             | Map(lambda e, x=IncomparableType(): (e, type(x).__name__)))
-    assert_that(pcoll, equal_to([(None, 'IncomparableType')]))
-    pipeline.run()
+    with TestPipeline() as pipeline:
+      pcoll = (pipeline
+               | beam.Create([None])
+               | Map(lambda e, x=IncomparableType(): (e, type(x).__name__)))
+      assert_that(pcoll, equal_to([(None, 'IncomparableType')]))
 
 
 class Bacon(PipelineOptions):
diff --git a/sdks/python/apache_beam/runners/common.pxd b/sdks/python/apache_beam/runners/common.pxd
index 37e05bf..00840fe 100644
--- a/sdks/python/apache_beam/runners/common.pxd
+++ b/sdks/python/apache_beam/runners/common.pxd
@@ -69,7 +69,6 @@
 
   cpdef invoke_process(self, WindowedValue windowed_value,
                        restriction_tracker=*,
-                       OutputProcessor output_processor=*,
                        additional_args=*, additional_kwargs=*)
   cpdef invoke_start_bundle(self)
   cpdef invoke_finish_bundle(self)
diff --git a/sdks/python/apache_beam/runners/common.py b/sdks/python/apache_beam/runners/common.py
index 429fdd7..a6145ac 100644
--- a/sdks/python/apache_beam/runners/common.py
+++ b/sdks/python/apache_beam/runners/common.py
@@ -340,7 +340,7 @@
   represented by a given DoFnSignature."""
 
   def __init__(self,
-               output_processor,  # type: Optional[_OutputProcessor]
+               output_processor,  # type: OutputProcessor
                signature  # type: DoFnSignature
               ):
     # type: (...) -> None
@@ -359,7 +359,7 @@
   @staticmethod
   def create_invoker(
       signature,  # type: DoFnSignature
-      output_processor=None,  # type: Optional[_OutputProcessor]
+      output_processor,  # type: OutputProcessor
       context=None,  # type: Optional[DoFnContext]
       side_inputs=None,   # type: Optional[List[sideinputs.SideInputMap]]
       input_args=None, input_kwargs=None,
@@ -410,7 +410,6 @@
   def invoke_process(self,
                      windowed_value,  # type: WindowedValue
                      restriction_tracker=None,  # type: Optional[iobase.RestrictionTracker]
-                     output_processor=None,  # type: Optional[OutputProcessor]
                      additional_args=None,
                      additional_kwargs=None
                     ):
@@ -481,7 +480,7 @@
   """An invoker that processes elements ignoring windowing information."""
 
   def __init__(self,
-               output_processor,  # type: Optional[_OutputProcessor]
+               output_processor,  # type: OutputProcessor
                signature  # type: DoFnSignature
               ):
     # type: (...) -> None
@@ -491,15 +490,11 @@
   def invoke_process(self,
                      windowed_value,  # type: WindowedValue
                      restriction_tracker=None,  # type: Optional[iobase.RestrictionTracker]
-                     output_processor=None,  # type: Optional[OutputProcessor]
                      additional_args=None,
                      additional_kwargs=None
                     ):
     # type: (...) -> None
-    if not output_processor:
-      output_processor = self.output_processor
-    # self.output_processor is Optional, but in practice it won't be None here
-    output_processor.process_outputs(  # type: ignore[union-attr]
+    self.output_processor.process_outputs(
         windowed_value, self.process_method(windowed_value.value))
 
 
@@ -507,7 +502,7 @@
   """An invoker that processes elements considering windowing information."""
 
   def __init__(self,
-               output_processor,  # type: Optional[_OutputProcessor]
+               output_processor,  # type: OutputProcessor
                signature,  # type: DoFnSignature
                context,  # type: DoFnContext
                side_inputs,  # type: Iterable[sideinputs.SideInputMap]
@@ -614,7 +609,6 @@
   def invoke_process(self,
                      windowed_value,  # type: WindowedValue
                      restriction_tracker=None,
-                     output_processor=None,  # type: Optional[OutputProcessor]
                      additional_args=None,
                      additional_kwargs=None
                     ):
@@ -624,9 +618,6 @@
     if not additional_kwargs:
       additional_kwargs = {}
 
-    if not output_processor:
-      assert self.output_processor is not None
-      output_processor = self.output_processor
     self.context.set_element(windowed_value)
     # Call for the process function for each window if has windowed side inputs
     # or if the process accesses the window parameter. We can just call it once
@@ -662,8 +653,7 @@
       try:
         self.current_windowed_value = windowed_value
         return self._invoke_process_per_window(
-            windowed_value, additional_args, additional_kwargs,
-            output_processor)
+            windowed_value, additional_args, additional_kwargs)
       finally:
         self.threadsafe_restriction_tracker = None
         self.current_windowed_value = windowed_value
@@ -672,17 +662,16 @@
       for w in windowed_value.windows:
         self._invoke_process_per_window(
             WindowedValue(windowed_value.value, windowed_value.timestamp, (w,)),
-            additional_args, additional_kwargs, output_processor)
+            additional_args, additional_kwargs)
     else:
       self._invoke_process_per_window(
-          windowed_value, additional_args, additional_kwargs, output_processor)
+          windowed_value, additional_args, additional_kwargs)
     return None
 
   def _invoke_process_per_window(self,
                                  windowed_value,  # type: WindowedValue
                                  additional_args,
                                  additional_kwargs,
-                                 output_processor  # type: OutputProcessor
                                 ):
     # type: (...) -> Optional[Tuple[WindowedValue, Timestamp]]
     if self.has_windowed_inputs:
@@ -751,11 +740,11 @@
           kwargs_for_process[key] = additional_kwargs[key]
 
     if kwargs_for_process:
-      output_processor.process_outputs(
+      self.output_processor.process_outputs(
           windowed_value,
           self.process_method(*args_for_process, **kwargs_for_process))
     else:
-      output_processor.process_outputs(
+      self.output_processor.process_outputs(
           windowed_value, self.process_method(*args_for_process))
 
     if self.is_splittable:
diff --git a/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py b/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py
index db35635..69b1fb8 100644
--- a/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py
+++ b/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py
@@ -500,7 +500,9 @@
     test_options = options.view_as(TestOptions)
     # If it is a dry run, return without submitting the job.
     if test_options.dry_run:
-      return None
+      result = PipelineResult(PipelineState.DONE)
+      result.wait_until_finish = lambda duration=None: None
+      return result
 
     # Get a Dataflow API client and set its options
     self.dataflow_client = apiclient.DataflowApplicationClient(options)
diff --git a/sdks/python/apache_beam/runners/dataflow/dataflow_runner_test.py b/sdks/python/apache_beam/runners/dataflow/dataflow_runner_test.py
index 74d6e57..d00066c 100644
--- a/sdks/python/apache_beam/runners/dataflow/dataflow_runner_test.py
+++ b/sdks/python/apache_beam/runners/dataflow/dataflow_runner_test.py
@@ -210,12 +210,12 @@
     self.default_properties.append('--experiments=beam_fn_api')
     self.default_properties.append('--worker_harness_container_image=FOO')
     remote_runner = DataflowRunner()
-    p = Pipeline(remote_runner,
-                 options=PipelineOptions(self.default_properties))
-    (p | ptransform.Create([1, 2, 3])  # pylint: disable=expression-not-assigned
-     | 'Do' >> ptransform.FlatMap(lambda x: [(x, x)])
-     | ptransform.GroupByKey())
-    p.run()
+    with Pipeline(
+        remote_runner,
+        options=PipelineOptions(self.default_properties)) as p:
+      (p | ptransform.Create([1, 2, 3])  # pylint: disable=expression-not-assigned
+       | 'Do' >> ptransform.FlatMap(lambda x: [(x, x)])
+       | ptransform.GroupByKey())
     self.assertEqual(
         list(remote_runner.proto_pipeline.components.environments.values()),
         [beam_runner_api_pb2.Environment(
@@ -225,20 +225,19 @@
 
   def test_remote_runner_translation(self):
     remote_runner = DataflowRunner()
-    p = Pipeline(remote_runner,
-                 options=PipelineOptions(self.default_properties))
+    with Pipeline(
+        remote_runner,
+        options=PipelineOptions(self.default_properties)) as p:
 
-    (p | ptransform.Create([1, 2, 3])  # pylint: disable=expression-not-assigned
-     | 'Do' >> ptransform.FlatMap(lambda x: [(x, x)])
-     | ptransform.GroupByKey())
-    p.run()
+      (p | ptransform.Create([1, 2, 3])  # pylint: disable=expression-not-assigned
+       | 'Do' >> ptransform.FlatMap(lambda x: [(x, x)])
+       | ptransform.GroupByKey())
 
   def test_streaming_create_translation(self):
     remote_runner = DataflowRunner()
     self.default_properties.append("--streaming")
-    p = Pipeline(remote_runner, PipelineOptions(self.default_properties))
-    p | ptransform.Create([1])  # pylint: disable=expression-not-assigned
-    p.run()
+    with Pipeline(remote_runner, PipelineOptions(self.default_properties)) as p:
+      p | ptransform.Create([1])  # pylint: disable=expression-not-assigned
     job_dict = json.loads(str(remote_runner.job))
     self.assertEqual(len(job_dict[u'steps']), 3)
 
@@ -252,11 +251,12 @@
   def test_biqquery_read_streaming_fail(self):
     remote_runner = DataflowRunner()
     self.default_properties.append("--streaming")
-    p = Pipeline(remote_runner, PipelineOptions(self.default_properties))
-    _ = p | beam.io.Read(beam.io.BigQuerySource('some.table'))
     with self.assertRaisesRegex(ValueError,
                                 r'source is not currently available'):
-      p.run()
+      with Pipeline(
+          remote_runner,
+          PipelineOptions(self.default_properties)) as p:
+        _ = p | beam.io.Read(beam.io.BigQuerySource('some.table'))
 
   # TODO(BEAM-8095): Segfaults in Python 3.7 with xdist.
   @pytest.mark.no_xdist
@@ -422,9 +422,8 @@
     remote_runner = DataflowRunner()
     self.default_properties.append('--min_cpu_platform=Intel Haswell')
 
-    p = Pipeline(remote_runner, PipelineOptions(self.default_properties))
-    p | ptransform.Create([1])  # pylint: disable=expression-not-assigned
-    p.run()
+    with Pipeline(remote_runner, PipelineOptions(self.default_properties)) as p:
+      p | ptransform.Create([1])  # pylint: disable=expression-not-assigned
     self.assertIn('min_cpu_platform=Intel Haswell',
                   remote_runner.job.options.view_as(DebugOptions).experiments)
 
@@ -434,9 +433,8 @@
     self.default_properties.append('--enable_streaming_engine')
     self.default_properties.append('--experiment=some_other_experiment')
 
-    p = Pipeline(remote_runner, PipelineOptions(self.default_properties))
-    p | ptransform.Create([1])  # pylint: disable=expression-not-assigned
-    p.run()
+    with Pipeline(remote_runner, PipelineOptions(self.default_properties)) as p:
+      p | ptransform.Create([1])  # pylint: disable=expression-not-assigned
 
     experiments_for_job = (
         remote_runner.job.options.view_as(DebugOptions).experiments)
@@ -449,9 +447,8 @@
     self.default_properties.append('--experiment=some_other_experiment')
     self.default_properties.append('--dataflow_worker_jar=test.jar')
 
-    p = Pipeline(remote_runner, PipelineOptions(self.default_properties))
-    p | ptransform.Create([1])  # pylint: disable=expression-not-assigned
-    p.run()
+    with Pipeline(remote_runner, PipelineOptions(self.default_properties)) as p:
+      p | ptransform.Create([1])  # pylint: disable=expression-not-assigned
 
     experiments_for_job = (
         remote_runner.job.options.view_as(DebugOptions).experiments)
@@ -463,9 +460,8 @@
     self.default_properties.append('--experiment=beam_fn_api')
     self.default_properties.append('--dataflow_worker_jar=test.jar')
 
-    p = Pipeline(remote_runner, PipelineOptions(self.default_properties))
-    p | ptransform.Create([1])  # pylint: disable=expression-not-assigned
-    p.run()
+    with Pipeline(remote_runner, PipelineOptions(self.default_properties)) as p:
+      p | ptransform.Create([1])  # pylint: disable=expression-not-assigned
 
     experiments_for_job = (
         remote_runner.job.options.view_as(DebugOptions).experiments)
@@ -475,9 +471,8 @@
   def test_use_fastavro_experiment_is_added_on_py3_and_onwards(self):
     remote_runner = DataflowRunner()
 
-    p = Pipeline(remote_runner, PipelineOptions(self.default_properties))
-    p | ptransform.Create([1])  # pylint: disable=expression-not-assigned
-    p.run()
+    with Pipeline(remote_runner, PipelineOptions(self.default_properties)) as p:
+      p | ptransform.Create([1])  # pylint: disable=expression-not-assigned
 
     self.assertEqual(
         sys.version_info[0] > 2,
@@ -488,9 +483,8 @@
     remote_runner = DataflowRunner()
     self.default_properties.append('--experiment=use_avro')
 
-    p = Pipeline(remote_runner, PipelineOptions(self.default_properties))
-    p | ptransform.Create([1])  # pylint: disable=expression-not-assigned
-    p.run()
+    with Pipeline(remote_runner, PipelineOptions(self.default_properties)) as p:
+      p | ptransform.Create([1])  # pylint: disable=expression-not-assigned
 
     debug_options = remote_runner.job.options.view_as(DebugOptions)
 
diff --git a/sdks/python/apache_beam/runners/dataflow/native_io/iobase_test.py b/sdks/python/apache_beam/runners/dataflow/native_io/iobase_test.py
index f3f2e75..ec69ee1 100644
--- a/sdks/python/apache_beam/runners/dataflow/native_io/iobase_test.py
+++ b/sdks/python/apache_beam/runners/dataflow/native_io/iobase_test.py
@@ -185,10 +185,9 @@
       def Write(self, value):
         self.written_values.append(value)
 
-    p = TestPipeline()
-    sink = FakeSink()
-    p | Create(['a', 'b', 'c']) | _NativeWrite(sink)  # pylint: disable=expression-not-assigned
-    p.run()
+    with TestPipeline() as p:
+      sink = FakeSink()
+      p | Create(['a', 'b', 'c']) | _NativeWrite(sink)  # pylint: disable=expression-not-assigned
 
     self.assertEqual(['a', 'b', 'c'], sorted(sink.written_values))
 
diff --git a/sdks/python/apache_beam/runners/dataflow/template_runner_test.py b/sdks/python/apache_beam/runners/dataflow/template_runner_test.py
index e6d0d66..9edb2db 100644
--- a/sdks/python/apache_beam/runners/dataflow/template_runner_test.py
+++ b/sdks/python/apache_beam/runners/dataflow/template_runner_test.py
@@ -54,19 +54,19 @@
     dummy_dir = tempfile.mkdtemp()
 
     remote_runner = DataflowRunner()
-    pipeline = Pipeline(remote_runner,
-                        options=PipelineOptions([
-                            '--dataflow_endpoint=ignored',
-                            '--sdk_location=' + dummy_file_name,
-                            '--job_name=test-job',
-                            '--project=test-project',
-                            '--staging_location=' + dummy_dir,
-                            '--temp_location=/dev/null',
-                            '--template_location=' + dummy_file_name,
-                            '--no_auth']))
+    with Pipeline(remote_runner,
+                  options=PipelineOptions([
+                      '--dataflow_endpoint=ignored',
+                      '--sdk_location=' + dummy_file_name,
+                      '--job_name=test-job',
+                      '--project=test-project',
+                      '--staging_location=' + dummy_dir,
+                      '--temp_location=/dev/null',
+                      '--template_location=' + dummy_file_name,
+                      '--no_auth'])) as pipeline:
 
-    pipeline | beam.Create([1, 2, 3]) | beam.Map(lambda x: x) # pylint: disable=expression-not-assigned
-    pipeline.run().wait_until_finish()
+      pipeline | beam.Create([1, 2, 3]) | beam.Map(lambda x: x) # pylint: disable=expression-not-assigned
+
     with open(dummy_file_name) as template_file:
       saved_job_dict = json.load(template_file)
       self.assertEqual(
diff --git a/sdks/python/apache_beam/runners/direct/direct_runner.py b/sdks/python/apache_beam/runners/direct/direct_runner.py
index d584f6b..b6902ff 100644
--- a/sdks/python/apache_beam/runners/direct/direct_runner.py
+++ b/sdks/python/apache_beam/runners/direct/direct_runner.py
@@ -118,6 +118,10 @@
 
     return outputs
 
+  # We must mark this method as not a test or else its name is a matcher for
+  # nosetest tests.
+  apply_TestStream.__test__ = False
+
   def run_pipeline(self, pipeline, options):
 
     from apache_beam.pipeline import PipelineVisitor
diff --git a/sdks/python/apache_beam/runners/direct/sdf_direct_runner.py b/sdks/python/apache_beam/runners/direct/sdf_direct_runner.py
index e7234de..7659a65 100644
--- a/sdks/python/apache_beam/runners/direct/sdf_direct_runner.py
+++ b/sdks/python/apache_beam/runners/direct/sdf_direct_runner.py
@@ -261,9 +261,11 @@
     self._restriction_tag = _ValueStateTag('restriction')
     self.watermark_hold_tag = _ValueStateTag('watermark_hold')
     self._process_element_invoker = None
+    self._output_processor = _OutputProcessor()
 
     self.sdf_invoker = DoFnInvoker.create_invoker(
         DoFnSignature(self.sdf), context=DoFnContext('unused_context'),
+        output_processor=self._output_processor,
         input_args=args_for_invoker, input_kwargs=kwargs_for_invoker)
 
     self._step_context = None
@@ -329,7 +331,8 @@
                       SDFProcessElementInvoker)
 
     output_values = self._process_element_invoker.invoke_process_element(
-        self.sdf_invoker, windowed_element, tracker, *args, **kwargs)
+        self.sdf_invoker, self._output_processor, windowed_element, tracker,
+        *args, **kwargs)
 
     sdf_result = None
     for output in output_values:
@@ -424,7 +427,7 @@
     raise ValueError
 
   def invoke_process_element(
-      self, sdf_invoker, element, tracker, *args, **kwargs):
+      self, sdf_invoker, output_processor, element, tracker, *args, **kwargs):
     """Invokes `process()` method of a Splittable `DoFn` for a given element.
 
      Args:
@@ -453,10 +456,10 @@
       checkpoint_state.residual_restriction = tracker.checkpoint()
       checkpoint_state.checkpointed = object()
 
-    output_processor = _OutputProcessor()
+    output_processor.reset()
     Timer(self._max_duration, initiate_checkpoint).start()
     sdf_invoker.invoke_process(
-        element, restriction_tracker=tracker, output_processor=output_processor,
+        element, restriction_tracker=tracker,
         additional_args=args, additional_kwargs=kwargs)
 
     assert output_processor.output_iter is not None
@@ -505,3 +508,6 @@
   def process_outputs(self, windowed_input_element, output_iter):
     # type: (WindowedValue, Iterable[Any]) -> None
     self.output_iter = output_iter
+
+  def reset(self):
+    self.output_iter = None
diff --git a/sdks/python/apache_beam/runners/interactive/interactive_beam.py b/sdks/python/apache_beam/runners/interactive/interactive_beam.py
index 3769dd7..d35c1d8 100644
--- a/sdks/python/apache_beam/runners/interactive/interactive_beam.py
+++ b/sdks/python/apache_beam/runners/interactive/interactive_beam.py
@@ -65,10 +65,9 @@
 
       class Foo(object)
         def run_pipeline(self):
-          p = beam.Pipeline()
-          init_pcoll = p |  'Init Create' >> beam.Create(range(10))
-          watch(locals())
-          p.run()
+          with beam.Pipeline() as p:
+            init_pcoll = p |  'Init Create' >> beam.Create(range(10))
+            watch(locals())
           return init_pcoll
       init_pcoll = Foo().run_pipeline()
 
diff --git a/sdks/python/apache_beam/runners/portability/fn_api_runner_test.py b/sdks/python/apache_beam/runners/portability/fn_api_runner_test.py
index 8125a5a..e82f31c 100644
--- a/sdks/python/apache_beam/runners/portability/fn_api_runner_test.py
+++ b/sdks/python/apache_beam/runners/portability/fn_api_runner_test.py
@@ -566,6 +566,12 @@
              p | 'd' >> beam.Create(additional)) | beam.Flatten()
       assert_that(res, equal_to(['a', 'b', 'c'] + additional))
 
+  def test_flatten_same_pcollections(self, with_transcoding=True):
+    with self.create_pipeline() as p:
+      pc = p | beam.Create(['a', 'b'])
+      assert_that((pc, pc, pc) | beam.Flatten(), equal_to(['a', 'b'] * 3))
+
+
   def test_combine_per_key(self):
     with self.create_pipeline() as p:
       res = (p
diff --git a/sdks/python/apache_beam/testing/test_pipeline.py b/sdks/python/apache_beam/testing/test_pipeline.py
index a34af95..0ca81ec 100644
--- a/sdks/python/apache_beam/testing/test_pipeline.py
+++ b/sdks/python/apache_beam/testing/test_pipeline.py
@@ -23,8 +23,7 @@
 
 import argparse
 import shlex
-
-from nose.plugins.skip import SkipTest
+from unittest import SkipTest
 
 from apache_beam.internal import pickler
 from apache_beam.options.pipeline_options import PipelineOptions
@@ -58,10 +57,9 @@
 
   For example, use assert_that for test validation::
 
-    pipeline = TestPipeline()
-    pcoll = ...
-    assert_that(pcoll, equal_to(...))
-    pipeline.run()
+    with TestPipeline() as pipeline:
+      pcoll = ...
+      assert_that(pcoll, equal_to(...))
   """
 
   def __init__(self,
diff --git a/sdks/python/apache_beam/testing/test_pipeline_test.py b/sdks/python/apache_beam/testing/test_pipeline_test.py
index 8cd4c88..59779cd 100644
--- a/sdks/python/apache_beam/testing/test_pipeline_test.py
+++ b/sdks/python/apache_beam/testing/test_pipeline_test.py
@@ -106,16 +106,16 @@
     self.assertEqual(test_pipeline.get_option(name), value)
 
   def test_skip_IT(self):
-    test_pipeline = TestPipeline(is_integration_test=True)
-    test_pipeline.run()
-    # Note that this will never be reached since it should be skipped above.
+    with TestPipeline(is_integration_test=True) as _:
+      # Note that this will never be reached since it should be skipped above.
+      pass
     self.fail()
 
   @mock.patch('apache_beam.testing.test_pipeline.Pipeline.run', autospec=True)
   def test_not_use_test_runner_api(self, mock_run):
-    test_pipeline = TestPipeline(argv=['--not-use-test-runner-api'],
-                                 blocking=False)
-    test_pipeline.run()
+    with TestPipeline(argv=['--not-use-test-runner-api'],
+                      blocking=False) as test_pipeline:
+      pass
     mock_run.assert_called_once_with(test_pipeline, test_runner_api=False)
 
 
diff --git a/sdks/python/apache_beam/testing/test_stream_test.py b/sdks/python/apache_beam/testing/test_stream_test.py
index 0aefbcb..ba599bd 100644
--- a/sdks/python/apache_beam/testing/test_stream_test.py
+++ b/sdks/python/apache_beam/testing/test_stream_test.py
@@ -114,20 +114,19 @@
 
     options = PipelineOptions()
     options.view_as(StandardOptions).streaming = True
-    p = TestPipeline(options=options)
-    my_record_fn = RecordFn()
-    records = p | test_stream | beam.ParDo(my_record_fn)
+    with TestPipeline(options=options) as p:
+      my_record_fn = RecordFn()
+      records = p | test_stream | beam.ParDo(my_record_fn)
 
-    assert_that(records, equal_to([
-        ('a', timestamp.Timestamp(10)),
-        ('b', timestamp.Timestamp(10)),
-        ('c', timestamp.Timestamp(10)),
-        ('d', timestamp.Timestamp(20)),
-        ('e', timestamp.Timestamp(20)),
-        ('late', timestamp.Timestamp(12)),
-        ('last', timestamp.Timestamp(310)),]))
+      assert_that(records, equal_to([
+          ('a', timestamp.Timestamp(10)),
+          ('b', timestamp.Timestamp(10)),
+          ('c', timestamp.Timestamp(10)),
+          ('d', timestamp.Timestamp(20)),
+          ('e', timestamp.Timestamp(20)),
+          ('late', timestamp.Timestamp(12)),
+          ('last', timestamp.Timestamp(310)),]))
 
-    p.run()
 
   def test_multiple_outputs(self):
     """Tests that the TestStream supports emitting to multiple PCollections."""
@@ -418,33 +417,31 @@
   def test_basic_execution_sideinputs(self):
     options = PipelineOptions()
     options.view_as(StandardOptions).streaming = True
-    p = TestPipeline(options=options)
+    with TestPipeline(options=options) as p:
 
-    main_stream = (p
-                   | 'main TestStream' >> TestStream()
-                   .advance_watermark_to(10)
-                   .add_elements(['e']))
-    side_stream = (p
-                   | 'side TestStream' >> TestStream()
-                   .add_elements([window.TimestampedValue(2, 2)])
-                   .add_elements([window.TimestampedValue(1, 1)])
-                   .add_elements([window.TimestampedValue(7, 7)])
-                   .add_elements([window.TimestampedValue(4, 4)])
-                  )
+      main_stream = (p
+                     | 'main TestStream' >> TestStream()
+                     .advance_watermark_to(10)
+                     .add_elements(['e']))
+      side_stream = (p
+                     | 'side TestStream' >> TestStream()
+                     .add_elements([window.TimestampedValue(2, 2)])
+                     .add_elements([window.TimestampedValue(1, 1)])
+                     .add_elements([window.TimestampedValue(7, 7)])
+                     .add_elements([window.TimestampedValue(4, 4)])
+                    )
 
-    class RecordFn(beam.DoFn):
-      def process(self,
-                  elm=beam.DoFn.ElementParam,
-                  ts=beam.DoFn.TimestampParam,
-                  side=beam.DoFn.SideInputParam):
-        yield (elm, ts, side)
+      class RecordFn(beam.DoFn):
+        def process(self,
+                    elm=beam.DoFn.ElementParam,
+                    ts=beam.DoFn.TimestampParam,
+                    side=beam.DoFn.SideInputParam):
+          yield (elm, ts, side)
 
-    records = (main_stream        # pylint: disable=unused-variable
-               | beam.ParDo(RecordFn(), beam.pvalue.AsList(side_stream)))
+      records = (main_stream        # pylint: disable=unused-variable
+                 | beam.ParDo(RecordFn(), beam.pvalue.AsList(side_stream)))
 
-    assert_that(records, equal_to([('e', Timestamp(10), [2, 1, 7, 4])]))
-
-    p.run()
+      assert_that(records, equal_to([('e', Timestamp(10), [2, 1, 7, 4])]))
 
   def test_basic_execution_batch_sideinputs_fixed_windows(self):
     options = PipelineOptions()
diff --git a/sdks/python/apache_beam/transforms/combiners_test.py b/sdks/python/apache_beam/transforms/combiners_test.py
index 1111778..412f079 100644
--- a/sdks/python/apache_beam/transforms/combiners_test.py
+++ b/sdks/python/apache_beam/transforms/combiners_test.py
@@ -95,86 +95,83 @@
 class CombineTest(unittest.TestCase):
 
   def test_builtin_combines(self):
-    pipeline = TestPipeline()
+    with TestPipeline() as pipeline:
 
-    vals = [6, 3, 1, 1, 9, 1, 5, 2, 0, 6]
-    mean = sum(vals) / float(len(vals))
-    size = len(vals)
+      vals = [6, 3, 1, 1, 9, 1, 5, 2, 0, 6]
+      mean = sum(vals) / float(len(vals))
+      size = len(vals)
 
-    # First for global combines.
-    pcoll = pipeline | 'start' >> Create(vals)
-    result_mean = pcoll | 'mean' >> combine.Mean.Globally()
-    result_count = pcoll | 'count' >> combine.Count.Globally()
-    assert_that(result_mean, equal_to([mean]), label='assert:mean')
-    assert_that(result_count, equal_to([size]), label='assert:size')
+      # First for global combines.
+      pcoll = pipeline | 'start' >> Create(vals)
+      result_mean = pcoll | 'mean' >> combine.Mean.Globally()
+      result_count = pcoll | 'count' >> combine.Count.Globally()
+      assert_that(result_mean, equal_to([mean]), label='assert:mean')
+      assert_that(result_count, equal_to([size]), label='assert:size')
 
-    # Again for per-key combines.
-    pcoll = pipeline | 'start-perkey' >> Create([('a', x) for x in vals])
-    result_key_mean = pcoll | 'mean-perkey' >> combine.Mean.PerKey()
-    result_key_count = pcoll | 'count-perkey' >> combine.Count.PerKey()
-    assert_that(result_key_mean, equal_to([('a', mean)]), label='key:mean')
-    assert_that(result_key_count, equal_to([('a', size)]), label='key:size')
-    pipeline.run()
+      # Again for per-key combines.
+      pcoll = pipeline | 'start-perkey' >> Create([('a', x) for x in vals])
+      result_key_mean = pcoll | 'mean-perkey' >> combine.Mean.PerKey()
+      result_key_count = pcoll | 'count-perkey' >> combine.Count.PerKey()
+      assert_that(result_key_mean, equal_to([('a', mean)]), label='key:mean')
+      assert_that(result_key_count, equal_to([('a', size)]), label='key:size')
 
   def test_top(self):
-    pipeline = TestPipeline()
+    with TestPipeline() as pipeline:
 
-    # First for global combines.
-    pcoll = pipeline | 'start' >> Create([6, 3, 1, 1, 9, 1, 5, 2, 0, 6])
-    result_top = pcoll | 'top' >> combine.Top.Largest(5)
-    result_bot = pcoll | 'bot' >> combine.Top.Smallest(4)
-    assert_that(result_top, equal_to([[9, 6, 6, 5, 3]]), label='assert:top')
-    assert_that(result_bot, equal_to([[0, 1, 1, 1]]), label='assert:bot')
+      # First for global combines.
+      pcoll = pipeline | 'start' >> Create([6, 3, 1, 1, 9, 1, 5, 2, 0, 6])
+      result_top = pcoll | 'top' >> combine.Top.Largest(5)
+      result_bot = pcoll | 'bot' >> combine.Top.Smallest(4)
+      assert_that(result_top, equal_to([[9, 6, 6, 5, 3]]), label='assert:top')
+      assert_that(result_bot, equal_to([[0, 1, 1, 1]]), label='assert:bot')
 
-    # Again for per-key combines.
-    pcoll = pipeline | 'start-perkey' >> Create(
-        [('a', x) for x in [6, 3, 1, 1, 9, 1, 5, 2, 0, 6]])
-    result_key_top = pcoll | 'top-perkey' >> combine.Top.LargestPerKey(5)
-    result_key_bot = pcoll | 'bot-perkey' >> combine.Top.SmallestPerKey(4)
-    assert_that(result_key_top, equal_to([('a', [9, 6, 6, 5, 3])]),
-                label='key:top')
-    assert_that(result_key_bot, equal_to([('a', [0, 1, 1, 1])]),
-                label='key:bot')
-    pipeline.run()
+      # Again for per-key combines.
+      pcoll = pipeline | 'start-perkey' >> Create(
+          [('a', x) for x in [6, 3, 1, 1, 9, 1, 5, 2, 0, 6]])
+      result_key_top = pcoll | 'top-perkey' >> combine.Top.LargestPerKey(5)
+      result_key_bot = pcoll | 'bot-perkey' >> combine.Top.SmallestPerKey(4)
+      assert_that(result_key_top, equal_to([('a', [9, 6, 6, 5, 3])]),
+                  label='key:top')
+      assert_that(result_key_bot, equal_to([('a', [0, 1, 1, 1])]),
+                  label='key:bot')
 
   @unittest.skipIf(sys.version_info[0] > 2, 'deprecated comparator')
   def test_top_py2(self):
-    pipeline = TestPipeline()
+    with TestPipeline() as pipeline:
 
-    # A parameter we'll be sharing with a custom comparator.
-    names = {0: 'zo',
-             1: 'one',
-             2: 'twoo',
-             3: 'three',
-             5: 'fiiive',
-             6: 'sssssix',
-             9: 'nniiinne'}
+      # A parameter we'll be sharing with a custom comparator.
+      names = {0: 'zo',
+               1: 'one',
+               2: 'twoo',
+               3: 'three',
+               5: 'fiiive',
+               6: 'sssssix',
+               9: 'nniiinne'}
 
-    # First for global combines.
-    pcoll = pipeline | 'start' >> Create([6, 3, 1, 1, 9, 1, 5, 2, 0, 6])
+      # First for global combines.
+      pcoll = pipeline | 'start' >> Create([6, 3, 1, 1, 9, 1, 5, 2, 0, 6])
 
-    result_cmp = pcoll | 'cmp' >> combine.Top.Of(
-        6,
-        lambda a, b, names: len(names[a]) < len(names[b]),
-        names)  # Note parameter passed to comparator.
-    result_cmp_rev = pcoll | 'cmp_rev' >> combine.Top.Of(
-        3,
-        lambda a, b, names: len(names[a]) < len(names[b]),
-        names,  # Note parameter passed to comparator.
-        reverse=True)
-    assert_that(result_cmp, equal_to([[9, 6, 6, 5, 3, 2]]), label='assert:cmp')
-    assert_that(result_cmp_rev, equal_to([[0, 1, 1]]), label='assert:cmp_rev')
+      result_cmp = pcoll | 'cmp' >> combine.Top.Of(
+          6,
+          lambda a, b, names: len(names[a]) < len(names[b]),
+          names)  # Note parameter passed to comparator.
+      result_cmp_rev = pcoll | 'cmp_rev' >> combine.Top.Of(
+          3,
+          lambda a, b, names: len(names[a]) < len(names[b]),
+          names,  # Note parameter passed to comparator.
+          reverse=True)
+      assert_that(result_cmp, equal_to([[9, 6, 6, 5, 3, 2]]), label='CheckCmp')
+      assert_that(result_cmp_rev, equal_to([[0, 1, 1]]), label='CheckCmpRev')
 
-    # Again for per-key combines.
-    pcoll = pipeline | 'start-perkye' >> Create(
-        [('a', x) for x in [6, 3, 1, 1, 9, 1, 5, 2, 0, 6]])
-    result_key_cmp = pcoll | 'cmp-perkey' >> combine.Top.PerKey(
-        6,
-        lambda a, b, names: len(names[a]) < len(names[b]),
-        names)  # Note parameter passed to comparator.
-    assert_that(result_key_cmp, equal_to([('a', [9, 6, 6, 5, 3, 2])]),
-                label='key:cmp')
-    pipeline.run()
+      # Again for per-key combines.
+      pcoll = pipeline | 'start-perkye' >> Create(
+          [('a', x) for x in [6, 3, 1, 1, 9, 1, 5, 2, 0, 6]])
+      result_key_cmp = pcoll | 'cmp-perkey' >> combine.Top.PerKey(
+          6,
+          lambda a, b, names: len(names[a]) < len(names[b]),
+          names)  # Note parameter passed to comparator.
+      assert_that(result_key_cmp, equal_to([('a', [9, 6, 6, 5, 3, 2])]),
+                  label='key:cmp')
 
   def test_empty_global_top(self):
     with TestPipeline() as p:
@@ -185,12 +182,11 @@
     elements = list(range(100))
     random.shuffle(elements)
 
-    pipeline = TestPipeline()
-    shards = [pipeline | 'Shard%s' % shard >> beam.Create(elements[shard::7])
-              for shard in range(7)]
-    assert_that(shards | beam.Flatten() | combine.Top.Largest(10),
-                equal_to([[99, 98, 97, 96, 95, 94, 93, 92, 91, 90]]))
-    pipeline.run()
+    with TestPipeline() as pipeline:
+      shards = [pipeline | 'Shard%s' % shard >> beam.Create(elements[shard::7])
+                for shard in range(7)]
+      assert_that(shards | beam.Flatten() | combine.Top.Largest(10),
+                  equal_to([[99, 98, 97, 96, 95, 94, 93, 92, 91, 90]]))
 
   def test_top_key(self):
     self.assertEqual(
@@ -272,22 +268,22 @@
     hc.assert_that(dd.items, hc.contains_inanyorder(*expected_items))
 
   def test_top_shorthands(self):
-    pipeline = TestPipeline()
+    with TestPipeline() as pipeline:
 
-    pcoll = pipeline | 'start' >> Create([6, 3, 1, 1, 9, 1, 5, 2, 0, 6])
-    result_top = pcoll | 'top' >> beam.CombineGlobally(combine.Largest(5))
-    result_bot = pcoll | 'bot' >> beam.CombineGlobally(combine.Smallest(4))
-    assert_that(result_top, equal_to([[9, 6, 6, 5, 3]]), label='assert:top')
-    assert_that(result_bot, equal_to([[0, 1, 1, 1]]), label='assert:bot')
+      pcoll = pipeline | 'start' >> Create([6, 3, 1, 1, 9, 1, 5, 2, 0, 6])
+      result_top = pcoll | 'top' >> beam.CombineGlobally(combine.Largest(5))
+      result_bot = pcoll | 'bot' >> beam.CombineGlobally(combine.Smallest(4))
+      assert_that(result_top, equal_to([[9, 6, 6, 5, 3]]), label='assert:top')
+      assert_that(result_bot, equal_to([[0, 1, 1, 1]]), label='assert:bot')
 
-    pcoll = pipeline | 'start-perkey' >> Create(
-        [('a', x) for x in [6, 3, 1, 1, 9, 1, 5, 2, 0, 6]])
-    result_ktop = pcoll | 'top-perkey' >> beam.CombinePerKey(combine.Largest(5))
-    result_kbot = pcoll | 'bot-perkey' >> beam.CombinePerKey(
-        combine.Smallest(4))
-    assert_that(result_ktop, equal_to([('a', [9, 6, 6, 5, 3])]), label='k:top')
-    assert_that(result_kbot, equal_to([('a', [0, 1, 1, 1])]), label='k:bot')
-    pipeline.run()
+      pcoll = pipeline | 'start-perkey' >> Create(
+          [('a', x) for x in [6, 3, 1, 1, 9, 1, 5, 2, 0, 6]])
+      result_ktop = pcoll | 'top-perkey' >> beam.CombinePerKey(
+          combine.Largest(5))
+      result_kbot = pcoll | 'bot-perkey' >> beam.CombinePerKey(
+          combine.Smallest(4))
+      assert_that(result_ktop, equal_to([('a', [9, 6, 6, 5, 3])]), label='ktop')
+      assert_that(result_kbot, equal_to([('a', [0, 1, 1, 1])]), label='kbot')
 
   def test_top_no_compact(self):
 
@@ -296,24 +292,23 @@
       def compact(self, accumulator):
         return accumulator
 
-    pipeline = TestPipeline()
-    pcoll = pipeline | 'Start' >> Create([6, 3, 1, 1, 9, 1, 5, 2, 0, 6])
-    result_top = pcoll | 'Top' >> beam.CombineGlobally(
-        TopCombineFnNoCompact(5, key=lambda x: x))
-    result_bot = pcoll | 'Bot' >> beam.CombineGlobally(
-        TopCombineFnNoCompact(4, reverse=True))
-    assert_that(result_top, equal_to([[9, 6, 6, 5, 3]]), label='Assert:Top')
-    assert_that(result_bot, equal_to([[0, 1, 1, 1]]), label='Assert:Bot')
+    with TestPipeline() as pipeline:
+      pcoll = pipeline | 'Start' >> Create([6, 3, 1, 1, 9, 1, 5, 2, 0, 6])
+      result_top = pcoll | 'Top' >> beam.CombineGlobally(
+          TopCombineFnNoCompact(5, key=lambda x: x))
+      result_bot = pcoll | 'Bot' >> beam.CombineGlobally(
+          TopCombineFnNoCompact(4, reverse=True))
+      assert_that(result_top, equal_to([[9, 6, 6, 5, 3]]), label='Assert:Top')
+      assert_that(result_bot, equal_to([[0, 1, 1, 1]]), label='Assert:Bot')
 
-    pcoll = pipeline | 'Start-Perkey' >> Create(
-        [('a', x) for x in [6, 3, 1, 1, 9, 1, 5, 2, 0, 6]])
-    result_ktop = pcoll | 'Top-PerKey' >> beam.CombinePerKey(
-        TopCombineFnNoCompact(5, key=lambda x: x))
-    result_kbot = pcoll | 'Bot-PerKey' >> beam.CombinePerKey(
-        TopCombineFnNoCompact(4, reverse=True))
-    assert_that(result_ktop, equal_to([('a', [9, 6, 6, 5, 3])]), label='K:Top')
-    assert_that(result_kbot, equal_to([('a', [0, 1, 1, 1])]), label='K:Bot')
-    pipeline.run()
+      pcoll = pipeline | 'Start-Perkey' >> Create(
+          [('a', x) for x in [6, 3, 1, 1, 9, 1, 5, 2, 0, 6]])
+      result_ktop = pcoll | 'Top-PerKey' >> beam.CombinePerKey(
+          TopCombineFnNoCompact(5, key=lambda x: x))
+      result_kbot = pcoll | 'Bot-PerKey' >> beam.CombinePerKey(
+          TopCombineFnNoCompact(4, reverse=True))
+      assert_that(result_ktop, equal_to([('a', [9, 6, 6, 5, 3])]), label='KTop')
+      assert_that(result_kbot, equal_to([('a', [0, 1, 1, 1])]), label='KBot')
 
   def test_global_sample(self):
     def is_good_sample(actual):
@@ -329,21 +324,20 @@
             label='check-%d' % ix)
 
   def test_per_key_sample(self):
-    pipeline = TestPipeline()
-    pcoll = pipeline | 'start-perkey' >> Create(
-        sum(([(i, 1), (i, 1), (i, 2), (i, 2)] for i in range(9)), []))
-    result = pcoll | 'sample' >> combine.Sample.FixedSizePerKey(3)
+    with TestPipeline() as pipeline:
+      pcoll = pipeline | 'start-perkey' >> Create(
+          sum(([(i, 1), (i, 1), (i, 2), (i, 2)] for i in range(9)), []))
+      result = pcoll | 'sample' >> combine.Sample.FixedSizePerKey(3)
 
-    def matcher():
-      def match(actual):
-        for _, samples in actual:
-          equal_to([3])([len(samples)])
-          num_ones = sum(1 for x in samples if x == 1)
-          num_twos = sum(1 for x in samples if x == 2)
-          equal_to([1, 2])([num_ones, num_twos])
-      return match
-    assert_that(result, matcher())
-    pipeline.run()
+      def matcher():
+        def match(actual):
+          for _, samples in actual:
+            equal_to([3])([len(samples)])
+            num_ones = sum(1 for x in samples if x == 1)
+            num_twos = sum(1 for x in samples if x == 2)
+            equal_to([1, 2])([num_ones, num_twos])
+        return match
+      assert_that(result, matcher())
 
   def test_tuple_combine_fn(self):
     with TestPipeline() as p:
@@ -365,30 +359,28 @@
       assert_that(result, equal_to([(1, 7.0 / 4, 3)]))
 
   def test_to_list_and_to_dict(self):
-    pipeline = TestPipeline()
-    the_list = [6, 3, 1, 1, 9, 1, 5, 2, 0, 6]
-    pcoll = pipeline | 'start' >> Create(the_list)
-    result = pcoll | 'to list' >> combine.ToList()
+    with TestPipeline() as pipeline:
+      the_list = [6, 3, 1, 1, 9, 1, 5, 2, 0, 6]
+      pcoll = pipeline | 'start' >> Create(the_list)
+      result = pcoll | 'to list' >> combine.ToList()
 
-    def matcher(expected):
-      def match(actual):
-        equal_to(expected[0])(actual[0])
-      return match
-    assert_that(result, matcher([the_list]))
-    pipeline.run()
+      def matcher(expected):
+        def match(actual):
+          equal_to(expected[0])(actual[0])
+        return match
+      assert_that(result, matcher([the_list]))
 
-    pipeline = TestPipeline()
-    pairs = [(1, 2), (3, 4), (5, 6)]
-    pcoll = pipeline | 'start-pairs' >> Create(pairs)
-    result = pcoll | 'to dict' >> combine.ToDict()
+    with TestPipeline() as pipeline:
+      pairs = [(1, 2), (3, 4), (5, 6)]
+      pcoll = pipeline | 'start-pairs' >> Create(pairs)
+      result = pcoll | 'to dict' >> combine.ToDict()
 
-    def matcher():
-      def match(actual):
-        equal_to([1])([len(actual)])
-        equal_to(pairs)(actual[0].items())
-      return match
-    assert_that(result, matcher())
-    pipeline.run()
+      def matcher():
+        def match(actual):
+          equal_to([1])([len(actual)])
+          equal_to(pairs)(actual[0].items())
+        return match
+      assert_that(result, matcher())
 
   def test_combine_globally_with_default(self):
     with TestPipeline() as p:
diff --git a/sdks/python/apache_beam/transforms/dofn_lifecycle_test.py b/sdks/python/apache_beam/transforms/dofn_lifecycle_test.py
index 0f2cc4c..bcba20d 100644
--- a/sdks/python/apache_beam/transforms/dofn_lifecycle_test.py
+++ b/sdks/python/apache_beam/transforms/dofn_lifecycle_test.py
@@ -78,12 +78,10 @@
 @attr('ValidatesRunner')
 class DoFnLifecycleTest(unittest.TestCase):
   def test_dofn_lifecycle(self):
-    p = TestPipeline()
-    _ = (p
-         | 'Start' >> beam.Create([1, 2, 3])
-         | 'Do' >> beam.ParDo(CallSequenceEnforcingDoFn()))
-    result = p.run()
-    result.wait_until_finish()
+    with TestPipeline() as p:
+      _ = (p
+           | 'Start' >> beam.Create([1, 2, 3])
+           | 'Do' >> beam.ParDo(CallSequenceEnforcingDoFn()))
     # Assumes that the worker is run in the same process as the test.
 
 
diff --git a/sdks/python/apache_beam/transforms/ptransform_test.py b/sdks/python/apache_beam/transforms/ptransform_test.py
index f85a2b9..ef3932e 100644
--- a/sdks/python/apache_beam/transforms/ptransform_test.py
+++ b/sdks/python/apache_beam/transforms/ptransform_test.py
@@ -118,11 +118,10 @@
       def process(self, element, addon):
         return [element + addon]
 
-    pipeline = TestPipeline()
-    pcoll = pipeline | 'Start' >> beam.Create([1, 2, 3])
-    result = pcoll | 'Do' >> beam.ParDo(AddNDoFn(), 10)
-    assert_that(result, equal_to([11, 12, 13]))
-    pipeline.run()
+    with TestPipeline() as pipeline:
+      pcoll = pipeline | 'Start' >> beam.Create([1, 2, 3])
+      result = pcoll | 'Do' >> beam.ParDo(AddNDoFn(), 10)
+      assert_that(result, equal_to([11, 12, 13]))
 
   def test_do_with_unconstructed_do_fn(self):
     class MyDoFn(beam.DoFn):
@@ -130,81 +129,74 @@
       def process(self):
         pass
 
-    pipeline = TestPipeline()
-    pcoll = pipeline | 'Start' >> beam.Create([1, 2, 3])
     with self.assertRaises(ValueError):
-      pcoll | 'Do' >> beam.ParDo(MyDoFn)  # Note the lack of ()'s
+      with TestPipeline() as pipeline:
+        pcoll = pipeline | 'Start' >> beam.Create([1, 2, 3])
+        pcoll | 'Do' >> beam.ParDo(MyDoFn)  # Note the lack of ()'s
 
   def test_do_with_callable(self):
-    pipeline = TestPipeline()
-    pcoll = pipeline | 'Start' >> beam.Create([1, 2, 3])
-    result = pcoll | 'Do' >> beam.FlatMap(lambda x, addon: [x + addon], 10)
-    assert_that(result, equal_to([11, 12, 13]))
-    pipeline.run()
+    with TestPipeline() as pipeline:
+      pcoll = pipeline | 'Start' >> beam.Create([1, 2, 3])
+      result = pcoll | 'Do' >> beam.FlatMap(lambda x, addon: [x + addon], 10)
+      assert_that(result, equal_to([11, 12, 13]))
 
   def test_do_with_side_input_as_arg(self):
-    pipeline = TestPipeline()
-    side = pipeline | 'Side' >> beam.Create([10])
-    pcoll = pipeline | 'Start' >> beam.Create([1, 2, 3])
-    result = pcoll | 'Do' >> beam.FlatMap(
-        lambda x, addon: [x + addon], pvalue.AsSingleton(side))
-    assert_that(result, equal_to([11, 12, 13]))
-    pipeline.run()
+    with TestPipeline() as pipeline:
+      side = pipeline | 'Side' >> beam.Create([10])
+      pcoll = pipeline | 'Start' >> beam.Create([1, 2, 3])
+      result = pcoll | 'Do' >> beam.FlatMap(
+          lambda x, addon: [x + addon], pvalue.AsSingleton(side))
+      assert_that(result, equal_to([11, 12, 13]))
 
   def test_do_with_side_input_as_keyword_arg(self):
-    pipeline = TestPipeline()
-    side = pipeline | 'Side' >> beam.Create([10])
-    pcoll = pipeline | 'Start' >> beam.Create([1, 2, 3])
-    result = pcoll | 'Do' >> beam.FlatMap(
-        lambda x, addon: [x + addon], addon=pvalue.AsSingleton(side))
-    assert_that(result, equal_to([11, 12, 13]))
-    pipeline.run()
+    with TestPipeline() as pipeline:
+      side = pipeline | 'Side' >> beam.Create([10])
+      pcoll = pipeline | 'Start' >> beam.Create([1, 2, 3])
+      result = pcoll | 'Do' >> beam.FlatMap(
+          lambda x, addon: [x + addon], addon=pvalue.AsSingleton(side))
+      assert_that(result, equal_to([11, 12, 13]))
 
   def test_do_with_do_fn_returning_string_raises_warning(self):
-    pipeline = TestPipeline()
-    pipeline._options.view_as(TypeOptions).runtime_type_check = True
-    pcoll = pipeline | 'Start' >> beam.Create(['2', '9', '3'])
-    pcoll | 'Do' >> beam.FlatMap(lambda x: x + '1')
-
-    # Since the DoFn directly returns a string we should get an error warning
-    # us.
     with self.assertRaises(typehints.TypeCheckError) as cm:
-      pipeline.run()
+      with TestPipeline() as pipeline:
+        pipeline._options.view_as(TypeOptions).runtime_type_check = True
+        pcoll = pipeline | 'Start' >> beam.Create(['2', '9', '3'])
+        pcoll | 'Do' >> beam.FlatMap(lambda x: x + '1')
+
+        # Since the DoFn directly returns a string we should get an
+        # error warning us when the pipeliene runs.
 
     expected_error_prefix = ('Returning a str from a ParDo or FlatMap '
                              'is discouraged.')
     self.assertStartswith(cm.exception.args[0], expected_error_prefix)
 
   def test_do_with_do_fn_returning_dict_raises_warning(self):
-    pipeline = TestPipeline()
-    pipeline._options.view_as(TypeOptions).runtime_type_check = True
-    pcoll = pipeline | 'Start' >> beam.Create(['2', '9', '3'])
-    pcoll | 'Do' >> beam.FlatMap(lambda x: {x: '1'})
-
-    # Since the DoFn directly returns a dict we should get an error warning
-    # us.
     with self.assertRaises(typehints.TypeCheckError) as cm:
-      pipeline.run()
+      with TestPipeline() as pipeline:
+        pipeline._options.view_as(TypeOptions).runtime_type_check = True
+        pcoll = pipeline | 'Start' >> beam.Create(['2', '9', '3'])
+        pcoll | 'Do' >> beam.FlatMap(lambda x: {x: '1'})
+
+        # Since the DoFn directly returns a dict we should get an error warning
+        # us when the pipeliene runs.
 
     expected_error_prefix = ('Returning a dict from a ParDo or FlatMap '
                              'is discouraged.')
     self.assertStartswith(cm.exception.args[0], expected_error_prefix)
 
   def test_do_with_multiple_outputs_maintains_unique_name(self):
-    pipeline = TestPipeline()
-    pcoll = pipeline | 'Start' >> beam.Create([1, 2, 3])
-    r1 = pcoll | 'A' >> beam.FlatMap(lambda x: [x + 1]).with_outputs(main='m')
-    r2 = pcoll | 'B' >> beam.FlatMap(lambda x: [x + 2]).with_outputs(main='m')
-    assert_that(r1.m, equal_to([2, 3, 4]), label='r1')
-    assert_that(r2.m, equal_to([3, 4, 5]), label='r2')
-    pipeline.run()
+    with TestPipeline() as pipeline:
+      pcoll = pipeline | 'Start' >> beam.Create([1, 2, 3])
+      r1 = pcoll | 'A' >> beam.FlatMap(lambda x: [x + 1]).with_outputs(main='m')
+      r2 = pcoll | 'B' >> beam.FlatMap(lambda x: [x + 2]).with_outputs(main='m')
+      assert_that(r1.m, equal_to([2, 3, 4]), label='r1')
+      assert_that(r2.m, equal_to([3, 4, 5]), label='r2')
 
   @attr('ValidatesRunner')
   def test_impulse(self):
-    pipeline = TestPipeline()
-    result = pipeline | beam.Impulse() | beam.Map(lambda _: 0)
-    assert_that(result, equal_to([0]))
-    pipeline.run()
+    with TestPipeline() as pipeline:
+      result = pipeline | beam.Impulse() | beam.Map(lambda _: 0)
+      assert_that(result, equal_to([0]))
 
   # TODO(BEAM-3544): Disable this test in streaming temporarily.
   # Remove sickbay-streaming tag after it's resolved.
@@ -246,14 +238,13 @@
         else:
           yield pvalue.TaggedOutput('odd', element)
 
-    pipeline = TestPipeline()
-    nums = pipeline | 'Some Numbers' >> beam.Create([1, 2, 3, 4])
-    results = nums | 'ClassifyNumbers' >> beam.ParDo(
-        SomeDoFn()).with_outputs('odd', 'even', main='main')
-    assert_that(results.main, equal_to([1, 2, 3, 4]))
-    assert_that(results.odd, equal_to([1, 3]), label='assert:odd')
-    assert_that(results.even, equal_to([2, 4]), label='assert:even')
-    pipeline.run()
+    with TestPipeline() as pipeline:
+      nums = pipeline | 'Some Numbers' >> beam.Create([1, 2, 3, 4])
+      results = nums | 'ClassifyNumbers' >> beam.ParDo(
+          SomeDoFn()).with_outputs('odd', 'even', main='main')
+      assert_that(results.main, equal_to([1, 2, 3, 4]))
+      assert_that(results.odd, equal_to([1, 3]), label='assert:odd')
+      assert_that(results.even, equal_to([2, 4]), label='assert:even')
 
   @attr('ValidatesRunner')
   def test_par_do_with_multiple_outputs_and_using_return(self):
@@ -262,55 +253,51 @@
         return [v, pvalue.TaggedOutput('even', v)]
       return [v, pvalue.TaggedOutput('odd', v)]
 
-    pipeline = TestPipeline()
-    nums = pipeline | 'Some Numbers' >> beam.Create([1, 2, 3, 4])
-    results = nums | 'ClassifyNumbers' >> beam.FlatMap(
-        some_fn).with_outputs('odd', 'even', main='main')
-    assert_that(results.main, equal_to([1, 2, 3, 4]))
-    assert_that(results.odd, equal_to([1, 3]), label='assert:odd')
-    assert_that(results.even, equal_to([2, 4]), label='assert:even')
-    pipeline.run()
+    with TestPipeline() as pipeline:
+      nums = pipeline | 'Some Numbers' >> beam.Create([1, 2, 3, 4])
+      results = nums | 'ClassifyNumbers' >> beam.FlatMap(
+          some_fn).with_outputs('odd', 'even', main='main')
+      assert_that(results.main, equal_to([1, 2, 3, 4]))
+      assert_that(results.odd, equal_to([1, 3]), label='assert:odd')
+      assert_that(results.even, equal_to([2, 4]), label='assert:even')
 
   @attr('ValidatesRunner')
   def test_undeclared_outputs(self):
-    pipeline = TestPipeline()
-    nums = pipeline | 'Some Numbers' >> beam.Create([1, 2, 3, 4])
-    results = nums | 'ClassifyNumbers' >> beam.FlatMap(
-        lambda x: [x,
-                   pvalue.TaggedOutput('even' if x % 2 == 0 else 'odd', x),
-                   pvalue.TaggedOutput('extra', x)]
-    ).with_outputs()
-    assert_that(results[None], equal_to([1, 2, 3, 4]))
-    assert_that(results.odd, equal_to([1, 3]), label='assert:odd')
-    assert_that(results.even, equal_to([2, 4]), label='assert:even')
-    pipeline.run()
+    with TestPipeline() as pipeline:
+      nums = pipeline | 'Some Numbers' >> beam.Create([1, 2, 3, 4])
+      results = nums | 'ClassifyNumbers' >> beam.FlatMap(
+          lambda x: [x,
+                     pvalue.TaggedOutput('even' if x % 2 == 0 else 'odd', x),
+                     pvalue.TaggedOutput('extra', x)]
+      ).with_outputs()
+      assert_that(results[None], equal_to([1, 2, 3, 4]))
+      assert_that(results.odd, equal_to([1, 3]), label='assert:odd')
+      assert_that(results.even, equal_to([2, 4]), label='assert:even')
 
   @attr('ValidatesRunner')
   def test_multiple_empty_outputs(self):
-    pipeline = TestPipeline()
-    nums = pipeline | 'Some Numbers' >> beam.Create([1, 3, 5])
-    results = nums | 'ClassifyNumbers' >> beam.FlatMap(
-        lambda x: [x,
-                   pvalue.TaggedOutput('even' if x % 2 == 0 else 'odd', x)]
-    ).with_outputs()
-    assert_that(results[None], equal_to([1, 3, 5]))
-    assert_that(results.odd, equal_to([1, 3, 5]), label='assert:odd')
-    assert_that(results.even, equal_to([]), label='assert:even')
-    pipeline.run()
+    with TestPipeline() as pipeline:
+      nums = pipeline | 'Some Numbers' >> beam.Create([1, 3, 5])
+      results = nums | 'ClassifyNumbers' >> beam.FlatMap(
+          lambda x: [x,
+                     pvalue.TaggedOutput('even' if x % 2 == 0 else 'odd', x)]
+      ).with_outputs()
+      assert_that(results[None], equal_to([1, 3, 5]))
+      assert_that(results.odd, equal_to([1, 3, 5]), label='assert:odd')
+      assert_that(results.even, equal_to([]), label='assert:even')
 
   def test_do_requires_do_fn_returning_iterable(self):
     # This function is incorrect because it returns an object that isn't an
     # iterable.
     def incorrect_par_do_fn(x):
       return x + 5
-    pipeline = TestPipeline()
-    pipeline._options.view_as(TypeOptions).runtime_type_check = True
-    pcoll = pipeline | 'Start' >> beam.Create([2, 9, 3])
-    pcoll | 'Do' >> beam.FlatMap(incorrect_par_do_fn)
-    # It's a requirement that all user-defined functions to a ParDo return
-    # an iterable.
     with self.assertRaises(typehints.TypeCheckError) as cm:
-      pipeline.run()
+      with TestPipeline() as pipeline:
+        pipeline._options.view_as(TypeOptions).runtime_type_check = True
+        pcoll = pipeline | 'Start' >> beam.Create([2, 9, 3])
+        pcoll | 'Do' >> beam.FlatMap(incorrect_par_do_fn)
+        # It's a requirement that all user-defined functions to a ParDo return
+        # an iterable.
 
     expected_error_prefix = 'FlatMap and ParDo must return an iterable.'
     self.assertStartswith(cm.exception.args[0], expected_error_prefix)
@@ -323,19 +310,18 @@
       def finish_bundle(self):
         yield WindowedValue('finish', -1, [window.GlobalWindow()])
 
-    pipeline = TestPipeline()
-    pcoll = pipeline | 'Start' >> beam.Create([1, 2, 3])
-    result = pcoll | 'Do' >> beam.ParDo(MyDoFn())
+    with TestPipeline() as pipeline:
+      pcoll = pipeline | 'Start' >> beam.Create([1, 2, 3])
+      result = pcoll | 'Do' >> beam.ParDo(MyDoFn())
 
-    # May have many bundles, but each has a start and finish.
-    def  matcher():
-      def match(actual):
-        equal_to(['finish'])(list(set(actual)))
-        equal_to([1])([actual.count('finish')])
-      return match
+      # May have many bundles, but each has a start and finish.
+      def  matcher():
+        def match(actual):
+          equal_to(['finish'])(list(set(actual)))
+          equal_to([1])([actual.count('finish')])
+        return match
 
-    assert_that(result, matcher())
-    pipeline.run()
+      assert_that(result, matcher())
 
   def test_do_fn_with_windowing_in_finish_bundle(self):
     windowfn = window.FixedWindows(2)
@@ -375,19 +361,18 @@
           yield 'started'
         self.state = 'process'
 
-    pipeline = TestPipeline()
-    pcoll = pipeline | 'Start' >> beam.Create([1, 2, 3])
-    result = pcoll | 'Do' >> beam.ParDo(MyDoFn())
+    with TestPipeline() as pipeline:
+      pcoll = pipeline | 'Start' >> beam.Create([1, 2, 3])
+      result = pcoll | 'Do' >> beam.ParDo(MyDoFn())
 
-    # May have many bundles, but each has a start and finish.
-    def  matcher():
-      def match(actual):
-        equal_to(['started'])(list(set(actual)))
-        equal_to([1])([actual.count('started')])
-      return match
+      # May have many bundles, but each has a start and finish.
+      def  matcher():
+        def match(actual):
+          equal_to(['started'])(list(set(actual)))
+          equal_to([1])([actual.count('started')])
+        return match
 
-    assert_that(result, matcher())
-    pipeline.run()
+      assert_that(result, matcher())
 
   def test_do_fn_with_start_error(self):
     class MyDoFn(beam.DoFn):
@@ -397,17 +382,15 @@
       def process(self, element):
         pass
 
-    pipeline = TestPipeline()
-    pipeline | 'Start' >> beam.Create([1, 2, 3]) | 'Do' >> beam.ParDo(MyDoFn())
     with self.assertRaises(RuntimeError):
-      pipeline.run()
+      with TestPipeline() as p:
+        p | 'Start' >> beam.Create([1, 2, 3]) | 'Do' >> beam.ParDo(MyDoFn())
 
   def test_filter(self):
-    pipeline = TestPipeline()
-    pcoll = pipeline | 'Start' >> beam.Create([1, 2, 3, 4])
-    result = pcoll | 'Filter' >> beam.Filter(lambda x: x % 2 == 0)
-    assert_that(result, equal_to([2, 4]))
-    pipeline.run()
+    with TestPipeline() as pipeline:
+      pcoll = pipeline | 'Start' >> beam.Create([1, 2, 3, 4])
+      result = pcoll | 'Filter' >> beam.Filter(lambda x: x % 2 == 0)
+      assert_that(result, equal_to([2, 4]))
 
   class _MeanCombineFn(beam.CombineFn):
 
@@ -430,68 +413,62 @@
 
   def test_combine_with_combine_fn(self):
     vals = [1, 2, 3, 4, 5, 6, 7]
-    pipeline = TestPipeline()
-    pcoll = pipeline | 'Start' >> beam.Create(vals)
-    result = pcoll | 'Mean' >> beam.CombineGlobally(self._MeanCombineFn())
-    assert_that(result, equal_to([sum(vals) // len(vals)]))
-    pipeline.run()
+    with TestPipeline() as pipeline:
+      pcoll = pipeline | 'Start' >> beam.Create(vals)
+      result = pcoll | 'Mean' >> beam.CombineGlobally(self._MeanCombineFn())
+      assert_that(result, equal_to([sum(vals) // len(vals)]))
 
   def test_combine_with_callable(self):
     vals = [1, 2, 3, 4, 5, 6, 7]
-    pipeline = TestPipeline()
-    pcoll = pipeline | 'Start' >> beam.Create(vals)
-    result = pcoll | beam.CombineGlobally(sum)
-    assert_that(result, equal_to([sum(vals)]))
-    pipeline.run()
+    with TestPipeline() as pipeline:
+      pcoll = pipeline | 'Start' >> beam.Create(vals)
+      result = pcoll | beam.CombineGlobally(sum)
+      assert_that(result, equal_to([sum(vals)]))
 
   def test_combine_with_side_input_as_arg(self):
     values = [1, 2, 3, 4, 5, 6, 7]
-    pipeline = TestPipeline()
-    pcoll = pipeline | 'Start' >> beam.Create(values)
-    divisor = pipeline | 'Divisor' >> beam.Create([2])
-    result = pcoll | 'Max' >> beam.CombineGlobally(
-        # Multiples of divisor only.
-        lambda vals, d: max(v for v in vals if v % d == 0),
-        pvalue.AsSingleton(divisor)).without_defaults()
-    filt_vals = [v for v in values if v % 2 == 0]
-    assert_that(result, equal_to([max(filt_vals)]))
-    pipeline.run()
+    with TestPipeline() as pipeline:
+      pcoll = pipeline | 'Start' >> beam.Create(values)
+      divisor = pipeline | 'Divisor' >> beam.Create([2])
+      result = pcoll | 'Max' >> beam.CombineGlobally(
+          # Multiples of divisor only.
+          lambda vals, d: max(v for v in vals if v % d == 0),
+          pvalue.AsSingleton(divisor)).without_defaults()
+      filt_vals = [v for v in values if v % 2 == 0]
+      assert_that(result, equal_to([max(filt_vals)]))
 
   def test_combine_per_key_with_combine_fn(self):
     vals_1 = [1, 2, 3, 4, 5, 6, 7]
     vals_2 = [2, 4, 6, 8, 10, 12, 14]
-    pipeline = TestPipeline()
-    pcoll = pipeline | 'Start' >> beam.Create(([('a', x) for x in vals_1] +
-                                               [('b', x) for x in vals_2]))
-    result = pcoll | 'Mean' >> beam.CombinePerKey(self._MeanCombineFn())
-    assert_that(result, equal_to([('a', sum(vals_1) // len(vals_1)),
-                                  ('b', sum(vals_2) // len(vals_2))]))
-    pipeline.run()
+    with TestPipeline() as pipeline:
+      pcoll = pipeline | 'Start' >> beam.Create(([('a', x) for x in vals_1] +
+                                                 [('b', x) for x in vals_2]))
+      result = pcoll | 'Mean' >> beam.CombinePerKey(self._MeanCombineFn())
+      assert_that(result, equal_to([('a', sum(vals_1) // len(vals_1)),
+                                    ('b', sum(vals_2) // len(vals_2))]))
 
   def test_combine_per_key_with_callable(self):
     vals_1 = [1, 2, 3, 4, 5, 6, 7]
     vals_2 = [2, 4, 6, 8, 10, 12, 14]
-    pipeline = TestPipeline()
-    pcoll = pipeline | 'Start' >> beam.Create(([('a', x) for x in vals_1] +
-                                               [('b', x) for x in vals_2]))
-    result = pcoll | beam.CombinePerKey(sum)
-    assert_that(result, equal_to([('a', sum(vals_1)), ('b', sum(vals_2))]))
-    pipeline.run()
+    with TestPipeline() as pipeline:
+      pcoll = pipeline | 'Start' >> beam.Create(([('a', x) for x in vals_1] +
+                                                 [('b', x) for x in vals_2]))
+      result = pcoll | beam.CombinePerKey(sum)
+      assert_that(result, equal_to([('a', sum(vals_1)), ('b', sum(vals_2))]))
 
   def test_combine_per_key_with_side_input_as_arg(self):
     vals_1 = [1, 2, 3, 4, 5, 6, 7]
     vals_2 = [2, 4, 6, 8, 10, 12, 14]
-    pipeline = TestPipeline()
-    pcoll = pipeline | 'Start' >> beam.Create(([('a', x) for x in vals_1] +
-                                               [('b', x) for x in vals_2]))
-    divisor = pipeline | 'Divisor' >> beam.Create([2])
-    result = pcoll | beam.CombinePerKey(
-        lambda vals, d: max(v for v in vals if v % d == 0),
-        pvalue.AsSingleton(divisor))  # Multiples of divisor only.
-    m_1 = max(v for v in vals_1 if v % 2 == 0)
-    m_2 = max(v for v in vals_2 if v % 2 == 0)
-    assert_that(result, equal_to([('a', m_1), ('b', m_2)]))
-    pipeline.run()
+    with TestPipeline() as pipeline:
+      pcoll = pipeline | 'Start' >> beam.Create(([('a', x) for x in vals_1] +
+                                                 [('b', x) for x in vals_2]))
+      divisor = pipeline | 'Divisor' >> beam.Create([2])
+      result = pcoll | beam.CombinePerKey(
+          lambda vals, d: max(v for v in vals if v % d == 0),
+          pvalue.AsSingleton(divisor))  # Multiples of divisor only.
+      m_1 = max(v for v in vals_1 if v % 2 == 0)
+      m_2 = max(v for v in vals_2 if v % 2 == 0)
+      assert_that(result, equal_to([('a', m_1), ('b', m_2)]))
 
   def test_group_by_key(self):
     pipeline = TestPipeline()
@@ -511,13 +488,12 @@
           sum_val += sum(value_list)
         return [(key, sum_val)]
 
-    pipeline = TestPipeline()
-    pcoll = pipeline | 'start' >> beam.Create(
-        [(1, 1), (1, 2), (1, 3), (1, 4)])
-    result = (pcoll | 'Group' >> beam.GroupByKey()
-              | 'Reiteration-Sum' >> beam.ParDo(MyDoFn()))
-    assert_that(result, equal_to([(1, 170)]))
-    pipeline.run()
+    with TestPipeline() as pipeline:
+      pcoll = pipeline | 'start' >> beam.Create(
+          [(1, 1), (1, 2), (1, 3), (1, 4)])
+      result = (pcoll | 'Group' >> beam.GroupByKey()
+                | 'Reiteration-Sum' >> beam.ParDo(MyDoFn()))
+      assert_that(result, equal_to([(1, 170)]))
 
   def test_partition_with_partition_fn(self):
 
@@ -526,36 +502,33 @@
       def partition_for(self, element, num_partitions, offset):
         return (element % 3) + offset
 
-    pipeline = TestPipeline()
-    pcoll = pipeline | 'Start' >> beam.Create([0, 1, 2, 3, 4, 5, 6, 7, 8])
-    # Attempt nominal partition operation.
-    partitions = pcoll | 'Part 1' >> beam.Partition(SomePartitionFn(), 4, 1)
-    assert_that(partitions[0], equal_to([]))
-    assert_that(partitions[1], equal_to([0, 3, 6]), label='p1')
-    assert_that(partitions[2], equal_to([1, 4, 7]), label='p2')
-    assert_that(partitions[3], equal_to([2, 5, 8]), label='p3')
-    pipeline.run()
+    with TestPipeline() as pipeline:
+      pcoll = pipeline | 'Start' >> beam.Create([0, 1, 2, 3, 4, 5, 6, 7, 8])
+      # Attempt nominal partition operation.
+      partitions = pcoll | 'Part 1' >> beam.Partition(SomePartitionFn(), 4, 1)
+      assert_that(partitions[0], equal_to([]))
+      assert_that(partitions[1], equal_to([0, 3, 6]), label='p1')
+      assert_that(partitions[2], equal_to([1, 4, 7]), label='p2')
+      assert_that(partitions[3], equal_to([2, 5, 8]), label='p3')
 
     # Check that a bad partition label will yield an error. For the
     # DirectRunner, this error manifests as an exception.
-    pipeline = TestPipeline()
-    pcoll = pipeline | 'Start' >> beam.Create([0, 1, 2, 3, 4, 5, 6, 7, 8])
-    partitions = pcoll | 'Part 2' >> beam.Partition(SomePartitionFn(), 4, 10000)
     with self.assertRaises(ValueError):
-      pipeline.run()
+      with TestPipeline() as pipeline:
+        pcoll = pipeline | 'Start' >> beam.Create([0, 1, 2, 3, 4, 5, 6, 7, 8])
+        partitions = pcoll | beam.Partition(SomePartitionFn(), 4, 10000)
 
   def test_partition_with_callable(self):
-    pipeline = TestPipeline()
-    pcoll = pipeline | 'Start' >> beam.Create([0, 1, 2, 3, 4, 5, 6, 7, 8])
-    partitions = (
-        pcoll | 'part' >> beam.Partition(
-            lambda e, n, offset: (e % 3) + offset, 4,
-            1))
-    assert_that(partitions[0], equal_to([]))
-    assert_that(partitions[1], equal_to([0, 3, 6]), label='p1')
-    assert_that(partitions[2], equal_to([1, 4, 7]), label='p2')
-    assert_that(partitions[3], equal_to([2, 5, 8]), label='p3')
-    pipeline.run()
+    with TestPipeline() as pipeline:
+      pcoll = pipeline | 'Start' >> beam.Create([0, 1, 2, 3, 4, 5, 6, 7, 8])
+      partitions = (
+          pcoll | 'part' >> beam.Partition(
+              lambda e, n, offset: (e % 3) + offset, 4,
+              1))
+      assert_that(partitions[0], equal_to([]))
+      assert_that(partitions[1], equal_to([0, 3, 6]), label='p1')
+      assert_that(partitions[2], equal_to([1, 4, 7]), label='p2')
+      assert_that(partitions[3], equal_to([2, 5, 8]), label='p3')
 
   def test_partition_followed_by_flatten_and_groupbykey(self):
     """Regression test for an issue with how partitions are handled."""
@@ -570,56 +543,50 @@
 
   @attr('ValidatesRunner')
   def test_flatten_pcollections(self):
-    pipeline = TestPipeline()
-    pcoll_1 = pipeline | 'Start 1' >> beam.Create([0, 1, 2, 3])
-    pcoll_2 = pipeline | 'Start 2' >> beam.Create([4, 5, 6, 7])
-    result = (pcoll_1, pcoll_2) | 'Flatten' >> beam.Flatten()
-    assert_that(result, equal_to([0, 1, 2, 3, 4, 5, 6, 7]))
-    pipeline.run()
+    with TestPipeline() as pipeline:
+      pcoll_1 = pipeline | 'Start 1' >> beam.Create([0, 1, 2, 3])
+      pcoll_2 = pipeline | 'Start 2' >> beam.Create([4, 5, 6, 7])
+      result = (pcoll_1, pcoll_2) | 'Flatten' >> beam.Flatten()
+      assert_that(result, equal_to([0, 1, 2, 3, 4, 5, 6, 7]))
 
   def test_flatten_no_pcollections(self):
-    pipeline = TestPipeline()
-    with self.assertRaises(ValueError):
-      () | 'PipelineArgMissing' >> beam.Flatten()
-    result = () | 'Empty' >> beam.Flatten(pipeline=pipeline)
-    assert_that(result, equal_to([]))
-    pipeline.run()
+    with TestPipeline() as pipeline:
+      with self.assertRaises(ValueError):
+        () | 'PipelineArgMissing' >> beam.Flatten()
+      result = () | 'Empty' >> beam.Flatten(pipeline=pipeline)
+      assert_that(result, equal_to([]))
 
   @attr('ValidatesRunner')
   def test_flatten_one_single_pcollection(self):
-    pipeline = TestPipeline()
-    input = [0, 1, 2, 3]
-    pcoll = pipeline | 'Input' >> beam.Create(input)
-    result = (pcoll,)| 'Single Flatten' >> beam.Flatten()
-    assert_that(result, equal_to(input))
-    pipeline.run()
+    with TestPipeline() as pipeline:
+      input = [0, 1, 2, 3]
+      pcoll = pipeline | 'Input' >> beam.Create(input)
+      result = (pcoll,)| 'Single Flatten' >> beam.Flatten()
+      assert_that(result, equal_to(input))
 
   # TODO(BEAM-9002): Does not work in streaming mode on Dataflow.
   @attr('ValidatesRunner', 'sickbay-streaming')
   def test_flatten_same_pcollections(self):
-    pipeline = TestPipeline()
-    pc = pipeline | beam.Create(['a', 'b'])
-    assert_that((pc, pc, pc) | beam.Flatten(), equal_to(['a', 'b'] * 3))
-    pipeline.run()
+    with TestPipeline() as pipeline:
+      pc = pipeline | beam.Create(['a', 'b'])
+      assert_that((pc, pc, pc) | beam.Flatten(), equal_to(['a', 'b'] * 3))
 
   def test_flatten_pcollections_in_iterable(self):
-    pipeline = TestPipeline()
-    pcoll_1 = pipeline | 'Start 1' >> beam.Create([0, 1, 2, 3])
-    pcoll_2 = pipeline | 'Start 2' >> beam.Create([4, 5, 6, 7])
-    result = [pcoll for pcoll in (pcoll_1, pcoll_2)] | beam.Flatten()
-    assert_that(result, equal_to([0, 1, 2, 3, 4, 5, 6, 7]))
-    pipeline.run()
+    with TestPipeline() as pipeline:
+      pcoll_1 = pipeline | 'Start 1' >> beam.Create([0, 1, 2, 3])
+      pcoll_2 = pipeline | 'Start 2' >> beam.Create([4, 5, 6, 7])
+      result = [pcoll for pcoll in (pcoll_1, pcoll_2)] | beam.Flatten()
+      assert_that(result, equal_to([0, 1, 2, 3, 4, 5, 6, 7]))
 
   @attr('ValidatesRunner')
   def test_flatten_a_flattened_pcollection(self):
-    pipeline = TestPipeline()
-    pcoll_1 = pipeline | 'Start 1' >> beam.Create([0, 1, 2, 3])
-    pcoll_2 = pipeline | 'Start 2' >> beam.Create([4, 5, 6, 7])
-    pcoll_3 = pipeline | 'Start 3' >> beam.Create([8, 9])
-    pcoll_12 = (pcoll_1, pcoll_2) | 'Flatten' >> beam.Flatten()
-    pcoll_123 = (pcoll_12, pcoll_3) | 'Flatten again' >> beam.Flatten()
-    assert_that(pcoll_123, equal_to([x for x in range(10)]))
-    pipeline.run()
+    with TestPipeline() as pipeline:
+      pcoll_1 = pipeline | 'Start 1' >> beam.Create([0, 1, 2, 3])
+      pcoll_2 = pipeline | 'Start 2' >> beam.Create([4, 5, 6, 7])
+      pcoll_3 = pipeline | 'Start 3' >> beam.Create([8, 9])
+      pcoll_12 = (pcoll_1, pcoll_2) | 'Flatten' >> beam.Flatten()
+      pcoll_123 = (pcoll_12, pcoll_3) | 'Flatten again' >> beam.Flatten()
+      assert_that(pcoll_123, equal_to([x for x in range(10)]))
 
   def test_flatten_input_type_must_be_iterable(self):
     # Inputs to flatten *must* be an iterable.
@@ -635,21 +602,20 @@
 
   @attr('ValidatesRunner')
   def test_flatten_multiple_pcollections_having_multiple_consumers(self):
-    pipeline = TestPipeline()
-    input = pipeline | 'Start' >> beam.Create(['AA', 'BBB', 'CC'])
+    with TestPipeline() as pipeline:
+      input = pipeline | 'Start' >> beam.Create(['AA', 'BBB', 'CC'])
 
-    def split_even_odd(element):
-      tag = 'even_length' if len(element) % 2 == 0 else 'odd_length'
-      return pvalue.TaggedOutput(tag, element)
+      def split_even_odd(element):
+        tag = 'even_length' if len(element) % 2 == 0 else 'odd_length'
+        return pvalue.TaggedOutput(tag, element)
 
-    even_length, odd_length = (input | beam.Map(split_even_odd)
-                               .with_outputs('even_length', 'odd_length'))
-    merged = (even_length, odd_length) | 'Flatten' >> beam.Flatten()
+      even_length, odd_length = (input | beam.Map(split_even_odd)
+                                 .with_outputs('even_length', 'odd_length'))
+      merged = (even_length, odd_length) | 'Flatten' >> beam.Flatten()
 
-    assert_that(merged, equal_to(['AA', 'BBB', 'CC']))
-    assert_that(even_length, equal_to(['AA', 'CC']), label='assert:even')
-    assert_that(odd_length, equal_to(['BBB']), label='assert:odd')
-    pipeline.run()
+      assert_that(merged, equal_to(['AA', 'BBB', 'CC']))
+      assert_that(even_length, equal_to(['AA', 'CC']), label='assert:even')
+      assert_that(odd_length, equal_to(['BBB']), label='assert:odd')
 
   def test_co_group_by_key_on_list(self):
     pipeline = TestPipeline()
@@ -690,12 +656,10 @@
     pipeline.run()
 
   def test_group_by_key_input_must_be_kv_pairs(self):
-    pipeline = TestPipeline()
-    pcolls = pipeline | 'A' >> beam.Create([1, 2, 3, 4, 5])
-
     with self.assertRaises(typehints.TypeCheckError) as e:
-      pcolls | 'D' >> beam.GroupByKey()
-      pipeline.run()
+      with TestPipeline() as pipeline:
+        pcolls = pipeline | 'A' >> beam.Create([1, 2, 3, 4, 5])
+        pcolls | 'D' >> beam.GroupByKey()
 
     self.assertStartswith(
         e.exception.args[0],
@@ -703,58 +667,52 @@
         'Tuple[TypeVariable[K], TypeVariable[V]]')
 
   def test_group_by_key_only_input_must_be_kv_pairs(self):
-    pipeline = TestPipeline()
-    pcolls = pipeline | 'A' >> beam.Create(['a', 'b', 'f'])
     with self.assertRaises(typehints.TypeCheckError) as cm:
-      pcolls | 'D' >> _GroupByKeyOnly()
-      pipeline.run()
+      with TestPipeline() as pipeline:
+        pcolls = pipeline | 'A' >> beam.Create(['a', 'b', 'f'])
+        pcolls | 'D' >> _GroupByKeyOnly()
 
     expected_error_prefix = ('Input type hint violation at D: expected '
                              'Tuple[TypeVariable[K], TypeVariable[V]]')
     self.assertStartswith(cm.exception.args[0], expected_error_prefix)
 
   def test_keys_and_values(self):
-    pipeline = TestPipeline()
-    pcoll = pipeline | 'Start' >> beam.Create(
-        [(3, 1), (2, 1), (1, 1), (3, 2), (2, 2), (3, 3)])
-    keys = pcoll.apply(beam.Keys('keys'))
-    vals = pcoll.apply(beam.Values('vals'))
-    assert_that(keys, equal_to([1, 2, 2, 3, 3, 3]), label='assert:keys')
-    assert_that(vals, equal_to([1, 1, 1, 2, 2, 3]), label='assert:vals')
-    pipeline.run()
+    with TestPipeline() as pipeline:
+      pcoll = pipeline | 'Start' >> beam.Create(
+          [(3, 1), (2, 1), (1, 1), (3, 2), (2, 2), (3, 3)])
+      keys = pcoll.apply(beam.Keys('keys'))
+      vals = pcoll.apply(beam.Values('vals'))
+      assert_that(keys, equal_to([1, 2, 2, 3, 3, 3]), label='assert:keys')
+      assert_that(vals, equal_to([1, 1, 1, 2, 2, 3]), label='assert:vals')
 
   def test_kv_swap(self):
-    pipeline = TestPipeline()
-    pcoll = pipeline | 'Start' >> beam.Create(
-        [(6, 3), (1, 2), (7, 1), (5, 2), (3, 2)])
-    result = pcoll.apply(beam.KvSwap(), label='swap')
-    assert_that(result, equal_to([(1, 7), (2, 1), (2, 3), (2, 5), (3, 6)]))
-    pipeline.run()
+    with TestPipeline() as pipeline:
+      pcoll = pipeline | 'Start' >> beam.Create(
+          [(6, 3), (1, 2), (7, 1), (5, 2), (3, 2)])
+      result = pcoll.apply(beam.KvSwap(), label='swap')
+      assert_that(result, equal_to([(1, 7), (2, 1), (2, 3), (2, 5), (3, 6)]))
 
   def test_distinct(self):
-    pipeline = TestPipeline()
-    pcoll = pipeline | 'Start' >> beam.Create(
-        [6, 3, 1, 1, 9, 'pleat', 'pleat', 'kazoo', 'navel'])
-    result = pcoll.apply(beam.Distinct())
-    assert_that(result, equal_to([1, 3, 6, 9, 'pleat', 'kazoo', 'navel']))
-    pipeline.run()
+    with TestPipeline() as pipeline:
+      pcoll = pipeline | 'Start' >> beam.Create(
+          [6, 3, 1, 1, 9, 'pleat', 'pleat', 'kazoo', 'navel'])
+      result = pcoll.apply(beam.Distinct())
+      assert_that(result, equal_to([1, 3, 6, 9, 'pleat', 'kazoo', 'navel']))
 
   def test_remove_duplicates(self):
-    pipeline = TestPipeline()
-    pcoll = pipeline | 'Start' >> beam.Create(
-        [6, 3, 1, 1, 9, 'pleat', 'pleat', 'kazoo', 'navel'])
-    result = pcoll.apply(beam.RemoveDuplicates())
-    assert_that(result, equal_to([1, 3, 6, 9, 'pleat', 'kazoo', 'navel']))
-    pipeline.run()
+    with TestPipeline() as pipeline:
+      pcoll = pipeline | 'Start' >> beam.Create(
+          [6, 3, 1, 1, 9, 'pleat', 'pleat', 'kazoo', 'navel'])
+      result = pcoll.apply(beam.RemoveDuplicates())
+      assert_that(result, equal_to([1, 3, 6, 9, 'pleat', 'kazoo', 'navel']))
 
   def test_chained_ptransforms(self):
-    pipeline = TestPipeline()
-    t = (beam.Map(lambda x: (x, 1))
-         | beam.GroupByKey()
-         | beam.Map(lambda x_ones: (x_ones[0], sum(x_ones[1]))))
-    result = pipeline | 'Start' >> beam.Create(['a', 'a', 'b']) | t
-    assert_that(result, equal_to([('a', 2), ('b', 1)]))
-    pipeline.run()
+    with TestPipeline() as pipeline:
+      t = (beam.Map(lambda x: (x, 1))
+           | beam.GroupByKey()
+           | beam.Map(lambda x_ones: (x_ones[0], sum(x_ones[1]))))
+      result = pipeline | 'Start' >> beam.Create(['a', 'a', 'b']) | t
+      assert_that(result, equal_to([('a', 2), ('b', 1)]))
 
   def test_apply_to_list(self):
     self.assertCountEqual(
@@ -850,47 +808,43 @@
 
   def test_chained_ptransforms(self):
     """Tests that chaining gets proper nesting."""
-    pipeline = TestPipeline()
-    map1 = 'Map1' >> beam.Map(lambda x: (x, 1))
-    gbk = 'Gbk' >> beam.GroupByKey()
-    map2 = 'Map2' >> beam.Map(lambda x_ones2: (x_ones2[0], sum(x_ones2[1])))
-    t = (map1 | gbk | map2)
-    result = pipeline | 'Start' >> beam.Create(['a', 'a', 'b']) | t
-    self.assertTrue('Map1|Gbk|Map2/Map1' in pipeline.applied_labels)
-    self.assertTrue('Map1|Gbk|Map2/Gbk' in pipeline.applied_labels)
-    self.assertTrue('Map1|Gbk|Map2/Map2' in pipeline.applied_labels)
-    assert_that(result, equal_to([('a', 2), ('b', 1)]))
-    pipeline.run()
+    with TestPipeline() as pipeline:
+      map1 = 'Map1' >> beam.Map(lambda x: (x, 1))
+      gbk = 'Gbk' >> beam.GroupByKey()
+      map2 = 'Map2' >> beam.Map(lambda x_ones2: (x_ones2[0], sum(x_ones2[1])))
+      t = (map1 | gbk | map2)
+      result = pipeline | 'Start' >> beam.Create(['a', 'a', 'b']) | t
+      self.assertTrue('Map1|Gbk|Map2/Map1' in pipeline.applied_labels)
+      self.assertTrue('Map1|Gbk|Map2/Gbk' in pipeline.applied_labels)
+      self.assertTrue('Map1|Gbk|Map2/Map2' in pipeline.applied_labels)
+      assert_that(result, equal_to([('a', 2), ('b', 1)]))
 
   def test_apply_custom_transform_without_label(self):
-    pipeline = TestPipeline()
-    pcoll = pipeline | 'PColl' >> beam.Create([1, 2, 3])
-    custom = PTransformLabelsTest.CustomTransform()
-    result = pipeline.apply(custom, pcoll)
-    self.assertTrue('CustomTransform' in pipeline.applied_labels)
-    self.assertTrue('CustomTransform/*Do*' in pipeline.applied_labels)
-    assert_that(result, equal_to([2, 3, 4]))
-    pipeline.run()
+    with TestPipeline() as pipeline:
+      pcoll = pipeline | 'PColl' >> beam.Create([1, 2, 3])
+      custom = PTransformLabelsTest.CustomTransform()
+      result = pipeline.apply(custom, pcoll)
+      self.assertTrue('CustomTransform' in pipeline.applied_labels)
+      self.assertTrue('CustomTransform/*Do*' in pipeline.applied_labels)
+      assert_that(result, equal_to([2, 3, 4]))
 
   def test_apply_custom_transform_with_label(self):
-    pipeline = TestPipeline()
-    pcoll = pipeline | 'PColl' >> beam.Create([1, 2, 3])
-    custom = PTransformLabelsTest.CustomTransform('*Custom*')
-    result = pipeline.apply(custom, pcoll)
-    self.assertTrue('*Custom*' in pipeline.applied_labels)
-    self.assertTrue('*Custom*/*Do*' in pipeline.applied_labels)
-    assert_that(result, equal_to([2, 3, 4]))
-    pipeline.run()
+    with TestPipeline() as pipeline:
+      pcoll = pipeline | 'PColl' >> beam.Create([1, 2, 3])
+      custom = PTransformLabelsTest.CustomTransform('*Custom*')
+      result = pipeline.apply(custom, pcoll)
+      self.assertTrue('*Custom*' in pipeline.applied_labels)
+      self.assertTrue('*Custom*/*Do*' in pipeline.applied_labels)
+      assert_that(result, equal_to([2, 3, 4]))
 
   def test_combine_without_label(self):
     vals = [1, 2, 3, 4, 5, 6, 7]
-    pipeline = TestPipeline()
-    pcoll = pipeline | 'Start' >> beam.Create(vals)
-    combine = beam.CombineGlobally(sum)
-    result = pcoll | combine
-    self.assertTrue('CombineGlobally(sum)' in pipeline.applied_labels)
-    assert_that(result, equal_to([sum(vals)]))
-    pipeline.run()
+    with TestPipeline() as pipeline:
+      pcoll = pipeline | 'Start' >> beam.Create(vals)
+      combine = beam.CombineGlobally(sum)
+      result = pcoll | combine
+      self.assertTrue('CombineGlobally(sum)' in pipeline.applied_labels)
+      assert_that(result, equal_to([sum(vals)]))
 
   def test_apply_ptransform_using_decorator(self):
     pipeline = TestPipeline()
@@ -903,13 +857,12 @@
 
   def test_combine_with_label(self):
     vals = [1, 2, 3, 4, 5, 6, 7]
-    pipeline = TestPipeline()
-    pcoll = pipeline | 'Start' >> beam.Create(vals)
-    combine = '*Sum*' >> beam.CombineGlobally(sum)
-    result = pcoll | combine
-    self.assertTrue('*Sum*' in pipeline.applied_labels)
-    assert_that(result, equal_to([sum(vals)]))
-    pipeline.run()
+    with TestPipeline() as pipeline:
+      pcoll = pipeline | 'Start' >> beam.Create(vals)
+      combine = '*Sum*' >> beam.CombineGlobally(sum)
+      result = pcoll | combine
+      self.assertTrue('*Sum*' in pipeline.applied_labels)
+      assert_that(result, equal_to([sum(vals)]))
 
   def check_label(self, ptransform, expected_label):
     pipeline = TestPipeline()
@@ -2226,11 +2179,10 @@
     def MyTransform(pcoll):
       return pcoll | beam.ParDo(lambda x: [x]).with_output_types(int)
 
-    p = TestPipeline()
-    _ = (p
-         | beam.Create([1, 2])
-         | MyTransform().with_output_types(int))
-    p.run()
+    with TestPipeline() as p:
+      _ = (p
+           | beam.Create([1, 2])
+           | MyTransform().with_output_types(int))
 
   def test_type_hints_arg(self):
     # Tests passing type hints via the magic 'type_hints' argument name.
@@ -2241,11 +2193,10 @@
               | beam.ParDo(lambda x: [x]).with_output_types(
                   type_hints.output_types[0][0]))
 
-    p = TestPipeline()
-    _ = (p
-         | beam.Create([1, 2])
-         | MyTransform('test').with_output_types(int))
-    p.run()
+    with TestPipeline() as p:
+      _ = (p
+           | beam.Create([1, 2])
+           | MyTransform('test').with_output_types(int))
 
 
 def _sort_lists(result):
diff --git a/sdks/python/apache_beam/transforms/stats_test.py b/sdks/python/apache_beam/transforms/stats_test.py
index 550c3f5..b7f9604 100644
--- a/sdks/python/apache_beam/transforms/stats_test.py
+++ b/sdks/python/apache_beam/transforms/stats_test.py
@@ -56,13 +56,12 @@
     test_input = [random.randint(0, 1000) for _ in range(100)]
 
     with self.assertRaises(ValueError) as e:
-      pipeline = TestPipeline()
-      _ = (pipeline
-           | 'create'
-           >> beam.Create(test_input)
-           | 'get_estimate'
-           >> beam.ApproximateUnique.Globally(size=sample_size))
-      pipeline.run()
+      with TestPipeline() as pipeline:
+        _ = (pipeline
+             | 'create'
+             >> beam.Create(test_input)
+             | 'get_estimate'
+             >> beam.ApproximateUnique.Globally(size=sample_size))
 
     expected_msg = beam.ApproximateUnique._INPUT_SIZE_ERR_MSG % (sample_size)
 
@@ -75,12 +74,11 @@
     test_input = [random.randint(0, 1000) for _ in range(100)]
 
     with self.assertRaises(ValueError) as e:
-      pipeline = TestPipeline()
-      _ = (pipeline
-           | 'create' >> beam.Create(test_input)
-           | 'get_estimate'
-           >> beam.ApproximateUnique.Globally(size=sample_size))
-      pipeline.run()
+      with TestPipeline() as pipeline:
+        _ = (pipeline
+             | 'create' >> beam.Create(test_input)
+             | 'get_estimate'
+             >> beam.ApproximateUnique.Globally(size=sample_size))
 
     expected_msg = beam.ApproximateUnique._INPUT_SIZE_ERR_MSG % (sample_size)
 
@@ -93,12 +91,11 @@
     test_input = [random.randint(0, 1000) for _ in range(100)]
 
     with self.assertRaises(ValueError) as e:
-      pipeline = TestPipeline()
-      _ = (pipeline
-           | 'create' >> beam.Create(test_input)
-           | 'get_estimate'
-           >> beam.ApproximateUnique.Globally(error=est_err))
-      pipeline.run()
+      with TestPipeline() as pipeline:
+        _ = (pipeline
+             | 'create' >> beam.Create(test_input)
+             | 'get_estimate'
+             >> beam.ApproximateUnique.Globally(error=est_err))
 
     expected_msg = beam.ApproximateUnique._INPUT_ERROR_ERR_MSG % (est_err)
 
@@ -111,12 +108,11 @@
     test_input = [random.randint(0, 1000) for _ in range(100)]
 
     with self.assertRaises(ValueError) as e:
-      pipeline = TestPipeline()
-      _ = (pipeline
-           | 'create' >> beam.Create(test_input)
-           | 'get_estimate'
-           >> beam.ApproximateUnique.Globally(error=est_err))
-      pipeline.run()
+      with TestPipeline() as pipeline:
+        _ = (pipeline
+             | 'create' >> beam.Create(test_input)
+             | 'get_estimate'
+             >> beam.ApproximateUnique.Globally(error=est_err))
 
     expected_msg = beam.ApproximateUnique._INPUT_ERROR_ERR_MSG % (est_err)
 
@@ -127,12 +123,11 @@
     test_input = [random.randint(0, 1000) for _ in range(100)]
 
     with self.assertRaises(ValueError) as e:
-      pipeline = TestPipeline()
-      _ = (pipeline
-           | 'create' >> beam.Create(test_input)
-           | 'get_estimate'
-           >> beam.ApproximateUnique.Globally())
-      pipeline.run()
+      with TestPipeline() as pipeline:
+        _ = (pipeline
+             | 'create' >> beam.Create(test_input)
+             | 'get_estimate'
+             >> beam.ApproximateUnique.Globally())
 
     expected_msg = beam.ApproximateUnique._NO_VALUE_ERR_MSG
     assert e.exception.args[0] == expected_msg
@@ -144,12 +139,11 @@
     sample_size = 30
 
     with self.assertRaises(ValueError) as e:
-      pipeline = TestPipeline()
-      _ = (pipeline
-           | 'create' >> beam.Create(test_input)
-           | 'get_estimate'
-           >> beam.ApproximateUnique.Globally(size=sample_size, error=est_err))
-      pipeline.run()
+      with TestPipeline() as pipeline:
+        _ = (pipeline
+             | 'create' >> beam.Create(test_input)
+             | 'get_estimate' >> beam.ApproximateUnique.Globally(
+                 size=sample_size, error=est_err))
 
     expected_msg = beam.ApproximateUnique._MULTI_VALUE_ERR_MSG % (
         sample_size, est_err)
@@ -178,18 +172,17 @@
 
     actual_count = len(set(test_input))
 
-    pipeline = TestPipeline()
-    result = (pipeline
-              | 'create' >> beam.Create(test_input)
-              | 'get_estimate'
-              >> beam.ApproximateUnique.Globally(size=sample_size)
-              | 'compare'
-              >> beam.FlatMap(lambda x: [abs(x - actual_count) * 1.0
-                                         / actual_count <= max_err]))
+    with TestPipeline() as pipeline:
+      result = (pipeline
+                | 'create' >> beam.Create(test_input)
+                | 'get_estimate'
+                >> beam.ApproximateUnique.Globally(size=sample_size)
+                | 'compare'
+                >> beam.FlatMap(lambda x: [abs(x - actual_count) * 1.0
+                                           / actual_count <= max_err]))
 
-    assert_that(result, equal_to([True]),
-                label='assert:global_by_size')
-    pipeline.run()
+      assert_that(result, equal_to([True]),
+                  label='assert:global_by_size')
 
   @retry(reraise=True, stop=stop_after_attempt(5))
   def test_approximate_unique_global_by_sample_size_with_duplicates(self):
@@ -200,18 +193,17 @@
     test_input = [10] * 50 + [20] * 50
     actual_count = len(set(test_input))
 
-    pipeline = TestPipeline()
-    result = (pipeline
-              | 'create' >> beam.Create(test_input)
-              | 'get_estimate'
-              >> beam.ApproximateUnique.Globally(size=sample_size)
-              | 'compare'
-              >> beam.FlatMap(lambda x: [abs(x - actual_count) * 1.0
-                                         / actual_count <= max_err]))
+    with TestPipeline() as pipeline:
+      result = (pipeline
+                | 'create' >> beam.Create(test_input)
+                | 'get_estimate'
+                >> beam.ApproximateUnique.Globally(size=sample_size)
+                | 'compare'
+                >> beam.FlatMap(lambda x: [abs(x - actual_count) * 1.0
+                                           / actual_count <= max_err]))
 
-    assert_that(result, equal_to([True]),
-                label='assert:global_by_size_with_duplicates')
-    pipeline.run()
+      assert_that(result, equal_to([True]),
+                  label='assert:global_by_size_with_duplicates')
 
   @retry(reraise=True, stop=stop_after_attempt(5))
   def test_approximate_unique_global_by_sample_size_with_small_population(self):
@@ -223,15 +215,14 @@
                   221, 829, 965, 729, 35, 33, 115, 894, 827, 364]
     actual_count = len(set(test_input))
 
-    pipeline = TestPipeline()
-    result = (pipeline
-              | 'create' >> beam.Create(test_input)
-              | 'get_estimate'
-              >> beam.ApproximateUnique.Globally(size=sample_size))
+    with TestPipeline() as pipeline:
+      result = (pipeline
+                | 'create' >> beam.Create(test_input)
+                | 'get_estimate'
+                >> beam.ApproximateUnique.Globally(size=sample_size))
 
-    assert_that(result, equal_to([actual_count]),
-                label='assert:global_by_sample_size_with_small_population')
-    pipeline.run()
+      assert_that(result, equal_to([actual_count]),
+                  label='assert:global_by_sample_size_with_small_population')
 
   @unittest.skip('Skip because hash function is not good enough. '
                  'TODO: BEAM-7654')
@@ -243,17 +234,16 @@
                   973, 386, 506, 546, 991, 450, 226, 889, 514, 693]
     actual_count = len(set(test_input))
 
-    pipeline = TestPipeline()
-    result = (pipeline
-              | 'create' >> beam.Create(test_input)
-              | 'get_estimate'
-              >> beam.ApproximateUnique.Globally(error=est_err)
-              | 'compare'
-              >> beam.FlatMap(lambda x: [abs(x - actual_count) * 1.0
-                                         / actual_count <= est_err]))
+    with TestPipeline() as pipeline:
+      result = (pipeline
+                | 'create' >> beam.Create(test_input)
+                | 'get_estimate'
+                >> beam.ApproximateUnique.Globally(error=est_err)
+                | 'compare'
+                >> beam.FlatMap(lambda x: [abs(x - actual_count) * 1.0
+                                           / actual_count <= est_err]))
 
-    assert_that(result, equal_to([True]), label='assert:global_by_error')
-    pipeline.run()
+      assert_that(result, equal_to([True]), label='assert:global_by_error')
 
   @retry(reraise=True, stop=stop_after_attempt(5))
   def test_approximate_unique_global_by_error_with_small_population(self):
@@ -266,15 +256,14 @@
                   756, 755, 839, 79, 393]
     actual_count = len(set(test_input))
 
-    pipeline = TestPipeline()
-    result = (pipeline
-              | 'create' >> beam.Create(test_input)
-              | 'get_estimate'
-              >> beam.ApproximateUnique.Globally(error=est_err))
+    with TestPipeline() as pipeline:
+      result = (pipeline
+                | 'create' >> beam.Create(test_input)
+                | 'get_estimate'
+                >> beam.ApproximateUnique.Globally(error=est_err))
 
-    assert_that(result, equal_to([actual_count]),
-                label='assert:global_by_error_with_small_population')
-    pipeline.run()
+      assert_that(result, equal_to([actual_count]),
+                  label='assert:global_by_error_with_small_population')
 
   @retry(reraise=True, stop=stop_after_attempt(5))
   def test_approximate_unique_perkey_by_size(self):
@@ -292,20 +281,19 @@
     for (x, y) in test_input:
       actual_count_dict[x].add(y)
 
-    pipeline = TestPipeline()
-    result = (pipeline
-              | 'create' >> beam.Create(test_input)
-              | 'get_estimate'
-              >> beam.ApproximateUnique.PerKey(size=sample_size)
-              | 'compare'
-              >> beam.FlatMap(lambda x: [abs(x[1]
-                                             - len(actual_count_dict[x[0]]))
-                                         * 1.0 / len(actual_count_dict[x[0]])
-                                         <= max_err]))
+    with TestPipeline() as pipeline:
+      result = (pipeline
+                | 'create' >> beam.Create(test_input)
+                | 'get_estimate'
+                >> beam.ApproximateUnique.PerKey(size=sample_size)
+                | 'compare'
+                >> beam.FlatMap(lambda x: [abs(x[1]
+                                               - len(actual_count_dict[x[0]]))
+                                           * 1.0 / len(actual_count_dict[x[0]])
+                                           <= max_err]))
 
-    assert_that(result, equal_to([True] * len(actual_count_dict)),
-                label='assert:perkey_by_size')
-    pipeline.run()
+      assert_that(result, equal_to([True] * len(actual_count_dict)),
+                  label='assert:perkey_by_size')
 
   @retry(reraise=True, stop=stop_after_attempt(5))
   def test_approximate_unique_perkey_by_error(self):
@@ -318,20 +306,19 @@
     for (x, y) in test_input:
       actual_count_dict[x].add(y)
 
-    pipeline = TestPipeline()
-    result = (pipeline
-              | 'create' >> beam.Create(test_input)
-              | 'get_estimate'
-              >> beam.ApproximateUnique.PerKey(error=est_err)
-              | 'compare'
-              >> beam.FlatMap(lambda x: [abs(x[1]
-                                             - len(actual_count_dict[x[0]]))
-                                         * 1.0 / len(actual_count_dict[x[0]])
-                                         <= est_err]))
+    with TestPipeline() as pipeline:
+      result = (pipeline
+                | 'create' >> beam.Create(test_input)
+                | 'get_estimate'
+                >> beam.ApproximateUnique.PerKey(error=est_err)
+                | 'compare'
+                >> beam.FlatMap(lambda x: [abs(x[1]
+                                               - len(actual_count_dict[x[0]]))
+                                           * 1.0 / len(actual_count_dict[x[0]])
+                                           <= est_err]))
 
-    assert_that(result, equal_to([True] * len(actual_count_dict)),
-                label='assert:perkey_by_error')
-    pipeline.run()
+      assert_that(result, equal_to([True] * len(actual_count_dict)),
+                  label='assert:perkey_by_error')
 
   @retry(reraise=True, stop=stop_after_attempt(5))
   def test_approximate_unique_globally_by_error_with_skewed_data(self):
@@ -341,18 +328,17 @@
                   6, 55, 1, 13, 90, 4, 18, 52, 33, 0, 77, 21, 26, 5, 18]
     actual_count = len(set(test_input))
 
-    pipeline = TestPipeline()
-    result = (pipeline
-              | 'create' >> beam.Create(test_input)
-              | 'get_estimate'
-              >> beam.ApproximateUnique.Globally(error=est_err)
-              | 'compare'
-              >> beam.FlatMap(lambda x: [abs(x - actual_count) * 1.0
-                                         / actual_count <= est_err]))
+    with TestPipeline() as pipeline:
+      result = (pipeline
+                | 'create' >> beam.Create(test_input)
+                | 'get_estimate'
+                >> beam.ApproximateUnique.Globally(error=est_err)
+                | 'compare'
+                >> beam.FlatMap(lambda x: [abs(x - actual_count) * 1.0
+                                           / actual_count <= est_err]))
 
-    assert_that(result, equal_to([True]),
-                label='assert:globally_by_error_with_skewed_data')
-    pipeline.run()
+      assert_that(result, equal_to([True]),
+                  label='assert:globally_by_error_with_skewed_data')
 
 
 class ApproximateQuantilesTest(unittest.TestCase):
diff --git a/sdks/python/apache_beam/transforms/transforms_keyword_only_args_test_py3.py b/sdks/python/apache_beam/transforms/transforms_keyword_only_args_test_py3.py
index 661d6ac..b220373 100644
--- a/sdks/python/apache_beam/transforms/transforms_keyword_only_args_test_py3.py
+++ b/sdks/python/apache_beam/transforms/transforms_keyword_only_args_test_py3.py
@@ -36,109 +36,106 @@
   _multiprocess_can_split_ = True
 
   def test_side_input_keyword_only_args(self):
-    pipeline = TestPipeline()
+    with TestPipeline() as pipeline:
 
-    def sort_with_side_inputs(x, *s, reverse=False):
-      for y in s:
-        yield sorted([x] + y, reverse=reverse)
+      def sort_with_side_inputs(x, *s, reverse=False):
+        for y in s:
+          yield sorted([x] + y, reverse=reverse)
 
-    def sort_with_side_inputs_without_default_values(x, *s, reverse):
-      for y in s:
-        yield sorted([x] + y, reverse=reverse)
+      def sort_with_side_inputs_without_default_values(x, *s, reverse):
+        for y in s:
+          yield sorted([x] + y, reverse=reverse)
 
-    pcol = pipeline | 'start' >> beam.Create([1, 2])
-    side = pipeline | 'side' >> beam.Create([3, 4])  # 2 values in side input.
-    result1 = pcol | 'compute1' >> beam.FlatMap(
-        sort_with_side_inputs,
-        beam.pvalue.AsList(side), reverse=True)
-    assert_that(result1, equal_to([[4, 3, 1], [4, 3, 2]]), label='assert1')
+      pcol = pipeline | 'start' >> beam.Create([1, 2])
+      side = pipeline | 'side' >> beam.Create([3, 4])  # 2 values in side input.
+      result1 = pcol | 'compute1' >> beam.FlatMap(
+          sort_with_side_inputs,
+          beam.pvalue.AsList(side), reverse=True)
+      assert_that(result1, equal_to([[4, 3, 1], [4, 3, 2]]), label='assert1')
 
-    result2 = pcol | 'compute2' >> beam.FlatMap(
-        sort_with_side_inputs,
-        beam.pvalue.AsList(side))
-    assert_that(result2, equal_to([[1, 3, 4], [2, 3, 4]]), label='assert2')
+      result2 = pcol | 'compute2' >> beam.FlatMap(
+          sort_with_side_inputs,
+          beam.pvalue.AsList(side))
+      assert_that(result2, equal_to([[1, 3, 4], [2, 3, 4]]), label='assert2')
 
-    result3 = pcol | 'compute3' >> beam.FlatMap(
-        sort_with_side_inputs)
-    assert_that(result3, equal_to([]), label='assert3')
+      result3 = pcol | 'compute3' >> beam.FlatMap(
+          sort_with_side_inputs)
+      assert_that(result3, equal_to([]), label='assert3')
 
-    result4 = pcol | 'compute4' >> beam.FlatMap(
-        sort_with_side_inputs, reverse=True)
-    assert_that(result4, equal_to([]), label='assert4')
+      result4 = pcol | 'compute4' >> beam.FlatMap(
+          sort_with_side_inputs, reverse=True)
+      assert_that(result4, equal_to([]), label='assert4')
 
-    result5 = pcol | 'compute5' >> beam.FlatMap(
-        sort_with_side_inputs_without_default_values,
-        beam.pvalue.AsList(side), reverse=True)
-    assert_that(result5, equal_to([[4, 3, 1], [4, 3, 2]]), label='assert5')
+      result5 = pcol | 'compute5' >> beam.FlatMap(
+          sort_with_side_inputs_without_default_values,
+          beam.pvalue.AsList(side), reverse=True)
+      assert_that(result5, equal_to([[4, 3, 1], [4, 3, 2]]), label='assert5')
 
-    result6 = pcol | 'compute6' >> beam.FlatMap(
-        sort_with_side_inputs_without_default_values,
-        beam.pvalue.AsList(side), reverse=False)
-    assert_that(result6, equal_to([[1, 3, 4], [2, 3, 4]]), label='assert6')
+      result6 = pcol | 'compute6' >> beam.FlatMap(
+          sort_with_side_inputs_without_default_values,
+          beam.pvalue.AsList(side), reverse=False)
+      assert_that(result6, equal_to([[1, 3, 4], [2, 3, 4]]), label='assert6')
 
-    result7 = pcol | 'compute7' >> beam.FlatMap(
-        sort_with_side_inputs_without_default_values, reverse=False)
-    assert_that(result7, equal_to([]), label='assert7')
+      result7 = pcol | 'compute7' >> beam.FlatMap(
+          sort_with_side_inputs_without_default_values, reverse=False)
+      assert_that(result7, equal_to([]), label='assert7')
 
-    result8 = pcol | 'compute8' >> beam.FlatMap(
-        sort_with_side_inputs_without_default_values, reverse=True)
-    assert_that(result8, equal_to([]), label='assert8')
+      result8 = pcol | 'compute8' >> beam.FlatMap(
+          sort_with_side_inputs_without_default_values, reverse=True)
+      assert_that(result8, equal_to([]), label='assert8')
 
-    pipeline.run()
 
   def test_combine_keyword_only_args(self):
-    pipeline = TestPipeline()
+    with TestPipeline() as pipeline:
 
-    def bounded_sum(values, *s, bound=500):
-      return min(sum(values) + sum(s), bound)
+      def bounded_sum(values, *s, bound=500):
+        return min(sum(values) + sum(s), bound)
 
-    def bounded_sum_without_default_values(values, *s, bound):
-      return min(sum(values) + sum(s), bound)
+      def bounded_sum_without_default_values(values, *s, bound):
+        return min(sum(values) + sum(s), bound)
 
-    pcoll = pipeline | 'start' >> beam.Create([6, 3, 1])
-    result1 = pcoll | 'sum1' >> beam.CombineGlobally(bounded_sum, 5, 8,
-                                                     bound=20)
-    result2 = pcoll | 'sum2' >> beam.CombineGlobally(bounded_sum, 0, 0)
-    result3 = pcoll | 'sum3' >> beam.CombineGlobally(bounded_sum)
-    result4 = pcoll | 'sum4' >> beam.CombineGlobally(bounded_sum, bound=5)
-    result5 = pcoll | 'sum5' >> beam.CombineGlobally(
-        bounded_sum_without_default_values, 5, 8, bound=20)
-    result6 = pcoll | 'sum6' >> beam.CombineGlobally(
-        bounded_sum_without_default_values, 0, 0, bound=500)
-    result7 = pcoll | 'sum7' >> beam.CombineGlobally(
-        bounded_sum_without_default_values, bound=500)
-    result8 = pcoll | 'sum8' >> beam.CombineGlobally(
-        bounded_sum_without_default_values, bound=5)
+      pcoll = pipeline | 'start' >> beam.Create([6, 3, 1])
+      result1 = pcoll | 'sum1' >> beam.CombineGlobally(bounded_sum, 5, 8,
+                                                       bound=20)
+      result2 = pcoll | 'sum2' >> beam.CombineGlobally(bounded_sum, 0, 0)
+      result3 = pcoll | 'sum3' >> beam.CombineGlobally(bounded_sum)
+      result4 = pcoll | 'sum4' >> beam.CombineGlobally(bounded_sum, bound=5)
+      result5 = pcoll | 'sum5' >> beam.CombineGlobally(
+          bounded_sum_without_default_values, 5, 8, bound=20)
+      result6 = pcoll | 'sum6' >> beam.CombineGlobally(
+          bounded_sum_without_default_values, 0, 0, bound=500)
+      result7 = pcoll | 'sum7' >> beam.CombineGlobally(
+          bounded_sum_without_default_values, bound=500)
+      result8 = pcoll | 'sum8' >> beam.CombineGlobally(
+          bounded_sum_without_default_values, bound=5)
 
-    assert_that(result1, equal_to([20]), label='assert1')
-    assert_that(result2, equal_to([10]), label='assert2')
-    assert_that(result3, equal_to([10]), label='assert3')
-    assert_that(result4, equal_to([5]), label='assert4')
-    assert_that(result5, equal_to([20]), label='assert5')
-    assert_that(result6, equal_to([10]), label='assert6')
-    assert_that(result7, equal_to([10]), label='assert7')
-    assert_that(result8, equal_to([5]), label='assert8')
+      assert_that(result1, equal_to([20]), label='assert1')
+      assert_that(result2, equal_to([10]), label='assert2')
+      assert_that(result3, equal_to([10]), label='assert3')
+      assert_that(result4, equal_to([5]), label='assert4')
+      assert_that(result5, equal_to([20]), label='assert5')
+      assert_that(result6, equal_to([10]), label='assert6')
+      assert_that(result7, equal_to([10]), label='assert7')
+      assert_that(result8, equal_to([5]), label='assert8')
 
-    pipeline.run()
 
   def test_do_fn_keyword_only_args(self):
-    pipeline = TestPipeline()
+    with TestPipeline() as pipeline:
 
-    class MyDoFn(beam.DoFn):
-      def process(self, element, *s, bound=500):
-        return [min(sum(s) + element, bound)]
+      class MyDoFn(beam.DoFn):
+        def process(self, element, *s, bound=500):
+          return [min(sum(s) + element, bound)]
 
-    pcoll = pipeline | 'start' >> beam.Create([6, 3, 1])
-    result1 = pcoll | 'sum1' >> beam.ParDo(MyDoFn(), 5, 8, bound=15)
-    result2 = pcoll | 'sum2' >> beam.ParDo(MyDoFn(), 5, 8)
-    result3 = pcoll | 'sum3' >> beam.ParDo(MyDoFn())
-    result4 = pcoll | 'sum4' >> beam.ParDo(MyDoFn(), bound=5)
+      pcoll = pipeline | 'start' >> beam.Create([6, 3, 1])
+      result1 = pcoll | 'sum1' >> beam.ParDo(MyDoFn(), 5, 8, bound=15)
+      result2 = pcoll | 'sum2' >> beam.ParDo(MyDoFn(), 5, 8)
+      result3 = pcoll | 'sum3' >> beam.ParDo(MyDoFn())
+      result4 = pcoll | 'sum4' >> beam.ParDo(MyDoFn(), bound=5)
 
-    assert_that(result1, equal_to([15, 15, 14]), label='assert1')
-    assert_that(result2, equal_to([19, 16, 14]), label='assert2')
-    assert_that(result3, equal_to([6, 3, 1]), label='assert3')
-    assert_that(result4, equal_to([5, 3, 1]), label='assert4')
-    pipeline.run()
+      assert_that(result1, equal_to([15, 15, 14]), label='assert1')
+      assert_that(result2, equal_to([19, 16, 14]), label='assert2')
+      assert_that(result3, equal_to([6, 3, 1]), label='assert3')
+      assert_that(result4, equal_to([5, 3, 1]), label='assert4')
 
 
 if __name__ == '__main__':
diff --git a/sdks/python/apache_beam/transforms/userstate_test.py b/sdks/python/apache_beam/transforms/userstate_test.py
index 9c79551..30aa88c 100644
--- a/sdks/python/apache_beam/transforms/userstate_test.py
+++ b/sdks/python/apache_beam/transforms/userstate_test.py
@@ -518,19 +518,16 @@
           aggregated_value += saved_value
         yield aggregated_value
 
-    p = TestPipeline()
-    values = p | beam.Create([('key', 1),
-                              ('key', 2),
-                              ('key', 3),
-                              ('key', 4),
-                              ('key', 3)], reshuffle=False)
-    actual_values = (values
-                     | beam.ParDo(SetStatefulDoFn()))
+    with TestPipeline() as p:
+      values = p | beam.Create([('key', 1),
+                                ('key', 2),
+                                ('key', 3),
+                                ('key', 4),
+                                ('key', 3)], reshuffle=False)
+      actual_values = (values
+                       | beam.ParDo(SetStatefulDoFn()))
+      assert_that(actual_values, equal_to([1, 3, 6, 10, 10]))
 
-    assert_that(actual_values, equal_to([1, 3, 6, 10, 10]))
-
-    result = p.run()
-    result.wait_until_finish()
 
   def test_stateful_set_state_clean_portably(self):
 
@@ -557,21 +554,19 @@
       def emit_values(self, set_state=beam.DoFn.StateParam(SET_STATE)):
         yield sorted(set_state.read())
 
-    p = TestPipeline()
-    values = p | beam.Create([('key', 1),
-                              ('key', 2),
-                              ('key', 3),
-                              ('key', 4),
-                              ('key', 5)])
-    actual_values = (values
-                     | beam.Map(lambda t: window.TimestampedValue(t, 1))
-                     | beam.WindowInto(window.FixedWindows(1))
-                     | beam.ParDo(SetStateClearingStatefulDoFn()))
+    with TestPipeline() as p:
+      values = p | beam.Create([('key', 1),
+                                ('key', 2),
+                                ('key', 3),
+                                ('key', 4),
+                                ('key', 5)])
+      actual_values = (values
+                       | beam.Map(lambda t: window.TimestampedValue(t, 1))
+                       | beam.WindowInto(window.FixedWindows(1))
+                       | beam.ParDo(SetStateClearingStatefulDoFn()))
 
-    assert_that(actual_values, equal_to([[100]]))
+      assert_that(actual_values, equal_to([[100]]))
 
-    result = p.run()
-    result.wait_until_finish()
 
   def test_stateful_dofn_nonkeyed_input(self):
     p = TestPipeline()
diff --git a/sdks/python/apache_beam/transforms/util_test.py b/sdks/python/apache_beam/transforms/util_test.py
index d7290ce..fc32874 100644
--- a/sdks/python/apache_beam/transforms/util_test.py
+++ b/sdks/python/apache_beam/transforms/util_test.py
@@ -284,23 +284,22 @@
         yield WindowedValue(
             element, expected_timestamp, [expected_window])
 
-    pipeline = TestPipeline()
-    data = [(1, 1), (2, 1), (3, 1), (1, 2), (2, 2), (1, 4)]
-    expected_windows = [
-        TestWindowedValue(kv, expected_timestamp, [expected_window])
-        for kv in data]
-    before_identity = (pipeline
-                       | 'start' >> beam.Create(data)
-                       | 'add_windows' >> beam.ParDo(AddWindowDoFn()))
-    assert_that(before_identity, equal_to(expected_windows),
-                label='before_identity', reify_windows=True)
-    after_identity = (before_identity
-                      | 'window' >> beam.WindowInto(
-                          beam.transforms.util._IdentityWindowFn(
-                              coders.IntervalWindowCoder())))
-    assert_that(after_identity, equal_to(expected_windows),
-                label='after_identity', reify_windows=True)
-    pipeline.run()
+    with TestPipeline() as pipeline:
+      data = [(1, 1), (2, 1), (3, 1), (1, 2), (2, 2), (1, 4)]
+      expected_windows = [
+          TestWindowedValue(kv, expected_timestamp, [expected_window])
+          for kv in data]
+      before_identity = (pipeline
+                         | 'start' >> beam.Create(data)
+                         | 'add_windows' >> beam.ParDo(AddWindowDoFn()))
+      assert_that(before_identity, equal_to(expected_windows),
+                  label='before_identity', reify_windows=True)
+      after_identity = (before_identity
+                        | 'window' >> beam.WindowInto(
+                            beam.transforms.util._IdentityWindowFn(
+                                coders.IntervalWindowCoder())))
+      assert_that(after_identity, equal_to(expected_windows),
+                  label='after_identity', reify_windows=True)
 
   def test_no_window_context_fails(self):
     expected_timestamp = timestamp.Timestamp(5)
@@ -311,40 +310,38 @@
       def process(self, element):
         yield window.TimestampedValue(element, expected_timestamp)
 
-    pipeline = TestPipeline()
-    data = [(1, 1), (2, 1), (3, 1), (1, 2), (2, 2), (1, 4)]
-    expected_windows = [
-        TestWindowedValue(kv, expected_timestamp, [expected_window])
-        for kv in data]
-    before_identity = (pipeline
-                       | 'start' >> beam.Create(data)
-                       | 'add_timestamps' >> beam.ParDo(AddTimestampDoFn()))
-    assert_that(before_identity, equal_to(expected_windows),
-                label='before_identity', reify_windows=True)
-    after_identity = (before_identity
-                      | 'window' >> beam.WindowInto(
-                          beam.transforms.util._IdentityWindowFn(
-                              coders.GlobalWindowCoder()))
-                      # This DoFn will return TimestampedValues, making
-                      # WindowFn.AssignContext passed to IdentityWindowFn
-                      # contain a window of None. IdentityWindowFn should
-                      # raise an exception.
-                      | 'add_timestamps2' >> beam.ParDo(AddTimestampDoFn()))
-    assert_that(after_identity, equal_to(expected_windows),
-                label='after_identity', reify_windows=True)
     with self.assertRaisesRegex(ValueError, r'window.*None.*add_timestamps2'):
-      pipeline.run()
+      with TestPipeline() as pipeline:
+        data = [(1, 1), (2, 1), (3, 1), (1, 2), (2, 2), (1, 4)]
+        expected_windows = [
+            TestWindowedValue(kv, expected_timestamp, [expected_window])
+            for kv in data]
+        before_identity = (pipeline
+                           | 'start' >> beam.Create(data)
+                           | 'add_timestamps' >> beam.ParDo(AddTimestampDoFn()))
+        assert_that(before_identity, equal_to(expected_windows),
+                    label='before_identity', reify_windows=True)
+        after_identity = (before_identity
+                          | 'window' >> beam.WindowInto(
+                              beam.transforms.util._IdentityWindowFn(
+                                  coders.GlobalWindowCoder()))
+                          # This DoFn will return TimestampedValues, making
+                          # WindowFn.AssignContext passed to IdentityWindowFn
+                          # contain a window of None. IdentityWindowFn should
+                          # raise an exception.
+                          | 'add_timestamps2' >> beam.ParDo(AddTimestampDoFn()))
+        assert_that(after_identity, equal_to(expected_windows),
+                    label='after_identity', reify_windows=True)
 
 class ReshuffleTest(unittest.TestCase):
 
   def test_reshuffle_contents_unchanged(self):
-    pipeline = TestPipeline()
-    data = [(1, 1), (2, 1), (3, 1), (1, 2), (2, 2), (1, 3)]
-    result = (pipeline
-              | beam.Create(data)
-              | beam.Reshuffle())
-    assert_that(result, equal_to(data))
-    pipeline.run()
+    with TestPipeline() as pipeline:
+      data = [(1, 1), (2, 1), (3, 1), (1, 2), (2, 2), (1, 3)]
+      result = (pipeline
+                | beam.Create(data)
+                | beam.Reshuffle())
+      assert_that(result, equal_to(data))
 
   def test_reshuffle_after_gbk_contents_unchanged(self):
     pipeline = TestPipeline()
@@ -362,74 +359,72 @@
     pipeline.run()
 
   def test_reshuffle_timestamps_unchanged(self):
-    pipeline = TestPipeline()
-    timestamp = 5
-    data = [(1, 1), (2, 1), (3, 1), (1, 2), (2, 2), (1, 3)]
-    expected_result = [TestWindowedValue(v, timestamp, [GlobalWindow()])
-                       for v in data]
-    before_reshuffle = (pipeline
-                        | 'start' >> beam.Create(data)
-                        | 'add_timestamp' >> beam.Map(
-                            lambda v: beam.window.TimestampedValue(v,
-                                                                   timestamp)))
-    assert_that(before_reshuffle, equal_to(expected_result),
-                label='before_reshuffle', reify_windows=True)
-    after_reshuffle = before_reshuffle | beam.Reshuffle()
-    assert_that(after_reshuffle, equal_to(expected_result),
-                label='after_reshuffle', reify_windows=True)
-    pipeline.run()
+    with TestPipeline() as pipeline:
+      timestamp = 5
+      data = [(1, 1), (2, 1), (3, 1), (1, 2), (2, 2), (1, 3)]
+      expected_result = [TestWindowedValue(v, timestamp, [GlobalWindow()])
+                         for v in data]
+      before_reshuffle = (pipeline
+                          | 'start' >> beam.Create(data)
+                          | 'add_timestamp' >> beam.Map(
+                              lambda v: beam.window.TimestampedValue(
+                                  v, timestamp)))
+      assert_that(before_reshuffle, equal_to(expected_result),
+                  label='before_reshuffle', reify_windows=True)
+      after_reshuffle = before_reshuffle | beam.Reshuffle()
+      assert_that(after_reshuffle, equal_to(expected_result),
+                  label='after_reshuffle', reify_windows=True)
 
   def test_reshuffle_windows_unchanged(self):
-    pipeline = TestPipeline()
-    data = [(1, 1), (2, 1), (3, 1), (1, 2), (2, 2), (1, 4)]
-    expected_data = [TestWindowedValue(v, t - .001, [w]) for (v, t, w) in [
-        ((1, contains_in_any_order([2, 1])), 4.0, IntervalWindow(1.0, 4.0)),
-        ((2, contains_in_any_order([2, 1])), 4.0, IntervalWindow(1.0, 4.0)),
-        ((3, [1]), 3.0, IntervalWindow(1.0, 3.0)),
-        ((1, [4]), 6.0, IntervalWindow(4.0, 6.0))]]
-    before_reshuffle = (pipeline
-                        | 'start' >> beam.Create(data)
-                        | 'add_timestamp' >> beam.Map(
-                            lambda v: beam.window.TimestampedValue(v, v[1]))
-                        | 'window' >> beam.WindowInto(Sessions(gap_size=2))
-                        | 'group_by_key' >> beam.GroupByKey())
-    assert_that(before_reshuffle, equal_to(expected_data),
-                label='before_reshuffle', reify_windows=True)
-    after_reshuffle = before_reshuffle | beam.Reshuffle()
-    assert_that(after_reshuffle, equal_to(expected_data),
-                label='after reshuffle', reify_windows=True)
-    pipeline.run()
+    with TestPipeline() as pipeline:
+      data = [(1, 1), (2, 1), (3, 1), (1, 2), (2, 2), (1, 4)]
+      expected_data = [TestWindowedValue(v, t - .001, [w]) for (v, t, w) in [
+          ((1, contains_in_any_order([2, 1])), 4.0, IntervalWindow(1.0, 4.0)),
+          ((2, contains_in_any_order([2, 1])), 4.0, IntervalWindow(1.0, 4.0)),
+          ((3, [1]), 3.0, IntervalWindow(1.0, 3.0)),
+          ((1, [4]), 6.0, IntervalWindow(4.0, 6.0))]]
+      before_reshuffle = (pipeline
+                          | 'start' >> beam.Create(data)
+                          | 'add_timestamp' >> beam.Map(
+                              lambda v: beam.window.TimestampedValue(v, v[1]))
+                          | 'window' >> beam.WindowInto(Sessions(gap_size=2))
+                          | 'group_by_key' >> beam.GroupByKey())
+      assert_that(before_reshuffle, equal_to(expected_data),
+                  label='before_reshuffle', reify_windows=True)
+      after_reshuffle = before_reshuffle | beam.Reshuffle()
+      assert_that(after_reshuffle, equal_to(expected_data),
+                  label='after reshuffle', reify_windows=True)
 
   def test_reshuffle_window_fn_preserved(self):
-    pipeline = TestPipeline()
-    data = [(1, 1), (2, 1), (3, 1), (1, 2), (2, 2), (1, 4)]
-    expected_windows = [TestWindowedValue(v, t, [w]) for (v, t, w) in [
-        ((1, 1), 1.0, IntervalWindow(1.0, 3.0)),
-        ((2, 1), 1.0, IntervalWindow(1.0, 3.0)),
-        ((3, 1), 1.0, IntervalWindow(1.0, 3.0)),
-        ((1, 2), 2.0, IntervalWindow(2.0, 4.0)),
-        ((2, 2), 2.0, IntervalWindow(2.0, 4.0)),
-        ((1, 4), 4.0, IntervalWindow(4.0, 6.0))]]
-    expected_merged_windows = [
-        TestWindowedValue(v, t - .001, [w]) for (v, t, w) in [
-            ((1, contains_in_any_order([2, 1])), 4.0, IntervalWindow(1.0, 4.0)),
-            ((2, contains_in_any_order([2, 1])), 4.0, IntervalWindow(1.0, 4.0)),
-            ((3, [1]), 3.0, IntervalWindow(1.0, 3.0)),
-            ((1, [4]), 6.0, IntervalWindow(4.0, 6.0))]]
-    before_reshuffle = (pipeline
-                        | 'start' >> beam.Create(data)
-                        | 'add_timestamp' >> beam.Map(
-                            lambda v: TimestampedValue(v, v[1]))
-                        | 'window' >> beam.WindowInto(Sessions(gap_size=2)))
-    assert_that(before_reshuffle, equal_to(expected_windows),
-                label='before_reshuffle', reify_windows=True)
-    after_reshuffle = before_reshuffle | beam.Reshuffle()
-    assert_that(after_reshuffle, equal_to(expected_windows),
-                label='after_reshuffle', reify_windows=True)
-    after_group = after_reshuffle | beam.GroupByKey()
-    assert_that(after_group, equal_to(expected_merged_windows),
-                label='after_group', reify_windows=True)
-    pipeline.run()
+    any_order = contains_in_any_order
+    with TestPipeline() as pipeline:
+      data = [(1, 1), (2, 1), (3, 1), (1, 2), (2, 2), (1, 4)]
+      expected_windows = [TestWindowedValue(v, t, [w]) for (v, t, w) in [
+          ((1, 1), 1.0, IntervalWindow(1.0, 3.0)),
+          ((2, 1), 1.0, IntervalWindow(1.0, 3.0)),
+          ((3, 1), 1.0, IntervalWindow(1.0, 3.0)),
+          ((1, 2), 2.0, IntervalWindow(2.0, 4.0)),
+          ((2, 2), 2.0, IntervalWindow(2.0, 4.0)),
+          ((1, 4), 4.0, IntervalWindow(4.0, 6.0))]]
+      expected_merged_windows = [
+          TestWindowedValue(v, t - .001, [w]) for (v, t, w) in [
+              ((1, any_order([2, 1])), 4.0, IntervalWindow(1.0, 4.0)),
+              ((2, any_order([2, 1])), 4.0, IntervalWindow(1.0, 4.0)),
+              ((3, [1]), 3.0, IntervalWindow(1.0, 3.0)),
+              ((1, [4]), 6.0, IntervalWindow(4.0, 6.0))]]
+      before_reshuffle = (pipeline
+                          | 'start' >> beam.Create(data)
+                          | 'add_timestamp' >> beam.Map(
+                              lambda v: TimestampedValue(v, v[1]))
+                          | 'window' >> beam.WindowInto(Sessions(gap_size=2)))
+      assert_that(before_reshuffle, equal_to(expected_windows),
+                  label='before_reshuffle', reify_windows=True)
+      after_reshuffle = before_reshuffle | beam.Reshuffle()
+      assert_that(after_reshuffle, equal_to(expected_windows),
+                  label='after_reshuffle', reify_windows=True)
+      after_group = after_reshuffle | beam.GroupByKey()
+      assert_that(after_group, equal_to(expected_merged_windows),
+                  label='after_group', reify_windows=True)
 
   def test_reshuffle_global_window(self):
     pipeline = TestPipeline()
@@ -584,16 +579,16 @@
     return data
 
   def test_in_global_window(self):
-    pipeline = TestPipeline()
-    collection = pipeline \
-                 | beam.Create(GroupIntoBatchesTest._create_test_data()) \
-                 | util.GroupIntoBatches(GroupIntoBatchesTest.BATCH_SIZE)
-    num_batches = collection | beam.combiners.Count.Globally()
-    assert_that(num_batches,
-                equal_to([int(math.ceil(GroupIntoBatchesTest.NUM_ELEMENTS /
-                                        GroupIntoBatchesTest.BATCH_SIZE))]))
-    pipeline.run()
+    with TestPipeline() as pipeline:
+      collection = pipeline \
+                   | beam.Create(GroupIntoBatchesTest._create_test_data()) \
+                   | util.GroupIntoBatches(GroupIntoBatchesTest.BATCH_SIZE)
+      num_batches = collection | beam.combiners.Count.Globally()
+      assert_that(num_batches,
+                  equal_to([int(math.ceil(GroupIntoBatchesTest.NUM_ELEMENTS /
+                                          GroupIntoBatchesTest.BATCH_SIZE))]))
 
+  @unittest.skip('BEAM-8748')
   def test_in_streaming_mode(self):
     timestamp_interval = 1
     offset = itertools.count(0)
@@ -609,26 +604,23 @@
                    .advance_watermark_to(start_time +
                                          GroupIntoBatchesTest.NUM_ELEMENTS)
                    .advance_watermark_to_infinity())
-    pipeline = TestPipeline(options=StandardOptions(streaming=True))
-    #  window duration is 6 and batch size is 5, so output batch size should be
-    #  5 (flush because of batchSize reached)
-    expected_0 = 5
-    # there is only one element left in the window so batch size should be 1
-    # (flush because of end of window reached)
-    expected_1 = 1
-    #  collection is 10 elements, there is only 4 left, so batch size should be
-    #  4 (flush because end of collection reached)
-    expected_2 = 4
+    with TestPipeline(options=StandardOptions(streaming=True)) as pipeline:
+      # window duration is 6 and batch size is 5, so output batch size
+      # should be 5 (flush because of batchSize reached)
+      expected_0 = 5
+      # there is only one element left in the window so batch size
+      # should be 1 (flush because of end of window reached)
+      expected_1 = 1
+      # collection is 10 elements, there is only 4 left, so batch size
+      # should be 4 (flush because end of collection reached)
+      expected_2 = 4
 
-    collection = pipeline | test_stream \
-                 | WindowInto(FixedWindows(window_duration)) \
-                 | util.GroupIntoBatches(GroupIntoBatchesTest.BATCH_SIZE)
-    num_elements_in_batches = collection | beam.Map(len)
-
-    result = pipeline.run()
-    result.wait_until_finish()
-    assert_that(num_elements_in_batches,
-                equal_to([expected_0, expected_1, expected_2]))
+      collection = pipeline | test_stream \
+                   | WindowInto(FixedWindows(window_duration)) \
+                   | util.GroupIntoBatches(GroupIntoBatchesTest.BATCH_SIZE)
+      num_elements_in_batches = collection | beam.Map(len)
+      assert_that(num_elements_in_batches,
+                  equal_to([expected_0, expected_1, expected_2]))
 
 
 class ToStringTest(unittest.TestCase):
diff --git a/sdks/python/apache_beam/typehints/opcodes.py b/sdks/python/apache_beam/typehints/opcodes.py
index 1109627..8034d0d 100644
--- a/sdks/python/apache_beam/typehints/opcodes.py
+++ b/sdks/python/apache_beam/typehints/opcodes.py
@@ -155,7 +155,7 @@
 
 def binary_subscr(state, unused_arg):
   index = state.stack.pop()
-  base = state.stack.pop()
+  base = Const.unwrap(state.stack.pop())
   if base in (str, unicode):
     out = base
   elif (isinstance(index, Const) and isinstance(index.value, int)
diff --git a/sdks/python/apache_beam/typehints/trivial_inference_test.py b/sdks/python/apache_beam/typehints/trivial_inference_test.py
index f163520..cd2ce29 100644
--- a/sdks/python/apache_beam/typehints/trivial_inference_test.py
+++ b/sdks/python/apache_beam/typehints/trivial_inference_test.py
@@ -72,6 +72,11 @@
     self.assertReturnType(str, lambda v: v[::-1], [str])
     self.assertReturnType(typehints.Any, lambda v: v[::-1], [typehints.Any])
     self.assertReturnType(typehints.Any, lambda v: v[::-1], [object])
+    if sys.version_info >= (3,):
+      # Test binary_subscr on a slice of a Const. On Py2.7 this will use the
+      # unsupported opcode SLICE+0.
+      test_list = ['a', 'b']
+      self.assertReturnType(typehints.List[str], lambda: test_list[:], [])
 
   def testUnpack(self):
     def reverse(a_b):
diff --git a/sdks/python/apache_beam/typehints/typed_pipeline_test.py b/sdks/python/apache_beam/typehints/typed_pipeline_test.py
index fafd386..52cd4ee 100644
--- a/sdks/python/apache_beam/typehints/typed_pipeline_test.py
+++ b/sdks/python/apache_beam/typehints/typed_pipeline_test.py
@@ -137,19 +137,18 @@
     self.assertEqual([1, 3], [1, 2, 3] | beam.Filter(filter_fn))
 
   def test_partition(self):
-    p = TestPipeline()
-    even, odd = (p
-                 | beam.Create([1, 2, 3])
-                 | 'even_odd' >> beam.Partition(lambda e, _: e % 2, 2))
-    self.assertIsNotNone(even.element_type)
-    self.assertIsNotNone(odd.element_type)
-    res_even = (even
-                | 'id_even' >> beam.ParDo(lambda e: [e]).with_input_types(int))
-    res_odd = (odd
-               | 'id_odd' >> beam.ParDo(lambda e: [e]).with_input_types(int))
-    assert_that(res_even, equal_to([2]), label='even_check')
-    assert_that(res_odd, equal_to([1, 3]), label='odd_check')
-    p.run()
+    with TestPipeline() as p:
+      even, odd = (p
+                   | beam.Create([1, 2, 3])
+                   | 'even_odd' >> beam.Partition(lambda e, _: e % 2, 2))
+      self.assertIsNotNone(even.element_type)
+      self.assertIsNotNone(odd.element_type)
+      res_even = (even
+                  | 'IdEven' >> beam.ParDo(lambda e: [e]).with_input_types(int))
+      res_odd = (odd
+                 | 'IdOdd' >> beam.ParDo(lambda e: [e]).with_input_types(int))
+      assert_that(res_even, equal_to([2]), label='even_check')
+      assert_that(res_odd, equal_to([1, 3]), label='odd_check')
 
   def test_typed_dofn_multi_output(self):
     class MyDoFn(beam.DoFn):
diff --git a/sdks/python/apache_beam/version.py b/sdks/python/apache_beam/version.py
index ba28fb7..be5d97f 100644
--- a/sdks/python/apache_beam/version.py
+++ b/sdks/python/apache_beam/version.py
@@ -18,4 +18,4 @@
 """Apache Beam SDK version information and utilities."""
 
 
-__version__ = '2.19.0.dev'
+__version__ = '2.20.0.dev'
diff --git a/sdks/python/build.gradle b/sdks/python/build.gradle
index d3f65a9..ca9cae8 100644
--- a/sdks/python/build.gradle
+++ b/sdks/python/build.gradle
@@ -48,7 +48,7 @@
       args '-c', ". ${envdir}/bin/activate && python setup.py -q sdist --formats zip,gztar --dist-dir ${buildDir}"
     }
 
-    def collection = fileTree(buildDir){ include "**/*${project['python_sdk_version']}*.tar.gz" exclude 'srcs/**'}
+    def collection = fileTree(buildDir){ include "**/*${project.sdk_version}*.tar.gz" exclude 'srcs/**'}
 
     // we need a fixed name for the artifact
     copy { from collection.singleFile; into buildDir; rename { tarball } }
diff --git a/sdks/python/container/py2/build.gradle b/sdks/python/container/py2/build.gradle
index 64f39f0..4e29de0 100644
--- a/sdks/python/container/py2/build.gradle
+++ b/sdks/python/container/py2/build.gradle
@@ -57,7 +57,7 @@
           root: project.rootProject.hasProperty(["docker-repository-root"]) ?
                   project.rootProject["docker-repository-root"] : "apachebeam",
           tag: project.rootProject.hasProperty(["docker-tag"]) ?
-                  project.rootProject["docker-tag"] : project['python_sdk_version'])
+                  project.rootProject["docker-tag"] : project.sdk_version)
   files "../Dockerfile", "./build"
   buildArgs(['py_version': "2.7"])
 }
diff --git a/sdks/python/container/py35/build.gradle b/sdks/python/container/py35/build.gradle
index 024847b..f8cd8c2 100644
--- a/sdks/python/container/py35/build.gradle
+++ b/sdks/python/container/py35/build.gradle
@@ -57,7 +57,7 @@
           root: project.rootProject.hasProperty(["docker-repository-root"]) ?
                   project.rootProject["docker-repository-root"] : "apachebeam",
           tag: project.rootProject.hasProperty(["docker-tag"]) ?
-                  project.rootProject["docker-tag"] : project['python_sdk_version'])
+                  project.rootProject["docker-tag"] : project.sdk_version)
   files "../Dockerfile", "./build"
   buildArgs(['py_version': "3.5"])
 }
diff --git a/sdks/python/container/py36/build.gradle b/sdks/python/container/py36/build.gradle
index f81f6ec..b7ced3d 100644
--- a/sdks/python/container/py36/build.gradle
+++ b/sdks/python/container/py36/build.gradle
@@ -57,7 +57,7 @@
           root: project.rootProject.hasProperty(["docker-repository-root"]) ?
                   project.rootProject["docker-repository-root"] : "apachebeam",
           tag: project.rootProject.hasProperty(["docker-tag"]) ?
-                  project.rootProject["docker-tag"] : project['python_sdk_version'])
+                  project.rootProject["docker-tag"] : project.sdk_version)
   files "../Dockerfile", "./build"
   buildArgs(['py_version': "3.6"])
 }
diff --git a/sdks/python/container/py37/build.gradle b/sdks/python/container/py37/build.gradle
index a7f10c4..d9e9b5d 100644
--- a/sdks/python/container/py37/build.gradle
+++ b/sdks/python/container/py37/build.gradle
@@ -57,7 +57,7 @@
           root: project.rootProject.hasProperty(["docker-repository-root"]) ?
                   project.rootProject["docker-repository-root"] : "apachebeam",
           tag: project.rootProject.hasProperty(["docker-tag"]) ?
-                  project.rootProject["docker-tag"] : project['python_sdk_version'])
+                  project.rootProject["docker-tag"] : project.sdk_version)
   files "../Dockerfile", "./build"
   buildArgs(['py_version': "3.7"])
 }
diff --git a/sdks/python/scripts/run_integration_test.sh b/sdks/python/scripts/run_integration_test.sh
index 1133147..0804ace 100755
--- a/sdks/python/scripts/run_integration_test.sh
+++ b/sdks/python/scripts/run_integration_test.sh
@@ -194,8 +194,10 @@
   fi
 
   # Install test dependencies for ValidatesRunner tests.
-  echo "pyhamcrest" > postcommit_requirements.txt
-  echo "mock" >> postcommit_requirements.txt
+  # pyhamcrest==1.10.0 doesn't work on Py2.
+  # See: https://github.com/hamcrest/PyHamcrest/issues/131.
+  echo "pyhamcrest!=1.10.0,<2.0.0" > postcommit_requirements.txt
+  echo "mock<3.0.0" >> postcommit_requirements.txt
 
   # Options used to run testing pipeline on Cloud Dataflow Service. Also used for
   # running on DirectRunner (some options ignored).
diff --git a/sdks/python/setup.py b/sdks/python/setup.py
index 8599b7f..a396d17 100644
--- a/sdks/python/setup.py
+++ b/sdks/python/setup.py
@@ -181,8 +181,9 @@
     'nose_xunitmp>=0.4.1',
     'pandas>=0.23.4,<0.25',
     'parameterized>=0.6.0,<0.8.0',
-    # pyhamcrest==1.10.0 requires Py3. Beam still supports Py2.
-    'pyhamcrest>=1.9,<1.10.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',
     'tenacity>=5.0.2,<6.0',
@@ -203,6 +204,8 @@
     'google-cloud-bigtable>=0.31.1,<1.1.0',
     # [BEAM-4543] googledatastore is not supported in Python 3.
     'proto-google-cloud-datastore-v1>=0.90.0,<=0.90.4; python_version < "3.0"',
+    'google-cloud-spanner>=1.7.1<1.8.0',
+    'grpcio-gcp>=0.2.2,<1',
 ]
 
 INTERACTIVE_BEAM = [
diff --git a/sdks/python/test-suites/direct/py2/build.gradle b/sdks/python/test-suites/direct/py2/build.gradle
index 436cd7e..acb7fee 100644
--- a/sdks/python/test-suites/direct/py2/build.gradle
+++ b/sdks/python/test-suites/direct/py2/build.gradle
@@ -77,11 +77,10 @@
 }
 
 task hdfsIntegrationTest {
-  dependsOn 'installGcpTest'
   doLast {
     exec {
       executable 'sh'
-      args '-c', ". ${envdir}/bin/activate && ${rootDir}/sdks/python/apache_beam/io/hdfs_integration_test/hdfs_integration_test.sh python:2"
+      args '-c', "${rootDir}/sdks/python/apache_beam/io/hdfs_integration_test/hdfs_integration_test.sh python:2"
     }
   }
 }
diff --git a/sdks/python/test-suites/direct/py37/build.gradle b/sdks/python/test-suites/direct/py37/build.gradle
index c5c30fb..1f54f90 100644
--- a/sdks/python/test-suites/direct/py37/build.gradle
+++ b/sdks/python/test-suites/direct/py37/build.gradle
@@ -56,11 +56,10 @@
 }
 
 task hdfsIntegrationTest {
-  dependsOn 'installGcpTest'
   doLast {
     exec {
       executable 'sh'
-      args '-c', ". ${envdir}/bin/activate && ${pythonDir}/apache_beam/io/hdfs_integration_test/hdfs_integration_test.sh python:3.7"
+      args '-c', "${pythonDir}/apache_beam/io/hdfs_integration_test/hdfs_integration_test.sh python:3.7"
     }
   }
 }
diff --git a/sdks/python/tox.ini b/sdks/python/tox.ini
index c78bddf..e6908d9 100644
--- a/sdks/python/tox.ini
+++ b/sdks/python/tox.ini
@@ -315,3 +315,30 @@
   coverage report --skip-covered
   # Generate report in xml format
   coverage xml
+
+[testenv:hdfs_integration_test]
+# Used by hdfs_integration_test.sh. Do not run this directly, as it depends on
+# nodes defined in hdfs_integration_test/docker-compose.yml.
+deps =
+  -r build-requirements.txt
+  gsutil==4.47
+  holdup==1.8.0
+extras =
+  gcp
+whitelist_externals =
+  echo
+  sleep
+passenv = HDFSCLI_CONFIG
+commands =
+  holdup -t 45 http://namenode:50070 http://datanode:50075
+  echo "Waiting for safe mode to end."
+  sleep 45
+  gsutil cp gs://dataflow-samples/shakespeare/kinglear.txt .
+  hdfscli -v -v -v upload -f kinglear.txt /
+  python -m apache_beam.examples.wordcount \
+      --input hdfs://kinglear* \
+      --output hdfs://py-wordcount-integration \
+      --hdfs_host namenode --hdfs_port 50070 --hdfs_user root
+# Disable pip check. TODO: remove this once gsutil does not conflict with
+# apache_beam (oauth2client).
+commands_pre =
diff --git a/vendor/sdks-java-extensions-protobuf/build.gradle b/vendor/sdks-java-extensions-protobuf/build.gradle
index e3f0c94..a174db6 100644
--- a/vendor/sdks-java-extensions-protobuf/build.gradle
+++ b/vendor/sdks-java-extensions-protobuf/build.gradle
@@ -16,14 +16,16 @@
  * limitations under the License.
  */
 
+import org.apache.beam.gradle.GrpcVendoring_1_26_0
+
 plugins { id 'org.apache.beam.module' }
 applyJavaNature(
   automaticModuleName: 'org.apache.beam.vendor.sdks.java.extensions.protobuf',
   exportJavadoc: false,
   shadowClosure: {
     dependencies {
-        include(dependency('com.google.guava:guava:26.0-jre'))
-        include(dependency('com.google.protobuf:protobuf-java:3.7.1'))
+        include(dependency("com.google.guava:guava:${GrpcVendoring_1_26_0.guava_version}"))
+        include(dependency("com.google.protobuf:protobuf-java:${GrpcVendoring_1_26_0.protobuf_version}"))
     }
     // We specifically relocate beam-sdks-extensions-protobuf under a vendored namespace
     // but also vendor guava and protobuf to the same vendored namespace as the model/*
@@ -32,10 +34,10 @@
     relocate "org.apache.beam.sdk.extensions.protobuf", "org.apache.beam.vendor.sdk.v2.sdk.extensions.protobuf"
 
     // guava uses the com.google.common and com.google.thirdparty package namespaces
-    relocate "com.google.common", "org.apache.beam.vendor.grpc.v1p21p0.com.google.common"
-    relocate "com.google.thirdparty", "org.apache.beam.vendor.grpc.v1p21p0.com.google.thirdparty"
+    relocate "com.google.common", "org.apache.beam.vendor.grpc.v1p26p0.com.google.common"
+    relocate "com.google.thirdparty", "org.apache.beam.vendor.grpc.v1p26p0.com.google.thirdparty"
 
-    relocate "com.google.protobuf", "org.apache.beam.vendor.grpc.v1p21p0.com.google.protobuf"
+    relocate "com.google.protobuf", "org.apache.beam.vendor.grpc.v1p26p0.com.google.protobuf"
   }
 )
 
@@ -54,7 +56,7 @@
 }
 
 dependencies {
-    compile 'com.google.guava:guava:26.0-jre'
-    compile 'com.google.protobuf:protobuf-java:3.7.1'
+    compile "com.google.guava:guava:${GrpcVendoring_1_26_0.guava_version}"
+    compile "com.google.protobuf:protobuf-java:${GrpcVendoring_1_26_0.protobuf_version}"
     shadow project(path: ":sdks:java:core", configuration: "shadow")
 }
diff --git a/website/src/_includes/section-menu/get-started.html b/website/src/_includes/section-menu/get-started.html
index 61270a4..96facb3 100644
--- a/website/src/_includes/section-menu/get-started.html
+++ b/website/src/_includes/section-menu/get-started.html
@@ -30,4 +30,4 @@
    </ul>
 </li>
 <li><a href="{{ site.baseurl }}/get-started/downloads">Downloads</a></li>
-
+<li><a href="{{ site.baseurl }}/security">Security</a></li>
diff --git a/website/src/_posts/2020-01-06-beam-2.17.0.md b/website/src/_posts/2020-01-06-beam-2.17.0.md
index c5c4716..5c7caeb 100644
--- a/website/src/_posts/2020-01-06-beam-2.17.0.md
+++ b/website/src/_posts/2020-01-06-beam-2.17.0.md
@@ -25,6 +25,8 @@
 -->
 
 We are happy to present the new 2.17.0 release of Beam. This release includes both improvements and new functionality.
+Users of the MongoDbIO connector are encouraged to upgrade to this release to address a [security vulnerability]({{ site.baseurl }}/security/CVE-2020-1929/).
+
 See the [download page]({{ site.baseurl }}/get-started/downloads/#2170-2020-01-06) for this release.<!--more-->
 For more information on changes in 2.17.0, check out the
 [detailed release notes](https://issues.apache.org/jira/secure/ReleaseNote.jspa?version=12345970&projectId=12319527).
diff --git a/website/src/security/CVE-2020-1929.md b/website/src/security/CVE-2020-1929.md
new file mode 100644
index 0000000..27facc4
--- /dev/null
+++ b/website/src/security/CVE-2020-1929.md
@@ -0,0 +1,17 @@
+---
+permalink: /security/CVE-2020-1929/
+redirect_to: /security/index.html#cve-2020-1929
+---
+<!--
+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.
+-->
diff --git a/website/src/security/index.md b/website/src/security/index.md
new file mode 100644
index 0000000..c8db8e1
--- /dev/null
+++ b/website/src/security/index.md
@@ -0,0 +1,56 @@
+---
+layout: section
+title: "Beam Security"
+permalink: security/
+section_menu: section-menu/get-started.html
+---
+<!--
+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.
+-->
+
+# Reporting Security Issues
+
+Apache Beam uses the standard process outlined by the [Apache Security
+Team](https://www.apache.org/security/) for reporting vulnerabilities. Note
+that vulnerabilities should not be publicly disclosed until the project has
+responded.
+
+To report a possible security vulnerability, please email
+`security@apache.org` and `pmc@beam.apache.org`. This is a non-public list
+that will reach the Beam PMC.
+
+# Known Security Issues
+
+## CVE-2020-1929
+
+[CVE-2020-1929] Apache Beam MongoDB IO connector disables certificate trust verification
+
+Severity: Major  
+Vendor: The Apache Software Foundation   
+
+Versions Affected:  
+Apache Beam 2.10.0 to 2.16.0
+
+Description:  
+The Apache Beam MongoDB connector in versions 2.10.0 to 2.16.0 has an option to
+disable SSL trust verification. However this configuration is not respected and
+the certificate verification disables trust verification in every case. This
+exclusion also gets registered globally which disables trust checking for any
+code running in the same JVM.
+
+Mitigation:  
+Users of the affected versions should apply one of the following mitigations:
+- Upgrade to Apache Beam 2.17.0 or later
+
+Acknowledgements:  
+This issue was reported (and fixed) by Colm Ó hÉigeartaigh.