[BEAM-12802] Define a prefetchable iterator and iterable and utility functions that work over them.

diff --git a/.gitignore b/.gitignore
index e88001a..33aac5b 100644
--- a/.gitignore
+++ b/.gitignore
@@ -17,6 +17,10 @@
 **/.gradletasknamecache
 **/generated/*
 
+# Ignore sources generated into the main tree
+**/src/main/generated/**
+**/src/test/generated_tests/**
+
 # Ignore files generated by the Maven build process.
 **/bin/**/*
 **/dependency-reduced-pom.xml
diff --git a/.test-infra/jenkins/job_PerformanceTests_KafkaIO_IT.groovy b/.test-infra/jenkins/job_PerformanceTests_KafkaIO_IT.groovy
index 87767f8..4c665a3 100644
--- a/.test-infra/jenkins/job_PerformanceTests_KafkaIO_IT.groovy
+++ b/.test-infra/jenkins/job_PerformanceTests_KafkaIO_IT.groovy
@@ -24,7 +24,8 @@
 
 job(jobName) {
   common.setTopLevelMainJobProperties(delegate)
-  common.setAutoJob(delegate, 'H */6 * * *')
+  // TODO(BEAM-9482): Re-enable once fixed.
+  // common.setAutoJob(delegate, 'H */6 * * *')
   common.enablePhraseTriggeringFromPullRequest(
       delegate,
       'Java KafkaIO Performance Test',
diff --git a/.test-infra/metrics/docker-compose.yml b/.test-infra/metrics/docker-compose.yml
index d47c7d7..dfa4a50 100644
--- a/.test-infra/metrics/docker-compose.yml
+++ b/.test-infra/metrics/docker-compose.yml
@@ -64,6 +64,7 @@
       - GF_SECURITY_ADMIN_PASSWORD=<GrafanaPasswordHere>
       - GF_AUTH_ANONYMOUS_ENABLED=true
       - GF_AUTH_ANONYMOUS_ORG_NAME=Beam
+      - GF_INSTALL_PLUGINS=marcusolsson-json-datasource
       - PSQL_DB_HOST=beampostgresql
       - PSQL_DB_PORT=5432
       - PSQL_DB_DBNAME=beam_metrics
diff --git a/.test-infra/metrics/grafana/dashboards/github_actions_post-commit_tests.json b/.test-infra/metrics/grafana/dashboards/github_actions_post-commit_tests.json
new file mode 100644
index 0000000..11ad969
--- /dev/null
+++ b/.test-infra/metrics/grafana/dashboards/github_actions_post-commit_tests.json
@@ -0,0 +1,558 @@
+{
+  "annotations": {
+    "list": [
+      {
+        "builtIn": 1,
+        "datasource": "-- Grafana --",
+        "enable": true,
+        "hide": true,
+        "iconColor": "rgba(0, 211, 255, 1)",
+        "name": "Annotations & Alerts",
+        "type": "dashboard"
+      }
+    ]
+  },
+  "editable": true,
+  "gnetId": null,
+  "graphTooltip": 0,
+  "id": 2,
+  "links": [],
+  "panels": [
+    {
+      "datasource": "Python Tests",
+      "fieldConfig": {
+        "defaults": {
+          "color": {
+            "mode": "palette-classic"
+          },
+          "custom": {
+            "hideFrom": {
+              "legend": false,
+              "tooltip": false,
+              "viz": false
+            }
+          },
+          "mappings": []
+        },
+        "overrides": [
+          {
+            "matcher": {
+              "id": "byFrameRefID",
+              "options": "A"
+            },
+            "properties": [
+              {
+                "id": "color",
+                "value": {
+                  "fixedColor": "green",
+                  "mode": "fixed"
+                }
+              }
+            ]
+          },
+          {
+            "matcher": {
+              "id": "byFrameRefID",
+              "options": "B"
+            },
+            "properties": [
+              {
+                "id": "color",
+                "value": {
+                  "fixedColor": "dark-red",
+                  "mode": "fixed"
+                }
+              }
+            ]
+          },
+          {
+            "matcher": {
+              "id": "byFrameRefID",
+              "options": "C"
+            },
+            "properties": [
+              {
+                "id": "color",
+                "value": {
+                  "fixedColor": "super-light-yellow",
+                  "mode": "fixed"
+                }
+              }
+            ]
+          }
+        ]
+      },
+      "gridPos": {
+        "h": 8,
+        "w": 12,
+        "x": 0,
+        "y": 0
+      },
+      "id": 6,
+      "options": {
+        "displayLabels": [
+          "value"
+        ],
+        "legend": {
+          "displayMode": "list",
+          "placement": "right",
+          "values": []
+        },
+        "pieType": "pie",
+        "reduceOptions": {
+          "calcs": [
+            "count"
+          ],
+          "fields": "/.*/",
+          "values": false
+        },
+        "tooltip": {
+          "mode": "single"
+        }
+      },
+      "pluginVersion": "8.0.4",
+      "targets": [
+        {
+          "body": "",
+          "cacheDurationSeconds": 300,
+          "fields": [
+            {
+              "jsonPath": "$.workflow_runs[?(@.conclusion == \"success\")]",
+              "name": "success"
+            }
+          ],
+          "hide": false,
+          "method": "GET",
+          "queryParams": "",
+          "refId": "A",
+          "urlPath": ""
+        },
+        {
+          "cacheDurationSeconds": 300,
+          "fields": [
+            {
+              "jsonPath": "$.workflow_runs[?(@.conclusion == \"failure\")]",
+              "name": "failure"
+            }
+          ],
+          "hide": false,
+          "method": "GET",
+          "queryParams": "",
+          "refId": "B",
+          "urlPath": ""
+        },
+        {
+          "cacheDurationSeconds": 300,
+          "fields": [
+            {
+              "jsonPath": "$.workflow_runs[?(@.conclusion == \"cancelled\")]",
+              "name": "cancelled"
+            }
+          ],
+          "hide": false,
+          "method": "GET",
+          "queryParams": "",
+          "refId": "C",
+          "urlPath": ""
+        }
+      ],
+      "title": "Python last 100 post-commit tests results",
+      "transformations": [],
+      "type": "piechart"
+    },
+    {
+      "datasource": "Python Tests",
+      "fieldConfig": {
+        "defaults": {
+          "color": {
+            "mode": "palette-classic"
+          },
+          "custom": {
+            "align": "center",
+            "displayMode": "color-background"
+          },
+          "mappings": [
+            {
+              "options": {
+                "cancelled": {
+                  "color": "super-light-yellow",
+                  "index": 2
+                },
+                "failure": {
+                  "color": "dark-red",
+                  "index": 1
+                },
+                "success": {
+                  "color": "green",
+                  "index": 0
+                }
+              },
+              "type": "value"
+            }
+          ],
+          "noValue": "Fetching status",
+          "thresholds": {
+            "mode": "absolute",
+            "steps": [
+              {
+                "color": "green",
+                "value": null
+              },
+              {
+                "color": "red",
+                "value": 80
+              }
+            ]
+          }
+        },
+        "overrides": [
+          {
+            "matcher": {
+              "id": "byName",
+              "options": "Workflow URL"
+            },
+            "properties": [
+              {
+                "id": "custom.displayMode",
+                "value": "auto"
+              },
+              {
+                "id": "custom.width",
+                "value": 415
+              }
+            ]
+          },
+          {
+            "matcher": {
+              "id": "byName",
+              "options": "Created At"
+            },
+            "properties": [
+              {
+                "id": "custom.displayMode",
+                "value": "auto"
+              }
+            ]
+          }
+        ]
+      },
+      "gridPos": {
+        "h": 8,
+        "w": 12,
+        "x": 12,
+        "y": 0
+      },
+      "id": 8,
+      "options": {
+        "showHeader": true,
+        "sortBy": []
+      },
+      "pluginVersion": "8.0.4",
+      "targets": [
+        {
+          "body": "",
+          "cacheDurationSeconds": 300,
+          "fields": [
+            {
+              "jsonPath": "$.workflow_runs[*].id",
+              "name": "Workflow ID"
+            },
+            {
+              "jsonPath": "$.workflow_runs[*].conclusion",
+              "name": "Test Result"
+            },
+            {
+              "jsonPath": "$.workflow_runs[*].html_url",
+              "name": "Workflow URL"
+            },
+            {
+              "jsonPath": "$.workflow_runs[*].created_at",
+              "name": "Created At"
+            }
+          ],
+          "method": "GET",
+          "queryParams": "",
+          "refId": "A",
+          "urlPath": ""
+        }
+      ],
+      "title": "Python Workflow Results",
+      "type": "table"
+    },
+    {
+      "datasource": null,
+      "fieldConfig": {
+        "defaults": {
+          "color": {
+            "mode": "palette-classic"
+          },
+          "custom": {
+            "hideFrom": {
+              "legend": false,
+              "tooltip": false,
+              "viz": false
+            }
+          },
+          "mappings": []
+        },
+        "overrides": [
+          {
+            "matcher": {
+              "id": "byFrameRefID",
+              "options": "A"
+            },
+            "properties": [
+              {
+                "id": "color",
+                "value": {
+                  "fixedColor": "green",
+                  "mode": "fixed"
+                }
+              }
+            ]
+          },
+          {
+            "matcher": {
+              "id": "byFrameRefID",
+              "options": "B"
+            },
+            "properties": [
+              {
+                "id": "color",
+                "value": {
+                  "fixedColor": "dark-red",
+                  "mode": "fixed"
+                }
+              }
+            ]
+          },
+          {
+            "matcher": {
+              "id": "byFrameRefID",
+              "options": "C"
+            },
+            "properties": [
+              {
+                "id": "color",
+                "value": {
+                  "fixedColor": "super-light-yellow",
+                  "mode": "fixed"
+                }
+              }
+            ]
+          }
+        ]
+      },
+      "gridPos": {
+        "h": 9,
+        "w": 12,
+        "x": 0,
+        "y": 8
+      },
+      "id": 2,
+      "options": {
+        "displayLabels": [
+          "value"
+        ],
+        "legend": {
+          "displayMode": "list",
+          "placement": "right",
+          "values": []
+        },
+        "pieType": "pie",
+        "reduceOptions": {
+          "calcs": [
+            "count"
+          ],
+          "fields": "/.*/",
+          "values": false
+        },
+        "tooltip": {
+          "mode": "single"
+        }
+      },
+      "pluginVersion": "8.0.4",
+      "targets": [
+        {
+          "cacheDurationSeconds": 300,
+          "fields": [
+            {
+              "jsonPath": "$.workflow_runs[?(@.conclusion == \"success\")]",
+              "name": "success"
+            }
+          ],
+          "hide": false,
+          "method": "GET",
+          "queryParams": "",
+          "refId": "A",
+          "urlPath": ""
+        },
+        {
+          "cacheDurationSeconds": 300,
+          "fields": [
+            {
+              "jsonPath": "$.workflow_runs[?(@.conclusion == \"failure\")]",
+              "name": "failure"
+            }
+          ],
+          "hide": false,
+          "method": "GET",
+          "queryParams": "",
+          "refId": "B",
+          "urlPath": ""
+        },
+        {
+          "cacheDurationSeconds": 300,
+          "fields": [
+            {
+              "jsonPath": "$.workflow_runs[?(@.conclusion == \"cancelled\")]",
+              "name": "cancelled"
+            }
+          ],
+          "hide": false,
+          "method": "GET",
+          "queryParams": "",
+          "refId": "C",
+          "urlPath": ""
+        }
+      ],
+      "title": "Java last 100 post-commit tests results",
+      "type": "piechart"
+    },
+    {
+      "datasource": null,
+      "fieldConfig": {
+        "defaults": {
+          "color": {
+            "mode": "palette-classic"
+          },
+          "custom": {
+            "align": "center",
+            "displayMode": "color-background-solid"
+          },
+          "mappings": [
+            {
+              "options": {
+                "cancelled": {
+                  "color": "super-light-yellow",
+                  "index": 2
+                },
+                "failure": {
+                  "color": "dark-red",
+                  "index": 1
+                },
+                "success": {
+                  "color": "green",
+                  "index": 0
+                }
+              },
+              "type": "value"
+            }
+          ],
+          "noValue": "Fetching status",
+          "thresholds": {
+            "mode": "absolute",
+            "steps": [
+              {
+                "color": "green",
+                "value": null
+              },
+              {
+                "color": "red",
+                "value": 80
+              }
+            ]
+          }
+        },
+        "overrides": [
+          {
+            "matcher": {
+              "id": "byName",
+              "options": "Workflow URL"
+            },
+            "properties": [
+              {
+                "id": "custom.displayMode",
+                "value": "auto"
+              },
+              {
+                "id": "custom.width",
+                "value": 410
+              }
+            ]
+          },
+          {
+            "matcher": {
+              "id": "byName",
+              "options": "Created At"
+            },
+            "properties": [
+              {
+                "id": "custom.displayMode",
+                "value": "auto"
+              }
+            ]
+          }
+        ]
+      },
+      "gridPos": {
+        "h": 9,
+        "w": 12,
+        "x": 12,
+        "y": 8
+      },
+      "id": 4,
+      "options": {
+        "showHeader": true,
+        "sortBy": []
+      },
+      "pluginVersion": "8.0.4",
+      "targets": [
+        {
+          "cacheDurationSeconds": 300,
+          "fields": [
+            {
+              "jsonPath": "$.workflow_runs[*].id",
+              "name": "Workflow ID"
+            },
+            {
+              "jsonPath": "$.workflow_runs[*].conclusion",
+              "name": "Test Result"
+            },
+            {
+              "jsonPath": "$.workflow_runs[*].html_url",
+              "name": "Workflow URL"
+            },
+            {
+              "jsonPath": "$.workflow_runs[*].created_at",
+              "name": "Created At"
+            }
+          ],
+          "method": "GET",
+          "queryParams": "",
+          "refId": "A",
+          "urlPath": ""
+        }
+      ],
+      "title": "Java Workflow Results",
+      "type": "table"
+    }
+  ],
+  "refresh": "",
+  "schemaVersion": 30,
+  "style": "dark",
+  "tags": [],
+  "templating": {
+    "list": []
+  },
+  "time": {
+    "from": "now-24h",
+    "to": "now"
+  },
+  "timepicker": {},
+  "timezone": "",
+  "title": "GitHub Actions Post Commit Tests",
+  "uid": "dYwQFp7nk",
+  "version": 103
+}
+
diff --git a/CHANGES.md b/CHANGES.md
index 7f6bf2e..5547c75 100644
--- a/CHANGES.md
+++ b/CHANGES.md
@@ -51,11 +51,12 @@
 * Fixed X (Java/Python) ([BEAM-X](https://issues.apache.org/jira/browse/BEAM-X)).
 -->
 # [2.34.0] - Unreleased
-
+* Add an [example](https://github.com/cometta/python-apache-beam-spark) of deploying Python Apache Beam job with Spark Cluster
 ## Highlights
 
 * New highly anticipated feature X added to Python SDK ([BEAM-X](https://issues.apache.org/jira/browse/BEAM-X)).
 * New highly anticipated feature Y added to Java SDK ([BEAM-Y](https://issues.apache.org/jira/browse/BEAM-Y)).
+* The Beam Java API for Calcite SqlTransform is no longer experimental ([BEAM-12680](https://issues.apache.org/jira/browse/BEAM-12680)).
 
 ## I/Os
 
@@ -64,6 +65,7 @@
 ## New Features / Improvements
 
 * X feature added (Java/Python) ([BEAM-X](https://issues.apache.org/jira/browse/BEAM-X)).
+* Upgrade to Calcite 1.26.0 ([BEAM-9379](https://issues.apache.org/jira/browse/BEAM-9379)).
 
 ## Breaking Changes
 
@@ -73,6 +75,7 @@
   * Code depending on beam imports need to include v2 on the module path.
     * Fix by'v2' to the import paths, turning  `.../sdks/go/...` to `.../sdks/v2/go/...`
   * No other code change should be required to use v2.33.0 of the Go SDK.
+* SQL Rows are no longer flattened ([BEAM-5505](https://issues.apache.org/jira/browse/BEAM-5505)).
 
 ## Deprecations
 
@@ -121,40 +124,12 @@
 
 * Fixed X (Java/Python) ([BEAM-X](https://issues.apache.org/jira/browse/BEAM-X)).
 
-
-# [2.33.0] - Unreleased
-
-## Highlights
-
-* New highly anticipated feature X added to Python SDK ([BEAM-X](https://issues.apache.org/jira/browse/BEAM-X)).
-* New highly anticipated feature Y added to Java SDK ([BEAM-Y](https://issues.apache.org/jira/browse/BEAM-Y)).
-
-## I/Os
-
-* Support for X source added (Java/Python) ([BEAM-X](https://issues.apache.org/jira/browse/BEAM-X)).
-
-## New Features / Improvements
-
-* X feature added (Java/Python) ([BEAM-X](https://issues.apache.org/jira/browse/BEAM-X)).
-
-## Breaking Changes
-
-* X behavior was changed ([BEAM-X](https://issues.apache.org/jira/browse/BEAM-X)).
-
-## Deprecations
-
-* X behavior is deprecated and will be removed in X versions ([BEAM-X](https://issues.apache.org/jira/browse/BEAM-X)).
-
-## Known Issues
-
-* Fixed X (Java/Python) ([BEAM-X](https://issues.apache.org/jira/browse/BEAM-X)).
-
 # [2.32.0] - 2021-08-25
 
 ## Highlights
 * The [Beam DataFrame
   API](https://beam.apache.org/documentation/dsls/dataframes/overview/) is no
-  longer experimental! We've spent the time since the [2.32.0 preview
+  longer experimental! We've spent the time since the [2.26.0 preview
   announcement](https://beam.apache.org/blog/dataframe-api-preview-available/)
   implementing the most frequently used pandas operations
   ([BEAM-9547](https://issues.apache.org/jira/browse/BEAM-9547)), improving
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 c4f5d85..57bae57 100644
--- a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy
+++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy
@@ -472,7 +472,7 @@
     def spark_version = "2.4.8"
     def spotbugs_version = "4.0.6"
     def testcontainers_version = "1.15.1"
-    def arrow_version = "4.0.0"
+    def arrow_version = "5.0.0"
     def jmh_version = "1.32"
 
     // A map of maps containing common libraries used per language. To use:
@@ -662,7 +662,7 @@
         vendored_bytebuddy_1_11_0                   : "org.apache.beam:beam-vendor-bytebuddy-1_11_0:0.1",
         vendored_grpc_1_36_0                        : "org.apache.beam:beam-vendor-grpc-1_36_0:0.2",
         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",
+        vendored_calcite_1_26_0                     : "org.apache.beam:beam-vendor-calcite-1_26_0:0.1",
         woodstox_core_asl                           : "org.codehaus.woodstox:woodstox-core-asl:4.4.1",
         zstd_jni                                    : "com.github.luben:zstd-jni:1.4.5-2",
         quickcheck_core                             : "com.pholser:junit-quickcheck-core:$quickcheck_version",
@@ -1043,6 +1043,7 @@
         }
         project.spotbugs {
           excludeFilter = project.rootProject.file('sdks/java/build-tools/src/main/resources/beam/spotbugs-filter.xml')
+          jvmArgs = ['-Xmx12g']
         }
         project.tasks.withType(com.github.spotbugs.snom.SpotBugsTask) {
           reports {
diff --git a/model/pipeline/src/main/proto/external_transforms.proto b/model/pipeline/src/main/proto/external_transforms.proto
index f2d47a1..a528e56 100644
--- a/model/pipeline/src/main/proto/external_transforms.proto
+++ b/model/pipeline/src/main/proto/external_transforms.proto
@@ -29,6 +29,7 @@
 option java_outer_classname = "ExternalTransforms";
 
 import "schema.proto";
+import "beam_runner_api.proto";
 
 // A configuration payload for an external transform.
 // Used as the payload of ExternalTransform as part of an ExpansionRequest.
@@ -40,3 +41,65 @@
   // schema.
   bytes payload = 2;
 }
+
+// Defines specific expansion methods that may be used to expand cross-language
+// transforms.
+// Has to be set as the URN of the transform of the expansion request.
+message ExpansionMethods {
+  enum Enum {
+    // Expand a Java transform using specified constructor and builder methods.
+    // Transform payload will be of type JavaClassLookupPayload.
+    JAVA_CLASS_LOOKUP = 0 [(org.apache.beam.model.pipeline.v1.beam_urn) =
+      "beam:expansion:payload:java_class_lookup:v1"];
+  }
+}
+
+// A configuration payload for an external transform.
+// Used to define a Java transform that can be directly instantiated by a Java
+// expansion service.
+message JavaClassLookupPayload {
+  // Name of the Java transform class.
+  string class_name = 1;
+
+  // A static method to construct the initial instance of the transform.
+  // If not provided, the transform should be instantiated using a class
+  // constructor.
+  string constructor_method = 2;
+
+  // The top level fields of the schema represent the method parameters in
+  // order.
+  // If able, top level field names are also verified against the method
+  // parameters for a match.
+  Schema constructor_schema = 3;
+
+  // A payload which can be decoded using beam:coder:row:v1 and the provided
+  // constructor schema.
+  bytes constructor_payload = 4;
+
+  // Set of builder methods and corresponding parameters to apply after the
+  // transform object is constructed.
+  // When constructing the transform object, given builder methods will be
+  // applied in order.
+  repeated BuilderMethod builder_methods = 5;
+}
+
+// This represents a builder method of the transform class that should be
+// applied in-order after instantiating the initial transform object.
+// Each builder method may take one or more parameters and has to return an
+// instance of the transform object.
+message BuilderMethod {
+  // Name of the builder method
+  string name = 1;
+
+  // The top level fields of the schema represent the method parameters in
+  // order.
+  // If able, top level field names are also verified against the method
+  // parameters for a match.
+  Schema schema = 2;
+
+  // A payload which can be decoded using beam:coder:row:v1 and the builder
+  // method schema.
+  bytes payload = 3;
+}
+
+
diff --git a/model/pipeline/src/main/proto/metrics.proto b/model/pipeline/src/main/proto/metrics.proto
index 8c8d981..10166e1 100644
--- a/model/pipeline/src/main/proto/metrics.proto
+++ b/model/pipeline/src/main/proto/metrics.proto
@@ -415,6 +415,8 @@
     BIGQUERY_QUERY_NAME = 15 [(label_props) = { name: "BIGQUERY_QUERY_NAME" }];
     GCS_BUCKET = 16 [(label_props) = { name: "GCS_BUCKET"}];
     GCS_PROJECT_ID = 17 [(label_props) = { name: "GCS_PROJECT_ID"}];
+    DATASTORE_PROJECT = 18 [(label_props) = { name: "DATASTORE_PROJECT" }];
+    DATASTORE_NAMESPACE = 19 [(label_props) = { name: "DATASTORE_NAMESPACE" }];
   }
 
   // A set of key and value labels which define the scope of the metric. For
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 82d8810..7060ca0 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
@@ -49,13 +49,18 @@
 })
 public class TestStreamTranslation {
 
-  public static TestStream<?> testStreamFromProtoPayload(
+  public static <T> TestStream<T> testStreamFromProtoPayload(
       RunnerApi.TestStreamPayload testStreamPayload, RehydratedComponents components)
       throws IOException {
 
-    Coder<Object> coder = (Coder<Object>) components.getCoder(testStreamPayload.getCoderId());
+    Coder<T> coder = (Coder<T>) components.getCoder(testStreamPayload.getCoderId());
 
-    List<TestStream.Event<Object>> events = new ArrayList<>();
+    return testStreamFromProtoPayload(testStreamPayload, coder);
+  }
+
+  public static <T> TestStream<T> testStreamFromProtoPayload(
+      RunnerApi.TestStreamPayload testStreamPayload, Coder<T> coder) throws IOException {
+    List<TestStream.Event<T>> events = new ArrayList<>();
 
     for (RunnerApi.TestStreamPayload.Event event : testStreamPayload.getEventsList()) {
       events.add(eventFromProto(event, coder));
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 c7f0274..965be82 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
@@ -79,11 +79,11 @@
   void deleteTimer(
       StateNamespace namespace, String timerId, String timerFamilyId, TimeDomain timeDomain);
 
-  /** @deprecated use {@link #deleteTimer(StateNamespace, String, TimeDomain)}. */
+  /** @deprecated use {@link #deleteTimer(StateNamespace, String, String, TimeDomain)}. */
   @Deprecated
   void deleteTimer(StateNamespace namespace, String timerId, String timerFamilyId);
 
-  /** @deprecated use {@link #deleteTimer(StateNamespace, String, TimeDomain)}. */
+  /** @deprecated use {@link #deleteTimer(StateNamespace, String, String, TimeDomain)}. */
   @Deprecated
   void deleteTimer(TimerData timerKey);
 
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/GcpResourceIdentifiers.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/GcpResourceIdentifiers.java
index 6ae35d2..800413c 100644
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/GcpResourceIdentifiers.java
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/GcpResourceIdentifiers.java
@@ -32,4 +32,9 @@
         "//bigquery.googleapis.com/projects/%s/datasets/%s/tables/%s",
         projectId, datasetId, tableId);
   }
+
+  public static String datastoreResource(String projectId, String namespace) {
+    return String.format(
+        "//bigtable.googleapis.com/projects/%s/namespaces/%s", projectId, namespace);
+  }
 }
diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/MonitoringInfoConstants.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/MonitoringInfoConstants.java
index 5161bb4..88e55db 100644
--- a/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/MonitoringInfoConstants.java
+++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/metrics/MonitoringInfoConstants.java
@@ -78,6 +78,8 @@
     public static final String BIGQUERY_TABLE = "BIGQUERY_TABLE";
     public static final String BIGQUERY_VIEW = "BIGQUERY_VIEW";
     public static final String BIGQUERY_QUERY_NAME = "BIGQUERY_QUERY_NAME";
+    public static final String DATASTORE_PROJECT = "DATASTORE_PROJECT";
+    public static final String DATASTORE_NAMESPACE = "DATASTORE_NAMESPACE";
 
     static {
       // Note: One benefit of defining these strings above, instead of pulling them in from
@@ -104,6 +106,9 @@
       checkArgument(BIGQUERY_VIEW.equals(extractLabel(MonitoringInfoLabels.BIGQUERY_VIEW)));
       checkArgument(
           BIGQUERY_QUERY_NAME.equals(extractLabel(MonitoringInfoLabels.BIGQUERY_QUERY_NAME)));
+      checkArgument(DATASTORE_PROJECT.equals(extractLabel(MonitoringInfoLabels.DATASTORE_PROJECT)));
+      checkArgument(
+          DATASTORE_NAMESPACE.equals(extractLabel(MonitoringInfoLabels.DATASTORE_NAMESPACE)));
     }
   }
 
diff --git a/runners/flink/job-server/flink_job_server.gradle b/runners/flink/job-server/flink_job_server.gradle
index ef3442e..9ab6c32 100644
--- a/runners/flink/job-server/flink_job_server.gradle
+++ b/runners/flink/job-server/flink_job_server.gradle
@@ -185,21 +185,12 @@
     testFilter: {
       // TODO(BEAM-10016)
       excludeTestsMatching 'org.apache.beam.sdk.transforms.FlattenTest.testFlattenWithDifferentInputAndOutputCoders2'
-      // TODO(BEAM-11310)
-      excludeTestsMatching 'org.apache.beam.sdk.transforms.ViewTest.testWindowedSideInputNotPresent'
-
-      // https://issues.apache.org/jira/browse/BEAM-12039
+      // TODO(BEAM-12039)
       excludeTestsMatching 'org.apache.beam.sdk.testing.TestStreamTest.testDiscardingMode'
-      // https://issues.apache.org/jira/browse/BEAM-12038
+      // TODO(BEAM-12038)
       excludeTestsMatching 'org.apache.beam.sdk.testing.TestStreamTest.testLateDataAccumulating'
-      // https://issues.apache.org/jira/browse/BEAM-12048
-      excludeTestsMatching 'org.apache.beam.sdk.testing.TestStreamTest.testMultipleStreams'
-      // https://issues.apache.org/jira/browse/BEAM-12710
+      // TODO(BEAM-12710)
       excludeTestsMatching 'org.apache.beam.sdk.testing.TestStreamTest.testFirstElementLate'
-      // https://issues.apache.org/jira/browse/BEAM-12050
-      excludeTestsMatching 'org.apache.beam.sdk.transforms.ParDoTest$TimerTests.testEventTimeTimerUnbounded'
-      excludeTestsMatching 'org.apache.beam.sdk.transforms.ParDoTest$TimerTests.testEventTimeTimerAlignUnbounded'
-      excludeTestsMatching 'org.apache.beam.sdk.transforms.ParDoTest$TimerTests.testEventTimeTimerAlignAfterGcTimeUnbounded'
     },
   )
 }
diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java
index fa7c1ec..c1220b0 100644
--- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java
+++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java
@@ -109,6 +109,7 @@
 import org.apache.beam.sdk.io.UnboundedSource;
 import org.apache.beam.sdk.io.WriteFiles;
 import org.apache.beam.sdk.io.WriteFilesResult;
+import org.apache.beam.sdk.io.fs.ResolveOptions;
 import org.apache.beam.sdk.io.fs.ResourceId;
 import org.apache.beam.sdk.io.gcp.pubsub.PubsubMessage;
 import org.apache.beam.sdk.io.gcp.pubsub.PubsubMessageWithAttributesAndMessageIdCoder;
@@ -849,6 +850,57 @@
     }
   }
 
+  @VisibleForTesting
+  protected RunnerApi.Pipeline resolveArtifacts(RunnerApi.Pipeline pipeline) {
+    RunnerApi.Pipeline.Builder pipelineBuilder = pipeline.toBuilder();
+    RunnerApi.Components.Builder componentsBuilder = pipelineBuilder.getComponentsBuilder();
+    componentsBuilder.clearEnvironments();
+    for (Map.Entry<String, RunnerApi.Environment> entry :
+        pipeline.getComponents().getEnvironmentsMap().entrySet()) {
+      RunnerApi.Environment.Builder environmentBuilder = entry.getValue().toBuilder();
+      environmentBuilder.clearDependencies();
+      for (RunnerApi.ArtifactInformation info : entry.getValue().getDependenciesList()) {
+        if (!BeamUrns.getUrn(RunnerApi.StandardArtifacts.Types.FILE).equals(info.getTypeUrn())) {
+          throw new RuntimeException(
+              String.format("unsupported artifact type %s", info.getTypeUrn()));
+        }
+        RunnerApi.ArtifactFilePayload filePayload;
+        try {
+          filePayload = RunnerApi.ArtifactFilePayload.parseFrom(info.getTypePayload());
+        } catch (InvalidProtocolBufferException e) {
+          throw new RuntimeException("Error parsing artifact file payload.", e);
+        }
+        if (!BeamUrns.getUrn(RunnerApi.StandardArtifacts.Roles.STAGING_TO)
+            .equals(info.getRoleUrn())) {
+          throw new RuntimeException(
+              String.format("unsupported artifact role %s", info.getRoleUrn()));
+        }
+        RunnerApi.ArtifactStagingToRolePayload stagingPayload;
+        try {
+          stagingPayload = RunnerApi.ArtifactStagingToRolePayload.parseFrom(info.getRolePayload());
+        } catch (InvalidProtocolBufferException e) {
+          throw new RuntimeException("Error parsing artifact staging_to role payload.", e);
+        }
+        environmentBuilder.addDependencies(
+            info.toBuilder()
+                .setTypeUrn(BeamUrns.getUrn(RunnerApi.StandardArtifacts.Types.URL))
+                .setTypePayload(
+                    RunnerApi.ArtifactUrlPayload.newBuilder()
+                        .setUrl(
+                            FileSystems.matchNewResource(options.getStagingLocation(), true)
+                                .resolve(
+                                    stagingPayload.getStagedName(),
+                                    ResolveOptions.StandardResolveOptions.RESOLVE_FILE)
+                                .toString())
+                        .setSha256(filePayload.getSha256())
+                        .build()
+                        .toByteString()));
+      }
+      componentsBuilder.putEnvironments(entry.getKey(), environmentBuilder.build());
+    }
+    return pipelineBuilder.build();
+  }
+
   private List<DataflowPackage> stageArtifacts(RunnerApi.Pipeline pipeline) {
     ImmutableList.Builder<StagedFile> filesToStageBuilder = ImmutableList.builder();
     for (Map.Entry<String, RunnerApi.Environment> entry :
@@ -952,6 +1004,10 @@
 
     RunnerApi.Pipeline portablePipelineProto =
         PipelineTranslation.toProto(pipeline, portableComponents, false);
+    // Note that `stageArtifacts` has to be called before `resolveArtifact` because
+    // `resolveArtifact` updates local paths to staged paths in pipeline proto.
+    List<DataflowPackage> packages = stageArtifacts(portablePipelineProto);
+    portablePipelineProto = resolveArtifacts(portablePipelineProto);
     LOG.debug("Portable pipeline proto:\n{}", TextFormat.printToString(portablePipelineProto));
     // Stage the portable pipeline proto, retrieving the staged pipeline path, then update
     // the options on the new job
@@ -976,7 +1032,6 @@
     RunnerApi.Pipeline dataflowV1PipelineProto =
         PipelineTranslation.toProto(pipeline, dataflowV1Components, true);
     LOG.debug("Dataflow v1 pipeline proto:\n{}", TextFormat.printToString(dataflowV1PipelineProto));
-    List<DataflowPackage> packages = stageArtifacts(dataflowV1PipelineProto);
 
     // Set a unique client_request_id in the CreateJob request.
     // This is used to ensure idempotence of job creation across retried
diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java
index 55aa182..f5bde8e 100644
--- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java
+++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java
@@ -1179,6 +1179,97 @@
   }
 
   @Test
+  public void testResolveArtifacts() throws IOException {
+    DataflowPipelineOptions options = buildPipelineOptions();
+    DataflowRunner runner = DataflowRunner.fromOptions(options);
+    String stagingLocation = options.getStagingLocation().replaceFirst("/$", "");
+    RunnerApi.ArtifactInformation fooLocalArtifact =
+        RunnerApi.ArtifactInformation.newBuilder()
+            .setTypeUrn(BeamUrns.getUrn(RunnerApi.StandardArtifacts.Types.FILE))
+            .setTypePayload(
+                RunnerApi.ArtifactFilePayload.newBuilder()
+                    .setPath("/tmp/foo.jar")
+                    .build()
+                    .toByteString())
+            .setRoleUrn(BeamUrns.getUrn(RunnerApi.StandardArtifacts.Roles.STAGING_TO))
+            .setRolePayload(
+                RunnerApi.ArtifactStagingToRolePayload.newBuilder()
+                    .setStagedName("foo_staged.jar")
+                    .build()
+                    .toByteString())
+            .build();
+    RunnerApi.ArtifactInformation barLocalArtifact =
+        RunnerApi.ArtifactInformation.newBuilder()
+            .setTypeUrn(BeamUrns.getUrn(RunnerApi.StandardArtifacts.Types.FILE))
+            .setTypePayload(
+                RunnerApi.ArtifactFilePayload.newBuilder()
+                    .setPath("/tmp/bar.jar")
+                    .build()
+                    .toByteString())
+            .setRoleUrn(BeamUrns.getUrn(RunnerApi.StandardArtifacts.Roles.STAGING_TO))
+            .setRolePayload(
+                RunnerApi.ArtifactStagingToRolePayload.newBuilder()
+                    .setStagedName("bar_staged.jar")
+                    .build()
+                    .toByteString())
+            .build();
+    RunnerApi.Pipeline pipeline =
+        RunnerApi.Pipeline.newBuilder()
+            .setComponents(
+                RunnerApi.Components.newBuilder()
+                    .putEnvironments(
+                        "env",
+                        RunnerApi.Environment.newBuilder()
+                            .addAllDependencies(
+                                ImmutableList.of(fooLocalArtifact, barLocalArtifact))
+                            .build()))
+            .build();
+
+    RunnerApi.ArtifactInformation fooStagedArtifact =
+        RunnerApi.ArtifactInformation.newBuilder()
+            .setTypeUrn(BeamUrns.getUrn(RunnerApi.StandardArtifacts.Types.URL))
+            .setTypePayload(
+                RunnerApi.ArtifactUrlPayload.newBuilder()
+                    .setUrl(stagingLocation + "/foo_staged.jar")
+                    .build()
+                    .toByteString())
+            .setRoleUrn(BeamUrns.getUrn(RunnerApi.StandardArtifacts.Roles.STAGING_TO))
+            .setRolePayload(
+                RunnerApi.ArtifactStagingToRolePayload.newBuilder()
+                    .setStagedName("foo_staged.jar")
+                    .build()
+                    .toByteString())
+            .build();
+    RunnerApi.ArtifactInformation barStagedArtifact =
+        RunnerApi.ArtifactInformation.newBuilder()
+            .setTypeUrn(BeamUrns.getUrn(RunnerApi.StandardArtifacts.Types.URL))
+            .setTypePayload(
+                RunnerApi.ArtifactUrlPayload.newBuilder()
+                    .setUrl(stagingLocation + "/bar_staged.jar")
+                    .build()
+                    .toByteString())
+            .setRoleUrn(BeamUrns.getUrn(RunnerApi.StandardArtifacts.Roles.STAGING_TO))
+            .setRolePayload(
+                RunnerApi.ArtifactStagingToRolePayload.newBuilder()
+                    .setStagedName("bar_staged.jar")
+                    .build()
+                    .toByteString())
+            .build();
+    RunnerApi.Pipeline expectedPipeline =
+        RunnerApi.Pipeline.newBuilder()
+            .setComponents(
+                RunnerApi.Components.newBuilder()
+                    .putEnvironments(
+                        "env",
+                        RunnerApi.Environment.newBuilder()
+                            .addAllDependencies(
+                                ImmutableList.of(fooStagedArtifact, barStagedArtifact))
+                            .build()))
+            .build();
+    assertThat(runner.resolveArtifacts(pipeline), equalTo(expectedPipeline));
+  }
+
+  @Test
   public void testGcpTempAndNoTempLocationSucceeds() throws Exception {
     DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class);
     options.setRunner(DataflowRunner.class);
diff --git a/runners/samza/job-server/build.gradle b/runners/samza/job-server/build.gradle
index 4c03123..266ec0e 100644
--- a/runners/samza/job-server/build.gradle
+++ b/runners/samza/job-server/build.gradle
@@ -91,7 +91,9 @@
             excludeCategories 'org.apache.beam.sdk.testing.UsesBundleFinalizer'
             excludeCategories 'org.apache.beam.sdk.testing.UsesOrderedListState'
             excludeCategories 'org.apache.beam.sdk.testing.UsesBoundedSplittableParDo'
-            excludeCategories 'org.apache.beam.sdk.testing.UsesTestStream'
+            excludeCategories 'org.apache.beam.sdk.testing.UsesTestStreamWithProcessingTime'
+            // TODO(BEAM-12821)
+            excludeCategories 'org.apache.beam.sdk.testing.UsesTestStreamWithMultipleStages'
             excludeCategories 'org.apache.beam.sdk.testing.UsesUnboundedSplittableParDo'
             excludeCategories 'org.apache.beam.sdk.testing.UsesSplittableParDoWithWindowedSideInputs'
             excludeCategories 'org.apache.beam.sdk.testing.UsesLoopingTimer'
@@ -102,6 +104,14 @@
             excludeTestsMatching "org.apache.beam.sdk.transforms.FlattenTest.testEmptyFlattenAsSideInput"
             excludeTestsMatching "org.apache.beam.sdk.transforms.FlattenTest.testFlattenPCollectionsEmptyThenParDo"
             excludeTestsMatching "org.apache.beam.sdk.transforms.FlattenTest.testFlattenPCollectionsEmpty"
+            // TODO(BEAM-10025)
+            excludeTestsMatching 'org.apache.beam.sdk.transforms.ParDoTest$TimerTests.testOutputTimestampDefaultUnbounded'
+            // TODO(BEAM-11479)
+            excludeTestsMatching 'org.apache.beam.sdk.transforms.ParDoTest$TimerTests.testOutputTimestamp'
+            // TODO(BEAM-12035)
+            excludeTestsMatching 'org.apache.beam.sdk.testing.TestStreamTest.testFirstElementLate'
+            // TODO(BEAM-12036)
+            excludeTestsMatching 'org.apache.beam.sdk.testing.TestStreamTest.testLateDataAccumulating'
         }
 )
 
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 9a8d852..6501247 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
@@ -28,6 +28,8 @@
 import org.apache.beam.runners.core.StateTag;
 import org.apache.beam.runners.core.TimerInternals;
 import org.apache.beam.runners.core.TimerInternalsFactory;
+import org.apache.beam.runners.samza.state.SamzaMapState;
+import org.apache.beam.runners.samza.state.SamzaSetState;
 import org.apache.beam.sdk.state.State;
 import org.apache.beam.sdk.state.StateContext;
 import org.apache.beam.sdk.state.TimeDomain;
@@ -87,8 +89,10 @@
     final List<State> states = threadLocalKeyedStates.get().states;
     states.forEach(
         state -> {
-          if (state instanceof SamzaStoreStateInternals.KeyValueIteratorState) {
-            ((SamzaStoreStateInternals.KeyValueIteratorState) state).closeIterators();
+          if (state instanceof SamzaMapState) {
+            ((SamzaMapState) state).closeIterators();
+          } else if (state instanceof SamzaSetState) {
+            ((SamzaSetState) state).closeIterators();
           }
         });
     states.clear();
diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SamzaStoreStateInternals.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SamzaStoreStateInternals.java
index 1861404..a9e38a7 100644
--- a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SamzaStoreStateInternals.java
+++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SamzaStoreStateInternals.java
@@ -307,11 +307,6 @@
     }
   }
 
-  /** An internal State interface that holds underlying KeyValueIterators. */
-  interface KeyValueIteratorState {
-    void closeIterators();
-  }
-
   private abstract class AbstractSamzaState<T> {
     private final StateNamespace namespace;
     private final String addressId;
@@ -537,7 +532,7 @@
     }
   }
 
-  private class SamzaSetStateImpl<T> implements SamzaSetState<T>, KeyValueIteratorState {
+  private class SamzaSetStateImpl<T> implements SamzaSetState<T> {
     private final SamzaMapStateImpl<T, Boolean> mapState;
 
     private SamzaSetStateImpl(
@@ -630,7 +625,7 @@
   }
 
   private class SamzaMapStateImpl<KeyT, ValueT> extends AbstractSamzaState<ValueT>
-      implements SamzaMapState<KeyT, ValueT>, KeyValueIteratorState {
+      implements SamzaMapState<KeyT, ValueT> {
 
     private final Coder<KeyT> keyCoder;
     private final int storeKeySize;
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 f791c3a..7814217 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
@@ -267,15 +267,13 @@
       }
 
       if (lastTimestamp != null) {
-        final TimerData lastTimerData =
-            TimerData.of(
-                timerData.getTimerId(),
-                timerData.getTimerFamilyId(),
-                timerData.getNamespace(),
-                new Instant(lastTimestamp),
-                new Instant(lastTimestamp),
-                timerData.getDomain());
-        deleteTimer(lastTimerData, false);
+        deleteTimer(
+            timerData.getNamespace(),
+            timerData.getTimerId(),
+            timerData.getTimerFamilyId(),
+            new Instant(lastTimestamp),
+            new Instant(lastTimestamp),
+            timerData.getDomain());
       }
 
       // persist it first
@@ -284,7 +282,7 @@
       // TO-DO: apply the same memory optimization over processing timers
       switch (timerData.getDomain()) {
         case EVENT_TIME:
-          /**
+          /*
            * To determine if the upcoming KeyedTimerData could be added to the Buffer while
            * guaranteeing the Buffer's timestamps are all <= than those in State Store to preserve
            * timestamp eviction priority:
@@ -319,36 +317,50 @@
       }
     }
 
+    /** @deprecated use {@link #deleteTimer(StateNamespace, String, String, TimeDomain)}. */
+    @Override
+    @Deprecated
+    public void deleteTimer(StateNamespace namespace, String timerId, String timerFamilyId) {
+      deleteTimer(namespace, timerId, timerFamilyId, TimeDomain.EVENT_TIME);
+    }
+
+    /** @deprecated use {@link #deleteTimer(StateNamespace, String, String, TimeDomain)}. */
+    @Override
+    @Deprecated
+    public void deleteTimer(TimerData timerData) {
+      deleteTimer(
+          timerData.getNamespace(),
+          timerData.getTimerId(),
+          timerData.getTimerFamilyId(),
+          timerData.getDomain());
+    }
+
     @Override
     public void deleteTimer(
         StateNamespace namespace, String timerId, String timerFamilyId, TimeDomain timeDomain) {
-      TimerKey<K> timerKey = TimerKey.of(key, namespace, timerId, timerFamilyId);
-
-      Long lastTimestamp = state.get(timerKey, timeDomain);
+      final TimerKey<K> timerKey = TimerKey.of(key, namespace, timerId, timerFamilyId);
+      final Long lastTimestamp = state.get(timerKey, timeDomain);
 
       if (lastTimestamp == null) {
         return;
       }
 
-      Instant timestamp = Instant.ofEpochMilli(lastTimestamp);
-      deleteTimer(TimerData.of(timerId, namespace, timestamp, timestamp, timeDomain));
+      final Instant timestamp = Instant.ofEpochMilli(lastTimestamp);
+      deleteTimer(namespace, timerId, timerFamilyId, timestamp, timestamp, timeDomain);
     }
 
-    @Override
-    public void deleteTimer(StateNamespace namespace, String timerId, String timerFamilyId) {
-      deleteTimer(namespace, timerId, timerFamilyId, TimeDomain.EVENT_TIME);
-    }
-
-    @Override
-    public void deleteTimer(TimerData timerData) {
-      deleteTimer(timerData, true);
-    }
-
-    private void deleteTimer(TimerData timerData, boolean updateState) {
+    private void deleteTimer(
+        StateNamespace namespace,
+        String timerId,
+        String timerFamilyId,
+        Instant timestamp,
+        Instant outputTimestamp,
+        TimeDomain timeDomain) {
+      final TimerData timerData =
+          TimerData.of(timerId, timerFamilyId, namespace, timestamp, outputTimestamp, timeDomain);
       final KeyedTimerData<K> keyedTimerData = new KeyedTimerData<>(keyBytes, key, timerData);
-      if (updateState) {
-        state.deletePersisted(keyedTimerData);
-      }
+
+      state.deletePersisted(keyedTimerData);
 
       switch (timerData.getDomain()) {
         case EVENT_TIME:
@@ -515,8 +527,7 @@
         maxEventTimeInBuffer = keyedTimerData.getTimerData().getTimestamp().getMillis();
       }
 
-      ((SamzaStoreStateInternals.KeyValueIteratorState) timestampSortedEventTimeTimerState)
-          .closeIterators();
+      timestampSortedEventTimeTimerState.closeIterators();
       LOG.info("Loaded {} event time timers in memory", eventTimeBuffer.size());
 
       if (eventTimeBuffer.size() < maxEventTimerBufferSize) {
@@ -544,7 +555,7 @@
             keyedTimerData, keyedTimerData.getTimerData().getTimestamp().getMillis());
         ++count;
       }
-      ((SamzaStoreStateInternals.KeyValueIteratorState) processingTimeTimerState).closeIterators();
+      processingTimeTimerState.closeIterators();
 
       LOG.info("Loaded {} processing time timers in memory", count);
     }
@@ -573,10 +584,9 @@
             timestampSortedEventTimeTimerState.add(keyedTimerData);
           }
         }
-        ((SamzaStoreStateInternals.KeyValueIteratorState) timestampSortedEventTimeTimerState)
-            .closeIterators();
+        timestampSortedEventTimeTimerState.closeIterators();
       }
-      ((SamzaStoreStateInternals.KeyValueIteratorState) eventTimeTimerState).closeIterators();
+      eventTimeTimerState.closeIterators();
 
       reloadEventTimeTimers();
       loadProcessingTimeTimers();
diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/state/SamzaMapState.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/state/SamzaMapState.java
index 2afba06..a8741fb 100644
--- a/runners/samza/src/main/java/org/apache/beam/runners/samza/state/SamzaMapState.java
+++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/state/SamzaMapState.java
@@ -34,4 +34,7 @@
    * @return a {@link ReadableState} of an iterator
    */
   ReadableState<Iterator<Map.Entry<KeyT, ValueT>>> readIterator();
+
+  /** Closes the iterator returned from {@link SamzaMapState#readIterator()}. */
+  void closeIterators();
 }
diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/state/SamzaSetState.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/state/SamzaSetState.java
index a6785c7..8af82fc 100644
--- a/runners/samza/src/main/java/org/apache/beam/runners/samza/state/SamzaSetState.java
+++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/state/SamzaSetState.java
@@ -33,4 +33,7 @@
    * @return a {@link ReadableState} of an iterator
    */
   ReadableState<Iterator<T>> readIterator();
+
+  /** Closes the iterator returned from {@link SamzaSetState#readIterator()}. */
+  void closeIterators();
 }
diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SamzaPortablePipelineTranslator.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SamzaPortablePipelineTranslator.java
index 407cc5d..9158cd4 100644
--- a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SamzaPortablePipelineTranslator.java
+++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SamzaPortablePipelineTranslator.java
@@ -106,6 +106,7 @@
           .put(PTransformTranslation.GROUP_BY_KEY_TRANSFORM_URN, new GroupByKeyTranslator<>())
           .put(PTransformTranslation.FLATTEN_TRANSFORM_URN, new FlattenPCollectionsTranslator<>())
           .put(PTransformTranslation.IMPULSE_TRANSFORM_URN, new ImpulseTranslator())
+          .put(PTransformTranslation.TEST_STREAM_TRANSFORM_URN, new SamzaTestStreamTranslator<>())
           .put(ExecutableStage.URN, new ParDoBoundMultiTranslator<>())
           .build();
     }
diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SamzaTestStreamSystemFactory.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SamzaTestStreamSystemFactory.java
index 96dc577..570be61 100644
--- a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SamzaTestStreamSystemFactory.java
+++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SamzaTestStreamSystemFactory.java
@@ -35,6 +35,7 @@
 import org.apache.samza.Partition;
 import org.apache.samza.SamzaException;
 import org.apache.samza.config.Config;
+import org.apache.samza.config.SystemConfig;
 import org.apache.samza.metrics.MetricsRegistry;
 import org.apache.samza.system.IncomingMessageEnvelope;
 import org.apache.samza.system.SystemAdmin;
@@ -55,9 +56,9 @@
 public class SamzaTestStreamSystemFactory implements SystemFactory {
   @Override
   public SystemConsumer getConsumer(String systemName, Config config, MetricsRegistry registry) {
-    final String streamPrefix = "systems." + systemName;
-    final Config scopedConfig = config.subset(streamPrefix + ".", true);
-    return new SmazaTestStreamSystemConsumer<>(getTestStream(scopedConfig));
+    final String streamPrefix = String.format(SystemConfig.SYSTEM_ID_PREFIX, systemName);
+    final Config scopedConfig = config.subset(streamPrefix, true);
+    return new SamzaTestStreamSystemConsumer<>(getTestStream(scopedConfig));
   }
 
   @Override
@@ -75,14 +76,13 @@
     @SuppressWarnings("unchecked")
     final SerializableFunction<String, TestStream<T>> testStreamDecoder =
         Base64Serializer.deserializeUnchecked(
-            config.get("testStreamDecoder"), SerializableFunction.class);
-    final TestStream<T> testStream = testStreamDecoder.apply(config.get("encodedTestStream"));
-    return testStream;
+            config.get(SamzaTestStreamTranslator.TEST_STREAM_DECODER), SerializableFunction.class);
+    return testStreamDecoder.apply(config.get(SamzaTestStreamTranslator.ENCODED_TEST_STREAM));
   }
 
   private static final String DUMMY_OFFSET = "0";
 
-  /** System admin for SmazaTestStreamSystem. */
+  /** System admin for SamzaTestStreamSystem. */
   public static class SamzaTestStreamSystemAdmin implements SystemAdmin {
     @Override
     public Map<SystemStreamPartition, String> getOffsetsAfter(
@@ -115,11 +115,11 @@
     }
   }
 
-  /** System consumer for SmazaTestStreamSystem. */
-  public static class SmazaTestStreamSystemConsumer<T> implements SystemConsumer {
+  /** System consumer for SamzaTestStreamSystem. */
+  public static class SamzaTestStreamSystemConsumer<T> implements SystemConsumer {
     TestStream<T> testStream;
 
-    public SmazaTestStreamSystemConsumer(TestStream<T> testStream) {
+    public SamzaTestStreamSystemConsumer(TestStream<T> testStream) {
       this.testStream = testStream;
     }
 
@@ -134,8 +134,7 @@
 
     @Override
     public Map<SystemStreamPartition, List<IncomingMessageEnvelope>> poll(
-        Set<SystemStreamPartition> systemStreamPartitions, long timeout)
-        throws InterruptedException {
+        Set<SystemStreamPartition> systemStreamPartitions, long timeout) {
       SystemStreamPartition ssp = systemStreamPartitions.iterator().next();
       ArrayList<IncomingMessageEnvelope> messages = new ArrayList<>();
 
diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SamzaTestStreamTranslator.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SamzaTestStreamTranslator.java
index ef38a79..e50dc2c 100644
--- a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SamzaTestStreamTranslator.java
+++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SamzaTestStreamTranslator.java
@@ -17,11 +17,16 @@
  */
 package org.apache.beam.runners.samza.translation;
 
+import java.io.IOException;
 import java.util.Map;
+import org.apache.beam.model.pipeline.v1.RunnerApi;
+import org.apache.beam.runners.core.construction.RehydratedComponents;
+import org.apache.beam.runners.core.construction.TestStreamTranslation;
 import org.apache.beam.runners.core.construction.graph.PipelineNode;
 import org.apache.beam.runners.core.construction.graph.QueryablePipeline;
 import org.apache.beam.runners.core.serialization.Base64Serializer;
 import org.apache.beam.runners.samza.runtime.OpMessage;
+import org.apache.beam.runners.samza.util.SamzaPipelineTranslatorUtils;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.coders.CoderException;
 import org.apache.beam.sdk.runners.TransformHierarchy;
@@ -29,8 +34,8 @@
 import org.apache.beam.sdk.transforms.SerializableFunction;
 import org.apache.beam.sdk.util.CoderUtils;
 import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.vendor.grpc.v1p36p0.com.google.protobuf.ByteString;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
-import org.apache.samza.SamzaException;
 import org.apache.samza.operators.KV;
 import org.apache.samza.serializers.KVSerde;
 import org.apache.samza.serializers.NoOpSerde;
@@ -40,11 +45,12 @@
 
 /**
  * Translate {@link org.apache.beam.sdk.testing.TestStream} to a samza message stream produced by
- * {@link
- * org.apache.beam.runners.samza.translation.SamzaTestStreamSystemFactory.SmazaTestStreamSystemConsumer}.
+ * {@link SamzaTestStreamSystemFactory.SamzaTestStreamSystemConsumer}.
  */
 @SuppressWarnings({"rawtypes"})
 public class SamzaTestStreamTranslator<T> implements TransformTranslator<TestStream<T>> {
+  public static final String ENCODED_TEST_STREAM = "encodedTestStream";
+  public static final String TEST_STREAM_DECODER = "testStreamDecoder";
 
   @Override
   public void translate(
@@ -53,15 +59,13 @@
     final String outputId = ctx.getIdForPValue(output);
     final Coder<T> valueCoder = testStream.getValueCoder();
     final TestStream.TestStreamCoder<T> testStreamCoder = TestStream.TestStreamCoder.of(valueCoder);
-    final GenericSystemDescriptor systemDescriptor =
-        new GenericSystemDescriptor(outputId, SamzaTestStreamSystemFactory.class.getName());
 
     // encode testStream as a string
     final String encodedTestStream;
     try {
       encodedTestStream = CoderUtils.encodeToBase64(testStreamCoder, testStream);
     } catch (CoderException e) {
-      throw new SamzaException("Could not encode TestStream.", e);
+      throw new RuntimeException("Could not encode TestStream.", e);
     }
 
     // the decoder for encodedTestStream
@@ -70,24 +74,12 @@
           try {
             return CoderUtils.decodeFromBase64(TestStream.TestStreamCoder.of(valueCoder), string);
           } catch (CoderException e) {
-            throw new SamzaException("Could not decode TestStream.", e);
+            throw new RuntimeException("Could not decode TestStream.", e);
           }
         };
 
-    final Map<String, String> systemConfig =
-        ImmutableMap.of(
-            "encodedTestStream",
-            encodedTestStream,
-            "testStreamDecoder",
-            Base64Serializer.serializeUnchecked(testStreamDecoder));
-    systemDescriptor.withSystemConfigs(systemConfig);
-
-    // The KvCoder is needed here for Samza not to crop the key.
-    final Serde<KV<?, OpMessage<byte[]>>> kvSerde = KVSerde.of(new NoOpSerde(), new NoOpSerde<>());
-    final GenericInputDescriptor<KV<?, OpMessage<byte[]>>> inputDescriptor =
-        systemDescriptor.getInputDescriptor(outputId, kvSerde);
-
-    ctx.registerInputMessageStream(output, inputDescriptor);
+    ctx.registerInputMessageStream(
+        output, createInputDescriptor(outputId, encodedTestStream, testStreamDecoder));
   }
 
   @Override
@@ -95,6 +87,62 @@
       PipelineNode.PTransformNode transform,
       QueryablePipeline pipeline,
       PortableTranslationContext ctx) {
-    throw new SamzaException("TestStream is not supported in portable by Samza runner");
+    final ByteString bytes = transform.getTransform().getSpec().getPayload();
+    final SerializableFunction<String, TestStream<T>> testStreamDecoder =
+        createTestStreamDecoder(pipeline.getComponents(), bytes);
+
+    final String outputId = ctx.getOutputId(transform);
+    final String escapedOutputId = SamzaPipelineTranslatorUtils.escape(outputId);
+
+    ctx.registerInputMessageStream(
+        outputId,
+        createInputDescriptor(
+            escapedOutputId, Base64Serializer.serializeUnchecked(bytes), testStreamDecoder));
+  }
+
+  @SuppressWarnings("unchecked")
+  private static <T> GenericInputDescriptor<KV<?, OpMessage<T>>> createInputDescriptor(
+      String id,
+      String encodedTestStream,
+      SerializableFunction<String, TestStream<T>> testStreamDecoder) {
+    final Map<String, String> systemConfig =
+        ImmutableMap.of(
+            ENCODED_TEST_STREAM,
+            encodedTestStream,
+            TEST_STREAM_DECODER,
+            Base64Serializer.serializeUnchecked(testStreamDecoder));
+    final GenericSystemDescriptor systemDescriptor =
+        new GenericSystemDescriptor(id, SamzaTestStreamSystemFactory.class.getName())
+            .withSystemConfigs(systemConfig);
+
+    // The KvCoder is needed here for Samza not to crop the key.
+    final Serde<KV<?, OpMessage<T>>> kvSerde = KVSerde.of(new NoOpSerde(), new NoOpSerde<>());
+    return systemDescriptor.getInputDescriptor(id, kvSerde);
+  }
+
+  @SuppressWarnings("unchecked")
+  private static <T> SerializableFunction<String, TestStream<T>> createTestStreamDecoder(
+      RunnerApi.Components components, ByteString payload) {
+    Coder<T> coder;
+    try {
+      coder =
+          (Coder<T>)
+              RehydratedComponents.forComponents(components)
+                  .getCoder(RunnerApi.TestStreamPayload.parseFrom(payload).getCoderId());
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+
+    // the decoder for encodedTestStream
+    return encodedTestStream -> {
+      try {
+        return TestStreamTranslation.testStreamFromProtoPayload(
+            RunnerApi.TestStreamPayload.parseFrom(
+                Base64Serializer.deserializeUnchecked(encodedTestStream, ByteString.class)),
+            coder);
+      } catch (IOException e) {
+        throw new RuntimeException("Could not decode TestStream.", e);
+      }
+    };
   }
 }
diff --git a/sdks/go/pkg/beam/core/runtime/exec/fn.go b/sdks/go/pkg/beam/core/runtime/exec/fn.go
index 595a013..b35fb41 100644
--- a/sdks/go/pkg/beam/core/runtime/exec/fn.go
+++ b/sdks/go/pkg/beam/core/runtime/exec/fn.go
@@ -251,7 +251,16 @@
 	return &n.ret, nil
 }
 
-func makeSideInputs(fn *funcx.Fn, in []*graph.Inbound, side []ReStream) ([]ReusableInput, error) {
+func makeSideInputs(ctx context.Context, w typex.Window, side []SideInputAdapter, reader StateReader, fn *funcx.Fn, in []*graph.Inbound) ([]ReusableInput, error) {
+	streams := make([]ReStream, len(side), len(side))
+	for i, adapter := range side {
+		s, err := adapter.NewIterable(ctx, reader, w)
+		if err != nil {
+			return nil, err
+		}
+		streams[i] = s
+	}
+
 	if len(side) == 0 {
 		return nil, nil // ok: no side input
 	}
@@ -268,8 +277,8 @@
 	offset := len(param) - len(side)
 
 	var ret []ReusableInput
-	for i := 0; i < len(side); i++ {
-		s, err := makeSideInput(in[i+1].Kind, fn.Param[param[i+offset]].T, side[i])
+	for i := 0; i < len(streams); i++ {
+		s, err := makeSideInput(in[i+1].Kind, fn.Param[param[i+offset]].T, streams[i])
 		if err != nil {
 			return nil, errors.WithContextf(err, "making side input %v for %v", i, fn)
 		}
diff --git a/sdks/go/pkg/beam/core/runtime/exec/pardo.go b/sdks/go/pkg/beam/core/runtime/exec/pardo.go
index 8cbd08a..fdf20b0 100644
--- a/sdks/go/pkg/beam/core/runtime/exec/pardo.go
+++ b/sdks/go/pkg/beam/core/runtime/exec/pardo.go
@@ -253,16 +253,7 @@
 
 	// Slow path: init side input for the given window
 
-	streams := make([]ReStream, len(n.Side), len(n.Side))
-	for i, adapter := range n.Side {
-		s, err := adapter.NewIterable(ctx, n.reader, w)
-		if err != nil {
-			return err
-		}
-		streams[i] = s
-	}
-
-	sideinput, err := makeSideInputs(n.Fn.ProcessElementFn(), n.Inbound, streams)
+	sideinput, err := makeSideInputs(ctx, w, n.Side, n.reader, n.Fn.ProcessElementFn(), n.Inbound)
 	if err != nil {
 		return err
 	}
diff --git a/sdks/go/test/run_validatesrunner_tests.sh b/sdks/go/test/run_validatesrunner_tests.sh
index c4c0dc5..49c196e 100755
--- a/sdks/go/test/run_validatesrunner_tests.sh
+++ b/sdks/go/test/run_validatesrunner_tests.sh
@@ -262,14 +262,12 @@
           --job-port $JOB_PORT \
           --expansion-port 0 \
           --artifact-port 0 &
-      SIMULTANEOUS=1
     elif [[ "$RUNNER" == "samza" ]]; then
       java \
           -jar $SAMZA_JOB_SERVER_JAR \
           --job-port $JOB_PORT \
           --expansion-port 0 \
           --artifact-port 0 &
-      SIMULTANEOUS=1
     elif [[ "$RUNNER" == "spark" ]]; then
       java \
           -jar $SPARK_JOB_SERVER_JAR \
@@ -277,7 +275,6 @@
           --job-port $JOB_PORT \
           --expansion-port 0 \
           --artifact-port 0 &
-      SIMULTANEOUS=1  # Spark runner fails if jobs are run concurrently.
     elif [[ "$RUNNER" == "portable" ]]; then
       python3 \
           -m apache_beam.runners.portability.local_job_service_main \
@@ -305,6 +302,11 @@
   fi
 fi
 
+# Disable parallelism on runners that don't support it.
+if [[ "$RUNNER" == "flink" || "$RUNNER" == "spark" || "$RUNNER" == "samza" ]]; then
+  SIMULTANEOUS=1
+fi
+
 if [[ "$RUNNER" == "dataflow" ]]; then
   # Verify docker and gcloud commands exist
   command -v docker
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/SerializableCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/SerializableCoder.java
index ffdf3fb..4f7addd 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/SerializableCoder.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/SerializableCoder.java
@@ -39,6 +39,9 @@
 /**
  * A {@link Coder} for Java classes that implement {@link Serializable}.
  *
+ * <p>{@link SerializableCoder} should be used only for objects that have proper {@link
+ * Object#equals} and {@link Object#hashCode} implementations.
+ *
  * <p>To use, specify the coder type on a PCollection:
  *
  * <pre>{@code
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/WriteFiles.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/WriteFiles.java
index e094aa1..9dabb40 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/WriteFiles.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/WriteFiles.java
@@ -27,6 +27,7 @@
 import java.util.List;
 import java.util.Map;
 import java.util.UUID;
+import java.util.concurrent.CompletionStage;
 import java.util.concurrent.ThreadLocalRandom;
 import org.apache.beam.sdk.annotations.Experimental;
 import org.apache.beam.sdk.annotations.Experimental.Kind;
@@ -70,6 +71,7 @@
 import org.apache.beam.sdk.transforms.windowing.PaneInfo;
 import org.apache.beam.sdk.transforms.windowing.Window;
 import org.apache.beam.sdk.util.CoderUtils;
+import org.apache.beam.sdk.util.MoreFutures;
 import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PCollection.IsBounded;
@@ -90,6 +92,7 @@
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.hash.Hashing;
 import org.checkerframework.checker.nullness.qual.Nullable;
 import org.joda.time.Duration;
+import org.joda.time.Instant;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -898,6 +901,15 @@
 
   private class WriteShardsIntoTempFilesFn
       extends DoFn<KV<ShardedKey<Integer>, Iterable<UserT>>, FileResult<DestinationT>> {
+    private transient @Nullable List<CompletionStage<Void>> closeFutures = null;
+    private transient @Nullable List<KV<Instant, FileResult<DestinationT>>> deferredOutput = null;
+
+    @StartBundle
+    public void startBundle() {
+      closeFutures = new ArrayList<>();
+      deferredOutput = new ArrayList<>();
+    }
+
     @ProcessElement
     public void processElement(ProcessContext c, BoundedWindow window) throws Exception {
       getDynamicDestinations().setSideInputAccessorFromProcessContext(c);
@@ -926,21 +938,43 @@
 
       // Close all writers.
       for (Map.Entry<DestinationT, Writer<DestinationT, OutputT>> entry : writers.entrySet()) {
-        Writer<DestinationT, OutputT> writer = entry.getValue();
-        try {
-          // Close the writer; if this throws let the error propagate.
-          writer.close();
-        } catch (Exception e) {
-          // If anything goes wrong, make sure to delete the temporary file.
-          writer.cleanup();
-          throw e;
-        }
         int shard = c.element().getKey().getShardNumber();
         checkArgument(
             shard != UNKNOWN_SHARDNUM,
             "Shard should have been set, but is unset for element %s",
             c.element());
-        c.output(new FileResult<>(writer.getOutputFile(), shard, window, c.pane(), entry.getKey()));
+        Writer<DestinationT, OutputT> writer = entry.getValue();
+        deferredOutput.add(
+            KV.of(
+                c.timestamp(),
+                new FileResult<>(writer.getOutputFile(), shard, window, c.pane(), entry.getKey())));
+        closeWriterInBackground(writer);
+      }
+    }
+
+    private void closeWriterInBackground(Writer<DestinationT, OutputT> writer) {
+      // Close in parallel so flushing of buffered writes to files for many windows happens in
+      // parallel.
+      closeFutures.add(
+          MoreFutures.runAsync(
+              () -> {
+                try {
+                  // Close the writer; if this throws let the error propagate.
+                  writer.close();
+                } catch (Exception e) {
+                  // If anything goes wrong, make sure to delete the temporary file.
+                  writer.cleanup();
+                  throw e;
+                }
+              }));
+    }
+
+    @FinishBundle
+    public void finishBundle(FinishBundleContext c) throws Exception {
+      MoreFutures.get(MoreFutures.allAsList(closeFutures));
+      // If all writers were closed without exception, output the results to the next stage.
+      for (KV<Instant, FileResult<DestinationT>> result : deferredOutput) {
+        c.output(result.getValue(), result.getKey(), result.getValue().getWindow());
       }
     }
   }
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptions.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptions.java
index 9ffd1f7..8a5e07b 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptions.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptions.java
@@ -136,6 +136,8 @@
  *   <li>Only getters may be annotated with {@link JsonIgnore @JsonIgnore}.
  *   <li>If any getter is annotated with {@link JsonIgnore @JsonIgnore}, then all getters for this
  *       property must be annotated with {@link JsonIgnore @JsonIgnore}.
+ *   <li>If any getter is annotated with {@link JsonDeserialize} and {@link JsonSerialize}, then all
+ *       getters for this property must also be.
  * </ul>
  *
  * <h3>Annotations For PipelineOptions</h3>
@@ -161,6 +163,10 @@
  * <p>{@link JsonIgnore @JsonIgnore} is used to prevent a property from being serialized and
  * available during execution of {@link DoFn}. See the Serialization section below for more details.
  *
+ * <p>{@link JsonSerialize @JsonSerialize} and {@link JsonDeserialize @JsonDeserialize} is used to
+ * control how a property is (de)serialized when the PipelineOptions are (de)serialized to JSON. See
+ * the Serialization section below for more details.
+ *
  * <h2>Registration Of PipelineOptions</h2>
  *
  * <p>Registration of {@link PipelineOptions} by an application guarantees that the {@link
@@ -198,6 +204,14 @@
  * Jackson's ability to automatically configure the {@link ObjectMapper} with additional modules via
  * {@link ObjectMapper#findModules()}.
  *
+ * <p>To further customize serialization, getter methods may be annotated with {@link
+ * JsonSerialize @JsonSerialize} and {@link JsonDeserialize @JsonDeserialize}. {@link
+ * JsonDeserialize @JsonDeserialize} is also used when parsing command line arguments.
+ *
+ * <p>Note: A property must be annotated with <b>BOTH</b>{@link JsonDeserialize @JsonDeserialize}
+ * and {@link JsonSerialize @JsonSerialize} or neither. It is an error to have a property annotated
+ * with only {@link JsonDeserialize @JsonDeserialize} or {@link JsonSerialize @JsonSerialize}.
+ *
  * <p>Note: It is an error to have the same property available in multiple interfaces with only some
  * of them being annotated with {@link JsonIgnore @JsonIgnore}. It is also an error to mark a setter
  * for a property with {@link JsonIgnore @JsonIgnore}.
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionsFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionsFactory.java
index de0e514..ea9d77a 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionsFactory.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionsFactory.java
@@ -22,8 +22,30 @@
 import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkNotNull;
 
 import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.core.JsonParseException;
+import com.fasterxml.jackson.core.JsonParser;
+import com.fasterxml.jackson.databind.BeanProperty;
+import com.fasterxml.jackson.databind.InjectableValues;
 import com.fasterxml.jackson.databind.JavaType;
+import com.fasterxml.jackson.databind.JsonDeserializer;
+import com.fasterxml.jackson.databind.JsonMappingException;
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.JsonSerializer;
 import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.annotation.JsonDeserialize;
+import com.fasterxml.jackson.databind.annotation.JsonSerialize;
+import com.fasterxml.jackson.databind.deser.DefaultDeserializationContext;
+import com.fasterxml.jackson.databind.deser.impl.MethodProperty;
+import com.fasterxml.jackson.databind.introspect.AnnotatedMember;
+import com.fasterxml.jackson.databind.introspect.AnnotatedMethod;
+import com.fasterxml.jackson.databind.introspect.AnnotationCollector;
+import com.fasterxml.jackson.databind.introspect.BeanPropertyDefinition;
+import com.fasterxml.jackson.databind.introspect.TypeResolutionContext;
+import com.fasterxml.jackson.databind.node.TreeTraversingParser;
+import com.fasterxml.jackson.databind.ser.DefaultSerializerProvider;
+import com.fasterxml.jackson.databind.type.TypeBindings;
+import com.fasterxml.jackson.databind.util.SimpleBeanPropertyDefinition;
+import com.fasterxml.jackson.databind.util.TokenBuffer;
 import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
 import java.beans.BeanInfo;
 import java.beans.IntrospectionException;
@@ -477,6 +499,17 @@
       new ObjectMapper()
           .registerModules(ObjectMapper.findModules(ReflectHelpers.findClassLoader()));
 
+  private static final DefaultDeserializationContext DESERIALIZATION_CONTEXT =
+      new DefaultDeserializationContext.Impl(MAPPER.getDeserializationContext().getFactory())
+          .createInstance(
+              MAPPER.getDeserializationConfig(),
+              new TokenBuffer(MAPPER, false).asParser(),
+              new InjectableValues.Std());
+
+  static final DefaultSerializerProvider SERIALIZER_PROVIDER =
+      new DefaultSerializerProvider.Impl()
+          .createInstance(MAPPER.getSerializationConfig(), MAPPER.getSerializerFactory());
+
   /** Classes that are used as the boundary in the stack trace to find the callers class name. */
   private static final ImmutableSet<String> PIPELINE_OPTIONS_FACTORY_CLASSES =
       ImmutableSet.of(PipelineOptionsFactory.class.getName(), Builder.class.getName());
@@ -1058,6 +1091,17 @@
     validateGettersHaveConsistentAnnotation(
         methodNameToAllMethodMap, descriptors, AnnotationPredicates.DEFAULT_VALUE);
 
+    // Verify that there is no getter with a mixed @JsonDeserialize annotation.
+    validateGettersHaveConsistentAnnotation(
+        methodNameToAllMethodMap, descriptors, AnnotationPredicates.JSON_DESERIALIZE);
+
+    // Verify that there is no getter with a mixed @JsonSerialize annotation.
+    validateGettersHaveConsistentAnnotation(
+        methodNameToAllMethodMap, descriptors, AnnotationPredicates.JSON_SERIALIZE);
+
+    // Verify that if a method has either @JsonSerialize or @JsonDeserialize then it has both.
+    validateMethodsHaveBothJsonSerializeAndDeserialize(descriptors);
+
     // Verify that no setter has @JsonIgnore.
     validateSettersDoNotHaveAnnotation(
         methodNameToAllMethodMap, descriptors, AnnotationPredicates.JSON_IGNORE);
@@ -1065,6 +1109,14 @@
     // Verify that no setter has @Default.
     validateSettersDoNotHaveAnnotation(
         methodNameToAllMethodMap, descriptors, AnnotationPredicates.DEFAULT_VALUE);
+
+    // Verify that no setter has @JsonDeserialize.
+    validateSettersDoNotHaveAnnotation(
+        methodNameToAllMethodMap, descriptors, AnnotationPredicates.JSON_DESERIALIZE);
+
+    // Verify that no setter has @JsonSerialize.
+    validateSettersDoNotHaveAnnotation(
+        methodNameToAllMethodMap, descriptors, AnnotationPredicates.JSON_SERIALIZE);
   }
 
   /** Validates that getters don't have mixed annotation. */
@@ -1246,6 +1298,31 @@
         iface.getName());
   }
 
+  private static void validateMethodsHaveBothJsonSerializeAndDeserialize(
+      List<PropertyDescriptor> descriptors) {
+    List<InconsistentJsonSerializeAndDeserializeAnnotation> inconsistentMethods =
+        Lists.newArrayList();
+    for (final PropertyDescriptor descriptor : descriptors) {
+      Method readMethod = descriptor.getReadMethod();
+      if (readMethod == null || IGNORED_METHODS.contains(descriptor.getReadMethod())) {
+        continue;
+      }
+
+      boolean hasJsonSerialize = AnnotationPredicates.JSON_SERIALIZE.forMethod.apply(readMethod);
+      boolean hasJsonDeserialize =
+          AnnotationPredicates.JSON_DESERIALIZE.forMethod.apply(readMethod);
+      if (hasJsonSerialize ^ hasJsonDeserialize) {
+        InconsistentJsonSerializeAndDeserializeAnnotation inconsistentAnnotation =
+            new InconsistentJsonSerializeAndDeserializeAnnotation();
+        inconsistentAnnotation.property = descriptor;
+        inconsistentAnnotation.hasJsonDeserializeAttribute = hasJsonDeserialize;
+        inconsistentMethods.add(inconsistentAnnotation);
+      }
+    }
+
+    throwForInconsistentJsonSerializeAndDeserializeAnnotation(inconsistentMethods);
+  }
+
   private static void checkInheritedFrom(
       Class<?> checkClass, Class fromClass, Set<Class<?>> nonPipelineOptions) {
     if (checkClass.equals(fromClass)) {
@@ -1415,6 +1492,38 @@
     }
   }
 
+  private static class InconsistentJsonSerializeAndDeserializeAnnotation {
+    PropertyDescriptor property;
+    boolean hasJsonDeserializeAttribute;
+  }
+
+  private static void throwForInconsistentJsonSerializeAndDeserializeAnnotation(
+      List<InconsistentJsonSerializeAndDeserializeAnnotation> inconsistentAnnotations)
+      throws IllegalArgumentException {
+    if (inconsistentAnnotations.isEmpty()) {
+      return;
+    }
+
+    StringBuilder builder =
+        new StringBuilder(
+            "Found incorrectly annotated property methods, if a method is annotated with either @JsonSerialize or @JsonDeserialize then it must be annotated with both.");
+
+    for (InconsistentJsonSerializeAndDeserializeAnnotation annotation : inconsistentAnnotations) {
+      String presentAnnotation;
+      if (annotation.hasJsonDeserializeAttribute) {
+        presentAnnotation = "JsonDeserialize";
+      } else {
+        presentAnnotation = "JsonSerialize";
+      }
+      builder.append(
+          String.format(
+              "%n  - Property [%s] had only @%s",
+              annotation.property.getName(), presentAnnotation));
+    }
+
+    throw new IllegalArgumentException(builder.toString());
+  }
+
   /** A {@link Comparator} that uses the classes name to compare them. */
   private static class ClassNameComparator implements Comparator<Class<?>> {
     static final ClassNameComparator INSTANCE = new ClassNameComparator();
@@ -1500,6 +1609,18 @@
               return false;
             });
 
+    static final AnnotationPredicates JSON_DESERIALIZE =
+        new AnnotationPredicates(
+            JsonDeserialize.class,
+            input -> JsonDeserialize.class.equals(input.annotationType()),
+            input -> input.isAnnotationPresent(JsonDeserialize.class));
+
+    static final AnnotationPredicates JSON_SERIALIZE =
+        new AnnotationPredicates(
+            JsonSerialize.class,
+            input -> JsonSerialize.class.equals(input.annotationType()),
+            input -> input.isAnnotationPresent(JsonSerialize.class));
+
     final Class<? extends Annotation> annotationClass;
     final Predicate<Annotation> forAnnotation;
     final Predicate<Method> forMethod;
@@ -1572,6 +1693,141 @@
     return builder.build();
   }
 
+  private static BeanProperty createBeanProperty(Method method) {
+    AnnotationCollector ac = AnnotationCollector.emptyCollector();
+    for (Annotation ann : method.getAnnotations()) {
+      ac = ac.addOrOverride(ann);
+    }
+
+    AnnotatedMethod annotatedMethod =
+        new AnnotatedMethod(
+            new TypeResolutionContext.Basic(MAPPER.getTypeFactory(), TypeBindings.emptyBindings()),
+            method,
+            ac.asAnnotationMap(),
+            null);
+
+    BeanPropertyDefinition propDef =
+        SimpleBeanPropertyDefinition.construct(MAPPER.getDeserializationConfig(), annotatedMethod);
+
+    JavaType type = MAPPER.constructType(method.getGenericReturnType());
+
+    try {
+      return new MethodProperty(
+          propDef,
+          type,
+          MAPPER.getDeserializationConfig().findTypeDeserializer(type),
+          annotatedMethod.getAllAnnotations(),
+          annotatedMethod);
+    } catch (JsonMappingException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  private static JsonDeserializer<Object> computeDeserializerForMethod(Method method) {
+    try {
+      BeanProperty prop = createBeanProperty(method);
+      AnnotatedMember annotatedMethod = prop.getMember();
+
+      Object maybeDeserializerClass =
+          DESERIALIZATION_CONTEXT.getAnnotationIntrospector().findDeserializer(annotatedMethod);
+
+      JsonDeserializer<Object> jsonDeserializer =
+          DESERIALIZATION_CONTEXT.deserializerInstance(annotatedMethod, maybeDeserializerClass);
+
+      if (jsonDeserializer == null) {
+        jsonDeserializer =
+            DESERIALIZATION_CONTEXT.findContextualValueDeserializer(prop.getType(), prop);
+      }
+      return jsonDeserializer;
+    } catch (JsonMappingException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  private static Optional<JsonSerializer<Object>> computeCustomSerializerForMethod(Method method) {
+    try {
+      BeanProperty prop = createBeanProperty(method);
+      AnnotatedMember annotatedMethod = prop.getMember();
+
+      Object maybeSerializerClass =
+          SERIALIZER_PROVIDER.getAnnotationIntrospector().findSerializer(annotatedMethod);
+
+      return Optional.fromNullable(
+          SERIALIZER_PROVIDER.serializerInstance(annotatedMethod, maybeSerializerClass));
+    } catch (JsonMappingException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  /**
+   * Get a {@link JsonDeserializer} for a given method. If the method is annotated with {@link
+   * JsonDeserialize} the specified deserializer from the annotation is returned, otherwise the
+   * default is returned.
+   */
+  private static JsonDeserializer<Object> getDeserializerForMethod(Method method) {
+    return CACHE
+        .get()
+        .deserializerCache
+        .computeIfAbsent(method, PipelineOptionsFactory::computeDeserializerForMethod);
+  }
+
+  /**
+   * Get a {@link JsonSerializer} for a given method. If the method is annotated with {@link
+   * JsonDeserialize} the specified serializer from the annotation is returned, otherwise null is
+   * returned.
+   */
+  static @Nullable JsonSerializer<Object> getCustomSerializerForMethod(Method method) {
+    return CACHE
+        .get()
+        .serializerCache
+        .computeIfAbsent(method, PipelineOptionsFactory::computeCustomSerializerForMethod)
+        .orNull();
+  }
+
+  static Object deserializeNode(JsonNode node, Method method) throws IOException {
+    if (node.isNull()) {
+      return null;
+    }
+
+    JsonParser parser = new TreeTraversingParser(node, MAPPER);
+    parser.nextToken();
+
+    JsonDeserializer<Object> jsonDeserializer = getDeserializerForMethod(method);
+    return jsonDeserializer.deserialize(parser, DESERIALIZATION_CONTEXT);
+  }
+
+  /**
+   * Attempt to parse an input string into an instance of `type` using an {@link ObjectMapper}.
+   *
+   * <p>If the getter method is annotated with {@link
+   * com.fasterxml.jackson.databind.annotation.JsonDeserialize} the specified deserializer will be
+   * used, otherwise the default ObjectMapper deserialization strategy is used.
+   *
+   * <p>Parsing is attempted twice, once with the raw string value. If that attempt fails, another
+   * attempt is made by wrapping the value in quotes so that it is interpreted as a JSON string.
+   */
+  private static Object tryParseObject(String value, Method method) throws IOException {
+
+    JsonNode tree;
+    try {
+      tree = MAPPER.readTree(value);
+    } catch (JsonParseException e) {
+      // try again, quoting the input string if it wasn't already
+      if (!(value.startsWith("\"") && value.endsWith("\""))) {
+        try {
+          tree = MAPPER.readTree("\"" + value + "\"");
+        } catch (JsonParseException inner) {
+          // rethrow the original exception rather the one thrown from the fallback attempt
+          throw e;
+        }
+      } else {
+        throw e;
+      }
+    }
+
+    return deserializeNode(tree, method);
+  }
+
   /**
    * Using the parsed string arguments, we convert the strings to the expected return type of the
    * methods that are found on the passed-in class.
@@ -1632,6 +1888,7 @@
         // Only allow empty argument values for String, String Array, and Collection<String>.
         Class<?> returnType = method.getReturnType();
         JavaType type = MAPPER.getTypeFactory().constructType(method.getGenericReturnType());
+
         if ("runner".equals(entry.getKey())) {
           String runner = Iterables.getOnlyElement(entry.getValue());
           final Map<String, Class<? extends PipelineRunner<?>>> pipelineRunners =
@@ -1680,7 +1937,7 @@
             checkEmptyStringAllowed(returnType, type, method.getGenericReturnType().toString());
           }
           try {
-            convertedOptions.put(entry.getKey(), MAPPER.readValue(value, type));
+            convertedOptions.put(entry.getKey(), tryParseObject(value, method));
           } catch (IOException e) {
             throw new IllegalArgumentException("Unable to parse JSON value " + value, e);
           }
@@ -1791,6 +2048,11 @@
     private final Map<Set<Class<? extends PipelineOptions>>, Registration<?>> combinedCache =
         Maps.newConcurrentMap();
 
+    private final Map<Method, JsonDeserializer<Object>> deserializerCache = Maps.newConcurrentMap();
+
+    private final Map<Method, Optional<JsonSerializer<Object>>> serializerCache =
+        Maps.newConcurrentMap();
+
     private Cache() {
       final ClassLoader loader = ReflectHelpers.findClassLoader();
       // Store the list of all available pipeline runners.
diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ProxyInvocationHandler.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ProxyInvocationHandler.java
index 97f673a..9283509 100644
--- a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ProxyInvocationHandler.java
+++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ProxyInvocationHandler.java
@@ -23,15 +23,14 @@
 import com.fasterxml.jackson.annotation.JsonIgnore;
 import com.fasterxml.jackson.core.JsonGenerator;
 import com.fasterxml.jackson.core.JsonParser;
-import com.fasterxml.jackson.core.JsonProcessingException;
 import com.fasterxml.jackson.databind.DeserializationContext;
-import com.fasterxml.jackson.databind.JavaType;
 import com.fasterxml.jackson.databind.JsonDeserializer;
 import com.fasterxml.jackson.databind.JsonNode;
 import com.fasterxml.jackson.databind.JsonSerializer;
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.fasterxml.jackson.databind.SerializerProvider;
 import com.fasterxml.jackson.databind.node.ObjectNode;
+import com.fasterxml.jackson.databind.util.TokenBuffer;
 import com.google.auto.value.AutoValue;
 import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
 import java.beans.PropertyDescriptor;
@@ -43,7 +42,6 @@
 import java.lang.reflect.Method;
 import java.lang.reflect.ParameterizedType;
 import java.lang.reflect.Proxy;
-import java.lang.reflect.Type;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.HashMap;
@@ -510,13 +508,13 @@
    * @return An object matching the return type of the method passed in.
    */
   private Object getValueFromJson(String propertyName, Method method) {
+    JsonNode jsonNode = jsonOptions.get(propertyName);
+    return getValueFromJson(jsonNode, method);
+  }
+
+  private static Object getValueFromJson(JsonNode node, Method method) {
     try {
-      JavaType type =
-          PipelineOptionsFactory.MAPPER
-              .getTypeFactory()
-              .constructType(method.getGenericReturnType());
-      JsonNode jsonNode = jsonOptions.get(propertyName);
-      return PipelineOptionsFactory.MAPPER.readValue(jsonNode.toString(), type);
+      return PipelineOptionsFactory.deserializeNode(node, method);
     } catch (IOException e) {
       throw new RuntimeException("Unable to parse representation", e);
     }
@@ -654,18 +652,35 @@
   }
 
   static class Serializer extends JsonSerializer<PipelineOptions> {
+    private void serializeEntry(
+        String name,
+        Object value,
+        JsonGenerator jgen,
+        Map<String, JsonSerializer<Object>> customSerializers)
+        throws IOException {
+
+      JsonSerializer<Object> customSerializer = customSerializers.get(name);
+      if (value == null || customSerializer == null || value instanceof JsonNode) {
+        jgen.writeObject(value);
+      } else {
+        customSerializer.serialize(value, jgen, PipelineOptionsFactory.SERIALIZER_PROVIDER);
+      }
+    }
+
     @Override
     public void serialize(PipelineOptions value, JsonGenerator jgen, SerializerProvider provider)
-        throws IOException, JsonProcessingException {
+        throws IOException {
       ProxyInvocationHandler handler = (ProxyInvocationHandler) Proxy.getInvocationHandler(value);
       synchronized (handler) {
+        PipelineOptionsFactory.Cache cache = PipelineOptionsFactory.CACHE.get();
         // We first filter out any properties that have been modified since
         // the last serialization of this PipelineOptions and then verify that
         // they are all serializable.
         Map<String, BoundValue> filteredOptions = Maps.newHashMap(handler.options);
-        PipelineOptionsFactory.Cache cache = PipelineOptionsFactory.CACHE.get();
+        Map<String, JsonSerializer<Object>> propertyToSerializer =
+            getSerializerMap(cache, handler.knownInterfaces);
         removeIgnoredOptions(cache, handler.knownInterfaces, filteredOptions);
-        ensureSerializable(cache, handler.knownInterfaces, filteredOptions);
+        ensureSerializable(cache, handler.knownInterfaces, filteredOptions, propertyToSerializer);
 
         // Now we create the map of serializable options by taking the original
         // set of serialized options (if any) and updating them with any properties
@@ -677,7 +692,15 @@
 
         jgen.writeStartObject();
         jgen.writeFieldName("options");
-        jgen.writeObject(serializableOptions);
+
+        jgen.writeStartObject();
+
+        for (Map.Entry<String, Object> entry : serializableOptions.entrySet()) {
+          jgen.writeFieldName(entry.getKey());
+          serializeEntry(entry.getKey(), entry.getValue(), jgen, propertyToSerializer);
+        }
+
+        jgen.writeEndObject();
 
         List<Map<String, Object>> serializedDisplayData = Lists.newArrayList();
         DisplayData displayData = DisplayData.from(value);
@@ -694,6 +717,23 @@
       }
     }
 
+    private Map<String, JsonSerializer<Object>> getSerializerMap(
+        PipelineOptionsFactory.Cache cache, Set<Class<? extends PipelineOptions>> interfaces) {
+
+      Map<String, JsonSerializer<Object>> propertyToSerializer = Maps.newHashMap();
+      for (PropertyDescriptor descriptor : cache.getPropertyDescriptors(interfaces)) {
+        if (descriptor.getReadMethod() != null) {
+          JsonSerializer<Object> maybeSerializer =
+              PipelineOptionsFactory.getCustomSerializerForMethod(descriptor.getReadMethod());
+          if (maybeSerializer != null) {
+            propertyToSerializer.put(descriptor.getName(), maybeSerializer);
+          }
+        }
+      }
+
+      return propertyToSerializer;
+    }
+
     /**
      * We remove all properties within the passed in options where there getter is annotated with
      * {@link JsonIgnore @JsonIgnore} from the passed in options using the passed in interfaces.
@@ -724,27 +764,27 @@
     private void ensureSerializable(
         PipelineOptionsFactory.Cache cache,
         Set<Class<? extends PipelineOptions>> interfaces,
-        Map<String, BoundValue> options)
+        Map<String, BoundValue> options,
+        Map<String, JsonSerializer<Object>> propertyToSerializer)
         throws IOException {
       // Construct a map from property name to the return type of the getter.
-      Map<String, Type> propertyToReturnType = Maps.newHashMap();
+      Map<String, Method> propertyToReadMethod = Maps.newHashMap();
       for (PropertyDescriptor descriptor : cache.getPropertyDescriptors(interfaces)) {
         if (descriptor.getReadMethod() != null) {
-          propertyToReturnType.put(
-              descriptor.getName(), descriptor.getReadMethod().getGenericReturnType());
+          propertyToReadMethod.put(descriptor.getName(), descriptor.getReadMethod());
         }
       }
 
       // Attempt to serialize and deserialize each property.
       for (Map.Entry<String, BoundValue> entry : options.entrySet()) {
         try {
-          String serializedValue =
-              PipelineOptionsFactory.MAPPER.writeValueAsString(entry.getValue().getValue());
-          JavaType type =
-              PipelineOptionsFactory.MAPPER
-                  .getTypeFactory()
-                  .constructType(propertyToReturnType.get(entry.getKey()));
-          PipelineOptionsFactory.MAPPER.readValue(serializedValue, type);
+          Object boundValue = entry.getValue().getValue();
+          if (boundValue != null) {
+            TokenBuffer buffer = new TokenBuffer(PipelineOptionsFactory.MAPPER, false);
+            serializeEntry(entry.getKey(), boundValue, buffer, propertyToSerializer);
+            Method method = propertyToReadMethod.get(entry.getKey());
+            getValueFromJson(buffer.asParser().<JsonNode>readValueAsTree(), method);
+          }
         } catch (Exception e) {
           throw new IOException(
               String.format(
@@ -759,7 +799,7 @@
   static class Deserializer extends JsonDeserializer<PipelineOptions> {
     @Override
     public PipelineOptions deserialize(JsonParser jp, DeserializationContext ctxt)
-        throws IOException, JsonProcessingException {
+        throws IOException {
       ObjectNode objectNode = jp.readValueAsTree();
       JsonNode rawOptionsNode = objectNode.get("options");
 
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsFactoryTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsFactoryTest.java
index 5fa7b1f..94fd3f4 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsFactoryTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsFactoryTest.java
@@ -33,6 +33,7 @@
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertSame;
+import static org.junit.Assert.assertThrows;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assume.assumeFalse;
 
@@ -49,7 +50,9 @@
 import com.fasterxml.jackson.databind.SerializerProvider;
 import com.fasterxml.jackson.databind.annotation.JsonDeserialize;
 import com.fasterxml.jackson.databind.annotation.JsonSerialize;
+import com.fasterxml.jackson.databind.deser.std.StdDeserializer;
 import com.fasterxml.jackson.databind.module.SimpleModule;
+import com.fasterxml.jackson.databind.ser.std.StdSerializer;
 import com.google.auto.service.AutoService;
 import java.io.ByteArrayOutputStream;
 import java.io.IOException;
@@ -57,6 +60,7 @@
 import java.util.Collection;
 import java.util.List;
 import java.util.Map;
+import java.util.Objects;
 import java.util.Set;
 import org.apache.beam.model.jobmanagement.v1.JobApi.PipelineOptionDescriptor;
 import org.apache.beam.model.jobmanagement.v1.JobApi.PipelineOptionType;
@@ -2080,6 +2084,161 @@
     }
   }
 
+  public static class SimpleParsedObject {
+    public String value;
+
+    public SimpleParsedObject(String value) {
+      this.value = value;
+    }
+  }
+
+  public interface OptionsWithParsing extends PipelineOptions {
+    SimpleParsedObject getSimple();
+
+    void setSimple(SimpleParsedObject value);
+  }
+
+  @Test
+  public void testAutoQuoteStringArgumentsForComplexObjects() {
+    OptionsWithParsing options =
+        PipelineOptionsFactory.fromArgs("--simple=test").as(OptionsWithParsing.class);
+
+    assertEquals("test", options.getSimple().value);
+  }
+
+  public static class ComplexType2 {
+    public String value;
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+      ComplexType2 that = (ComplexType2) o;
+      return value.equals(that.value);
+    }
+
+    @Override
+    public int hashCode() {
+      return value.hashCode();
+    }
+  }
+
+  public interface OptionsWithJsonDeserialize1 extends PipelineOptions {
+    @JsonDeserialize(using = ComplexType2Deserializer1.class)
+    @JsonSerialize(using = ComplexType2Serializer1.class)
+    ComplexType2 getComplexType();
+
+    void setComplexType(ComplexType2 value);
+  }
+
+  public interface OptionsWithJsonDeserialize2 extends PipelineOptions {
+    @JsonDeserialize(using = ComplexType2Deserializer2.class)
+    ComplexType2 getComplexType();
+
+    void setComplexType(ComplexType2 value);
+  }
+
+  public static class ComplexType2Deserializer1 extends StdDeserializer<ComplexType2> {
+    public ComplexType2Deserializer1() {
+      super(ComplexType2.class);
+    }
+
+    @Override
+    public ComplexType2 deserialize(JsonParser p, DeserializationContext ctxt)
+        throws IOException, JsonProcessingException {
+      ComplexType2 ct = new ComplexType2();
+      ct.value = p.getText();
+      return ct;
+    }
+  }
+
+  public static class ComplexType2Serializer1 extends StdSerializer<ComplexType2> {
+    public ComplexType2Serializer1() {
+      super(ComplexType2.class);
+    }
+
+    @Override
+    public void serialize(ComplexType2 value, JsonGenerator gen, SerializerProvider provider)
+        throws IOException {
+      gen.writeString(value.value);
+    }
+  }
+
+  public static class ComplexType2Deserializer2 extends StdDeserializer<ComplexType2> {
+    public ComplexType2Deserializer2() {
+      super(ComplexType2.class);
+    }
+
+    @Override
+    public ComplexType2 deserialize(JsonParser p, DeserializationContext ctxt)
+        throws IOException, JsonProcessingException {
+      ComplexType2 ct = new ComplexType2();
+      ct.value = p.getText();
+      return ct;
+    }
+  }
+
+  @Test
+  public void testJsonDeserializeAttribute_NoConflict() {
+    OptionsWithJsonDeserialize1 options =
+        PipelineOptionsFactory.fromArgs("--complexType=test").as(OptionsWithJsonDeserialize1.class);
+
+    assertEquals("test", options.getComplexType().value);
+  }
+
+  @Test
+  public void testJsonDeserializeAttribute_Conflict() {
+    OptionsWithJsonDeserialize1 options =
+        PipelineOptionsFactory.fromArgs("--complexType=test").as(OptionsWithJsonDeserialize1.class);
+
+    IllegalArgumentException thrown =
+        assertThrows(
+            IllegalArgumentException.class, () -> options.as(OptionsWithJsonDeserialize2.class));
+    assertThat(
+        thrown.getMessage(),
+        containsString("Property [complexType] is marked with contradictory annotations"));
+  }
+
+  public interface InconsistentJsonDeserializeAttributes extends PipelineOptions {
+    @JsonDeserialize()
+    String getString();
+
+    void setString(String value);
+  }
+
+  public interface InconsistentJsonSerializeAttributes extends PipelineOptions {
+    @JsonSerialize()
+    String getString();
+
+    void setString(String value);
+  }
+
+  @Test
+  public void testJsonDeserializeAttributeValidation() {
+    IllegalArgumentException thrown =
+        assertThrows(
+            IllegalArgumentException.class,
+            () ->
+                PipelineOptionsFactory.fromArgs("--string=test")
+                    .as(InconsistentJsonDeserializeAttributes.class));
+    assertThat(thrown.getMessage(), containsString("Property [string] had only @JsonDeserialize"));
+  }
+
+  @Test
+  public void testJsonSerializeAttributeValidation() {
+    IllegalArgumentException thrown =
+        assertThrows(
+            IllegalArgumentException.class,
+            () ->
+                PipelineOptionsFactory.fromArgs("--string=test")
+                    .as(InconsistentJsonSerializeAttributes.class));
+    assertThat(thrown.getMessage(), containsString("Property [string] had only @JsonSerialize"));
+  }
+
   /** Test interface. */
   public interface TestDescribeOptions extends PipelineOptions {
     String getString();
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ProxyInvocationHandlerTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ProxyInvocationHandlerTest.java
index b590b00..4cf9b2f 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ProxyInvocationHandlerTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/ProxyInvocationHandlerTest.java
@@ -36,8 +36,17 @@
 import com.fasterxml.jackson.annotation.JsonIgnore;
 import com.fasterxml.jackson.annotation.JsonProperty;
 import com.fasterxml.jackson.annotation.JsonValue;
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.core.JsonParser;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.DeserializationContext;
 import com.fasterxml.jackson.databind.JsonMappingException;
 import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.SerializerProvider;
+import com.fasterxml.jackson.databind.annotation.JsonDeserialize;
+import com.fasterxml.jackson.databind.annotation.JsonSerialize;
+import com.fasterxml.jackson.databind.deser.std.StdDeserializer;
+import com.fasterxml.jackson.databind.ser.std.StdSerializer;
 import com.google.common.testing.EqualsTester;
 import java.io.IOException;
 import java.io.NotSerializableException;
@@ -904,6 +913,24 @@
     void setValue(Object value);
   }
 
+  public interface PrimitiveIntOptions extends PipelineOptions {
+    int getInt();
+
+    void setInt(int value);
+  }
+
+  @Test
+  public void testPrimitiveIntegerFromJsonOptions() throws Exception {
+    String optionsJson =
+        "{\"options\":{\"appName\":\"ProxyInvocationHandlerTest\",\"optionsId\":1,\"int\":\"100\"},\"display_data\":[{\"namespace\":\"org.apache.beam.sdk.options.ProxyInvocationHandlerTest$DisplayDataOptions\",\"key\":\"int\",\"type\":\"INTEGER\",\"value\":100},{\"namespace\":\"org.apache.beam.sdk.options.ApplicationNameOptions\",\"key\":\"appName\",\"type\":\"STRING\",\"value\":\"ProxyInvocationHandlerTest\"}]}";
+
+    PrimitiveIntOptions options =
+        MAPPER.readValue(optionsJson, PipelineOptions.class).as(PrimitiveIntOptions.class);
+
+    int value = options.getInt();
+    assertEquals(100, value);
+  }
+
   @Test
   public void testDisplayDataInheritanceNamespace() {
     ExtendsBaseOptions options = PipelineOptionsFactory.as(ExtendsBaseOptions.class);
@@ -969,6 +996,56 @@
     void setBar(String value);
   }
 
+  public static class JacksonObject {
+    String value;
+  }
+
+  public static class JacksonObjectSerializer extends StdSerializer<JacksonObject> {
+    public JacksonObjectSerializer() {
+      super(JacksonObject.class);
+    }
+
+    @Override
+    public void serialize(JacksonObject value, JsonGenerator gen, SerializerProvider provider)
+        throws IOException {
+      gen.writeString(value.value);
+    }
+  }
+
+  public static class JacksonObjectDeserializer extends StdDeserializer<JacksonObject> {
+    public JacksonObjectDeserializer() {
+      super(JacksonObject.class);
+    }
+
+    @Override
+    public JacksonObject deserialize(JsonParser p, DeserializationContext ctxt)
+        throws IOException, JsonProcessingException {
+      JacksonObject obj = new JacksonObject();
+      obj.value = p.getValueAsString();
+      return obj;
+    }
+  }
+
+  public interface JacksonOptions extends PipelineOptions {
+    @JsonSerialize(using = JacksonObjectSerializer.class)
+    @JsonDeserialize(using = JacksonObjectDeserializer.class)
+    JacksonObject getJacksonObject();
+
+    void setJacksonObject(JacksonObject value);
+  }
+
+  @Test
+  public void testJacksonSerializeAndDeserialize() throws Exception {
+    JacksonOptions options = PipelineOptionsFactory.as(JacksonOptions.class);
+    JacksonObject value = new JacksonObject();
+    value.value = "foo";
+
+    options.setJacksonObject(value);
+
+    JacksonOptions deserializedOptions = serializeDeserialize(JacksonOptions.class, options);
+    assertEquals(options.getJacksonObject().value, deserializedOptions.getJacksonObject().value);
+  }
+
   @Test
   public void testDisplayDataExcludesDefaultValues() {
     PipelineOptions options = PipelineOptionsFactory.as(HasDefaults.class);
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 6d33b30..87f7eff 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
@@ -86,27 +86,27 @@
   @Category({ValidatesRunner.class, UsesTestStream.class})
   public void testLateDataAccumulating() {
     Instant instant = new Instant(0);
-    TestStream<Integer> source =
-        TestStream.create(VarIntCoder.of())
+    TestStream<Long> source =
+        TestStream.create(VarLongCoder.of())
             .addElements(
-                TimestampedValue.of(1, instant),
-                TimestampedValue.of(2, instant),
-                TimestampedValue.of(3, instant))
+                TimestampedValue.of(1L, instant),
+                TimestampedValue.of(2L, instant),
+                TimestampedValue.of(3L, instant))
             .advanceWatermarkTo(instant.plus(Duration.standardMinutes(6)))
             // These elements are late but within the allowed lateness
-            .addElements(TimestampedValue.of(4, instant), TimestampedValue.of(5, instant))
+            .addElements(TimestampedValue.of(4L, instant), TimestampedValue.of(5L, instant))
             .advanceWatermarkTo(instant.plus(Duration.standardMinutes(20)))
             // These elements are droppably late
             .addElements(
-                TimestampedValue.of(-1, instant),
-                TimestampedValue.of(-2, instant),
-                TimestampedValue.of(-3, instant))
+                TimestampedValue.of(-1L, instant),
+                TimestampedValue.of(-2L, instant),
+                TimestampedValue.of(-3L, instant))
             .advanceWatermarkToInfinity();
 
-    PCollection<Integer> windowed =
+    PCollection<Long> windowed =
         p.apply(source)
             .apply(
-                Window.<Integer>into(FixedWindows.of(Duration.standardMinutes(5)))
+                Window.<Long>into(FixedWindows.of(Duration.standardMinutes(5)))
                     .triggering(
                         AfterWatermark.pastEndOfWindow()
                             .withEarlyFirings(
@@ -115,19 +115,19 @@
                             .withLateFirings(AfterPane.elementCountAtLeast(1)))
                     .accumulatingFiredPanes()
                     .withAllowedLateness(Duration.standardMinutes(5), ClosingBehavior.FIRE_ALWAYS));
-    PCollection<Integer> triggered =
+    PCollection<Long> triggered =
         windowed
             .apply(WithKeys.of(1))
             .apply(GroupByKey.create())
             .apply(Values.create())
             .apply(Flatten.iterables());
     PCollection<Long> count =
-        windowed.apply(Combine.globally(Count.<Integer>combineFn()).withoutDefaults());
-    PCollection<Integer> sum = windowed.apply(Sum.integersGlobally().withoutDefaults());
+        windowed.apply(Combine.globally(Count.<Long>combineFn()).withoutDefaults());
+    PCollection<Long> sum = windowed.apply(Sum.longsGlobally().withoutDefaults());
 
     IntervalWindow window = new IntervalWindow(instant, instant.plus(Duration.standardMinutes(5L)));
-    PAssert.that(triggered).inFinalPane(window).containsInAnyOrder(1, 2, 3, 4, 5);
-    PAssert.that(triggered).inOnTimePane(window).containsInAnyOrder(1, 2, 3);
+    PAssert.that(triggered).inFinalPane(window).containsInAnyOrder(1L, 2L, 3L, 4L, 5L);
+    PAssert.that(triggered).inOnTimePane(window).containsInAnyOrder(1L, 2L, 3L);
     PAssert.that(count)
         .inWindow(window)
         .satisfies(
@@ -141,8 +141,8 @@
         .inWindow(window)
         .satisfies(
             input -> {
-              for (Integer sum1 : input) {
-                assertThat(sum1, allOf(greaterThanOrEqualTo(6), lessThanOrEqualTo(15)));
+              for (Long sum1 : input) {
+                assertThat(sum1, allOf(greaterThanOrEqualTo(6L), lessThanOrEqualTo(15L)));
               }
               return null;
             });
@@ -297,8 +297,10 @@
             .addElements("foo", "bar")
             .advanceWatermarkToInfinity();
 
-    TestStream<Integer> other =
-        TestStream.create(VarIntCoder.of()).addElements(1, 2, 3, 4).advanceWatermarkToInfinity();
+    TestStream<Long> other =
+        TestStream.create(VarLongCoder.of())
+            .addElements(1L, 2L, 3L, 4L)
+            .advanceWatermarkToInfinity();
 
     PCollection<String> createStrings =
         p.apply("CreateStrings", stream)
@@ -309,15 +311,15 @@
                     .withAllowedLateness(Duration.ZERO)
                     .accumulatingFiredPanes());
     PAssert.that(createStrings).containsInAnyOrder("foo", "bar");
-    PCollection<Integer> createInts =
+    PCollection<Long> createInts =
         p.apply("CreateInts", other)
             .apply(
                 "WindowInts",
-                Window.<Integer>configure()
+                Window.<Long>configure()
                     .triggering(AfterPane.elementCountAtLeast(4))
                     .withAllowedLateness(Duration.ZERO)
                     .accumulatingFiredPanes());
-    PAssert.that(createInts).containsInAnyOrder(1, 2, 3, 4);
+    PAssert.that(createInts).containsInAnyOrder(1L, 2L, 3L, 4L);
 
     p.run();
   }
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DeduplicateTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DeduplicateTest.java
index 00ee2d2..0a8dc54 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DeduplicateTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DeduplicateTest.java
@@ -23,7 +23,7 @@
 
 import org.apache.beam.sdk.coders.KvCoder;
 import org.apache.beam.sdk.coders.StringUtf8Coder;
-import org.apache.beam.sdk.coders.VarIntCoder;
+import org.apache.beam.sdk.coders.VarLongCoder;
 import org.apache.beam.sdk.state.TimeDomain;
 import org.apache.beam.sdk.testing.NeedsRunner;
 import org.apache.beam.sdk.testing.PAssert;
@@ -177,27 +177,28 @@
   @Category({NeedsRunner.class, UsesTestStreamWithProcessingTime.class})
   public void testRepresentativeValuesWithCoder() {
     Instant base = new Instant(0);
-    TestStream<KV<Integer, String>> values =
-        TestStream.create(KvCoder.of(VarIntCoder.of(), StringUtf8Coder.of()))
+    TestStream<KV<Long, String>> values =
+        TestStream.create(KvCoder.of(VarLongCoder.of(), StringUtf8Coder.of()))
             .advanceWatermarkTo(base)
             .addElements(
-                TimestampedValue.of(KV.of(1, "k1"), base),
-                TimestampedValue.of(KV.of(2, "k2"), base.plus(Duration.standardSeconds(10))),
-                TimestampedValue.of(KV.of(3, "k3"), base.plus(Duration.standardSeconds(20))))
+                TimestampedValue.of(KV.of(1L, "k1"), base),
+                TimestampedValue.of(KV.of(2L, "k2"), base.plus(Duration.standardSeconds(10))),
+                TimestampedValue.of(KV.of(3L, "k3"), base.plus(Duration.standardSeconds(20))))
             .advanceProcessingTime(Duration.standardMinutes(1))
             .addElements(
-                TimestampedValue.of(KV.of(1, "k1"), base.plus(Duration.standardSeconds(30))),
-                TimestampedValue.of(KV.of(2, "k2"), base.plus(Duration.standardSeconds(40))),
-                TimestampedValue.of(KV.of(3, "k3"), base.plus(Duration.standardSeconds(50))))
+                TimestampedValue.of(KV.of(1L, "k1"), base.plus(Duration.standardSeconds(30))),
+                TimestampedValue.of(KV.of(2L, "k2"), base.plus(Duration.standardSeconds(40))),
+                TimestampedValue.of(KV.of(3L, "k3"), base.plus(Duration.standardSeconds(50))))
             .advanceWatermarkToInfinity();
 
-    PCollection<KV<Integer, String>> distinctValues =
+    PCollection<KV<Long, String>> distinctValues =
         p.apply(values)
             .apply(
-                Deduplicate.withRepresentativeValueFn(new Keys<Integer>())
-                    .withRepresentativeCoder(VarIntCoder.of()));
+                Deduplicate.withRepresentativeValueFn(new Keys<Long>())
+                    .withRepresentativeCoder(VarLongCoder.of()));
 
-    PAssert.that(distinctValues).containsInAnyOrder(KV.of(1, "k1"), KV.of(2, "k2"), KV.of(3, "k3"));
+    PAssert.that(distinctValues)
+        .containsInAnyOrder(KV.of(1L, "k1"), KV.of(2L, "k2"), KV.of(3L, "k3"));
     p.run();
   }
 
@@ -205,27 +206,28 @@
   @Category({NeedsRunner.class, UsesTestStreamWithProcessingTime.class})
   public void testTriggeredRepresentativeValuesWithType() {
     Instant base = new Instant(0);
-    TestStream<KV<Integer, String>> values =
-        TestStream.create(KvCoder.of(VarIntCoder.of(), StringUtf8Coder.of()))
+    TestStream<KV<Long, String>> values =
+        TestStream.create(KvCoder.of(VarLongCoder.of(), StringUtf8Coder.of()))
             .advanceWatermarkTo(base)
             .addElements(
-                TimestampedValue.of(KV.of(1, "k1"), base),
-                TimestampedValue.of(KV.of(2, "k2"), base.plus(Duration.standardSeconds(10))),
-                TimestampedValue.of(KV.of(3, "k3"), base.plus(Duration.standardSeconds(20))))
+                TimestampedValue.of(KV.of(1L, "k1"), base),
+                TimestampedValue.of(KV.of(2L, "k2"), base.plus(Duration.standardSeconds(10))),
+                TimestampedValue.of(KV.of(3L, "k3"), base.plus(Duration.standardSeconds(20))))
             .advanceProcessingTime(Duration.standardMinutes(1))
             .addElements(
-                TimestampedValue.of(KV.of(1, "k1"), base.plus(Duration.standardSeconds(30))),
-                TimestampedValue.of(KV.of(2, "k2"), base.plus(Duration.standardSeconds(40))),
-                TimestampedValue.of(KV.of(3, "k3"), base.plus(Duration.standardSeconds(50))))
+                TimestampedValue.of(KV.of(1L, "k1"), base.plus(Duration.standardSeconds(30))),
+                TimestampedValue.of(KV.of(2L, "k2"), base.plus(Duration.standardSeconds(40))),
+                TimestampedValue.of(KV.of(3L, "k3"), base.plus(Duration.standardSeconds(50))))
             .advanceWatermarkToInfinity();
 
-    PCollection<KV<Integer, String>> distinctValues =
+    PCollection<KV<Long, String>> distinctValues =
         p.apply(values)
             .apply(
-                Deduplicate.withRepresentativeValueFn(new Keys<Integer>())
-                    .withRepresentativeCoder(VarIntCoder.of()));
+                Deduplicate.withRepresentativeValueFn(new Keys<Long>())
+                    .withRepresentativeCoder(VarLongCoder.of()));
 
-    PAssert.that(distinctValues).containsInAnyOrder(KV.of(1, "k1"), KV.of(2, "k2"), KV.of(3, "k3"));
+    PAssert.that(distinctValues)
+        .containsInAnyOrder(KV.of(1L, "k1"), KV.of(2L, "k2"), KV.of(3L, "k3"));
     p.run();
   }
 
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DistinctTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DistinctTest.java
index cc59cea..124395d 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DistinctTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DistinctTest.java
@@ -32,7 +32,7 @@
 import java.util.Set;
 import org.apache.beam.sdk.coders.KvCoder;
 import org.apache.beam.sdk.coders.StringUtf8Coder;
-import org.apache.beam.sdk.coders.VarIntCoder;
+import org.apache.beam.sdk.coders.VarLongCoder;
 import org.apache.beam.sdk.testing.NeedsRunner;
 import org.apache.beam.sdk.testing.PAssert;
 import org.apache.beam.sdk.testing.TestPipeline;
@@ -217,25 +217,25 @@
   @Category({NeedsRunner.class, UsesTestStreamWithProcessingTime.class})
   public void testTriggeredDistinctRepresentativeValues() {
     Instant base = new Instant(0);
-    TestStream<KV<Integer, String>> values =
-        TestStream.create(KvCoder.of(VarIntCoder.of(), StringUtf8Coder.of()))
+    TestStream<KV<Long, String>> values =
+        TestStream.create(KvCoder.of(VarLongCoder.of(), StringUtf8Coder.of()))
             .advanceWatermarkTo(base)
             .addElements(
-                TimestampedValue.of(KV.of(1, "k1"), base),
-                TimestampedValue.of(KV.of(2, "k2"), base.plus(Duration.standardSeconds(10))),
-                TimestampedValue.of(KV.of(3, "k3"), base.plus(Duration.standardSeconds(20))))
+                TimestampedValue.of(KV.of(1L, "k1"), base),
+                TimestampedValue.of(KV.of(2L, "k2"), base.plus(Duration.standardSeconds(10))),
+                TimestampedValue.of(KV.of(3L, "k3"), base.plus(Duration.standardSeconds(20))))
             .advanceProcessingTime(Duration.standardMinutes(1))
             .addElements(
-                TimestampedValue.of(KV.of(1, "k1"), base.plus(Duration.standardSeconds(30))),
-                TimestampedValue.of(KV.of(2, "k2"), base.plus(Duration.standardSeconds(40))),
-                TimestampedValue.of(KV.of(3, "k3"), base.plus(Duration.standardSeconds(50))))
+                TimestampedValue.of(KV.of(1L, "k1"), base.plus(Duration.standardSeconds(30))),
+                TimestampedValue.of(KV.of(2L, "k2"), base.plus(Duration.standardSeconds(40))),
+                TimestampedValue.of(KV.of(3L, "k3"), base.plus(Duration.standardSeconds(50))))
             .advanceWatermarkToInfinity();
 
-    PCollection<KV<Integer, String>> distinctValues =
+    PCollection<KV<Long, String>> distinctValues =
         triggeredDistinctRepresentativePipeline
             .apply(values)
             .apply(
-                Window.<KV<Integer, String>>into(FixedWindows.of(Duration.standardMinutes(1)))
+                Window.<KV<Long, String>>into(FixedWindows.of(Duration.standardMinutes(1)))
                     .triggering(
                         Repeatedly.forever(
                             AfterProcessingTime.pastFirstElementInPane()
@@ -243,10 +243,11 @@
                     .withAllowedLateness(Duration.ZERO)
                     .accumulatingFiredPanes())
             .apply(
-                Distinct.withRepresentativeValueFn(new Keys<Integer>())
-                    .withRepresentativeType(TypeDescriptor.of(Integer.class)));
+                Distinct.withRepresentativeValueFn(new Keys<Long>())
+                    .withRepresentativeType(TypeDescriptor.of(Long.class)));
 
-    PAssert.that(distinctValues).containsInAnyOrder(KV.of(1, "k1"), KV.of(2, "k2"), KV.of(3, "k3"));
+    PAssert.that(distinctValues)
+        .containsInAnyOrder(KV.of(1L, "k1"), KV.of(2L, "k2"), KV.of(3L, "k3"));
     triggeredDistinctRepresentativePipeline.run();
   }
 
@@ -258,18 +259,18 @@
   @Category({NeedsRunner.class, UsesTestStreamWithProcessingTime.class})
   public void testTriggeredDistinctRepresentativeValuesEmpty() {
     Instant base = new Instant(0);
-    TestStream<KV<Integer, String>> values =
-        TestStream.create(KvCoder.of(VarIntCoder.of(), StringUtf8Coder.of()))
+    TestStream<KV<Long, String>> values =
+        TestStream.create(KvCoder.of(VarLongCoder.of(), StringUtf8Coder.of()))
             .advanceWatermarkTo(base)
-            .addElements(TimestampedValue.of(KV.of(1, "k1"), base))
+            .addElements(TimestampedValue.of(KV.of(1L, "k1"), base))
             .advanceProcessingTime(Duration.standardMinutes(1))
             .advanceWatermarkToInfinity();
 
-    PCollection<KV<Integer, String>> distinctValues =
+    PCollection<KV<Long, String>> distinctValues =
         triggeredDistinctRepresentativePipeline
             .apply(values)
             .apply(
-                Window.<KV<Integer, String>>into(FixedWindows.of(Duration.standardMinutes(1)))
+                Window.<KV<Long, String>>into(FixedWindows.of(Duration.standardMinutes(1)))
                     .triggering(
                         AfterWatermark.pastEndOfWindow()
                             .withEarlyFirings(
@@ -278,10 +279,10 @@
                     .withAllowedLateness(Duration.ZERO)
                     .discardingFiredPanes())
             .apply(
-                Distinct.withRepresentativeValueFn(new Keys<Integer>())
-                    .withRepresentativeType(TypeDescriptor.of(Integer.class)));
+                Distinct.withRepresentativeValueFn(new Keys<Long>())
+                    .withRepresentativeType(TypeDescriptor.of(Long.class)));
 
-    PAssert.that(distinctValues).containsInAnyOrder(KV.of(1, "k1"));
+    PAssert.that(distinctValues).containsInAnyOrder(KV.of(1L, "k1"));
     triggeredDistinctRepresentativePipeline.run();
   }
 
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 25a574a..e0ab582 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
@@ -4000,37 +4000,37 @@
     public void testSimpleProcessingTimerTimer() throws Exception {
       final String timerId = "foo";
 
-      DoFn<KV<String, Integer>, Integer> fn =
-          new DoFn<KV<String, Integer>, Integer>() {
+      DoFn<KV<String, Long>, Long> fn =
+          new DoFn<KV<String, Long>, Long>() {
 
             @TimerId(timerId)
             private final TimerSpec spec = TimerSpecs.timer(TimeDomain.PROCESSING_TIME);
 
             @ProcessElement
-            public void processElement(@TimerId(timerId) Timer timer, OutputReceiver<Integer> r) {
+            public void processElement(@TimerId(timerId) Timer timer, OutputReceiver<Long> r) {
               timer.offset(Duration.standardSeconds(1)).setRelative();
-              r.output(3);
+              r.output(3L);
             }
 
             @OnTimer(timerId)
-            public void onTimer(TimeDomain timeDomain, OutputReceiver<Integer> r) {
+            public void onTimer(TimeDomain timeDomain, OutputReceiver<Long> r) {
               if (timeDomain.equals(TimeDomain.PROCESSING_TIME)) {
-                r.output(42);
+                r.output(42L);
               }
             }
           };
 
-      TestStream<KV<String, Integer>> stream =
-          TestStream.create(KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of()))
-              .addElements(KV.of("hello", 37))
+      TestStream<KV<String, Long>> stream =
+          TestStream.create(KvCoder.of(StringUtf8Coder.of(), VarLongCoder.of()))
+              .addElements(KV.of("hello", 37L))
               .advanceProcessingTime(
                   Duration.millis(
                           DateTimeUtils.currentTimeMillis() / 1000 * 1000) // round to seconds
                       .plus(Duration.standardMinutes(2)))
               .advanceWatermarkToInfinity();
 
-      PCollection<Integer> output = pipeline.apply(stream).apply(ParDo.of(fn));
-      PAssert.that(output).containsInAnyOrder(3, 42);
+      PCollection<Long> output = pipeline.apply(stream).apply(ParDo.of(fn));
+      PAssert.that(output).containsInAnyOrder(3L, 42L);
       pipeline.run();
     }
 
@@ -4039,33 +4039,33 @@
     public void testEventTimeTimerUnbounded() throws Exception {
       final String timerId = "foo";
 
-      DoFn<KV<String, Integer>, Integer> fn =
-          new DoFn<KV<String, Integer>, Integer>() {
+      DoFn<KV<String, Long>, Long> fn =
+          new DoFn<KV<String, Long>, Long>() {
 
             @TimerId(timerId)
             private final TimerSpec spec = TimerSpecs.timer(TimeDomain.EVENT_TIME);
 
             @ProcessElement
-            public void processElement(@TimerId(timerId) Timer timer, OutputReceiver<Integer> r) {
+            public void processElement(@TimerId(timerId) Timer timer, OutputReceiver<Long> r) {
               timer.offset(Duration.standardSeconds(1)).setRelative();
-              r.output(3);
+              r.output(3L);
             }
 
             @OnTimer(timerId)
-            public void onTimer(OutputReceiver<Integer> r) {
-              r.output(42);
+            public void onTimer(OutputReceiver<Long> r) {
+              r.output(42L);
             }
           };
 
-      TestStream<KV<String, Integer>> stream =
-          TestStream.create(KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of()))
+      TestStream<KV<String, Long>> stream =
+          TestStream.create(KvCoder.of(StringUtf8Coder.of(), VarLongCoder.of()))
               .advanceWatermarkTo(new Instant(0))
-              .addElements(KV.of("hello", 37))
+              .addElements(KV.of("hello", 37L))
               .advanceWatermarkTo(new Instant(0).plus(Duration.standardSeconds(1)))
               .advanceWatermarkToInfinity();
 
-      PCollection<Integer> output = pipeline.apply(stream).apply(ParDo.of(fn));
-      PAssert.that(output).containsInAnyOrder(3, 42);
+      PCollection<Long> output = pipeline.apply(stream).apply(ParDo.of(fn));
+      PAssert.that(output).containsInAnyOrder(3L, 42L);
       pipeline.run();
     }
 
@@ -4074,8 +4074,8 @@
     public void testEventTimeTimerAlignUnbounded() throws Exception {
       final String timerId = "foo";
 
-      DoFn<KV<String, Integer>, KV<Integer, Instant>> fn =
-          new DoFn<KV<String, Integer>, KV<Integer, Instant>>() {
+      DoFn<KV<String, Long>, KV<Long, Instant>> fn =
+          new DoFn<KV<String, Long>, KV<Long, Instant>>() {
 
             @TimerId(timerId)
             private final TimerSpec spec = TimerSpecs.timer(TimeDomain.EVENT_TIME);
@@ -4084,34 +4084,33 @@
             public void processElement(
                 @TimerId(timerId) Timer timer,
                 @Timestamp Instant timestamp,
-                OutputReceiver<KV<Integer, Instant>> r) {
+                OutputReceiver<KV<Long, Instant>> r) {
               timer
                   .align(Duration.standardMinutes(1))
                   .offset(Duration.standardSeconds(1))
                   .setRelative();
-              r.output(KV.of(3, timestamp));
+              r.output(KV.of(3L, timestamp));
             }
 
             @OnTimer(timerId)
-            public void onTimer(
-                @Timestamp Instant timestamp, OutputReceiver<KV<Integer, Instant>> r) {
-              r.output(KV.of(42, timestamp));
+            public void onTimer(@Timestamp Instant timestamp, OutputReceiver<KV<Long, Instant>> r) {
+              r.output(KV.of(42L, timestamp));
             }
           };
 
-      TestStream<KV<String, Integer>> stream =
-          TestStream.create(KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of()))
+      TestStream<KV<String, Long>> stream =
+          TestStream.create(KvCoder.of(StringUtf8Coder.of(), VarLongCoder.of()))
               .advanceWatermarkTo(new Instant(0).plus(Duration.standardSeconds(5)))
-              .addElements(KV.of("hello", 37))
+              .addElements(KV.of("hello", 37L))
               .advanceWatermarkTo(new Instant(0).plus(Duration.standardMinutes(1)))
               .advanceWatermarkToInfinity();
 
-      PCollection<KV<Integer, Instant>> output = pipeline.apply(stream).apply(ParDo.of(fn));
+      PCollection<KV<Long, Instant>> output = pipeline.apply(stream).apply(ParDo.of(fn));
       PAssert.that(output)
           .containsInAnyOrder(
-              KV.of(3, new Instant(0).plus(Duration.standardSeconds(5))),
+              KV.of(3L, new Instant(0).plus(Duration.standardSeconds(5))),
               KV.of(
-                  42,
+                  42L,
                   new Instant(
                       Duration.standardMinutes(1).minus(Duration.standardSeconds(1)).getMillis())));
       pipeline.run();
@@ -4122,8 +4121,8 @@
     public void testEventTimeTimerAlignAfterGcTimeUnbounded() throws Exception {
       final String timerId = "foo";
 
-      DoFn<KV<String, Integer>, KV<Integer, Instant>> fn =
-          new DoFn<KV<String, Integer>, KV<Integer, Instant>>() {
+      DoFn<KV<String, Long>, KV<Long, Instant>> fn =
+          new DoFn<KV<String, Long>, KV<Long, Instant>>() {
 
             @TimerId(timerId)
             private final TimerSpec spec = TimerSpecs.timer(TimeDomain.EVENT_TIME);
@@ -4132,29 +4131,28 @@
             public void processElement(ProcessContext context, @TimerId(timerId) Timer timer) {
               // This aligned time will exceed the END_OF_GLOBAL_WINDOW
               timer.align(Duration.standardDays(1)).setRelative();
-              context.output(KV.of(3, context.timestamp()));
+              context.output(KV.of(3L, context.timestamp()));
             }
 
             @OnTimer(timerId)
-            public void onTimer(
-                @Timestamp Instant timestamp, OutputReceiver<KV<Integer, Instant>> r) {
-              r.output(KV.of(42, timestamp));
+            public void onTimer(@Timestamp Instant timestamp, OutputReceiver<KV<Long, Instant>> r) {
+              r.output(KV.of(42L, timestamp));
             }
           };
 
-      TestStream<KV<String, Integer>> stream =
-          TestStream.create(KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of()))
+      TestStream<KV<String, Long>> stream =
+          TestStream.create(KvCoder.of(StringUtf8Coder.of(), VarLongCoder.of()))
               // See GlobalWindow,
               // END_OF_GLOBAL_WINDOW is TIMESTAMP_MAX_VALUE.minus(Duration.standardDays(1))
               .advanceWatermarkTo(GlobalWindow.INSTANCE.maxTimestamp())
-              .addElements(KV.of("hello", 37))
+              .addElements(KV.of("hello", 37L))
               .advanceWatermarkToInfinity();
 
-      PCollection<KV<Integer, Instant>> output = pipeline.apply(stream).apply(ParDo.of(fn));
+      PCollection<KV<Long, Instant>> output = pipeline.apply(stream).apply(ParDo.of(fn));
       PAssert.that(output)
           .containsInAnyOrder(
-              KV.of(3, GlobalWindow.INSTANCE.maxTimestamp()),
-              KV.of(42, GlobalWindow.INSTANCE.maxTimestamp()));
+              KV.of(3L, GlobalWindow.INSTANCE.maxTimestamp()),
+              KV.of(42L, GlobalWindow.INSTANCE.maxTimestamp()));
       pipeline.run();
     }
 
@@ -4597,8 +4595,8 @@
     })
     public void testOutputTimestampWithProcessingTime() {
       final String timerId = "foo";
-      DoFn<KV<String, Integer>, KV<String, Integer>> fn1 =
-          new DoFn<KV<String, Integer>, KV<String, Integer>>() {
+      DoFn<KV<String, Long>, KV<String, Long>> fn1 =
+          new DoFn<KV<String, Long>, KV<String, Long>>() {
 
             @TimerId(timerId)
             private final TimerSpec timer = TimerSpecs.timer(TimeDomain.PROCESSING_TIME);
@@ -4607,21 +4605,21 @@
             public void processElement(
                 @TimerId(timerId) Timer timer,
                 @Timestamp Instant timestamp,
-                OutputReceiver<KV<String, Integer>> o) {
+                OutputReceiver<KV<String, Long>> o) {
               timer
                   .withOutputTimestamp(timestamp.plus(Duration.standardSeconds(5)))
                   .offset(Duration.standardSeconds(10))
                   .setRelative();
               // Output a message. This will cause the next DoFn to set a timer as well.
-              o.output(KV.of("foo", 100));
+              o.output(KV.of("foo", 100L));
             }
 
             @OnTimer(timerId)
             public void onTimer(OnTimerContext c, BoundedWindow w) {}
           };
 
-      DoFn<KV<String, Integer>, Integer> fn2 =
-          new DoFn<KV<String, Integer>, Integer>() {
+      DoFn<KV<String, Long>, Long> fn2 =
+          new DoFn<KV<String, Long>, Long>() {
 
             @TimerId(timerId)
             private final TimerSpec timer = TimerSpecs.timer(TimeDomain.EVENT_TIME);
@@ -4646,29 +4644,29 @@
             @OnTimer(timerId)
             public void onTimer(
                 @StateId("timerFired") ValueState<Boolean> timerFiredState,
-                OutputReceiver<Integer> o) {
+                OutputReceiver<Long> o) {
               timerFiredState.write(true);
-              o.output(100);
+              o.output(100L);
             }
           };
 
-      TestStream<KV<String, Integer>> stream =
-          TestStream.create(KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of()))
+      TestStream<KV<String, Long>> stream =
+          TestStream.create(KvCoder.of(StringUtf8Coder.of(), VarLongCoder.of()))
               .advanceProcessingTime(Duration.standardSeconds(1))
               // Cause fn2 to set a timer.
-              .addElements(KV.of("key", 1))
+              .addElements(KV.of("key", 1L))
               // Normally this would case fn2's timer to expire, but it shouldn't here because of
               // the output timestamp.
               .advanceProcessingTime(Duration.standardSeconds(9))
               .advanceWatermarkTo(new Instant(11))
               // If the timer fired, then this would case fn2 to fail with an assertion error.
-              .addElements(KV.of("key", 1))
+              .addElements(KV.of("key", 1L))
               .advanceProcessingTime(Duration.standardSeconds(100))
               .advanceWatermarkToInfinity();
-      PCollection<Integer> output =
+      PCollection<Long> output =
           pipeline.apply(stream).apply("first", ParDo.of(fn1)).apply("second", ParDo.of(fn2));
 
-      PAssert.that(output).containsInAnyOrder(100); // result output
+      PAssert.that(output).containsInAnyOrder(100L); // result output
       pipeline.run();
     }
 
@@ -4867,36 +4865,36 @@
 
       final String timerId = "processing-timer";
 
-      DoFn<KV<String, Integer>, Integer> fn =
-          new DoFn<KV<String, Integer>, Integer>() {
+      DoFn<KV<String, Long>, Long> fn =
+          new DoFn<KV<String, Long>, Long>() {
 
             @TimerId(timerId)
             private final TimerSpec spec = TimerSpecs.timer(TimeDomain.PROCESSING_TIME);
 
             @ProcessElement
-            public void processElement(@TimerId(timerId) Timer timer, OutputReceiver<Integer> r) {
+            public void processElement(@TimerId(timerId) Timer timer, OutputReceiver<Long> r) {
               timer.offset(Duration.standardSeconds(1)).setRelative();
               timer.clear();
-              r.output(3);
+              r.output(3L);
             }
 
             @OnTimer(timerId)
-            public void onTimer(TimeDomain timeDomain, OutputReceiver<Integer> r) {
-              r.output(42);
+            public void onTimer(TimeDomain timeDomain, OutputReceiver<Long> r) {
+              r.output(42L);
             }
           };
 
-      TestStream<KV<String, Integer>> stream =
-          TestStream.create(KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of()))
-              .addElements(KV.of("hello", 37))
+      TestStream<KV<String, Long>> stream =
+          TestStream.create(KvCoder.of(StringUtf8Coder.of(), VarLongCoder.of()))
+              .addElements(KV.of("hello", 37L))
               .advanceProcessingTime(
                   Duration.millis(
                           DateTimeUtils.currentTimeMillis() / 1000 * 1000) // round to seconds
                       .plus(Duration.standardMinutes(2)))
               .advanceWatermarkToInfinity();
 
-      PCollection<Integer> output = pipeline.apply(stream).apply(ParDo.of(fn));
-      PAssert.that(output).containsInAnyOrder(3);
+      PCollection<Long> output = pipeline.apply(stream).apply(ParDo.of(fn));
+      PAssert.that(output).containsInAnyOrder(3L);
       pipeline.run();
     }
 
@@ -4905,34 +4903,34 @@
     public void testSetAndClearEventTimeTimer() {
       final String timerId = "event-timer";
 
-      DoFn<KV<String, Integer>, Integer> fn =
-          new DoFn<KV<String, Integer>, Integer>() {
+      DoFn<KV<String, Long>, Long> fn =
+          new DoFn<KV<String, Long>, Long>() {
 
             @TimerId(timerId)
             private final TimerSpec spec = TimerSpecs.timer(TimeDomain.EVENT_TIME);
 
             @ProcessElement
-            public void processElement(@TimerId(timerId) Timer timer, OutputReceiver<Integer> r) {
+            public void processElement(@TimerId(timerId) Timer timer, OutputReceiver<Long> r) {
               timer.offset(Duration.standardSeconds(1)).setRelative();
               timer.clear();
-              r.output(3);
+              r.output(3L);
             }
 
             @OnTimer(timerId)
-            public void onTimer(OutputReceiver<Integer> r) {
-              r.output(42);
+            public void onTimer(OutputReceiver<Long> r) {
+              r.output(42L);
             }
           };
 
-      TestStream<KV<String, Integer>> stream =
-          TestStream.create(KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of()))
+      TestStream<KV<String, Long>> stream =
+          TestStream.create(KvCoder.of(StringUtf8Coder.of(), VarLongCoder.of()))
               .advanceWatermarkTo(new Instant(0))
-              .addElements(KV.of("hello", 37))
+              .addElements(KV.of("hello", 37L))
               .advanceWatermarkTo(new Instant(0).plus(Duration.standardSeconds(1)))
               .advanceWatermarkToInfinity();
 
-      PCollection<Integer> output = pipeline.apply(stream).apply(ParDo.of(fn));
-      PAssert.that(output).containsInAnyOrder(3);
+      PCollection<Long> output = pipeline.apply(stream).apply(ParDo.of(fn));
+      PAssert.that(output).containsInAnyOrder(3L);
       pipeline.run();
     }
 
@@ -4946,35 +4944,35 @@
     public void testClearUnsetProcessingTimeTimer() {
       final String timerId = "processing-timer";
 
-      DoFn<KV<String, Integer>, Integer> fn =
-          new DoFn<KV<String, Integer>, Integer>() {
+      DoFn<KV<String, Long>, Long> fn =
+          new DoFn<KV<String, Long>, Long>() {
 
             @TimerId(timerId)
             private final TimerSpec spec = TimerSpecs.timer(TimeDomain.PROCESSING_TIME);
 
             @ProcessElement
-            public void processElement(@TimerId(timerId) Timer timer, OutputReceiver<Integer> r) {
+            public void processElement(@TimerId(timerId) Timer timer, OutputReceiver<Long> r) {
               timer.clear();
-              r.output(3);
+              r.output(3L);
             }
 
             @OnTimer(timerId)
-            public void onTimer(TimeDomain timeDomain, OutputReceiver<Integer> r) {
-              r.output(42);
+            public void onTimer(TimeDomain timeDomain, OutputReceiver<Long> r) {
+              r.output(42L);
             }
           };
 
-      TestStream<KV<String, Integer>> stream =
-          TestStream.create(KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of()))
-              .addElements(KV.of("hello", 37))
+      TestStream<KV<String, Long>> stream =
+          TestStream.create(KvCoder.of(StringUtf8Coder.of(), VarLongCoder.of()))
+              .addElements(KV.of("hello", 37L))
               .advanceProcessingTime(
                   Duration.millis(
                           DateTimeUtils.currentTimeMillis() / 1000 * 1000) // round to seconds
                       .plus(Duration.standardMinutes(4)))
               .advanceWatermarkToInfinity();
 
-      PCollection<Integer> output = pipeline.apply(stream).apply(ParDo.of(fn));
-      PAssert.that(output).containsInAnyOrder(3);
+      PCollection<Long> output = pipeline.apply(stream).apply(ParDo.of(fn));
+      PAssert.that(output).containsInAnyOrder(3L);
       pipeline.run();
     }
 
@@ -4983,33 +4981,33 @@
     public void testClearUnsetEventTimeTimer() {
       final String timerId = "event-timer";
 
-      DoFn<KV<String, Integer>, Integer> fn =
-          new DoFn<KV<String, Integer>, Integer>() {
+      DoFn<KV<String, Long>, Long> fn =
+          new DoFn<KV<String, Long>, Long>() {
 
             @TimerId(timerId)
             private final TimerSpec spec = TimerSpecs.timer(TimeDomain.EVENT_TIME);
 
             @ProcessElement
-            public void processElement(@TimerId(timerId) Timer timer, OutputReceiver<Integer> r) {
+            public void processElement(@TimerId(timerId) Timer timer, OutputReceiver<Long> r) {
               timer.clear();
-              r.output(3);
+              r.output(3L);
             }
 
             @OnTimer(timerId)
-            public void onTimer(OutputReceiver<Integer> r) {
-              r.output(42);
+            public void onTimer(OutputReceiver<Long> r) {
+              r.output(42L);
             }
           };
 
-      TestStream<KV<String, Integer>> stream =
-          TestStream.create(KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of()))
+      TestStream<KV<String, Long>> stream =
+          TestStream.create(KvCoder.of(StringUtf8Coder.of(), VarLongCoder.of()))
               .advanceWatermarkTo(new Instant(0))
-              .addElements(KV.of("hello", 37))
+              .addElements(KV.of("hello", 37L))
               .advanceWatermarkTo(new Instant(0).plus(Duration.standardSeconds(1)))
               .advanceWatermarkToInfinity();
 
-      PCollection<Integer> output = pipeline.apply(stream).apply(ParDo.of(fn));
-      PAssert.that(output).containsInAnyOrder(3);
+      PCollection<Long> output = pipeline.apply(stream).apply(ParDo.of(fn));
+      PAssert.that(output).containsInAnyOrder(3L);
       pipeline.run();
     }
 
@@ -5024,8 +5022,8 @@
       final String timerId = "processing-timer";
       final String clearTimerId = "clear-timer";
 
-      DoFn<KV<String, Integer>, Integer> fn =
-          new DoFn<KV<String, Integer>, Integer>() {
+      DoFn<KV<String, Long>, Long> fn =
+          new DoFn<KV<String, Long>, Long>() {
 
             @TimerId(timerId)
             private final TimerSpec spec = TimerSpecs.timer(TimeDomain.PROCESSING_TIME);
@@ -5037,30 +5035,29 @@
             public void processElement(
                 @TimerId(timerId) Timer timer,
                 @TimerId(clearTimerId) Timer clearTimer,
-                OutputReceiver<Integer> r) {
+                OutputReceiver<Long> r) {
               timer.offset(Duration.standardSeconds(1)).setRelative();
               clearTimer.offset(Duration.standardSeconds(2)).setRelative();
 
-              r.output(3);
+              r.output(3L);
             }
 
             @OnTimer(timerId)
-            public void onTimer(
-                OutputReceiver<Integer> r, @TimerId(clearTimerId) Timer clearTimer) {
-              r.output(42);
+            public void onTimer(OutputReceiver<Long> r, @TimerId(clearTimerId) Timer clearTimer) {
+              r.output(42L);
               clearTimer.clear();
             }
 
             // This should never fire since we clear the timer in the earlier timer.
             @OnTimer(clearTimerId)
-            public void clearTimer(OutputReceiver<Integer> r) {
-              r.output(43);
+            public void clearTimer(OutputReceiver<Long> r) {
+              r.output(43L);
             }
           };
 
-      TestStream<KV<String, Integer>> stream =
-          TestStream.create(KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of()))
-              .addElements(KV.of("hello", 37))
+      TestStream<KV<String, Long>> stream =
+          TestStream.create(KvCoder.of(StringUtf8Coder.of(), VarLongCoder.of()))
+              .addElements(KV.of("hello", 37L))
               .advanceProcessingTime(
                   Duration.millis(
                           DateTimeUtils.currentTimeMillis() / 1000 * 1000) // round to seconds
@@ -5071,8 +5068,8 @@
                       .plus(Duration.standardMinutes(4)))
               .advanceWatermarkToInfinity();
 
-      PCollection<Integer> output = pipeline.apply(stream).apply(ParDo.of(fn));
-      PAssert.that(output).containsInAnyOrder(3, 42);
+      PCollection<Long> output = pipeline.apply(stream).apply(ParDo.of(fn));
+      PAssert.that(output).containsInAnyOrder(3L, 42L);
       pipeline.run();
     }
 
@@ -5082,8 +5079,8 @@
       final String timerId = "event-timer";
       final String clearTimerId = "clear-timer";
 
-      DoFn<KV<String, Integer>, Integer> fn =
-          new DoFn<KV<String, Integer>, Integer>() {
+      DoFn<KV<String, Long>, Long> fn =
+          new DoFn<KV<String, Long>, Long>() {
 
             @TimerId(timerId)
             private final TimerSpec spec = TimerSpecs.timer(TimeDomain.EVENT_TIME);
@@ -5095,36 +5092,35 @@
             public void processElement(
                 @TimerId(timerId) Timer timer,
                 @TimerId(clearTimerId) Timer clearTimer,
-                OutputReceiver<Integer> r) {
+                OutputReceiver<Long> r) {
               timer.offset(Duration.standardSeconds(1)).setRelative();
               clearTimer.offset(Duration.standardSeconds(2)).setRelative();
 
-              r.output(3);
+              r.output(3L);
             }
 
             @OnTimer(timerId)
-            public void onTimer(
-                OutputReceiver<Integer> r, @TimerId(clearTimerId) Timer clearTimer) {
-              r.output(42);
+            public void onTimer(OutputReceiver<Long> r, @TimerId(clearTimerId) Timer clearTimer) {
+              r.output(42L);
               clearTimer.clear();
             }
 
             // This should never fire since we clear the timer in the earlier timer.
             @OnTimer(clearTimerId)
-            public void clearTimer(OutputReceiver<Integer> r) {
-              r.output(43);
+            public void clearTimer(OutputReceiver<Long> r) {
+              r.output(43L);
             }
           };
 
-      TestStream<KV<String, Integer>> stream =
-          TestStream.create(KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of()))
+      TestStream<KV<String, Long>> stream =
+          TestStream.create(KvCoder.of(StringUtf8Coder.of(), VarLongCoder.of()))
               .advanceWatermarkTo(new Instant(0))
-              .addElements(KV.of("hello", 37))
+              .addElements(KV.of("hello", 37L))
               .advanceWatermarkTo(new Instant(0).plus(Duration.standardSeconds(1)))
               .advanceWatermarkToInfinity();
 
-      PCollection<Integer> output = pipeline.apply(stream).apply(ParDo.of(fn));
-      PAssert.that(output).containsInAnyOrder(3, 42);
+      PCollection<Long> output = pipeline.apply(stream).apply(ParDo.of(fn));
+      PAssert.that(output).containsInAnyOrder(3L, 42L);
       pipeline.run();
     }
 
@@ -5138,39 +5134,39 @@
     public void testSetProcessingTimerAfterClear() {
       final String timerId = "processing-timer";
 
-      DoFn<KV<String, Integer>, Integer> fn =
-          new DoFn<KV<String, Integer>, Integer>() {
+      DoFn<KV<String, Long>, Long> fn =
+          new DoFn<KV<String, Long>, Long>() {
 
             @TimerId(timerId)
             private final TimerSpec spec = TimerSpecs.timer(TimeDomain.PROCESSING_TIME);
 
             @ProcessElement
             public void processElement(
-                @Element KV<String, Integer> e,
+                @Element KV<String, Long> e,
                 @TimerId(timerId) Timer timer,
-                OutputReceiver<Integer> r) {
+                OutputReceiver<Long> r) {
               timer.clear();
               timer.offset(Duration.standardSeconds(1)).setRelative();
-              r.output(3);
+              r.output(3L);
             }
 
             @OnTimer(timerId)
-            public void onTimer(TimeDomain timeDomain, OutputReceiver<Integer> r) {
-              r.output(42);
+            public void onTimer(TimeDomain timeDomain, OutputReceiver<Long> r) {
+              r.output(42L);
             }
           };
 
-      TestStream<KV<String, Integer>> stream =
-          TestStream.create(KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of()))
-              .addElements(KV.of("hello", 37), KV.of("hello", 38))
+      TestStream<KV<String, Long>> stream =
+          TestStream.create(KvCoder.of(StringUtf8Coder.of(), VarLongCoder.of()))
+              .addElements(KV.of("hello", 37L), KV.of("hello", 38L))
               .advanceProcessingTime(
                   Duration.millis(
                           DateTimeUtils.currentTimeMillis() / 1000 * 1000) // round to seconds
                       .plus(Duration.standardMinutes(2)))
               .advanceWatermarkToInfinity();
 
-      PCollection<Integer> output = pipeline.apply(stream).apply(ParDo.of(fn));
-      PAssert.that(output).containsInAnyOrder(3, 3, 42);
+      PCollection<Long> output = pipeline.apply(stream).apply(ParDo.of(fn));
+      PAssert.that(output).containsInAnyOrder(3L, 3L, 42L);
       pipeline.run();
     }
 
@@ -5179,34 +5175,34 @@
     public void testSetEventTimerAfterClear() {
       final String timerId = "event-timer";
 
-      DoFn<KV<String, Integer>, Integer> fn =
-          new DoFn<KV<String, Integer>, Integer>() {
+      DoFn<KV<String, Long>, Long> fn =
+          new DoFn<KV<String, Long>, Long>() {
 
             @TimerId(timerId)
             private final TimerSpec spec = TimerSpecs.timer(TimeDomain.EVENT_TIME);
 
             @ProcessElement
-            public void processElement(@TimerId(timerId) Timer timer, OutputReceiver<Integer> r) {
+            public void processElement(@TimerId(timerId) Timer timer, OutputReceiver<Long> r) {
               timer.clear();
               timer.offset(Duration.standardSeconds(1)).setRelative();
-              r.output(3);
+              r.output(3L);
             }
 
             @OnTimer(timerId)
-            public void onTimer(OutputReceiver<Integer> r) {
-              r.output(42);
+            public void onTimer(OutputReceiver<Long> r) {
+              r.output(42L);
             }
           };
 
-      TestStream<KV<String, Integer>> stream =
-          TestStream.create(KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of()))
+      TestStream<KV<String, Long>> stream =
+          TestStream.create(KvCoder.of(StringUtf8Coder.of(), VarLongCoder.of()))
               .advanceWatermarkTo(new Instant(0))
-              .addElements(KV.of("hello", 37), KV.of("hello", 38))
+              .addElements(KV.of("hello", 37L), KV.of("hello", 38L))
               .advanceWatermarkTo(new Instant(0).plus(Duration.standardSeconds(1)))
               .advanceWatermarkToInfinity();
 
-      PCollection<Integer> output = pipeline.apply(stream).apply(ParDo.of(fn));
-      PAssert.that(output).containsInAnyOrder(3, 3, 42);
+      PCollection<Long> output = pipeline.apply(stream).apply(ParDo.of(fn));
+      PAssert.that(output).containsInAnyOrder(3L, 3L, 42L);
       pipeline.run();
     }
   }
@@ -5775,36 +5771,36 @@
     public void testTimerFamilyProcessingTime() throws Exception {
       final String timerId = "foo";
 
-      DoFn<KV<String, Integer>, Integer> fn =
-          new DoFn<KV<String, Integer>, Integer>() {
+      DoFn<KV<String, Long>, Long> fn =
+          new DoFn<KV<String, Long>, Long>() {
 
             @TimerFamily(timerId)
             private final TimerSpec spec = TimerSpecs.timerMap(TimeDomain.PROCESSING_TIME);
 
             @ProcessElement
             public void processElement(
-                @TimerFamily(timerId) TimerMap timerMap, OutputReceiver<Integer> r) {
+                @TimerFamily(timerId) TimerMap timerMap, OutputReceiver<Long> r) {
               Timer timer = timerMap.get("timerId1");
               timer.offset(Duration.standardSeconds(1)).setRelative();
-              r.output(3);
+              r.output(3L);
             }
 
             @OnTimerFamily(timerId)
-            public void onTimer(TimeDomain timeDomain, OutputReceiver<Integer> r) {
+            public void onTimer(TimeDomain timeDomain, OutputReceiver<Long> r) {
               if (timeDomain.equals(TimeDomain.PROCESSING_TIME)) {
-                r.output(42);
+                r.output(42L);
               }
             }
           };
 
-      TestStream<KV<String, Integer>> stream =
-          TestStream.create(KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of()))
-              .addElements(KV.of("hello", 37))
+      TestStream<KV<String, Long>> stream =
+          TestStream.create(KvCoder.of(StringUtf8Coder.of(), VarLongCoder.of()))
+              .addElements(KV.of("hello", 37L))
               .advanceProcessingTime(Duration.standardSeconds(2))
               .advanceWatermarkToInfinity();
 
-      PCollection<Integer> output = pipeline.apply(stream).apply(ParDo.of(fn));
-      PAssert.that(output).containsInAnyOrder(3, 42);
+      PCollection<Long> output = pipeline.apply(stream).apply(ParDo.of(fn));
+      PAssert.that(output).containsInAnyOrder(3L, 42L);
       pipeline.run();
     }
   }
diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ViewTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ViewTest.java
index 76b38d0..2dcce79 100644
--- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ViewTest.java
+++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ViewTest.java
@@ -44,6 +44,7 @@
 import org.apache.beam.sdk.coders.NullableCoder;
 import org.apache.beam.sdk.coders.StringUtf8Coder;
 import org.apache.beam.sdk.coders.VarIntCoder;
+import org.apache.beam.sdk.coders.VarLongCoder;
 import org.apache.beam.sdk.coders.VoidCoder;
 import org.apache.beam.sdk.testing.NeedsRunner;
 import org.apache.beam.sdk.testing.PAssert;
@@ -156,30 +157,30 @@
   @Test
   @Category({ValidatesRunner.class, UsesTestStream.class})
   public void testWindowedSideInputNotPresent() {
-    PCollection<KV<Integer, Integer>> input =
+    PCollection<KV<Long, Long>> input =
         pipeline.apply(
-            TestStream.create(KvCoder.of(VarIntCoder.of(), VarIntCoder.of()))
+            TestStream.create(KvCoder.of(VarLongCoder.of(), VarLongCoder.of()))
                 .advanceWatermarkTo(new Instant(0))
-                .addElements(TimestampedValue.of(KV.of(1000, 1000), new Instant(1000)))
+                .addElements(TimestampedValue.of(KV.of(1000L, 1000L), new Instant(1000L)))
                 .advanceWatermarkTo(new Instant(20000))
                 .advanceWatermarkToInfinity());
 
-    final PCollectionView<Integer> view =
+    final PCollectionView<Long> view =
         input
             .apply(Values.create())
             .apply("SideWindowInto", Window.into(FixedWindows.of(Duration.standardSeconds(100))))
-            .apply("ViewCombine", Combine.globally(Sum.ofIntegers()).withoutDefaults())
+            .apply("ViewCombine", Combine.globally(Sum.ofLongs()).withoutDefaults())
             .apply("Rewindow", Window.into(FixedWindows.of(Duration.standardSeconds(10))))
-            .apply(View.<Integer>asSingleton().withDefaultValue(0));
+            .apply(View.<Long>asSingleton().withDefaultValue(0L));
 
-    PCollection<Integer> output =
+    PCollection<Long> output =
         input
             .apply("MainWindowInto", Window.into(FixedWindows.of(Duration.standardSeconds(10))))
             .apply(GroupByKey.create())
             .apply(
                 "OutputSideInputs",
                 ParDo.of(
-                        new DoFn<KV<Integer, Iterable<Integer>>, Integer>() {
+                        new DoFn<KV<Long, Iterable<Long>>, Long>() {
                           @ProcessElement
                           public void processElement(ProcessContext c) {
                             c.output(c.sideInput(view));
@@ -189,7 +190,7 @@
 
     PAssert.that(output)
         .inWindow(new IntervalWindow(new Instant(0), new Instant(10000)))
-        .containsInAnyOrder(0);
+        .containsInAnyOrder(0L);
 
     pipeline.run();
   }
diff --git a/sdks/java/expansion-service/build.gradle b/sdks/java/expansion-service/build.gradle
index a626303..2a0ffd0 100644
--- a/sdks/java/expansion-service/build.gradle
+++ b/sdks/java/expansion-service/build.gradle
@@ -38,6 +38,9 @@
   compile project(path: ":sdks:java:core", configuration: "shadow")
   compile project(path: ":runners:core-construction-java")
   compile project(path: ":runners:java-fn-execution")
+  compile library.java.jackson_annotations
+  compile library.java.jackson_databind
+  compile library.java.jackson_dataformat_yaml
   compile library.java.vendored_grpc_1_36_0
   compile library.java.vendored_guava_26_0_jre
   compile library.java.slf4j_api
diff --git a/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionService.java b/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionService.java
index eaa1cbe..6e1f3d3 100644
--- a/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionService.java
+++ b/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionService.java
@@ -17,6 +17,7 @@
  */
 package org.apache.beam.sdk.expansion.service;
 
+import static org.apache.beam.runners.core.construction.BeamUrns.getUrn;
 import static org.apache.beam.runners.core.construction.resources.PipelineResources.detectClassPathResourcesToStage;
 import static org.apache.beam.sdk.util.Preconditions.checkArgumentNotNull;
 
@@ -35,8 +36,10 @@
 import java.util.stream.Collectors;
 import org.apache.beam.model.expansion.v1.ExpansionApi;
 import org.apache.beam.model.expansion.v1.ExpansionServiceGrpc;
+import org.apache.beam.model.pipeline.v1.ExternalTransforms.ExpansionMethods;
 import org.apache.beam.model.pipeline.v1.ExternalTransforms.ExternalConfigurationPayload;
 import org.apache.beam.model.pipeline.v1.RunnerApi;
+import org.apache.beam.model.pipeline.v1.SchemaApi;
 import org.apache.beam.runners.core.construction.Environments;
 import org.apache.beam.runners.core.construction.PipelineTranslation;
 import org.apache.beam.runners.core.construction.RehydratedComponents;
@@ -49,6 +52,7 @@
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.sdk.coders.RowCoder;
 import org.apache.beam.sdk.expansion.ExternalTransformRegistrar;
+import org.apache.beam.sdk.expansion.service.JavaClassLookupTransformProvider.AllowList;
 import org.apache.beam.sdk.options.ExperimentalOptions;
 import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.options.PipelineOptionsFactory;
@@ -70,6 +74,7 @@
 import org.apache.beam.sdk.values.POutput;
 import org.apache.beam.sdk.values.Row;
 import org.apache.beam.sdk.values.TupleTag;
+import org.apache.beam.vendor.grpc.v1p36p0.com.google.protobuf.ByteString;
 import org.apache.beam.vendor.grpc.v1p36p0.io.grpc.Server;
 import org.apache.beam.vendor.grpc.v1p36p0.io.grpc.ServerBuilder;
 import org.apache.beam.vendor.grpc.v1p36p0.io.grpc.stub.StreamObserver;
@@ -172,8 +177,8 @@
       return configurationClass;
     }
 
-    private static <ConfigT> Row decodeRow(ExternalConfigurationPayload payload) {
-      Schema payloadSchema = SchemaTranslation.schemaFromProto(payload.getSchema());
+    static <ConfigT> Row decodeConfigObjectRow(SchemaApi.Schema schema, ByteString payload) {
+      Schema payloadSchema = SchemaTranslation.schemaFromProto(schema);
 
       if (payloadSchema.getFieldCount() == 0) {
         return Row.withSchema(Schema.of()).build();
@@ -200,7 +205,7 @@
 
       Row configRow;
       try {
-        configRow = RowCoder.of(payloadSchema).decode(payload.getPayload().newInput());
+        configRow = RowCoder.of(payloadSchema).decode(payload.newInput());
       } catch (IOException e) {
         throw new RuntimeException("Error decoding payload", e);
       }
@@ -247,7 +252,7 @@
       SerializableFunction<Row, ConfigT> fromRowFunc =
           SCHEMA_REGISTRY.getFromRowFunction(configurationClass);
 
-      Row payloadRow = decodeRow(payload);
+      Row payloadRow = decodeConfigObjectRow(payload.getSchema(), payload.getPayload());
 
       if (!payloadRow.getSchema().assignableTo(configSchema)) {
         throw new IllegalArgumentException(
@@ -263,7 +268,7 @@
     private static <ConfigT> ConfigT payloadToConfigSetters(
         ExternalConfigurationPayload payload, Class<ConfigT> configurationClass)
         throws ReflectiveOperationException {
-      Row configRow = decodeRow(payload);
+      Row configRow = decodeConfigObjectRow(payload.getSchema(), payload.getPayload());
 
       Constructor<ConfigT> constructor = configurationClass.getDeclaredConstructor();
       constructor.setAccessible(true);
@@ -459,13 +464,22 @@
                       }
                     }));
 
-    @Nullable
-    TransformProvider transformProvider =
-        getRegisteredTransforms().get(request.getTransform().getSpec().getUrn());
-    if (transformProvider == null) {
-      throw new UnsupportedOperationException(
-          "Unknown urn: " + request.getTransform().getSpec().getUrn());
+    String urn = request.getTransform().getSpec().getUrn();
+
+    TransformProvider transformProvider = null;
+    if (getUrn(ExpansionMethods.Enum.JAVA_CLASS_LOOKUP).equals(urn)) {
+      AllowList allowList =
+          pipelineOptions.as(ExpansionServiceOptions.class).getJavaClassLookupAllowlist();
+      assert allowList != null;
+      transformProvider = new JavaClassLookupTransformProvider(allowList);
+    } else {
+      transformProvider = getRegisteredTransforms().get(urn);
+      if (transformProvider == null) {
+        throw new UnsupportedOperationException(
+            "Unknown urn: " + request.getTransform().getSpec().getUrn());
+      }
     }
+
     Map<String, PCollection<?>> outputs =
         transformProvider.apply(
             pipeline,
diff --git a/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionServiceOptions.java b/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionServiceOptions.java
new file mode 100644
index 0000000..79e870c
--- /dev/null
+++ b/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/ExpansionServiceOptions.java
@@ -0,0 +1,75 @@
+/*
+ * 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.expansion.service;
+
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.dataformat.yaml.YAMLFactory;
+import java.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import org.apache.beam.sdk.expansion.service.JavaClassLookupTransformProvider.AllowList;
+import org.apache.beam.sdk.options.Default;
+import org.apache.beam.sdk.options.DefaultValueFactory;
+import org.apache.beam.sdk.options.Description;
+import org.apache.beam.sdk.options.PipelineOptions;
+
+/** Options used to configure the {@link ExpansionService}. */
+public interface ExpansionServiceOptions extends PipelineOptions {
+
+  @Description("Allow list for Java class based transform expansion")
+  @Default.InstanceFactory(JavaClassLookupAllowListFactory.class)
+  AllowList getJavaClassLookupAllowlist();
+
+  void setJavaClassLookupAllowlist(AllowList file);
+
+  @Description("Allow list file for Java class based transform expansion")
+  String getJavaClassLookupAllowlistFile();
+
+  void setJavaClassLookupAllowlistFile(String file);
+
+  /**
+   * Loads the allow list from {@link #getJavaClassLookupAllowlistFile}, defaulting to an empty
+   * {@link JavaClassLookupTransformProvider.AllowList}.
+   */
+  class JavaClassLookupAllowListFactory implements DefaultValueFactory<AllowList> {
+
+    @Override
+    public AllowList create(PipelineOptions options) {
+      String allowListFile =
+          options.as(ExpansionServiceOptions.class).getJavaClassLookupAllowlistFile();
+      if (allowListFile != null) {
+        ObjectMapper mapper = new ObjectMapper(new YAMLFactory());
+        File allowListFileObj = new File(allowListFile);
+        if (!allowListFileObj.exists()) {
+          throw new IllegalArgumentException(
+              "Allow list file " + allowListFile + " does not exist");
+        }
+        try {
+          return mapper.readValue(allowListFileObj, AllowList.class);
+        } catch (IOException e) {
+          throw new IllegalArgumentException(
+              "Could not load the provided allowlist file " + allowListFile, e);
+        }
+      }
+
+      // By default produces an empty allow-list.
+      return new AutoValue_JavaClassLookupTransformProvider_AllowList(
+          JavaClassLookupTransformProvider.ALLOW_LIST_VERSION, new ArrayList<>());
+    }
+  }
+}
diff --git a/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/JavaClassLookupTransformProvider.java b/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/JavaClassLookupTransformProvider.java
new file mode 100644
index 0000000..d32c7e4
--- /dev/null
+++ b/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/JavaClassLookupTransformProvider.java
@@ -0,0 +1,526 @@
+/*
+ * 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.expansion.service;
+
+import static org.apache.beam.runners.core.construction.BeamUrns.getUrn;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.auto.value.AutoValue;
+import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
+import java.io.IOException;
+import java.lang.annotation.Annotation;
+import java.lang.reflect.Array;
+import java.lang.reflect.Constructor;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.lang.reflect.ParameterizedType;
+import java.lang.reflect.Type;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.stream.Collectors;
+import org.apache.beam.model.pipeline.v1.ExternalTransforms.BuilderMethod;
+import org.apache.beam.model.pipeline.v1.ExternalTransforms.ExpansionMethods;
+import org.apache.beam.model.pipeline.v1.ExternalTransforms.JavaClassLookupPayload;
+import org.apache.beam.model.pipeline.v1.RunnerApi.FunctionSpec;
+import org.apache.beam.model.pipeline.v1.SchemaApi;
+import org.apache.beam.repackaged.core.org.apache.commons.lang3.ClassUtils;
+import org.apache.beam.sdk.coders.RowCoder;
+import org.apache.beam.sdk.expansion.service.ExpansionService.TransformProvider;
+import org.apache.beam.sdk.schemas.JavaFieldSchema;
+import org.apache.beam.sdk.schemas.NoSuchSchemaException;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.schemas.Schema.Field;
+import org.apache.beam.sdk.schemas.Schema.TypeName;
+import org.apache.beam.sdk.schemas.SchemaRegistry;
+import org.apache.beam.sdk.schemas.SchemaTranslation;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.SerializableFunction;
+import org.apache.beam.sdk.util.common.ReflectHelpers;
+import org.apache.beam.sdk.values.PInput;
+import org.apache.beam.sdk.values.POutput;
+import org.apache.beam.sdk.values.Row;
+import org.apache.beam.vendor.grpc.v1p36p0.com.google.protobuf.ByteString;
+import org.apache.beam.vendor.grpc.v1p36p0.com.google.protobuf.InvalidProtocolBufferException;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+/**
+ * A transform provider that can be used to directly instantiate a transform using Java class name
+ * and builder methods.
+ *
+ * @param <InputT> input {@link PInput} type of the transform
+ * @param <OutputT> output {@link POutput} type of the transform
+ */
+@SuppressWarnings({"argument.type.incompatible", "assignment.type.incompatible"})
+@SuppressFBWarnings("UWF_UNWRITTEN_PUBLIC_OR_PROTECTED_FIELD")
+class JavaClassLookupTransformProvider<InputT extends PInput, OutputT extends POutput>
+    implements TransformProvider<PInput, POutput> {
+
+  public static final String ALLOW_LIST_VERSION = "v1";
+  private static final SchemaRegistry SCHEMA_REGISTRY = SchemaRegistry.createDefault();
+  private final AllowList allowList;
+
+  public JavaClassLookupTransformProvider(AllowList allowList) {
+    if (!allowList.getVersion().equals(ALLOW_LIST_VERSION)) {
+      throw new IllegalArgumentException("Unknown allow-list version");
+    }
+    this.allowList = allowList;
+  }
+
+  @Override
+  public PTransform<PInput, POutput> getTransform(FunctionSpec spec) {
+    JavaClassLookupPayload payload;
+    try {
+      payload = JavaClassLookupPayload.parseFrom(spec.getPayload());
+    } catch (InvalidProtocolBufferException e) {
+      throw new IllegalArgumentException(
+          "Invalid payload type for URN " + getUrn(ExpansionMethods.Enum.JAVA_CLASS_LOOKUP), e);
+    }
+
+    String className = payload.getClassName();
+    try {
+      AllowedClass allowlistClass = null;
+      if (this.allowList != null) {
+        for (AllowedClass cls : this.allowList.getAllowedClasses()) {
+          if (cls.getClassName().equals(className)) {
+            if (allowlistClass != null) {
+              throw new IllegalArgumentException(
+                  "Found two matching allowlist classes " + allowlistClass + " and " + cls);
+            }
+            allowlistClass = cls;
+          }
+        }
+      }
+      if (allowlistClass == null) {
+        throw new UnsupportedOperationException(
+            "The provided allow list does not enable expanding a transform class by the name "
+                + className
+                + ".");
+      }
+      Class<PTransform<InputT, OutputT>> transformClass =
+          (Class<PTransform<InputT, OutputT>>)
+              ReflectHelpers.findClassLoader().loadClass(className);
+      PTransform<PInput, POutput> transform;
+      Row constructorRow =
+          decodeRow(payload.getConstructorSchema(), payload.getConstructorPayload());
+      if (payload.getConstructorMethod().isEmpty()) {
+        Constructor<?>[] constructors = transformClass.getConstructors();
+        Constructor<PTransform<InputT, OutputT>> constructor =
+            findMappingConstructor(constructors, payload);
+        Object[] parameterValues =
+            getParameterValues(
+                constructor.getParameters(),
+                constructorRow,
+                constructor.getGenericParameterTypes());
+        transform = (PTransform<PInput, POutput>) constructor.newInstance(parameterValues);
+      } else {
+        Method[] methods = transformClass.getMethods();
+        Method method = findMappingConstructorMethod(methods, payload, allowlistClass);
+        Object[] parameterValues =
+            getParameterValues(
+                method.getParameters(), constructorRow, method.getGenericParameterTypes());
+        transform = (PTransform<PInput, POutput>) method.invoke(null /* static */, parameterValues);
+      }
+      return applyBuilderMethods(transform, payload, allowlistClass);
+    } catch (ClassNotFoundException e) {
+      throw new IllegalArgumentException("Could not find class " + className, e);
+    } catch (InstantiationException
+        | IllegalArgumentException
+        | IllegalAccessException
+        | InvocationTargetException e) {
+      throw new IllegalArgumentException("Could not instantiate class " + className, e);
+    }
+  }
+
+  private PTransform<PInput, POutput> applyBuilderMethods(
+      PTransform<PInput, POutput> transform,
+      JavaClassLookupPayload payload,
+      AllowedClass allowListClass) {
+    for (BuilderMethod builderMethod : payload.getBuilderMethodsList()) {
+      Method method = getMethod(transform, builderMethod, allowListClass);
+      try {
+        Row builderMethodRow = decodeRow(builderMethod.getSchema(), builderMethod.getPayload());
+        transform =
+            (PTransform<PInput, POutput>)
+                method.invoke(
+                    transform,
+                    getParameterValues(
+                        method.getParameters(),
+                        builderMethodRow,
+                        method.getGenericParameterTypes()));
+      } catch (IllegalAccessException | InvocationTargetException e) {
+        throw new IllegalArgumentException(
+            "Could not invoke the builder method "
+                + builderMethod
+                + " on transform "
+                + transform
+                + " with parameter schema "
+                + builderMethod.getSchema(),
+            e);
+      }
+    }
+
+    return transform;
+  }
+
+  private boolean isBuilderMethodForName(
+      Method method, String nameFromPayload, AllowedClass allowListClass) {
+    // Lookup based on method annotations
+    for (Annotation annotation : method.getAnnotations()) {
+      if (annotation instanceof MultiLanguageBuilderMethod) {
+        if (nameFromPayload.equals(((MultiLanguageBuilderMethod) annotation).name())) {
+          if (allowListClass.getAllowedBuilderMethods().contains(nameFromPayload)) {
+            return true;
+          } else {
+            throw new RuntimeException(
+                "Builder method " + nameFromPayload + " has to be explicitly allowed");
+          }
+        }
+      }
+    }
+
+    // Lookup based on the method name.
+    boolean match = method.getName().equals(nameFromPayload);
+    String consideredMethodName = method.getName();
+
+    // We provide a simplification for common Java builder pattern naming convention where builder
+    // methods start with "with". In this case, for a builder method name in the form "withXyz",
+    // users may just use "xyz". If additional updates to the method name are needed the transform
+    // has to be updated by adding annotations.
+    if (!match && consideredMethodName.length() > 4 && consideredMethodName.startsWith("with")) {
+      consideredMethodName =
+          consideredMethodName.substring(4, 5).toLowerCase() + consideredMethodName.substring(5);
+      match = consideredMethodName.equals(nameFromPayload);
+    }
+    if (match && !allowListClass.getAllowedBuilderMethods().contains(consideredMethodName)) {
+      throw new RuntimeException(
+          "Builder method name " + consideredMethodName + " has to be explicitly allowed");
+    }
+    return match;
+  }
+
+  private Method getMethod(
+      PTransform<PInput, POutput> transform,
+      BuilderMethod builderMethod,
+      AllowedClass allowListClass) {
+
+    Row builderMethodRow = decodeRow(builderMethod.getSchema(), builderMethod.getPayload());
+
+    List<Method> matchingMethods =
+        Arrays.stream(transform.getClass().getMethods())
+            .filter(m -> isBuilderMethodForName(m, builderMethod.getName(), allowListClass))
+            .filter(m -> parametersCompatible(m.getParameters(), builderMethodRow))
+            .filter(m -> PTransform.class.isAssignableFrom(m.getReturnType()))
+            .collect(Collectors.toList());
+
+    if (matchingMethods.size() != 1) {
+      throw new RuntimeException(
+          "Expected to find exactly one matching method in transform "
+              + transform
+              + " for BuilderMethod"
+              + builderMethod
+              + " but found "
+              + matchingMethods.size());
+    }
+    return matchingMethods.get(0);
+  }
+
+  private static boolean isPrimitiveOrWrapperOrString(java.lang.Class<?> type) {
+    return ClassUtils.isPrimitiveOrWrapper(type) || type == String.class;
+  }
+
+  private Schema getParameterSchema(Class<?> parameterClass) {
+    Schema parameterSchema;
+    try {
+      parameterSchema = SCHEMA_REGISTRY.getSchema(parameterClass);
+    } catch (NoSuchSchemaException e) {
+
+      SCHEMA_REGISTRY.registerSchemaProvider(parameterClass, new JavaFieldSchema());
+      try {
+        parameterSchema = SCHEMA_REGISTRY.getSchema(parameterClass);
+      } catch (NoSuchSchemaException e1) {
+        throw new RuntimeException(e1);
+      }
+      if (parameterSchema != null && parameterSchema.getFieldCount() == 0) {
+        throw new RuntimeException(
+            "Could not determine a valid schema for parameter class " + parameterClass);
+      }
+    }
+    return parameterSchema;
+  }
+
+  private boolean parametersCompatible(
+      java.lang.reflect.Parameter[] methodParameters, Row constructorRow) {
+    Schema constructorSchema = constructorRow.getSchema();
+    if (methodParameters.length != constructorSchema.getFieldCount()) {
+      return false;
+    }
+
+    for (int i = 0; i < methodParameters.length; i++) {
+      java.lang.reflect.Parameter parameterFromReflection = methodParameters[i];
+      Field parameterFromPayload = constructorSchema.getField(i);
+
+      String paramNameFromReflection = parameterFromReflection.getName();
+      if (!paramNameFromReflection.startsWith("arg")
+          && !paramNameFromReflection.equals(parameterFromPayload.getName())) {
+        // Parameter name through reflection is from the class file (not through synthesizing,
+        // hence we can validate names)
+        return false;
+      }
+
+      Class<?> parameterClass = parameterFromReflection.getType();
+      if (isPrimitiveOrWrapperOrString(parameterClass)) {
+        continue;
+      }
+
+      // We perform additional validation for arrays and non-primitive types.
+      if (parameterClass.isArray()) {
+        Class<?> arrayFieldClass = parameterClass.getComponentType();
+        if (parameterFromPayload.getType().getTypeName() != TypeName.ARRAY) {
+          throw new RuntimeException(
+              "Expected a schema with a single array field but received "
+                  + parameterFromPayload.getType().getTypeName());
+        }
+
+        // Following is a best-effort validation that may not cover all cases. Idea is to resolve
+        // ambiguities as much as possible to determine an exact match for the given set of
+        // parameters. If there are ambiguities, the expansion will fail.
+        if (!isPrimitiveOrWrapperOrString(arrayFieldClass)) {
+          @Nullable Collection<Row> values = constructorRow.getArray(i);
+          Schema arrayFieldSchema = getParameterSchema(arrayFieldClass);
+          if (arrayFieldSchema == null) {
+            throw new RuntimeException("Could not determine a schema for type " + arrayFieldClass);
+          }
+          if (values != null) {
+            @Nullable Row firstItem = values.iterator().next();
+            if (firstItem != null && !(firstItem.getSchema().assignableTo(arrayFieldSchema))) {
+              return false;
+            }
+          }
+        }
+      } else if (constructorRow.getValue(i) instanceof Row) {
+        @Nullable Row parameterRow = constructorRow.getRow(i);
+        Schema schema = getParameterSchema(parameterClass);
+        if (schema == null) {
+          throw new RuntimeException("Could not determine a schema for type " + parameterClass);
+        }
+        if (parameterRow != null && !parameterRow.getSchema().assignableTo(schema)) {
+          return false;
+        }
+      }
+    }
+    return true;
+  }
+
+  private @Nullable Object getDecodedValueFromRow(
+      Class<?> type, Object valueFromRow, @Nullable Type genericType) {
+    if (isPrimitiveOrWrapperOrString(type)) {
+      if (!isPrimitiveOrWrapperOrString(valueFromRow.getClass())) {
+        throw new IllegalArgumentException(
+            "Expected a Java primitive value but received " + valueFromRow);
+      }
+      return valueFromRow;
+    } else if (type.isArray()) {
+      Class<?> arrayComponentClass = type.getComponentType();
+      return getDecodedArrayValueFromRow(arrayComponentClass, valueFromRow);
+    } else if (Collection.class.isAssignableFrom(type)) {
+      List<Object> originalList = (List) valueFromRow;
+      List<Object> decodedList = new ArrayList<>();
+      for (Object obj : originalList) {
+        if (genericType instanceof ParameterizedType) {
+          Class<?> elementType =
+              (Class<?>) ((ParameterizedType) genericType).getActualTypeArguments()[0];
+          decodedList.add(getDecodedValueFromRow(elementType, obj, null));
+        } else {
+          throw new RuntimeException("Could not determine the generic type of the list");
+        }
+      }
+      return decodedList;
+    } else if (valueFromRow instanceof Row) {
+      Row row = (Row) valueFromRow;
+      SerializableFunction<Row, ?> fromRowFunc;
+      try {
+        fromRowFunc = SCHEMA_REGISTRY.getFromRowFunction(type);
+      } catch (NoSuchSchemaException e) {
+        throw new IllegalArgumentException(
+            "Could not determine the row function for class " + type, e);
+      }
+      return fromRowFunc.apply(row);
+    }
+    throw new RuntimeException("Could not decode the value from Row " + valueFromRow);
+  }
+
+  private Object[] getParameterValues(
+      java.lang.reflect.Parameter[] parameters, Row constrtuctorRow, Type[] genericTypes) {
+    ArrayList<Object> parameterValues = new ArrayList<>();
+    for (int i = 0; i < parameters.length; ++i) {
+      java.lang.reflect.Parameter parameter = parameters[i];
+      Class<?> parameterClass = parameter.getType();
+      Object parameterValue =
+          getDecodedValueFromRow(parameterClass, constrtuctorRow.getValue(i), genericTypes[i]);
+      parameterValues.add(parameterValue);
+    }
+
+    return parameterValues.toArray();
+  }
+
+  private Object[] getDecodedArrayValueFromRow(Class<?> arrayComponentType, Object valueFromRow) {
+    List<Object> originalValues = (List<Object>) valueFromRow;
+    List<Object> decodedValues = new ArrayList<>();
+    for (Object obj : originalValues) {
+      decodedValues.add(getDecodedValueFromRow(arrayComponentType, obj, null));
+    }
+
+    // We have to construct and return an array of the correct type. Otherwise Java reflection
+    // constructor/method invocations that use the returned value may consider the array as varargs
+    // (different parameters).
+    Object valueTypeArray = Array.newInstance(arrayComponentType, decodedValues.size());
+    for (int i = 0; i < decodedValues.size(); i++) {
+      Array.set(valueTypeArray, i, arrayComponentType.cast(decodedValues.get(i)));
+    }
+    return (Object[]) valueTypeArray;
+  }
+
+  private Constructor<PTransform<InputT, OutputT>> findMappingConstructor(
+      Constructor<?>[] constructors, JavaClassLookupPayload payload) {
+    Row constructorRow = decodeRow(payload.getConstructorSchema(), payload.getConstructorPayload());
+
+    List<Constructor<?>> mappingConstructors =
+        Arrays.stream(constructors)
+            .filter(c -> c.getParameterCount() == payload.getConstructorSchema().getFieldsCount())
+            .filter(c -> parametersCompatible(c.getParameters(), constructorRow))
+            .collect(Collectors.toList());
+    if (mappingConstructors.size() != 1) {
+      throw new RuntimeException(
+          "Expected to find a single mapping constructor but found " + mappingConstructors.size());
+    }
+    return (Constructor<PTransform<InputT, OutputT>>) mappingConstructors.get(0);
+  }
+
+  private boolean isConstructorMethodForName(
+      Method method, String nameFromPayload, AllowedClass allowListClass) {
+    for (Annotation annotation : method.getAnnotations()) {
+      if (annotation instanceof MultiLanguageConstructorMethod) {
+        if (nameFromPayload.equals(((MultiLanguageConstructorMethod) annotation).name())) {
+          if (allowListClass.getAllowedConstructorMethods().contains(nameFromPayload)) {
+            return true;
+          } else {
+            throw new RuntimeException(
+                "Constructor method " + nameFromPayload + " needs to be explicitly allowed");
+          }
+        }
+      }
+    }
+    if (method.getName().equals(nameFromPayload)) {
+      if (allowListClass.getAllowedConstructorMethods().contains(nameFromPayload)) {
+        return true;
+      } else {
+        throw new RuntimeException(
+            "Constructor method " + nameFromPayload + " needs to be explicitly allowed");
+      }
+    }
+    return false;
+  }
+
+  private Method findMappingConstructorMethod(
+      Method[] methods, JavaClassLookupPayload payload, AllowedClass allowListClass) {
+
+    Row constructorRow = decodeRow(payload.getConstructorSchema(), payload.getConstructorPayload());
+
+    List<Method> mappingConstructorMethods =
+        Arrays.stream(methods)
+            .filter(
+                m -> isConstructorMethodForName(m, payload.getConstructorMethod(), allowListClass))
+            .filter(m -> m.getParameterCount() == payload.getConstructorSchema().getFieldsCount())
+            .filter(m -> parametersCompatible(m.getParameters(), constructorRow))
+            .collect(Collectors.toList());
+
+    if (mappingConstructorMethods.size() != 1) {
+      throw new RuntimeException(
+          "Expected to find a single mapping constructor method but found "
+              + mappingConstructorMethods.size()
+              + " Payload was "
+              + payload);
+    }
+    return mappingConstructorMethods.get(0);
+  }
+
+  @AutoValue
+  public abstract static class AllowList {
+
+    public abstract String getVersion();
+
+    public abstract List<AllowedClass> getAllowedClasses();
+
+    @JsonCreator
+    static AllowList create(
+        @JsonProperty("version") String version,
+        @JsonProperty("allowedClasses") @javax.annotation.Nullable
+            List<AllowedClass> allowedClasses) {
+      if (allowedClasses == null) {
+        allowedClasses = new ArrayList<>();
+      }
+      return new AutoValue_JavaClassLookupTransformProvider_AllowList(version, allowedClasses);
+    }
+  }
+
+  @AutoValue
+  public abstract static class AllowedClass {
+
+    public abstract String getClassName();
+
+    public abstract List<String> getAllowedBuilderMethods();
+
+    public abstract List<String> getAllowedConstructorMethods();
+
+    @JsonCreator
+    static AllowedClass create(
+        @JsonProperty("className") String className,
+        @JsonProperty("allowedBuilderMethods") @javax.annotation.Nullable
+            List<String> allowedBuilderMethods,
+        @JsonProperty("allowedConstructorMethods") @javax.annotation.Nullable
+            List<String> allowedConstructorMethods) {
+      if (allowedBuilderMethods == null) {
+        allowedBuilderMethods = new ArrayList<>();
+      }
+      if (allowedConstructorMethods == null) {
+        allowedConstructorMethods = new ArrayList<>();
+      }
+      return new AutoValue_JavaClassLookupTransformProvider_AllowedClass(
+          className, allowedBuilderMethods, allowedConstructorMethods);
+    }
+  }
+
+  static Row decodeRow(SchemaApi.Schema schema, ByteString payload) {
+    Schema payloadSchema = SchemaTranslation.schemaFromProto(schema);
+
+    if (payloadSchema.getFieldCount() == 0) {
+      return Row.withSchema(Schema.of()).build();
+    }
+
+    Row row;
+    try {
+      row = RowCoder.of(payloadSchema).decode(payload.newInput());
+    } catch (IOException e) {
+      throw new RuntimeException("Error decoding payload", e);
+    }
+    return row;
+  }
+}
diff --git a/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/MultiLanguageBuilderMethod.java b/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/MultiLanguageBuilderMethod.java
new file mode 100644
index 0000000..3ee9ef5
--- /dev/null
+++ b/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/MultiLanguageBuilderMethod.java
@@ -0,0 +1,31 @@
+/*
+ * 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.expansion.service;
+
+import java.lang.annotation.Documented;
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+
+@Documented
+@Target({ElementType.METHOD})
+@Retention(RetentionPolicy.RUNTIME)
+public @interface MultiLanguageBuilderMethod {
+  String name();
+}
diff --git a/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/MultiLanguageConstructorMethod.java b/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/MultiLanguageConstructorMethod.java
new file mode 100644
index 0000000..e89f460
--- /dev/null
+++ b/sdks/java/expansion-service/src/main/java/org/apache/beam/sdk/expansion/service/MultiLanguageConstructorMethod.java
@@ -0,0 +1,31 @@
+/*
+ * 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.expansion.service;
+
+import java.lang.annotation.Documented;
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+
+@Documented
+@Target({ElementType.METHOD})
+@Retention(RetentionPolicy.RUNTIME)
+public @interface MultiLanguageConstructorMethod {
+  String name();
+}
diff --git a/sdks/java/expansion-service/src/test/java/org/apache/beam/sdk/expansion/service/ExpansionServiceTest.java b/sdks/java/expansion-service/src/test/java/org/apache/beam/sdk/expansion/service/ExpansionServiceTest.java
index 5e2a243..e8ecf46 100644
--- a/sdks/java/expansion-service/src/test/java/org/apache/beam/sdk/expansion/service/ExpansionServiceTest.java
+++ b/sdks/java/expansion-service/src/test/java/org/apache/beam/sdk/expansion/service/ExpansionServiceTest.java
@@ -90,7 +90,8 @@
 
   /** Registers a single test transformation. */
   @AutoService(ExpansionService.ExpansionServiceRegistrar.class)
-  public static class TestTransforms implements ExpansionService.ExpansionServiceRegistrar {
+  public static class TestTransformRegistrar implements ExpansionService.ExpansionServiceRegistrar {
+
     @Override
     public Map<String, ExpansionService.TransformProvider> knownTransforms() {
       return ImmutableMap.of(TEST_URN, spec -> Count.perElement());
@@ -140,9 +141,9 @@
   }
 
   @Test
-  public void testConstructGenerateSequence() {
+  public void testConstructGenerateSequenceWithRegistration() {
     ExternalTransforms.ExternalConfigurationPayload payload =
-        encodeRow(
+        encodeRowIntoExternalConfigurationPayload(
             Row.withSchema(
                     Schema.of(
                         Field.of("start", FieldType.INT64),
@@ -176,7 +177,7 @@
   @Test
   public void testCompoundCodersForExternalConfiguration_setters() throws Exception {
     ExternalTransforms.ExternalConfigurationPayload externalConfig =
-        encodeRow(
+        encodeRowIntoExternalConfigurationPayload(
             Row.withSchema(
                     Schema.of(
                         Field.nullable("config_key1", FieldType.INT64),
@@ -253,7 +254,7 @@
   @Test
   public void testCompoundCodersForExternalConfiguration_schemas() throws Exception {
     ExternalTransforms.ExternalConfigurationPayload externalConfig =
-        encodeRow(
+        encodeRowIntoExternalConfigurationPayload(
             Row.withSchema(
                     Schema.of(
                         Field.nullable("configKey1", FieldType.INT64),
@@ -320,7 +321,7 @@
   @Test
   public void testExternalConfiguration_simpleSchema() throws Exception {
     ExternalTransforms.ExternalConfigurationPayload externalConfig =
-        encodeRow(
+        encodeRowIntoExternalConfigurationPayload(
             Row.withSchema(
                     Schema.of(
                         Field.of("bar", FieldType.STRING),
@@ -350,7 +351,8 @@
     abstract List<String> getList();
   }
 
-  private static ExternalTransforms.ExternalConfigurationPayload encodeRow(Row row) {
+  private static ExternalTransforms.ExternalConfigurationPayload
+      encodeRowIntoExternalConfigurationPayload(Row row) {
     ByteString.Output outputStream = ByteString.newOutput();
     try {
       SchemaCoder.of(row.getSchema()).encode(row, outputStream);
diff --git a/sdks/java/expansion-service/src/test/java/org/apache/beam/sdk/expansion/service/JavaCLassLookupTransformProviderTest.java b/sdks/java/expansion-service/src/test/java/org/apache/beam/sdk/expansion/service/JavaCLassLookupTransformProviderTest.java
new file mode 100644
index 0000000..5244108
--- /dev/null
+++ b/sdks/java/expansion-service/src/test/java/org/apache/beam/sdk/expansion/service/JavaCLassLookupTransformProviderTest.java
@@ -0,0 +1,1111 @@
+/*
+ * 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.expansion.service;
+
+import static org.apache.beam.runners.core.construction.BeamUrns.getUrn;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.anyOf;
+import static org.hamcrest.Matchers.containsString;
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertThrows;
+import static org.junit.Assert.assertTrue;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.Serializable;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import org.apache.beam.model.expansion.v1.ExpansionApi;
+import org.apache.beam.model.pipeline.v1.ExternalTransforms;
+import org.apache.beam.model.pipeline.v1.ExternalTransforms.BuilderMethod;
+import org.apache.beam.model.pipeline.v1.ExternalTransforms.ExpansionMethods;
+import org.apache.beam.model.pipeline.v1.RunnerApi;
+import org.apache.beam.model.pipeline.v1.RunnerApi.ParDoPayload;
+import org.apache.beam.model.pipeline.v1.SchemaApi;
+import org.apache.beam.runners.core.construction.ParDoTranslation;
+import org.apache.beam.runners.core.construction.PipelineTranslation;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.schemas.Schema.Field;
+import org.apache.beam.sdk.schemas.Schema.FieldType;
+import org.apache.beam.sdk.schemas.SchemaCoder;
+import org.apache.beam.sdk.schemas.SchemaTranslation;
+import org.apache.beam.sdk.transforms.Create;
+import org.apache.beam.sdk.transforms.DoFn;
+import org.apache.beam.sdk.transforms.PTransform;
+import org.apache.beam.sdk.transforms.ParDo;
+import org.apache.beam.sdk.values.PBegin;
+import org.apache.beam.sdk.values.PCollection;
+import org.apache.beam.sdk.values.Row;
+import org.apache.beam.vendor.grpc.v1p36p0.com.google.protobuf.ByteString;
+import org.apache.beam.vendor.grpc.v1p36p0.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.io.Resources;
+import org.hamcrest.Matchers;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+/** Tests for {@link JavaCLassLookupTransformProvider}. */
+@RunWith(JUnit4.class)
+public class JavaCLassLookupTransformProviderTest {
+
+  private static final String TEST_URN = "test:beam:transforms:count";
+
+  private static final String TEST_NAME = "TestName";
+
+  private static final String TEST_NAMESPACE = "namespace";
+
+  private static ExpansionService expansionService;
+
+  @BeforeClass
+  public static void setupExpansionService() {
+    PipelineOptionsFactory.register(ExpansionServiceOptions.class);
+    URL allowListFile = Resources.getResource("./test_allowlist.yaml");
+    System.out.println("Exists: " + new File(allowListFile.getPath()).exists());
+    expansionService =
+        new ExpansionService(
+            new String[] {"--javaClassLookupAllowlistFile=" + allowListFile.getPath()});
+  }
+
+  static class DummyDoFn extends DoFn<String, String> {
+    String strField1;
+    String strField2;
+    int intField1;
+    Double doubleWrapperField;
+    String[] strArrayField;
+    DummyComplexType complexTypeField;
+    DummyComplexType[] complexTypeArrayField;
+    List<String> strListField;
+    List<DummyComplexType> complexTypeListField;
+
+    private DummyDoFn(
+        String strField1,
+        String strField2,
+        int intField1,
+        Double doubleWrapperField,
+        String[] strArrayField,
+        DummyComplexType complexTypeField,
+        DummyComplexType[] complexTypeArrayField,
+        List<String> strListField,
+        List<DummyComplexType> complexTypeListField) {
+      this.intField1 = intField1;
+      this.strField1 = strField1;
+      this.strField2 = strField2;
+      this.doubleWrapperField = doubleWrapperField;
+      this.strArrayField = strArrayField;
+      this.complexTypeField = complexTypeField;
+      this.complexTypeArrayField = complexTypeArrayField;
+      this.strListField = strListField;
+      this.complexTypeListField = complexTypeListField;
+    }
+
+    @ProcessElement
+    public void processElement(ProcessContext c) {
+      c.output(c.element());
+    }
+  }
+
+  public static class DummyComplexType implements Serializable {
+    String complexTypeStrField;
+    int complexTypeIntField;
+
+    public DummyComplexType() {}
+
+    public DummyComplexType(String complexTypeStrField, int complexTypeIntField) {
+      this.complexTypeStrField = complexTypeStrField;
+      this.complexTypeIntField = complexTypeIntField;
+    }
+
+    @Override
+    public int hashCode() {
+      return this.complexTypeStrField.hashCode() + this.complexTypeIntField * 31;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+      if (!(obj instanceof DummyComplexType)) {
+        return false;
+      }
+      DummyComplexType toCompare = (DummyComplexType) obj;
+      return (this.complexTypeIntField == toCompare.complexTypeIntField)
+          && (this.complexTypeStrField.equals(toCompare.complexTypeStrField));
+    }
+  }
+
+  public static class DummyTransform extends PTransform<PBegin, PCollection<String>> {
+    String strField1;
+    String strField2;
+    int intField1;
+    Double doubleWrapperField;
+    String[] strArrayField;
+    DummyComplexType complexTypeField;
+    DummyComplexType[] complexTypeArrayField;
+    List<String> strListField;
+    List<DummyComplexType> complexTypeListField;
+
+    @Override
+    public PCollection<String> expand(PBegin input) {
+      return input
+          .apply("MyCreateTransform", Create.of("aaa", "bbb", "ccc"))
+          .apply(
+              "MyParDoTransform",
+              ParDo.of(
+                  new DummyDoFn(
+                      this.strField1,
+                      this.strField2,
+                      this.intField1,
+                      this.doubleWrapperField,
+                      this.strArrayField,
+                      this.complexTypeField,
+                      this.complexTypeArrayField,
+                      this.strListField,
+                      this.complexTypeListField)));
+    }
+  }
+
+  public static class DummyTransformWithConstructor extends DummyTransform {
+
+    public DummyTransformWithConstructor(String strField1) {
+      this.strField1 = strField1;
+    }
+  }
+
+  public static class DummyTransformWithConstructorAndBuilderMethods extends DummyTransform {
+
+    public DummyTransformWithConstructorAndBuilderMethods(String strField1) {
+      this.strField1 = strField1;
+    }
+
+    public DummyTransformWithConstructorAndBuilderMethods withStrField2(String strField2) {
+      this.strField2 = strField2;
+      return this;
+    }
+
+    public DummyTransformWithConstructorAndBuilderMethods withIntField1(int intField1) {
+      this.intField1 = intField1;
+      return this;
+    }
+  }
+
+  public static class DummyTransformWithMultiArgumentBuilderMethod extends DummyTransform {
+
+    public DummyTransformWithMultiArgumentBuilderMethod(String strField1) {
+      this.strField1 = strField1;
+    }
+
+    public DummyTransformWithMultiArgumentBuilderMethod withFields(
+        String strField2, int intField1) {
+      this.strField2 = strField2;
+      this.intField1 = intField1;
+      return this;
+    }
+  }
+
+  public static class DummyTransformWithMultiArgumentConstructor extends DummyTransform {
+
+    public DummyTransformWithMultiArgumentConstructor(String strField1, String strField2) {
+      this.strField1 = strField1;
+      this.strField2 = strField2;
+    }
+  }
+
+  public static class DummyTransformWithConstructorMethod extends DummyTransform {
+
+    public static DummyTransformWithConstructorMethod from(String strField1) {
+      DummyTransformWithConstructorMethod transform = new DummyTransformWithConstructorMethod();
+      transform.strField1 = strField1;
+      return transform;
+    }
+  }
+
+  public static class DummyTransformWithConstructorMethodAndBuilderMethods extends DummyTransform {
+
+    public static DummyTransformWithConstructorMethodAndBuilderMethods from(String strField1) {
+      DummyTransformWithConstructorMethodAndBuilderMethods transform =
+          new DummyTransformWithConstructorMethodAndBuilderMethods();
+      transform.strField1 = strField1;
+      return transform;
+    }
+
+    public DummyTransformWithConstructorMethodAndBuilderMethods withStrField2(String strField2) {
+      this.strField2 = strField2;
+      return this;
+    }
+
+    public DummyTransformWithConstructorMethodAndBuilderMethods withIntField1(int intField1) {
+      this.intField1 = intField1;
+      return this;
+    }
+  }
+
+  public static class DummyTransformWithMultiLanguageAnnotations extends DummyTransform {
+
+    @MultiLanguageConstructorMethod(name = "create_transform")
+    public static DummyTransformWithMultiLanguageAnnotations from(String strField1) {
+      DummyTransformWithMultiLanguageAnnotations transform =
+          new DummyTransformWithMultiLanguageAnnotations();
+      transform.strField1 = strField1;
+      return transform;
+    }
+
+    @MultiLanguageBuilderMethod(name = "abc")
+    public DummyTransformWithMultiLanguageAnnotations withStrField2(String strField2) {
+      this.strField2 = strField2;
+      return this;
+    }
+
+    @MultiLanguageBuilderMethod(name = "xyz")
+    public DummyTransformWithMultiLanguageAnnotations withIntField1(int intField1) {
+      this.intField1 = intField1;
+      return this;
+    }
+  }
+
+  public static class DummyTransformWithWrapperTypes extends DummyTransform {
+    public DummyTransformWithWrapperTypes(String strField1) {
+      this.strField1 = strField1;
+    }
+
+    public DummyTransformWithWrapperTypes withDoubleWrapperField(Double doubleWrapperField) {
+      this.doubleWrapperField = doubleWrapperField;
+      return this;
+    }
+  }
+
+  public static class DummyTransformWithComplexTypes extends DummyTransform {
+    public DummyTransformWithComplexTypes(String strField1) {
+      this.strField1 = strField1;
+    }
+
+    public DummyTransformWithComplexTypes withComplexTypeField(DummyComplexType complexTypeField) {
+      this.complexTypeField = complexTypeField;
+      return this;
+    }
+  }
+
+  public static class DummyTransformWithArray extends DummyTransform {
+    public DummyTransformWithArray(String strField1) {
+      this.strField1 = strField1;
+    }
+
+    public DummyTransformWithArray withStrArrayField(String[] strArrayField) {
+      this.strArrayField = strArrayField;
+      return this;
+    }
+  }
+
+  public static class DummyTransformWithList extends DummyTransform {
+    public DummyTransformWithList(String strField1) {
+      this.strField1 = strField1;
+    }
+
+    public DummyTransformWithList withStrListField(List<String> strListField) {
+      this.strListField = strListField;
+      return this;
+    }
+  }
+
+  public static class DummyTransformWithComplexTypeArray extends DummyTransform {
+    public DummyTransformWithComplexTypeArray(String strField1) {
+      this.strField1 = strField1;
+    }
+
+    public DummyTransformWithComplexTypeArray withComplexTypeArrayField(
+        DummyComplexType[] complexTypeArrayField) {
+      this.complexTypeArrayField = complexTypeArrayField;
+      return this;
+    }
+  }
+
+  public static class DummyTransformWithComplexTypeList extends DummyTransform {
+    public DummyTransformWithComplexTypeList(String strField1) {
+      this.strField1 = strField1;
+    }
+
+    public DummyTransformWithComplexTypeList withComplexTypeListField(
+        List<DummyComplexType> complexTypeListField) {
+      this.complexTypeListField = complexTypeListField;
+      return this;
+    }
+  }
+
+  void testClassLookupExpansionRequestConstruction(
+      ExternalTransforms.JavaClassLookupPayload payload, Map<String, Object> fieldsToVerify) {
+    Pipeline p = Pipeline.create();
+
+    RunnerApi.Pipeline pipelineProto = PipelineTranslation.toProto(p);
+
+    ExpansionApi.ExpansionRequest request =
+        ExpansionApi.ExpansionRequest.newBuilder()
+            .setComponents(pipelineProto.getComponents())
+            .setTransform(
+                RunnerApi.PTransform.newBuilder()
+                    .setUniqueName(TEST_NAME)
+                    .setSpec(
+                        RunnerApi.FunctionSpec.newBuilder()
+                            .setUrn(getUrn(ExpansionMethods.Enum.JAVA_CLASS_LOOKUP))
+                            .setPayload(payload.toByteString())))
+            .setNamespace(TEST_NAMESPACE)
+            .build();
+    ExpansionApi.ExpansionResponse response = expansionService.expand(request);
+    RunnerApi.PTransform expandedTransform = response.getTransform();
+    assertEquals(TEST_NAMESPACE + TEST_NAME, expandedTransform.getUniqueName());
+    assertThat(expandedTransform.getInputsCount(), Matchers.is(0));
+    assertThat(expandedTransform.getOutputsCount(), Matchers.is(1));
+    assertEquals(2, expandedTransform.getSubtransformsCount());
+    assertEquals(2, expandedTransform.getSubtransformsCount());
+    assertThat(
+        expandedTransform.getSubtransforms(0),
+        anyOf(containsString("MyCreateTransform"), containsString("MyParDoTransform")));
+    assertThat(
+        expandedTransform.getSubtransforms(1),
+        anyOf(containsString("MyCreateTransform"), containsString("MyParDoTransform")));
+
+    org.apache.beam.model.pipeline.v1.RunnerApi.PTransform userParDoTransform = null;
+    for (String transformId : response.getComponents().getTransformsMap().keySet()) {
+      if (transformId.contains("ParMultiDo-Dummy-")) {
+        userParDoTransform = response.getComponents().getTransformsMap().get(transformId);
+      }
+    }
+    assertNotNull(userParDoTransform);
+    ParDoPayload parDoPayload = null;
+    try {
+      parDoPayload = ParDoPayload.parseFrom(userParDoTransform.getSpec().getPayload());
+    } catch (InvalidProtocolBufferException e) {
+      throw new RuntimeException(e);
+    }
+    assertNotNull(parDoPayload);
+    DummyDoFn doFn =
+        (DummyDoFn)
+            ParDoTranslation.doFnWithExecutionInformationFromProto(parDoPayload.getDoFn())
+                .getDoFn();
+    System.out.println("DoFn" + doFn);
+
+    List<String> verifiedFields = new ArrayList<>();
+    if (fieldsToVerify.keySet().contains("strField1")) {
+      assertEquals(doFn.strField1, fieldsToVerify.get("strField1"));
+      verifiedFields.add("strField1");
+    }
+    if (fieldsToVerify.keySet().contains("strField2")) {
+      assertEquals(doFn.strField2, fieldsToVerify.get("strField2"));
+      verifiedFields.add("strField2");
+    }
+    if (fieldsToVerify.keySet().contains("intField1")) {
+      assertEquals(doFn.intField1, fieldsToVerify.get("intField1"));
+      verifiedFields.add("intField1");
+    }
+    if (fieldsToVerify.keySet().contains("doubleWrapperField")) {
+      assertEquals(doFn.doubleWrapperField, fieldsToVerify.get("doubleWrapperField"));
+      verifiedFields.add("doubleWrapperField");
+    }
+    if (fieldsToVerify.containsKey("complexTypeStrField")) {
+      assertEquals(
+          doFn.complexTypeField.complexTypeStrField, fieldsToVerify.get("complexTypeStrField"));
+      verifiedFields.add("complexTypeStrField");
+    }
+    if (fieldsToVerify.containsKey("complexTypeIntField")) {
+      assertEquals(
+          doFn.complexTypeField.complexTypeIntField, fieldsToVerify.get("complexTypeIntField"));
+      verifiedFields.add("complexTypeIntField");
+    }
+
+    if (fieldsToVerify.keySet().contains("strArrayField")) {
+      assertArrayEquals(doFn.strArrayField, (String[]) fieldsToVerify.get("strArrayField"));
+      verifiedFields.add("strArrayField");
+    }
+
+    if (fieldsToVerify.keySet().contains("strListField")) {
+      assertEquals(doFn.strListField, (List) fieldsToVerify.get("strListField"));
+      verifiedFields.add("strListField");
+    }
+
+    if (fieldsToVerify.keySet().contains("complexTypeArrayField")) {
+      assertArrayEquals(
+          doFn.complexTypeArrayField,
+          (DummyComplexType[]) fieldsToVerify.get("complexTypeArrayField"));
+      verifiedFields.add("complexTypeArrayField");
+    }
+
+    if (fieldsToVerify.keySet().contains("complexTypeListField")) {
+      assertEquals(doFn.complexTypeListField, (List) fieldsToVerify.get("complexTypeListField"));
+      verifiedFields.add("complexTypeListField");
+    }
+
+    List<String> unverifiedFields = new ArrayList<>(fieldsToVerify.keySet());
+    unverifiedFields.removeAll(verifiedFields);
+    if (!unverifiedFields.isEmpty()) {
+      throw new RuntimeException("Failed to verify some fields: " + unverifiedFields);
+    }
+  }
+
+  @Test
+  public void testJavaClassLookupWithConstructor() {
+    ExternalTransforms.JavaClassLookupPayload.Builder payloadBuilder =
+        ExternalTransforms.JavaClassLookupPayload.newBuilder();
+    payloadBuilder.setClassName(
+        "org.apache.beam.sdk.expansion.service.JavaCLassLookupTransformProviderTest$DummyTransformWithConstructor");
+
+    Row constructorRow =
+        Row.withSchema(Schema.of(Field.of("strField1", FieldType.STRING)))
+            .withFieldValue("strField1", "test_str_1")
+            .build();
+
+    payloadBuilder.setConstructorSchema(getProtoSchemaFromRow(constructorRow));
+    payloadBuilder.setConstructorPayload(getProtoPayloadFromRow(constructorRow));
+
+    testClassLookupExpansionRequestConstruction(
+        payloadBuilder.build(), ImmutableMap.of("strField1", "test_str_1"));
+  }
+
+  @Test
+  public void testJavaClassLookupWithConstructorMethod() {
+    ExternalTransforms.JavaClassLookupPayload.Builder payloadBuilder =
+        ExternalTransforms.JavaClassLookupPayload.newBuilder();
+    payloadBuilder.setClassName(
+        "org.apache.beam.sdk.expansion.service.JavaCLassLookupTransformProviderTest$DummyTransformWithConstructorMethod");
+
+    payloadBuilder.setConstructorMethod("from");
+    Row constructorRow =
+        Row.withSchema(Schema.of(Field.of("strField1", FieldType.STRING)))
+            .withFieldValue("strField1", "test_str_1")
+            .build();
+
+    payloadBuilder.setConstructorSchema(getProtoSchemaFromRow(constructorRow));
+    payloadBuilder.setConstructorPayload(getProtoPayloadFromRow(constructorRow));
+
+    testClassLookupExpansionRequestConstruction(
+        payloadBuilder.build(), ImmutableMap.of("strField1", "test_str_1"));
+  }
+
+  @Test
+  public void testJavaClassLookupWithConstructorAndBuilderMethods() {
+    ExternalTransforms.JavaClassLookupPayload.Builder payloadBuilder =
+        ExternalTransforms.JavaClassLookupPayload.newBuilder();
+    payloadBuilder.setClassName(
+        "org.apache.beam.sdk.expansion.service.JavaCLassLookupTransformProviderTest$DummyTransformWithConstructorAndBuilderMethods");
+
+    Row constructorRow =
+        Row.withSchema(Schema.of(Field.of("strField1", FieldType.STRING)))
+            .withFieldValue("strField1", "test_str_1")
+            .build();
+
+    payloadBuilder.setConstructorSchema(getProtoSchemaFromRow(constructorRow));
+    payloadBuilder.setConstructorPayload(getProtoPayloadFromRow(constructorRow));
+
+    BuilderMethod.Builder builderMethodBuilder = BuilderMethod.newBuilder();
+    builderMethodBuilder.setName("withStrField2");
+    Row builderMethodRow =
+        Row.withSchema(Schema.of(Field.of("strField2", FieldType.STRING)))
+            .withFieldValue("strField2", "test_str_2")
+            .build();
+    builderMethodBuilder.setSchema(getProtoSchemaFromRow(builderMethodRow));
+    builderMethodBuilder.setPayload(getProtoPayloadFromRow(builderMethodRow));
+
+    payloadBuilder.addBuilderMethods(builderMethodBuilder);
+
+    builderMethodBuilder = BuilderMethod.newBuilder();
+    builderMethodBuilder.setName("withIntField1");
+    builderMethodRow =
+        Row.withSchema(Schema.of(Field.of("intField1", FieldType.INT32)))
+            .withFieldValue("intField1", 10)
+            .build();
+    builderMethodBuilder.setSchema(getProtoSchemaFromRow(builderMethodRow));
+    builderMethodBuilder.setPayload(getProtoPayloadFromRow(builderMethodRow));
+
+    payloadBuilder.addBuilderMethods(builderMethodBuilder);
+
+    testClassLookupExpansionRequestConstruction(
+        payloadBuilder.build(),
+        ImmutableMap.of("strField1", "test_str_1", "strField2", "test_str_2", "intField1", 10));
+  }
+
+  @Test
+  public void testJavaClassLookupWithMultiArgumentConstructor() {
+    ExternalTransforms.JavaClassLookupPayload.Builder payloadBuilder =
+        ExternalTransforms.JavaClassLookupPayload.newBuilder();
+    payloadBuilder.setClassName(
+        "org.apache.beam.sdk.expansion.service.JavaCLassLookupTransformProviderTest$DummyTransformWithMultiArgumentConstructor");
+
+    Row constructorRow =
+        Row.withSchema(
+                Schema.of(
+                    Field.of("strField1", FieldType.STRING),
+                    Field.of("strField2", FieldType.STRING)))
+            .withFieldValue("strField1", "test_str_1")
+            .withFieldValue("strField2", "test_str_2")
+            .build();
+
+    payloadBuilder.setConstructorSchema(getProtoSchemaFromRow(constructorRow));
+    payloadBuilder.setConstructorPayload(getProtoPayloadFromRow(constructorRow));
+
+    testClassLookupExpansionRequestConstruction(
+        payloadBuilder.build(),
+        ImmutableMap.of("strField1", "test_str_1", "strField2", "test_str_2"));
+  }
+
+  @Test
+  public void testJavaClassLookupWithMultiArgumentBuilderMethod() {
+    ExternalTransforms.JavaClassLookupPayload.Builder payloadBuilder =
+        ExternalTransforms.JavaClassLookupPayload.newBuilder();
+    payloadBuilder.setClassName(
+        "org.apache.beam.sdk.expansion.service.JavaCLassLookupTransformProviderTest$DummyTransformWithMultiArgumentBuilderMethod");
+
+    Row constructorRow =
+        Row.withSchema(Schema.of(Field.of("strField1", FieldType.STRING)))
+            .withFieldValue("strField1", "test_str_1")
+            .build();
+
+    payloadBuilder.setConstructorSchema(getProtoSchemaFromRow(constructorRow));
+    payloadBuilder.setConstructorPayload(getProtoPayloadFromRow(constructorRow));
+
+    BuilderMethod.Builder builderMethodBuilder = BuilderMethod.newBuilder();
+    builderMethodBuilder.setName("withFields");
+    Row builderMethodRow =
+        Row.withSchema(
+                Schema.of(
+                    Field.of("strField2", FieldType.STRING),
+                    Field.of("intField1", FieldType.INT32)))
+            .withFieldValue("strField2", "test_str_2")
+            .withFieldValue("intField1", 10)
+            .build();
+    builderMethodBuilder.setSchema(getProtoSchemaFromRow(builderMethodRow));
+    builderMethodBuilder.setPayload(getProtoPayloadFromRow(builderMethodRow));
+
+    payloadBuilder.addBuilderMethods(builderMethodBuilder);
+
+    testClassLookupExpansionRequestConstruction(
+        payloadBuilder.build(),
+        ImmutableMap.of("strField1", "test_str_1", "strField2", "test_str_2", "intField1", 10));
+  }
+
+  @Test
+  public void testJavaClassLookupWithWrapperTypes() {
+    ExternalTransforms.JavaClassLookupPayload.Builder payloadBuilder =
+        ExternalTransforms.JavaClassLookupPayload.newBuilder();
+    payloadBuilder.setClassName(
+        "org.apache.beam.sdk.expansion.service.JavaCLassLookupTransformProviderTest$DummyTransformWithWrapperTypes");
+
+    Row constructorRow =
+        Row.withSchema(Schema.of(Field.of("strField1", FieldType.STRING)))
+            .withFieldValue("strField1", "test_str_1")
+            .build();
+
+    payloadBuilder.setConstructorSchema(getProtoSchemaFromRow(constructorRow));
+    payloadBuilder.setConstructorPayload(getProtoPayloadFromRow(constructorRow));
+
+    BuilderMethod.Builder builderMethodBuilder = BuilderMethod.newBuilder();
+    builderMethodBuilder.setName("withDoubleWrapperField");
+    Row builderMethodRow =
+        Row.withSchema(Schema.of(Field.of("doubleWrapperField", FieldType.DOUBLE)))
+            .withFieldValue("doubleWrapperField", 123.56)
+            .build();
+    builderMethodBuilder.setSchema(getProtoSchemaFromRow(builderMethodRow));
+    builderMethodBuilder.setPayload(getProtoPayloadFromRow(builderMethodRow));
+
+    payloadBuilder.addBuilderMethods(builderMethodBuilder);
+
+    testClassLookupExpansionRequestConstruction(
+        payloadBuilder.build(), ImmutableMap.of("doubleWrapperField", 123.56));
+  }
+
+  @Test
+  public void testJavaClassLookupWithComplexTypes() {
+    ExternalTransforms.JavaClassLookupPayload.Builder payloadBuilder =
+        ExternalTransforms.JavaClassLookupPayload.newBuilder();
+    payloadBuilder.setClassName(
+        "org.apache.beam.sdk.expansion.service.JavaCLassLookupTransformProviderTest$DummyTransformWithComplexTypes");
+
+    Row constructorRow =
+        Row.withSchema(Schema.of(Field.of("strField1", FieldType.STRING)))
+            .withFieldValue("strField1", "test_str_1")
+            .build();
+    payloadBuilder.setConstructorSchema(getProtoSchemaFromRow(constructorRow));
+    payloadBuilder.setConstructorPayload(getProtoPayloadFromRow(constructorRow));
+
+    Schema complexTypeSchema =
+        Schema.builder()
+            .addStringField("complexTypeStrField")
+            .addInt32Field("complexTypeIntField")
+            .build();
+
+    BuilderMethod.Builder builderMethodBuilder = BuilderMethod.newBuilder();
+    builderMethodBuilder.setName("withComplexTypeField");
+
+    Row builderMethodParamRow =
+        Row.withSchema(complexTypeSchema)
+            .withFieldValue("complexTypeStrField", "complex_type_str_1")
+            .withFieldValue("complexTypeIntField", 123)
+            .build();
+
+    Schema builderMethodSchema =
+        Schema.builder().addRowField("complexTypeField", complexTypeSchema).build();
+    Row builderMethodRow =
+        Row.withSchema(builderMethodSchema)
+            .withFieldValue("complexTypeField", builderMethodParamRow)
+            .build();
+    builderMethodBuilder.setSchema(getProtoSchemaFromRow(builderMethodRow));
+    builderMethodBuilder.setPayload(getProtoPayloadFromRow(builderMethodRow));
+
+    payloadBuilder.addBuilderMethods(builderMethodBuilder);
+
+    testClassLookupExpansionRequestConstruction(
+        payloadBuilder.build(),
+        ImmutableMap.of("complexTypeStrField", "complex_type_str_1", "complexTypeIntField", 123));
+  }
+
+  @Test
+  public void testJavaClassLookupWithSimpleArrayType() {
+    ExternalTransforms.JavaClassLookupPayload.Builder payloadBuilder =
+        ExternalTransforms.JavaClassLookupPayload.newBuilder();
+    payloadBuilder.setClassName(
+        "org.apache.beam.sdk.expansion.service.JavaCLassLookupTransformProviderTest$DummyTransformWithArray");
+
+    Row constructorRow =
+        Row.withSchema(Schema.of(Field.of("strField1", FieldType.STRING)))
+            .withFieldValue("strField1", "test_str_1")
+            .build();
+
+    payloadBuilder.setConstructorSchema(getProtoSchemaFromRow(constructorRow));
+    payloadBuilder.setConstructorPayload(getProtoPayloadFromRow(constructorRow));
+
+    BuilderMethod.Builder builderMethodBuilder = BuilderMethod.newBuilder();
+    builderMethodBuilder.setName("withStrArrayField");
+
+    Schema builderMethodSchema =
+        Schema.builder().addArrayField("strArrayField", FieldType.STRING).build();
+
+    Row builderMethodRow =
+        Row.withSchema(builderMethodSchema)
+            .withFieldValue(
+                "strArrayField", ImmutableList.of("test_str_1", "test_str_2", "test_str_3"))
+            .build();
+
+    builderMethodBuilder.setSchema(getProtoSchemaFromRow(builderMethodRow));
+    builderMethodBuilder.setPayload(getProtoPayloadFromRow(builderMethodRow));
+
+    payloadBuilder.addBuilderMethods(builderMethodBuilder);
+
+    String[] resultArray = {"test_str_1", "test_str_2", "test_str_3"};
+    testClassLookupExpansionRequestConstruction(
+        payloadBuilder.build(), ImmutableMap.of("strArrayField", resultArray));
+  }
+
+  @Test
+  public void testJavaClassLookupWithSimpleListType() {
+    ExternalTransforms.JavaClassLookupPayload.Builder payloadBuilder =
+        ExternalTransforms.JavaClassLookupPayload.newBuilder();
+    payloadBuilder.setClassName(
+        "org.apache.beam.sdk.expansion.service.JavaCLassLookupTransformProviderTest$DummyTransformWithList");
+
+    Row constructorRow =
+        Row.withSchema(Schema.of(Field.of("strField1", FieldType.STRING)))
+            .withFieldValue("strField1", "test_str_1")
+            .build();
+
+    payloadBuilder.setConstructorSchema(getProtoSchemaFromRow(constructorRow));
+    payloadBuilder.setConstructorPayload(getProtoPayloadFromRow(constructorRow));
+
+    BuilderMethod.Builder builderMethodBuilder = BuilderMethod.newBuilder();
+    builderMethodBuilder.setName("withStrListField");
+
+    Schema builderMethodSchema =
+        Schema.builder().addIterableField("strListField", FieldType.STRING).build();
+
+    Row builderMethodRow =
+        Row.withSchema(builderMethodSchema)
+            .withFieldValue(
+                "strListField", ImmutableList.of("test_str_1", "test_str_2", "test_str_3"))
+            .build();
+
+    builderMethodBuilder.setSchema(getProtoSchemaFromRow(builderMethodRow));
+    builderMethodBuilder.setPayload(getProtoPayloadFromRow(builderMethodRow));
+
+    payloadBuilder.addBuilderMethods(builderMethodBuilder);
+
+    List<String> resultList = new ArrayList<>();
+    resultList.add("test_str_1");
+    resultList.add("test_str_2");
+    resultList.add("test_str_3");
+    testClassLookupExpansionRequestConstruction(
+        payloadBuilder.build(), ImmutableMap.of("strListField", resultList));
+  }
+
+  @Test
+  public void testJavaClassLookupWithComplexArrayType() {
+    ExternalTransforms.JavaClassLookupPayload.Builder payloadBuilder =
+        ExternalTransforms.JavaClassLookupPayload.newBuilder();
+    payloadBuilder.setClassName(
+        "org.apache.beam.sdk.expansion.service.JavaCLassLookupTransformProviderTest$DummyTransformWithComplexTypeArray");
+
+    Schema complexTypeSchema =
+        Schema.builder()
+            .addStringField("complexTypeStrField")
+            .addInt32Field("complexTypeIntField")
+            .build();
+
+    Schema builderMethodSchema =
+        Schema.builder()
+            .addArrayField("complexTypeArrayField", FieldType.row(complexTypeSchema))
+            .build();
+
+    Row constructorRow =
+        Row.withSchema(Schema.of(Field.of("strField1", FieldType.STRING)))
+            .withFieldValue("strField1", "test_str_1")
+            .build();
+
+    payloadBuilder.setConstructorSchema(getProtoSchemaFromRow(constructorRow));
+    payloadBuilder.setConstructorPayload(getProtoPayloadFromRow(constructorRow));
+
+    List<Row> complexTypeList = new ArrayList<>();
+    complexTypeList.add(
+        Row.withSchema(complexTypeSchema)
+            .withFieldValue("complexTypeStrField", "complex_type_str_1")
+            .withFieldValue("complexTypeIntField", 123)
+            .build());
+    complexTypeList.add(
+        Row.withSchema(complexTypeSchema)
+            .withFieldValue("complexTypeStrField", "complex_type_str_2")
+            .withFieldValue("complexTypeIntField", 456)
+            .build());
+    complexTypeList.add(
+        Row.withSchema(complexTypeSchema)
+            .withFieldValue("complexTypeStrField", "complex_type_str_3")
+            .withFieldValue("complexTypeIntField", 789)
+            .build());
+
+    BuilderMethod.Builder builderMethodBuilder = BuilderMethod.newBuilder();
+    builderMethodBuilder.setName("withComplexTypeArrayField");
+
+    Row builderMethodRow =
+        Row.withSchema(builderMethodSchema)
+            .withFieldValue("complexTypeArrayField", complexTypeList)
+            .build();
+
+    builderMethodBuilder.setSchema(getProtoSchemaFromRow(builderMethodRow));
+    builderMethodBuilder.setPayload(getProtoPayloadFromRow(builderMethodRow));
+
+    payloadBuilder.addBuilderMethods(builderMethodBuilder);
+
+    ArrayList<DummyComplexType> resultList = new ArrayList<>();
+    resultList.add(new DummyComplexType("complex_type_str_1", 123));
+    resultList.add(new DummyComplexType("complex_type_str_2", 456));
+    resultList.add(new DummyComplexType("complex_type_str_3", 789));
+
+    testClassLookupExpansionRequestConstruction(
+        payloadBuilder.build(),
+        ImmutableMap.of("complexTypeArrayField", resultList.toArray(new DummyComplexType[0])));
+  }
+
+  @Test
+  public void testJavaClassLookupWithComplexListType() {
+    ExternalTransforms.JavaClassLookupPayload.Builder payloadBuilder =
+        ExternalTransforms.JavaClassLookupPayload.newBuilder();
+    payloadBuilder.setClassName(
+        "org.apache.beam.sdk.expansion.service.JavaCLassLookupTransformProviderTest$DummyTransformWithComplexTypeList");
+
+    Schema complexTypeSchema =
+        Schema.builder()
+            .addStringField("complexTypeStrField")
+            .addInt32Field("complexTypeIntField")
+            .build();
+
+    Schema builderMethodSchema =
+        Schema.builder()
+            .addIterableField("complexTypeListField", FieldType.row(complexTypeSchema))
+            .build();
+
+    Row constructorRow =
+        Row.withSchema(Schema.of(Field.of("strField1", FieldType.STRING)))
+            .withFieldValue("strField1", "test_str_1")
+            .build();
+
+    payloadBuilder.setConstructorSchema(getProtoSchemaFromRow(constructorRow));
+    payloadBuilder.setConstructorPayload(getProtoPayloadFromRow(constructorRow));
+
+    List<Row> complexTypeList = new ArrayList<>();
+    complexTypeList.add(
+        Row.withSchema(complexTypeSchema)
+            .withFieldValue("complexTypeStrField", "complex_type_str_1")
+            .withFieldValue("complexTypeIntField", 123)
+            .build());
+    complexTypeList.add(
+        Row.withSchema(complexTypeSchema)
+            .withFieldValue("complexTypeStrField", "complex_type_str_2")
+            .withFieldValue("complexTypeIntField", 456)
+            .build());
+    complexTypeList.add(
+        Row.withSchema(complexTypeSchema)
+            .withFieldValue("complexTypeStrField", "complex_type_str_3")
+            .withFieldValue("complexTypeIntField", 789)
+            .build());
+
+    BuilderMethod.Builder builderMethodBuilder = BuilderMethod.newBuilder();
+    builderMethodBuilder.setName("withComplexTypeListField");
+
+    Row builderMethodRow =
+        Row.withSchema(builderMethodSchema)
+            .withFieldValue("complexTypeListField", complexTypeList)
+            .build();
+
+    builderMethodBuilder.setSchema(getProtoSchemaFromRow(builderMethodRow));
+    builderMethodBuilder.setPayload(getProtoPayloadFromRow(builderMethodRow));
+
+    payloadBuilder.addBuilderMethods(builderMethodBuilder);
+
+    ArrayList<DummyComplexType> resultList = new ArrayList<>();
+    resultList.add(new DummyComplexType("complex_type_str_1", 123));
+    resultList.add(new DummyComplexType("complex_type_str_2", 456));
+    resultList.add(new DummyComplexType("complex_type_str_3", 789));
+
+    testClassLookupExpansionRequestConstruction(
+        payloadBuilder.build(), ImmutableMap.of("complexTypeListField", resultList));
+  }
+
+  @Test
+  public void testJavaClassLookupWithConstructorMethodAndBuilderMethods() {
+    ExternalTransforms.JavaClassLookupPayload.Builder payloadBuilder =
+        ExternalTransforms.JavaClassLookupPayload.newBuilder();
+    payloadBuilder.setClassName(
+        "org.apache.beam.sdk.expansion.service.JavaCLassLookupTransformProviderTest$DummyTransformWithConstructorMethodAndBuilderMethods");
+    payloadBuilder.setConstructorMethod("from");
+
+    Row constructorRow =
+        Row.withSchema(Schema.of(Field.of("strField1", FieldType.STRING)))
+            .withFieldValue("strField1", "test_str_1")
+            .build();
+
+    payloadBuilder.setConstructorSchema(getProtoSchemaFromRow(constructorRow));
+    payloadBuilder.setConstructorPayload(getProtoPayloadFromRow(constructorRow));
+
+    BuilderMethod.Builder builderMethodBuilder = BuilderMethod.newBuilder();
+    builderMethodBuilder.setName("withStrField2");
+
+    Row builderMethodRow =
+        Row.withSchema(Schema.of(Field.of("strField2", FieldType.STRING)))
+            .withFieldValue("strField2", "test_str_2")
+            .build();
+    builderMethodBuilder.setSchema(getProtoSchemaFromRow(builderMethodRow));
+    builderMethodBuilder.setPayload(getProtoPayloadFromRow(builderMethodRow));
+
+    payloadBuilder.addBuilderMethods(builderMethodBuilder);
+
+    builderMethodBuilder = BuilderMethod.newBuilder();
+    builderMethodBuilder.setName("withIntField1");
+
+    builderMethodRow =
+        Row.withSchema(Schema.of(Field.of("intField1", FieldType.INT32)))
+            .withFieldValue("intField1", 10)
+            .build();
+    builderMethodBuilder.setSchema(getProtoSchemaFromRow(builderMethodRow));
+    builderMethodBuilder.setPayload(getProtoPayloadFromRow(builderMethodRow));
+
+    payloadBuilder.addBuilderMethods(builderMethodBuilder);
+
+    testClassLookupExpansionRequestConstruction(
+        payloadBuilder.build(),
+        ImmutableMap.of("strField1", "test_str_1", "strField2", "test_str_2", "intField1", 10));
+  }
+
+  @Test
+  public void testJavaClassLookupWithSimplifiedBuilderMethodNames() {
+    ExternalTransforms.JavaClassLookupPayload.Builder payloadBuilder =
+        ExternalTransforms.JavaClassLookupPayload.newBuilder();
+    payloadBuilder.setClassName(
+        "org.apache.beam.sdk.expansion.service.JavaCLassLookupTransformProviderTest$DummyTransformWithConstructorMethodAndBuilderMethods");
+    payloadBuilder.setConstructorMethod("from");
+
+    Row constructorRow =
+        Row.withSchema(Schema.of(Field.of("strField1", FieldType.STRING)))
+            .withFieldValue("strField1", "test_str_1")
+            .build();
+
+    payloadBuilder.setConstructorSchema(getProtoSchemaFromRow(constructorRow));
+    payloadBuilder.setConstructorPayload(getProtoPayloadFromRow(constructorRow));
+
+    BuilderMethod.Builder builderMethodBuilder = BuilderMethod.newBuilder();
+    builderMethodBuilder.setName("strField2");
+    Row builderMethodRow =
+        Row.withSchema(Schema.of(Field.of("strField2", FieldType.STRING)))
+            .withFieldValue("strField2", "test_str_2")
+            .build();
+    builderMethodBuilder.setSchema(getProtoSchemaFromRow(builderMethodRow));
+    builderMethodBuilder.setPayload(getProtoPayloadFromRow(builderMethodRow));
+
+    payloadBuilder.addBuilderMethods(builderMethodBuilder);
+
+    builderMethodBuilder = BuilderMethod.newBuilder();
+    builderMethodBuilder.setName("intField1");
+    builderMethodRow =
+        Row.withSchema(Schema.of(Field.of("intField1", FieldType.INT32)))
+            .withFieldValue("intField1", 10)
+            .build();
+    builderMethodBuilder.setSchema(getProtoSchemaFromRow(builderMethodRow));
+    builderMethodBuilder.setPayload(getProtoPayloadFromRow(builderMethodRow));
+
+    payloadBuilder.addBuilderMethods(builderMethodBuilder);
+
+    testClassLookupExpansionRequestConstruction(
+        payloadBuilder.build(),
+        ImmutableMap.of("strField1", "test_str_1", "strField2", "test_str_2", "intField1", 10));
+  }
+
+  @Test
+  public void testJavaClassLookupWithAnnotations() {
+    ExternalTransforms.JavaClassLookupPayload.Builder payloadBuilder =
+        ExternalTransforms.JavaClassLookupPayload.newBuilder();
+    payloadBuilder.setClassName(
+        "org.apache.beam.sdk.expansion.service.JavaCLassLookupTransformProviderTest$DummyTransformWithMultiLanguageAnnotations");
+    payloadBuilder.setConstructorMethod("create_transform");
+    Row constructorRow =
+        Row.withSchema(Schema.of(Field.of("strField1", FieldType.STRING)))
+            .withFieldValue("strField1", "test_str_1")
+            .build();
+
+    payloadBuilder.setConstructorSchema(getProtoSchemaFromRow(constructorRow));
+    payloadBuilder.setConstructorPayload(getProtoPayloadFromRow(constructorRow));
+
+    BuilderMethod.Builder builderMethodBuilder = BuilderMethod.newBuilder();
+    builderMethodBuilder.setName("abc");
+    Row builderMethodRow =
+        Row.withSchema(Schema.of(Field.of("strField2", FieldType.STRING)))
+            .withFieldValue("strField2", "test_str_2")
+            .build();
+    builderMethodBuilder.setSchema(getProtoSchemaFromRow(builderMethodRow));
+    builderMethodBuilder.setPayload(getProtoPayloadFromRow(builderMethodRow));
+
+    payloadBuilder.addBuilderMethods(builderMethodBuilder);
+
+    builderMethodBuilder = BuilderMethod.newBuilder();
+    builderMethodBuilder.setName("xyz");
+    builderMethodRow =
+        Row.withSchema(Schema.of(Field.of("intField1", FieldType.INT32)))
+            .withFieldValue("intField1", 10)
+            .build();
+    builderMethodBuilder.setSchema(getProtoSchemaFromRow(builderMethodRow));
+    builderMethodBuilder.setPayload(getProtoPayloadFromRow(builderMethodRow));
+
+    payloadBuilder.addBuilderMethods(builderMethodBuilder);
+
+    testClassLookupExpansionRequestConstruction(
+        payloadBuilder.build(),
+        ImmutableMap.of("strField1", "test_str_1", "strField2", "test_str_2", "intField1", 10));
+  }
+
+  @Test
+  public void testJavaClassLookupClassNotAvailable() {
+    ExternalTransforms.JavaClassLookupPayload.Builder payloadBuilder =
+        ExternalTransforms.JavaClassLookupPayload.newBuilder();
+    payloadBuilder.setClassName(
+        "org.apache.beam.sdk.expansion.service.JavaCLassLookupTransformProviderTest$UnavailableClass");
+    Row constructorRow =
+        Row.withSchema(Schema.of(Field.of("strField1", FieldType.STRING)))
+            .withFieldValue("strField1", "test_str_1")
+            .build();
+
+    payloadBuilder.setConstructorSchema(getProtoSchemaFromRow(constructorRow));
+    payloadBuilder.setConstructorPayload(getProtoPayloadFromRow(constructorRow));
+
+    RuntimeException thrown =
+        assertThrows(
+            RuntimeException.class,
+            () ->
+                testClassLookupExpansionRequestConstruction(
+                    payloadBuilder.build(), ImmutableMap.of()));
+    assertTrue(thrown.getMessage().contains("does not enable"));
+  }
+
+  @Test
+  public void testJavaClassLookupIncorrectConstructionParameter() {
+    ExternalTransforms.JavaClassLookupPayload.Builder payloadBuilder =
+        ExternalTransforms.JavaClassLookupPayload.newBuilder();
+    payloadBuilder.setClassName(
+        "org.apache.beam.sdk.expansion.service.JavaCLassLookupTransformProviderTest$DummyTransformWithConstructor");
+    Row constructorRow =
+        Row.withSchema(Schema.of(Field.of("incorrectField", FieldType.STRING)))
+            .withFieldValue("incorrectField", "test_str_1")
+            .build();
+
+    payloadBuilder.setConstructorSchema(getProtoSchemaFromRow(constructorRow));
+    payloadBuilder.setConstructorPayload(getProtoPayloadFromRow(constructorRow));
+
+    RuntimeException thrown =
+        assertThrows(
+            RuntimeException.class,
+            () ->
+                testClassLookupExpansionRequestConstruction(
+                    payloadBuilder.build(), ImmutableMap.of()));
+    assertTrue(thrown.getMessage().contains("Expected to find a single mapping constructor"));
+  }
+
+  @Test
+  public void testJavaClassLookupIncorrectBuilderMethodParameter() {
+    ExternalTransforms.JavaClassLookupPayload.Builder payloadBuilder =
+        ExternalTransforms.JavaClassLookupPayload.newBuilder();
+    payloadBuilder.setClassName(
+        "org.apache.beam.sdk.expansion.service.JavaCLassLookupTransformProviderTest$DummyTransformWithConstructorAndBuilderMethods");
+    Row constructorRow =
+        Row.withSchema(Schema.of(Field.of("strField1", FieldType.STRING)))
+            .withFieldValue("strField1", "test_str_1")
+            .build();
+
+    payloadBuilder.setConstructorSchema(getProtoSchemaFromRow(constructorRow));
+    payloadBuilder.setConstructorPayload(getProtoPayloadFromRow(constructorRow));
+
+    BuilderMethod.Builder builderMethodBuilder = BuilderMethod.newBuilder();
+    builderMethodBuilder.setName("withStrField2");
+    Row builderMethodRow =
+        Row.withSchema(Schema.of(Field.of("incorrectParam", FieldType.STRING)))
+            .withFieldValue("incorrectParam", "test_str_2")
+            .build();
+    builderMethodBuilder.setSchema(getProtoSchemaFromRow(builderMethodRow));
+    builderMethodBuilder.setPayload(getProtoPayloadFromRow(builderMethodRow));
+
+    payloadBuilder.addBuilderMethods(builderMethodBuilder);
+
+    RuntimeException thrown =
+        assertThrows(
+            RuntimeException.class,
+            () ->
+                testClassLookupExpansionRequestConstruction(
+                    payloadBuilder.build(), ImmutableMap.of()));
+    assertTrue(thrown.getMessage().contains("Expected to find exactly one matching method"));
+  }
+
+  private SchemaApi.Schema getProtoSchemaFromRow(Row row) {
+    return SchemaTranslation.schemaToProto(row.getSchema(), true);
+  }
+
+  private ByteString getProtoPayloadFromRow(Row row) {
+    ByteString.Output outputStream = ByteString.newOutput();
+    try {
+      SchemaCoder.of(row.getSchema()).encode(row, outputStream);
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+
+    return outputStream.toByteString();
+  }
+}
diff --git a/sdks/java/expansion-service/src/test/resources/test_allowlist.yaml b/sdks/java/expansion-service/src/test/resources/test_allowlist.yaml
new file mode 100644
index 0000000..ad11523
--- /dev/null
+++ b/sdks/java/expansion-service/src/test/resources/test_allowlist.yaml
@@ -0,0 +1,67 @@
+#
+# 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.
+#
+
+version: v1
+allowedClasses:
+- className: org.apache.beam.sdk.expansion.service.JavaCLassLookupTransformProviderTest$DummyTransformWithConstructor
+- className: org.apache.beam.sdk.expansion.service.JavaCLassLookupTransformProviderTest$DummyTransformWithConstructorMethod
+  allowedConstructorMethods:
+  - from
+- className: org.apache.beam.sdk.expansion.service.JavaCLassLookupTransformProviderTest$DummyTransformWithConstructorAndBuilderMethods
+  allowedBuilderMethods:
+  - withStrField2
+  - withIntField1
+- className: org.apache.beam.sdk.expansion.service.JavaCLassLookupTransformProviderTest$DummyTransformWithMultiArgumentConstructor
+- className: org.apache.beam.sdk.expansion.service.JavaCLassLookupTransformProviderTest$DummyTransformWithMultiArgumentBuilderMethod
+  allowedBuilderMethods:
+  - withFields
+- className: org.apache.beam.sdk.expansion.service.JavaCLassLookupTransformProviderTest$DummyTransformWithConstructorMethodAndBuilderMethods
+  allowedConstructorMethods:
+  - from
+  allowedBuilderMethods:
+  - withStrField2
+  - withIntField1
+  - strField2
+  - intField1
+- className: org.apache.beam.sdk.expansion.service.JavaCLassLookupTransformProviderTest$DummyTransformWithMultiLanguageAnnotations
+  allowedConstructorMethods:
+  - create_transform
+  allowedBuilderMethods:
+  - abc
+  - xyz
+- className: org.apache.beam.sdk.expansion.service.JavaCLassLookupTransformProviderTest$DummyTransformWithWrapperTypes
+  allowedBuilderMethods:
+  - withDoubleWrapperField
+- className: org.apache.beam.sdk.expansion.service.JavaCLassLookupTransformProviderTest$DummyTransformWithComplexTypes
+  allowedBuilderMethods:
+  - withComplexTypeField
+- className: org.apache.beam.sdk.expansion.service.JavaCLassLookupTransformProviderTest$DummyTransformWithArray
+  allowedBuilderMethods:
+  - withStrArrayField
+- className: org.apache.beam.sdk.expansion.service.JavaCLassLookupTransformProviderTest$DummyTransformWithList
+  allowedBuilderMethods:
+  - withStrListField
+- className: org.apache.beam.sdk.expansion.service.JavaCLassLookupTransformProviderTest$DummyTransformWithComplexTypeArray
+  allowedBuilderMethods:
+  - withComplexTypeArrayField
+- className: org.apache.beam.sdk.expansion.service.JavaCLassLookupTransformProviderTest$DummyTransformWithComplexTypeList
+  allowedBuilderMethods:
+  - withComplexTypeListField
+
+
diff --git a/sdks/java/extensions/arrow/build.gradle b/sdks/java/extensions/arrow/build.gradle
index b60b8d7..9cf7a48 100644
--- a/sdks/java/extensions/arrow/build.gradle
+++ b/sdks/java/extensions/arrow/build.gradle
@@ -21,8 +21,6 @@
 
 description = "Apache Beam :: SDKs :: Java :: Extensions :: Arrow"
 
-def arrow_version = "4.0.0"
-
 dependencies {
     compile library.java.vendored_guava_26_0_jre
     compile project(path: ":sdks:java:core", configuration: "shadow")
diff --git a/sdks/java/extensions/sql/build.gradle b/sdks/java/extensions/sql/build.gradle
index d3e1eb0..c7a73db 100644
--- a/sdks/java/extensions/sql/build.gradle
+++ b/sdks/java/extensions/sql/build.gradle
@@ -56,7 +56,7 @@
   javacc "net.java.dev.javacc:javacc:4.0"
   fmppTask "com.googlecode.fmpp-maven-plugin:fmpp-maven-plugin:1.0"
   fmppTask "org.freemarker:freemarker:2.3.31"
-  fmppTemplates library.java.vendored_calcite_1_20_0
+  fmppTemplates library.java.vendored_calcite_1_26_0
   compile project(path: ":sdks:java:core", configuration: "shadow")
   compile project(":sdks:java:extensions:join-library")
   permitUnusedDeclared project(":sdks:java:extensions:join-library") // BEAM-11761
@@ -66,7 +66,7 @@
   compile library.java.commons_csv
   compile library.java.jackson_databind
   compile library.java.joda_time
-  compile library.java.vendored_calcite_1_20_0
+  compile library.java.vendored_calcite_1_26_0
   compile "com.alibaba:fastjson:1.2.69"
   compile "org.codehaus.janino:janino:3.0.11"
   compile "org.codehaus.janino:commons-compiler:3.0.11"
@@ -83,7 +83,7 @@
   provided library.java.jackson_dataformat_xml
   provided library.java.hadoop_client
   provided library.java.kafka_clients
-  testCompile library.java.vendored_calcite_1_20_0
+  testCompile library.java.vendored_calcite_1_26_0
   testCompile library.java.vendored_guava_26_0_jre
   testCompile library.java.junit
   testCompile library.java.quickcheck_core
@@ -123,11 +123,11 @@
   into "${project.buildDir}/templates-fmpp"
   filter{
     line ->
-      line.replace('import org.apache.calcite.', 'import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.')
+      line.replace('import org.apache.calcite.', 'import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.')
   }
   filter{
     line ->
-      line.replace('import static org.apache.calcite.', 'import static org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.')
+      line.replace('import static org.apache.calcite.', 'import static org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.')
   }
 }
 
diff --git a/sdks/java/extensions/sql/datacatalog/src/main/java/org/apache/beam/sdk/extensions/sql/example/BeamSqlDataCatalogExample.java b/sdks/java/extensions/sql/datacatalog/src/main/java/org/apache/beam/sdk/extensions/sql/example/BeamSqlDataCatalogExample.java
index fd56ff4..9e73ab2 100644
--- a/sdks/java/extensions/sql/datacatalog/src/main/java/org/apache/beam/sdk/extensions/sql/example/BeamSqlDataCatalogExample.java
+++ b/sdks/java/extensions/sql/datacatalog/src/main/java/org/apache/beam/sdk/extensions/sql/example/BeamSqlDataCatalogExample.java
@@ -31,7 +31,7 @@
 import org.apache.beam.sdk.transforms.MapElements;
 import org.apache.beam.sdk.values.Row;
 import org.apache.beam.sdk.values.TypeDescriptor;
-import org.apache.beam.vendor.calcite.v1_20_0.com.google.common.base.Strings;
+import org.apache.beam.vendor.calcite.v1_26_0.com.google.common.base.Strings;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
diff --git a/sdks/java/extensions/sql/datacatalog/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/datacatalog/DataCatalogTableProvider.java b/sdks/java/extensions/sql/datacatalog/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/datacatalog/DataCatalogTableProvider.java
index 59b968e..cd81ce0 100644
--- a/sdks/java/extensions/sql/datacatalog/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/datacatalog/DataCatalogTableProvider.java
+++ b/sdks/java/extensions/sql/datacatalog/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/datacatalog/DataCatalogTableProvider.java
@@ -46,7 +46,7 @@
 import org.apache.beam.sdk.extensions.sql.meta.provider.pubsub.PubsubTableProvider;
 import org.apache.beam.sdk.extensions.sql.meta.provider.text.TextTableProvider;
 import org.apache.beam.sdk.schemas.Schema;
-import org.apache.beam.vendor.calcite.v1_20_0.com.google.common.collect.ImmutableList;
+import org.apache.beam.vendor.calcite.v1_26_0.com.google.common.collect.ImmutableList;
 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.ImmutableSet;
 import org.checkerframework.checker.nullness.qual.Nullable;
diff --git a/sdks/java/extensions/sql/datacatalog/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/datacatalog/SchemaUtils.java b/sdks/java/extensions/sql/datacatalog/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/datacatalog/SchemaUtils.java
index 1ee0847..7c729c4 100644
--- a/sdks/java/extensions/sql/datacatalog/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/datacatalog/SchemaUtils.java
+++ b/sdks/java/extensions/sql/datacatalog/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/datacatalog/SchemaUtils.java
@@ -28,8 +28,8 @@
 import org.apache.beam.sdk.schemas.Schema.Field;
 import org.apache.beam.sdk.schemas.Schema.FieldType;
 import org.apache.beam.sdk.schemas.logicaltypes.SqlTypes;
-import org.apache.beam.vendor.calcite.v1_20_0.com.google.common.base.Strings;
-import org.apache.beam.vendor.calcite.v1_20_0.com.google.common.collect.ImmutableMap;
+import org.apache.beam.vendor.calcite.v1_26_0.com.google.common.base.Strings;
+import org.apache.beam.vendor.calcite.v1_26_0.com.google.common.collect.ImmutableMap;
 
 @Experimental(Kind.SCHEMAS)
 @SuppressWarnings({
diff --git a/sdks/java/extensions/sql/jdbc/src/test/java/org/apache/beam/sdk/extensions/sql/jdbc/BeamSqlLineTest.java b/sdks/java/extensions/sql/jdbc/src/test/java/org/apache/beam/sdk/extensions/sql/jdbc/BeamSqlLineTest.java
index 48f7f95..e308fa5 100644
--- a/sdks/java/extensions/sql/jdbc/src/test/java/org/apache/beam/sdk/extensions/sql/jdbc/BeamSqlLineTest.java
+++ b/sdks/java/extensions/sql/jdbc/src/test/java/org/apache/beam/sdk/extensions/sql/jdbc/BeamSqlLineTest.java
@@ -21,12 +21,13 @@
 import static org.apache.beam.sdk.extensions.sql.jdbc.BeamSqlLineTestingUtils.toLines;
 import static org.hamcrest.CoreMatchers.everyItem;
 import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.is;
+import static org.hamcrest.Matchers.oneOf;
 
 import java.io.ByteArrayOutputStream;
 import java.io.File;
 import java.util.Arrays;
 import java.util.List;
-import org.hamcrest.collection.IsIn;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.TemporaryFolder;
@@ -94,7 +95,7 @@
     List<List<String>> lines = toLines(byteArrayOutputStream);
     assertThat(
         Arrays.asList(Arrays.asList("3", "hello", "2018-05-28")),
-        everyItem(IsIn.isOneOf(lines.toArray())));
+        everyItem(is(oneOf(lines.toArray()))));
   }
 
   @Test
@@ -104,7 +105,7 @@
         buildArgs(
             "CREATE EXTERNAL TABLE table_test (col_a VARCHAR, col_b VARCHAR, "
                 + "col_c VARCHAR, col_x TINYINT, col_y INT, col_z BIGINT) TYPE 'test';",
-            "INSERT INTO table_test VALUES ('a', 'b', 'c', 1, 2, 3);",
+            "INSERT INTO table_test VALUES ('a', 'b', 'c', 1, 2, CAST(3 AS BIGINT));",
             "SELECT * FROM table_test;");
 
     BeamSqlLine.runSqlLine(args, null, byteArrayOutputStream, null);
@@ -114,7 +115,7 @@
         Arrays.asList(
             Arrays.asList("col_a", "col_b", "col_c", "col_x", "col_y", "col_z"),
             Arrays.asList("a", "b", "c", "1", "2", "3")),
-        everyItem(IsIn.isOneOf(lines.toArray())));
+        everyItem(is(oneOf(lines.toArray()))));
   }
 
   @Test
@@ -129,8 +130,7 @@
     BeamSqlLine.runSqlLine(args, null, byteArrayOutputStream, null);
 
     List<List<String>> lines = toLines(byteArrayOutputStream);
-    assertThat(
-        Arrays.asList(Arrays.asList("3", "hello")), everyItem(IsIn.isOneOf(lines.toArray())));
+    assertThat(Arrays.asList(Arrays.asList("3", "hello")), everyItem(is(oneOf(lines.toArray()))));
   }
 
   @Test
@@ -149,7 +149,7 @@
     List<List<String>> lines = toLines(byteArrayOutputStream);
     assertThat(
         Arrays.asList(Arrays.asList("3", "2"), Arrays.asList("4", "1")),
-        everyItem(IsIn.isOneOf(lines.toArray())));
+        everyItem(is(oneOf(lines.toArray()))));
   }
 
   @Test
@@ -169,7 +169,7 @@
     assertThat(
         Arrays.asList(
             Arrays.asList("2018-07-01 21:26:06", "1"), Arrays.asList("2018-07-01 21:26:07", "1")),
-        everyItem(IsIn.isOneOf(lines.toArray())));
+        everyItem(is(oneOf(lines.toArray()))));
   }
 
   @Test
@@ -195,6 +195,6 @@
             Arrays.asList("2018-07-01 21:26:09", "2"),
             Arrays.asList("2018-07-01 21:26:10", "2"),
             Arrays.asList("2018-07-01 21:26:11", "1")),
-        everyItem(IsIn.isOneOf(lines.toArray())));
+        everyItem(is(oneOf(lines.toArray()))));
   }
 }
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 1194baa..d2808cd 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
@@ -46,9 +46,9 @@
 import org.apache.beam.sdk.transforms.ParDo;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.Row;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptRule;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.tools.RuleSet;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.tools.RuleSets;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptRule;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.tools.RuleSet;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.tools.RuleSets;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
 import org.junit.Before;
 import org.junit.BeforeClass;
diff --git a/sdks/java/extensions/sql/src/main/codegen/config.fmpp b/sdks/java/extensions/sql/src/main/codegen/config.fmpp
index 3b2c73e..ca59015 100644
--- a/sdks/java/extensions/sql/src/main/codegen/config.fmpp
+++ b/sdks/java/extensions/sql/src/main/codegen/config.fmpp
@@ -21,10 +21,10 @@
 
       # List of import statements.
       imports: [
-        "org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.schema.ColumnStrategy"
-        "org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlCreate"
-        "org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlDrop"
-        "org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.type.SqlTypeName"
+        "org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.schema.ColumnStrategy"
+        "org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlCreate"
+        "org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlDrop"
+        "org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.type.SqlTypeName"
         "org.apache.beam.sdk.extensions.sql.impl.parser.SqlCreateExternalTable"
         "org.apache.beam.sdk.extensions.sql.impl.parser.SqlCreateFunction"
         "org.apache.beam.sdk.extensions.sql.impl.parser.SqlDdlNodes"
@@ -46,6 +46,7 @@
       # List of keywords from "keywords" section that are not reserved.
       nonReservedKeywords: [
         "A"
+        "ABSENT"
         "ABSOLUTE"
         "ACTION"
         "ADA"
@@ -68,11 +69,11 @@
         "CATALOG_NAME"
         "CENTURY"
         "CHAIN"
+        "CHARACTERISTICS"
+        "CHARACTERS"
         "CHARACTER_SET_CATALOG"
         "CHARACTER_SET_NAME"
         "CHARACTER_SET_SCHEMA"
-        "CHARACTERISTICS"
-        "CHARACTERS"
         "CLASS_ORIGIN"
         "COBOL"
         "COLLATION"
@@ -83,13 +84,14 @@
         "COMMAND_FUNCTION"
         "COMMAND_FUNCTION_CODE"
         "COMMITTED"
+        "CONDITIONAL"
         "CONDITION_NUMBER"
         "CONNECTION"
         "CONNECTION_NAME"
         "CONSTRAINT_CATALOG"
         "CONSTRAINT_NAME"
-        "CONSTRAINT_SCHEMA"
         "CONSTRAINTS"
+        "CONSTRAINT_SCHEMA"
         "CONSTRUCTOR"
         "CONTINUE"
         "CURSOR_NAME"
@@ -97,6 +99,7 @@
         "DATABASE"
         "DATETIME_INTERVAL_CODE"
         "DATETIME_INTERVAL_PRECISION"
+        "DAYS"
         "DECADE"
         "DEFAULTS"
         "DEFERRABLE"
@@ -116,13 +119,16 @@
         "DOY"
         "DYNAMIC_FUNCTION"
         "DYNAMIC_FUNCTION_CODE"
+        "ENCODING"
         "EPOCH"
+        "ERROR"
         "EXCEPTION"
         "EXCLUDE"
         "EXCLUDING"
         "FINAL"
         "FIRST"
         "FOLLOWING"
+        "FORMAT"
         "FORTRAN"
         "FOUND"
         "FRAC_SECOND"
@@ -134,6 +140,9 @@
         "GOTO"
         "GRANTED"
         "HIERARCHY"
+        "HOP"
+        "HOURS"
+        "IGNORE"
         "IMMEDIATE"
         "IMMEDIATELY"
         "IMPLEMENTATION"
@@ -145,8 +154,8 @@
         "INSTANTIABLE"
         "INVOKER"
         "ISODOW"
-        "ISOYEAR"
         "ISOLATION"
+        "ISOYEAR"
         "JAVA"
         "JSON"
         "K"
@@ -163,13 +172,15 @@
         "MAP"
         "MATCHED"
         "MAXVALUE"
-        "MICROSECOND"
         "MESSAGE_LENGTH"
         "MESSAGE_OCTET_LENGTH"
         "MESSAGE_TEXT"
-        "MILLISECOND"
+        "MICROSECOND"
         "MILLENNIUM"
+        "MILLISECOND"
+        "MINUTES"
         "MINVALUE"
+        "MONTHS"
         "MORE_"
         "MUMPS"
         "NAME"
@@ -198,6 +209,7 @@
         "PARAMETER_SPECIFIC_SCHEMA"
         "PARTIAL"
         "PASCAL"
+        "PASSING"
         "PASSTHROUGH"
         "PAST"
         "PATH"
@@ -214,24 +226,28 @@
         "RELATIVE"
         "REPEATABLE"
         "REPLACE"
+        "RESPECT"
         "RESTART"
         "RESTRICT"
         "RETURNED_CARDINALITY"
         "RETURNED_LENGTH"
         "RETURNED_OCTET_LENGTH"
         "RETURNED_SQLSTATE"
+        "RETURNING"
         "ROLE"
         "ROUTINE"
         "ROUTINE_CATALOG"
         "ROUTINE_NAME"
         "ROUTINE_SCHEMA"
         "ROW_COUNT"
+        "SCALAR"
         "SCALE"
         "SCHEMA"
         "SCHEMA_NAME"
         "SCOPE_CATALOGS"
         "SCOPE_NAME"
         "SCOPE_SCHEMA"
+        "SECONDS"
         "SECTION"
         "SECURITY"
         "SELF"
@@ -272,8 +288,8 @@
         "SQL_INTERVAL_YEAR"
         "SQL_INTERVAL_YEAR_TO_MONTH"
         "SQL_LONGVARBINARY"
-        "SQL_LONGVARNCHAR"
         "SQL_LONGVARCHAR"
+        "SQL_LONGVARNCHAR"
         "SQL_NCHAR"
         "SQL_NCLOB"
         "SQL_NUMERIC"
@@ -316,9 +332,11 @@
         "TRIGGER_CATALOG"
         "TRIGGER_NAME"
         "TRIGGER_SCHEMA"
+        "TUMBLE"
         "TYPE"
         "UNBOUNDED"
         "UNCOMMITTED"
+        "UNCONDITIONAL"
         "UNDER"
         "UNNAMED"
         "USAGE"
@@ -326,13 +344,17 @@
         "USER_DEFINED_TYPE_CODE"
         "USER_DEFINED_TYPE_NAME"
         "USER_DEFINED_TYPE_SCHEMA"
+        "UTF16"
+        "UTF32"
+        "UTF8"
         "VERSION"
         "VIEW"
         "WEEK"
-        "WRAPPER"
         "WORK"
+        "WRAPPER"
         "WRITE"
         "XML"
+        "YEARS"
         "ZONE"
 
         # added in Beam
@@ -344,29 +366,49 @@
         "TBLPROPERTIES"
       ]
 
+      # List of non-reserved keywords to add;
+      # items in this list become non-reserved
+      nonReservedKeywordsToAdd: [
+      ]
+
+      # List of non-reserved keywords to remove;
+      # items in this list become reserved
+      nonReservedKeywordsToRemove: [
+      ]
+
       # List of additional join types. Each is a method with no arguments.
       # Example: LeftSemiJoin()
       joinTypes: [
       ]
 
       # List of methods for parsing custom SQL statements.
+      # Return type of method implementation should be 'SqlNode'.
+      # Example: SqlShowDatabases(), SqlShowTables().
       statementParserMethods: [
         "SqlSetOptionBeam(Span.of(), null)"
-        "SqlCreateExternalTable()"
-        "SqlCreateFunction()"
       ]
 
       # List of methods for parsing custom literals.
+      # Return type of method implementation should be "SqlNode".
       # Example: ParseJsonLiteral().
       literalParserMethods: [
       ]
 
       # List of methods for parsing custom data types.
+      # Return type of method implementation should be "SqlTypeNameSpec".
+      # Example: SqlParseTimeStampZ().
       dataTypeParserMethods: [
       ]
 
+      # List of methods for parsing builtin function calls.
+      # Return type of method implementation should be "SqlNode".
+      # Example: DateFunctionCall().
+      builtinFunctionCallMethods: [
+      ]
+
       # List of methods for parsing extensions to "ALTER <scope>" calls.
       # Each must accept arguments "(SqlParserPos pos, String scope)".
+      # Example: "SqlUploadJarNode"
       alterStatementParserMethods: [
         "SqlSetOptionBeam"
       ]
@@ -374,7 +416,9 @@
       # List of methods for parsing extensions to "CREATE [OR REPLACE]" calls.
       # Each must accept arguments "(SqlParserPos pos, boolean replace)".
       createStatementParserMethods: [
-        SqlCreateTableNotSupportedMessage
+        "SqlCreateExternalTable"
+        "SqlCreateFunction"
+        "SqlCreateTableNotSupportedMessage"
       ]
 
       # List of methods for parsing extensions to "DROP" calls.
@@ -383,6 +427,14 @@
         "SqlDropTable"
       ]
 
+      # Binary operators tokens
+      binaryOperatorsTokens: [
+      ]
+
+      # Binary operators initialization
+      extraBinaryExpressions: [
+      ]
+
       # List of files in @includes directory that have parser method
       # implementations for parsing custom SQL statements, literals or types
       # given as part of "statementParserMethods", "literalParserMethods" or
@@ -391,16 +443,14 @@
         "parserImpls.ftl"
       ]
 
-      # List of methods for parsing builtin function calls.
-      builtinFunctionCallMethods: [
-      ]
-
+      includePosixOperators: false
       includeCompoundIdentifier: true
       includeBraces: true
       includeAdditionalDeclarations: false
 
     }
 }
+
 freemarkerLinks: {
     includes: includes/
 }
diff --git a/sdks/java/extensions/sql/src/main/codegen/includes/parserImpls.ftl b/sdks/java/extensions/sql/src/main/codegen/includes/parserImpls.ftl
index f33f281..8983cc9 100644
--- a/sdks/java/extensions/sql/src/main/codegen/includes/parserImpls.ftl
+++ b/sdks/java/extensions/sql/src/main/codegen/includes/parserImpls.ftl
@@ -146,10 +146,8 @@
  *   ( LOCATION location_string )?
  *   ( TBLPROPERTIES tbl_properties )?
  */
-SqlCreate SqlCreateExternalTable() :
+SqlCreate SqlCreateExternalTable(Span s, boolean replace) :
 {
-    final Span s = Span.of();
-    final boolean replace = false;
     final boolean ifNotExists;
     final SqlIdentifier id;
     List<Schema.Field> fieldList = null;
@@ -160,7 +158,7 @@
 }
 {
 
-    <CREATE> <EXTERNAL> <TABLE> {
+    <EXTERNAL> <TABLE> {
         s.add(this);
     }
 
@@ -191,15 +189,13 @@
     }
 }
 
-SqlCreate SqlCreateFunction() :
+SqlCreate SqlCreateFunction(Span s, boolean replace) :
 {
-    final Span s = Span.of();
     boolean isAggregate = false;
     final SqlIdentifier name;
     final SqlNode jarName;
 }
 {
-    <CREATE>
     (
         <AGGREGATE> {
             isAggregate = true;
@@ -215,6 +211,7 @@
         return
             new SqlCreateFunction(
                 s.end(this),
+                replace,
                 name,
                 jarName,
                 isAggregate);
@@ -324,7 +321,7 @@
 Schema.FieldType SimpleType() :
 {
     final Span s = Span.of();
-    final SqlTypeName simpleTypeName;
+    final SqlTypeNameSpec simpleTypeName;
 }
 {
     simpleTypeName = SqlTypeName(s)
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/SqlTransform.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/SqlTransform.java
index a73cfac..5ddaca5 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/SqlTransform.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/SqlTransform.java
@@ -44,8 +44,8 @@
 import org.apache.beam.sdk.values.PValue;
 import org.apache.beam.sdk.values.Row;
 import org.apache.beam.sdk.values.TupleTag;
-import org.apache.beam.vendor.calcite.v1_20_0.com.google.common.collect.ImmutableList;
-import org.apache.beam.vendor.calcite.v1_20_0.com.google.common.collect.ImmutableMap;
+import org.apache.beam.vendor.calcite.v1_26_0.com.google.common.collect.ImmutableList;
+import org.apache.beam.vendor.calcite.v1_26_0.com.google.common.collect.ImmutableMap;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.MoreObjects;
 import org.checkerframework.checker.nullness.qual.Nullable;
 
@@ -107,12 +107,7 @@
  * }</pre>
  */
 @AutoValue
-@Experimental
 @AutoValue.CopyAnnotations
-@SuppressWarnings({
-  "rawtypes", // TODO(https://issues.apache.org/jira/browse/BEAM-10556)
-  "nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
-})
 public abstract class SqlTransform extends PTransform<PInput, PCollection<Row>> {
   static final String PCOLLECTION_NAME = "PCOLLECTION";
 
@@ -122,9 +117,9 @@
 
   abstract QueryParameters queryParameters();
 
-  abstract List<UdfDefinition> udfDefinitions();
+  abstract @Experimental List<UdfDefinition> udfDefinitions();
 
-  abstract List<UdafDefinition> udafDefinitions();
+  abstract @Experimental List<UdafDefinition> udafDefinitions();
 
   abstract boolean autoLoading();
 
@@ -154,8 +149,9 @@
 
     tableProviderMap().forEach(sqlEnvBuilder::addSchema);
 
-    if (defaultTableProvider() != null) {
-      sqlEnvBuilder.setCurrentSchema(defaultTableProvider());
+    final @Nullable String defaultTableProvider = defaultTableProvider();
+    if (defaultTableProvider != null) {
+      sqlEnvBuilder.setCurrentSchema(defaultTableProvider);
     }
 
     sqlEnvBuilder.setQueryPlannerClassName(
@@ -223,7 +219,7 @@
    * <p>Any available implementation of {@link QueryPlanner} can be used as the query planner in
    * {@link SqlTransform}. An implementation can be specified globally for the entire pipeline with
    * {@link BeamSqlPipelineOptions#getPlannerName()}. The global planner can be overridden
-   * per-transform with {@link #withQueryPlannerClass(Class<? extends QueryPlanner>)}.
+   * per-transform with {@link #withQueryPlannerClass(Class)}.
    */
   public static SqlTransform query(String queryString) {
     return builder().setQueryString(queryString).build();
@@ -239,6 +235,7 @@
     return withTableProvider(name, tableProvider).toBuilder().setDefaultTableProvider(name).build();
   }
 
+  @Experimental
   public SqlTransform withQueryPlannerClass(Class<? extends QueryPlanner> clazz) {
     return toBuilder().setQueryPlannerClassName(clazz.getName()).build();
   }
@@ -265,6 +262,7 @@
    *
    * <p>Refer to {@link BeamSqlUdf} for more about how to implement a UDF in BeamSql.
    */
+  @Experimental
   public SqlTransform registerUdf(String functionName, Class<? extends BeamSqlUdf> clazz) {
     return registerUdf(functionName, clazz, BeamSqlUdf.UDF_METHOD);
   }
@@ -273,6 +271,7 @@
    * Register {@link SerializableFunction} as a UDF function used in this query. Note, {@link
    * SerializableFunction} must have a constructor without arguments.
    */
+  @Experimental
   public SqlTransform registerUdf(String functionName, SerializableFunction sfn) {
     return registerUdf(functionName, sfn.getClass(), "apply");
   }
@@ -288,6 +287,7 @@
   }
 
   /** register a {@link Combine.CombineFn} as UDAF function used in this query. */
+  @Experimental
   public SqlTransform registerUdaf(String functionName, Combine.CombineFn combineFn) {
     ImmutableList<UdafDefinition> newUdafs =
         ImmutableList.<UdafDefinition>builder()
@@ -311,6 +311,7 @@
   }
 
   @AutoValue.Builder
+  @AutoValue.CopyAnnotations
   abstract static class Builder {
     abstract Builder setQueryString(String queryString);
 
@@ -318,9 +319,9 @@
 
     abstract Builder setDdlStrings(List<String> ddlStrings);
 
-    abstract Builder setUdfDefinitions(List<UdfDefinition> udfDefinitions);
+    abstract @Experimental Builder setUdfDefinitions(List<UdfDefinition> udfDefinitions);
 
-    abstract Builder setUdafDefinitions(List<UdafDefinition> udafDefinitions);
+    abstract @Experimental Builder setUdafDefinitions(List<UdafDefinition> udafDefinitions);
 
     abstract Builder setAutoLoading(boolean autoLoading);
 
@@ -335,7 +336,7 @@
 
   @AutoValue
   @AutoValue.CopyAnnotations
-  @SuppressWarnings({"rawtypes"})
+  @Experimental
   abstract static class UdfDefinition {
     abstract String udfName();
 
@@ -350,7 +351,7 @@
 
   @AutoValue
   @AutoValue.CopyAnnotations
-  @SuppressWarnings({"rawtypes"})
+  @Experimental
   abstract static class UdafDefinition {
     abstract String udafName();
 
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/TableNameExtractionUtils.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/TableNameExtractionUtils.java
index c6b1774..5b1c416 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/TableNameExtractionUtils.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/TableNameExtractionUtils.java
@@ -23,13 +23,13 @@
 import java.util.Collections;
 import java.util.List;
 import org.apache.beam.sdk.extensions.sql.impl.TableName;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlAsOperator;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlCall;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlIdentifier;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlJoin;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlSelect;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlSetOperator;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlAsOperator;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlCall;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlIdentifier;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlJoin;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlSelect;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlSetOperator;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
 
 /**
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/BeamCalciteSchema.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/BeamCalciteSchema.java
index d3d376d..accbf63 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/BeamCalciteSchema.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/BeamCalciteSchema.java
@@ -24,13 +24,13 @@
 import java.util.Set;
 import org.apache.beam.sdk.extensions.sql.meta.Table;
 import org.apache.beam.sdk.extensions.sql.meta.provider.TableProvider;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.linq4j.tree.Expression;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.type.RelProtoDataType;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.schema.Function;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.schema.Schema;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.schema.SchemaPlus;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.schema.SchemaVersion;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.schema.Schemas;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.linq4j.tree.Expression;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.type.RelProtoDataType;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.schema.Function;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.schema.Schema;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.schema.SchemaPlus;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.schema.SchemaVersion;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.schema.Schemas;
 import org.checkerframework.checker.nullness.qual.Nullable;
 
 /** Adapter from {@link TableProvider} to {@link Schema}. */
@@ -101,7 +101,7 @@
   }
 
   @Override
-  public org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.schema.Table getTable(
+  public org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.schema.Table getTable(
       String name) {
     Table table = tableProvider.getTable(name);
     if (table == null) {
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/BeamCalciteSchemaFactory.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/BeamCalciteSchemaFactory.java
index b7b2a17..08ba342 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/BeamCalciteSchemaFactory.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/BeamCalciteSchemaFactory.java
@@ -27,16 +27,16 @@
 import org.apache.beam.sdk.extensions.sql.meta.provider.TableProvider;
 import org.apache.beam.sdk.extensions.sql.meta.store.InMemoryMetaStore;
 import org.apache.beam.sdk.extensions.sql.meta.store.MetaStore;
-import org.apache.beam.vendor.calcite.v1_20_0.com.google.common.collect.ImmutableMap;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.jdbc.CalciteConnection;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.linq4j.tree.Expression;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.type.RelProtoDataType;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.schema.Function;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.schema.Schema;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.schema.SchemaFactory;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.schema.SchemaPlus;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.schema.SchemaVersion;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.schema.Table;
+import org.apache.beam.vendor.calcite.v1_26_0.com.google.common.collect.ImmutableMap;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.jdbc.CalciteConnection;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.linq4j.tree.Expression;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.type.RelProtoDataType;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.schema.Function;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.schema.Schema;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.schema.SchemaFactory;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.schema.SchemaPlus;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.schema.SchemaVersion;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.schema.Table;
 
 /**
  * Factory classes that Calcite uses to create initial schema for JDBC connection.
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/BeamCalciteTable.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/BeamCalciteTable.java
index deade43..d74e791 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/BeamCalciteTable.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/BeamCalciteTable.java
@@ -27,21 +27,21 @@
 import org.apache.beam.sdk.extensions.sql.impl.utils.CalciteUtils;
 import org.apache.beam.sdk.extensions.sql.meta.BeamSqlTable;
 import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.vendor.calcite.v1_20_0.com.google.common.collect.ImmutableMap;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.adapter.java.AbstractQueryableTable;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.linq4j.QueryProvider;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.linq4j.Queryable;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptCluster;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptTable;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.prepare.Prepare;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.TableModify;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.type.RelDataType;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.type.RelDataTypeFactory;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.schema.ModifiableTable;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.schema.SchemaPlus;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.schema.TranslatableTable;
+import org.apache.beam.vendor.calcite.v1_26_0.com.google.common.collect.ImmutableMap;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.adapter.java.AbstractQueryableTable;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.linq4j.QueryProvider;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.linq4j.Queryable;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptCluster;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptTable;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.prepare.Prepare;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.core.TableModify;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.type.RelDataType;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.schema.ModifiableTable;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.schema.SchemaPlus;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.schema.TranslatableTable;
 
 /** Adapter from {@link BeamSqlTable} to a calcite Table. */
 @SuppressWarnings({
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/BeamSqlEnv.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/BeamSqlEnv.java
index 848135b..655a3fc 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/BeamSqlEnv.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/BeamSqlEnv.java
@@ -17,7 +17,7 @@
  */
 package org.apache.beam.sdk.extensions.sql.impl;
 
-import static org.apache.beam.vendor.calcite.v1_20_0.com.google.common.base.Preconditions.checkNotNull;
+import static org.apache.beam.vendor.calcite.v1_26_0.com.google.common.base.Preconditions.checkNotNull;
 
 import java.sql.SQLException;
 import java.util.AbstractMap.SimpleEntry;
@@ -31,6 +31,7 @@
 import org.apache.beam.sdk.annotations.Internal;
 import org.apache.beam.sdk.extensions.sql.BeamSqlUdf;
 import org.apache.beam.sdk.extensions.sql.impl.QueryPlanner.QueryParameters;
+import org.apache.beam.sdk.extensions.sql.impl.parser.BeamSqlParser;
 import org.apache.beam.sdk.extensions.sql.impl.planner.BeamRuleSets;
 import org.apache.beam.sdk.extensions.sql.impl.rel.BeamRelNode;
 import org.apache.beam.sdk.extensions.sql.meta.BeamSqlTable;
@@ -42,12 +43,12 @@
 import org.apache.beam.sdk.options.PipelineOptionsFactory;
 import org.apache.beam.sdk.transforms.Combine.CombineFn;
 import org.apache.beam.sdk.transforms.SerializableFunction;
-import org.apache.beam.vendor.calcite.v1_20_0.com.google.common.base.Strings;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.jdbc.CalcitePrepare;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptUtil;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.schema.Function;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlExecutableStatement;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.tools.RuleSet;
+import org.apache.beam.vendor.calcite.v1_26_0.com.google.common.base.Strings;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.jdbc.CalcitePrepare;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptUtil;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.schema.Function;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlKind;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.tools.RuleSet;
 
 /**
  * Contains the metadata of tables/UDF functions, and exposes APIs to
@@ -112,12 +113,11 @@
   }
 
   public boolean isDdl(String sqlStatement) throws ParseException {
-    return planner.parse(sqlStatement) instanceof SqlExecutableStatement;
+    return planner.parse(sqlStatement).getKind().belongsTo(SqlKind.DDL);
   }
 
   public void executeDdl(String sqlStatement) throws ParseException {
-    SqlExecutableStatement ddl = (SqlExecutableStatement) planner.parse(sqlStatement);
-    ddl.execute(getContext());
+    BeamSqlParser.DDL_EXECUTOR.executeDdl(getContext(), planner.parse(sqlStatement));
   }
 
   public CalcitePrepare.Context getContext() {
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/BeamSqlPipelineOptionsRegistrar.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/BeamSqlPipelineOptionsRegistrar.java
index 5a1d313..ae88d40 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/BeamSqlPipelineOptionsRegistrar.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/BeamSqlPipelineOptionsRegistrar.java
@@ -20,7 +20,7 @@
 import com.google.auto.service.AutoService;
 import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.options.PipelineOptionsRegistrar;
-import org.apache.beam.vendor.calcite.v1_20_0.com.google.common.collect.ImmutableList;
+import org.apache.beam.vendor.calcite.v1_26_0.com.google.common.collect.ImmutableList;
 
 /** {@link AutoService} registrar for {@link BeamSqlPipelineOptions}. */
 @AutoService(PipelineOptionsRegistrar.class)
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/BeamTableStatistics.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/BeamTableStatistics.java
index b5d6a2e..d66275d 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/BeamTableStatistics.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/BeamTableStatistics.java
@@ -18,16 +18,17 @@
 package org.apache.beam.sdk.extensions.sql.impl;
 
 import java.io.Serializable;
+import java.util.Collections;
 import java.util.List;
 import org.apache.beam.sdk.annotations.Experimental;
 import org.apache.beam.sdk.annotations.Internal;
-import org.apache.beam.vendor.calcite.v1_20_0.com.google.common.collect.ImmutableList;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelCollation;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelDistribution;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelDistributionTraitDef;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelReferentialConstraint;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.schema.Statistic;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.util.ImmutableBitSet;
+import org.apache.beam.vendor.calcite.v1_26_0.com.google.common.collect.ImmutableList;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelCollation;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelDistribution;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelDistributionTraitDef;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelReferentialConstraint;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.schema.Statistic;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.util.ImmutableBitSet;
 
 /** This class stores row count statistics. */
 @Experimental
@@ -77,6 +78,11 @@
   }
 
   @Override
+  public List<ImmutableBitSet> getKeys() {
+    return Collections.emptyList();
+  }
+
+  @Override
   public List<RelReferentialConstraint> getReferentialConstraints() {
     return ImmutableList.of();
   }
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/CalciteConnectionWrapper.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/CalciteConnectionWrapper.java
index 0bdab10..c99076d 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/CalciteConnectionWrapper.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/CalciteConnectionWrapper.java
@@ -35,14 +35,14 @@
 import java.util.Map;
 import java.util.Properties;
 import java.util.concurrent.Executor;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.adapter.java.JavaTypeFactory;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.config.CalciteConnectionConfig;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.jdbc.CalciteConnection;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.jdbc.CalcitePrepare;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.linq4j.Enumerator;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.linq4j.Queryable;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.linq4j.tree.Expression;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.schema.SchemaPlus;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.config.CalciteConnectionConfig;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.jdbc.CalciteConnection;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.jdbc.CalcitePrepare;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.linq4j.Enumerator;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.linq4j.Queryable;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.linq4j.tree.Expression;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.schema.SchemaPlus;
 
 /**
  * Abstract wrapper for {@link CalciteConnection} to simplify extension.
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/CalciteFactoryWrapper.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/CalciteFactoryWrapper.java
index 6bd714f..8fb51b7 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/CalciteFactoryWrapper.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/CalciteFactoryWrapper.java
@@ -21,18 +21,18 @@
 import java.sql.SQLException;
 import java.util.Properties;
 import java.util.TimeZone;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.adapter.java.JavaTypeFactory;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.avatica.AvaticaConnection;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.avatica.AvaticaFactory;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.avatica.AvaticaPreparedStatement;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.avatica.AvaticaResultSet;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.avatica.AvaticaSpecificDatabaseMetaData;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.avatica.AvaticaStatement;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.avatica.Meta;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.avatica.QueryState;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.avatica.UnregisteredDriver;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.jdbc.CalciteFactory;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.jdbc.CalciteSchema;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.avatica.AvaticaConnection;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.avatica.AvaticaFactory;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.avatica.AvaticaPreparedStatement;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.avatica.AvaticaResultSet;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.avatica.AvaticaSpecificDatabaseMetaData;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.avatica.AvaticaStatement;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.avatica.Meta;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.avatica.QueryState;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.avatica.UnregisteredDriver;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.jdbc.CalciteFactory;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.jdbc.CalciteSchema;
 
 /**
  * Wrapper for {@link CalciteFactory}.
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/CalciteQueryPlanner.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/CalciteQueryPlanner.java
index ff5deb4..8c1a05b 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/CalciteQueryPlanner.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/CalciteQueryPlanner.java
@@ -23,49 +23,49 @@
 import java.util.Map;
 import java.util.ServiceLoader;
 import java.util.stream.Collectors;
-import org.apache.beam.sdk.extensions.sql.impl.QueryPlanner.Factory;
 import org.apache.beam.sdk.extensions.sql.impl.QueryPlanner.QueryParameters.Kind;
 import org.apache.beam.sdk.extensions.sql.impl.planner.BeamCostModel;
 import org.apache.beam.sdk.extensions.sql.impl.planner.RelMdNodeStats;
 import org.apache.beam.sdk.extensions.sql.impl.rel.BeamLogicalConvention;
 import org.apache.beam.sdk.extensions.sql.impl.rel.BeamRelNode;
 import org.apache.beam.sdk.extensions.sql.impl.udf.BeamBuiltinFunctionProvider;
-import org.apache.beam.vendor.calcite.v1_20_0.com.google.common.collect.ImmutableList;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.config.CalciteConnectionConfig;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.jdbc.CalciteSchema;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.Contexts;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.ConventionTraitDef;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptCost;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptPlanner.CannotPlanException;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptUtil;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelTraitDef;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelTraitSet;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.prepare.CalciteCatalogReader;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelRoot;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.metadata.BuiltInMetadata;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.metadata.ChainedRelMetadataProvider;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.metadata.JaninoRelMetadataProvider;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.metadata.MetadataDef;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.metadata.MetadataHandler;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.metadata.ReflectiveRelMetadataProvider;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.metadata.RelMetadataProvider;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.metadata.RelMetadataQuery;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.schema.SchemaPlus;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlOperatorTable;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.fun.SqlStdOperatorTable;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.parser.SqlParseException;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.parser.SqlParser;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.parser.SqlParserImplFactory;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.util.ChainedSqlOperatorTable;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.tools.FrameworkConfig;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.tools.Frameworks;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.tools.Planner;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.tools.RelConversionException;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.tools.RuleSet;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.tools.ValidationException;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.util.BuiltInMethod;
+import org.apache.beam.vendor.calcite.v1_26_0.com.google.common.collect.ImmutableList;
+import org.apache.beam.vendor.calcite.v1_26_0.com.google.common.collect.Table;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.config.CalciteConnectionConfig;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.jdbc.CalciteSchema;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.Contexts;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.ConventionTraitDef;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptCost;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptPlanner.CannotPlanException;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptUtil;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelTraitDef;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelTraitSet;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.prepare.CalciteCatalogReader;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelRoot;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.metadata.BuiltInMetadata;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.metadata.ChainedRelMetadataProvider;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.metadata.JaninoRelMetadataProvider;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.metadata.MetadataDef;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.metadata.MetadataHandler;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.metadata.ReflectiveRelMetadataProvider;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.metadata.RelMetadataProvider;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.schema.SchemaPlus;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlOperatorTable;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.parser.SqlParseException;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.parser.SqlParser;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.parser.SqlParserImplFactory;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.util.SqlOperatorTables;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.tools.FrameworkConfig;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.tools.Frameworks;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.tools.Planner;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.tools.RelConversionException;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.tools.RuleSet;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.tools.ValidationException;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.util.BuiltInMethod;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -148,7 +148,7 @@
         .ruleSets(ruleSets.toArray(new RuleSet[0]))
         .costFactory(BeamCostModel.FACTORY)
         .typeSystem(connection.getTypeFactory().getTypeSystem())
-        .operatorTable(ChainedSqlOperatorTable.of(opTab0, catalogReader))
+        .operatorTable(SqlOperatorTables.chain(opTab0, catalogReader))
         .build();
   }
 
@@ -244,14 +244,13 @@
       // here and based on the design we also need to remove the cached values
 
       // We need to first remove the cached values.
-      List<List> costKeys =
-          mq.map.entrySet().stream()
+      List<Table.Cell<RelNode, List, Object>> costKeys =
+          mq.map.cellSet().stream()
               .filter(entry -> entry.getValue() instanceof BeamCostModel)
               .filter(entry -> ((BeamCostModel) entry.getValue()).isInfinite())
-              .map(Map.Entry::getKey)
               .collect(Collectors.toList());
 
-      costKeys.forEach(mq.map::remove);
+      costKeys.forEach(cell -> mq.map.remove(cell.getRowKey(), cell.getColumnKey()));
 
       return ((BeamRelNode) rel).beamComputeSelfCost(rel.getCluster().getPlanner(), mq);
     }
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/JavaUdfLoader.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/JavaUdfLoader.java
index 429ee63..75a20e2 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/JavaUdfLoader.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/JavaUdfLoader.java
@@ -40,7 +40,7 @@
 import org.apache.beam.sdk.extensions.sql.udf.UdfProvider;
 import org.apache.beam.sdk.io.FileSystems;
 import org.apache.beam.sdk.io.fs.ResourceId;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.commons.codec.digest.DigestUtils;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.commons.codec.digest.DigestUtils;
 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/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/JdbcConnection.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/JdbcConnection.java
index 3ae7177..2ff8d6f 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/JdbcConnection.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/JdbcConnection.java
@@ -24,10 +24,10 @@
 import org.apache.beam.sdk.extensions.sql.meta.provider.TableProvider;
 import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.values.KV;
-import org.apache.beam.vendor.calcite.v1_20_0.com.google.common.collect.ImmutableMap;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.jdbc.CalciteConnection;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.jdbc.CalciteSchema;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.schema.SchemaPlus;
+import org.apache.beam.vendor.calcite.v1_26_0.com.google.common.collect.ImmutableMap;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.jdbc.CalciteConnection;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.jdbc.CalciteSchema;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.schema.SchemaPlus;
 import org.checkerframework.checker.nullness.qual.Nullable;
 
 /**
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/JdbcDriver.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/JdbcDriver.java
index 931d7b5..1c08ea0 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/JdbcDriver.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/JdbcDriver.java
@@ -17,8 +17,8 @@
  */
 package org.apache.beam.sdk.extensions.sql.impl;
 
-import static org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.config.CalciteConnectionProperty.SCHEMA_FACTORY;
-import static org.apache.beam.vendor.calcite.v1_20_0.org.codehaus.commons.compiler.CompilerFactoryFactory.getDefaultCompilerFactory;
+import static org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.config.CalciteConnectionProperty.SCHEMA_FACTORY;
+import static org.apache.beam.vendor.calcite.v1_26_0.org.codehaus.commons.compiler.CompilerFactoryFactory.getDefaultCompilerFactory;
 
 import com.fasterxml.jackson.databind.ObjectMapper;
 import com.google.auto.service.AutoService;
@@ -32,19 +32,18 @@
 import org.apache.beam.sdk.extensions.sql.impl.planner.BeamRuleSets;
 import org.apache.beam.sdk.extensions.sql.meta.provider.TableProvider;
 import org.apache.beam.sdk.options.PipelineOptions;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.avatica.AvaticaFactory;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.jdbc.CalciteConnection;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.jdbc.CalciteFactory;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.jdbc.Driver;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptPlanner;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptRule;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelTraitDef;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.prepare.CalcitePrepareImpl;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelCollationTraitDef;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.rules.CalcRemoveRule;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.rules.SortRemoveRule;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.runtime.Hook;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.tools.RuleSet;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.adapter.enumerable.EnumerableRules;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.avatica.AvaticaFactory;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.jdbc.CalciteConnection;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.jdbc.CalciteFactory;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.jdbc.Driver;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptPlanner;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptRule;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelTraitDef;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelCollationTraitDef;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.rules.CoreRules;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.runtime.Hook;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.tools.RuleSet;
 
 /**
  * Calcite JDBC driver with Beam defaults.
@@ -86,10 +85,10 @@
                   planner.addRule(rule);
                 }
               }
-              planner.removeRule(CalcRemoveRule.INSTANCE);
-              planner.removeRule(SortRemoveRule.INSTANCE);
+              planner.removeRule(CoreRules.CALC_REMOVE);
+              planner.removeRule(CoreRules.SORT_REMOVE);
 
-              for (RelOptRule rule : CalcitePrepareImpl.ENUMERABLE_RULES) {
+              for (RelOptRule rule : EnumerableRules.ENUMERABLE_RULES) {
                 planner.removeRule(rule);
               }
 
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/JdbcFactory.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/JdbcFactory.java
index 22a6a52..19f1e4c 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/JdbcFactory.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/JdbcFactory.java
@@ -18,26 +18,26 @@
 package org.apache.beam.sdk.extensions.sql.impl;
 
 import static org.apache.beam.sdk.extensions.sql.impl.JdbcDriver.TOP_LEVEL_BEAM_SCHEMA;
-import static org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.avatica.BuiltInConnectionProperty.TIME_ZONE;
-import static org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.config.CalciteConnectionProperty.LEX;
-import static org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.config.CalciteConnectionProperty.PARSER_FACTORY;
-import static org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.config.CalciteConnectionProperty.SCHEMA;
-import static org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.config.CalciteConnectionProperty.SCHEMA_FACTORY;
-import static org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.config.CalciteConnectionProperty.TYPE_SYSTEM;
+import static org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.avatica.BuiltInConnectionProperty.TIME_ZONE;
+import static org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.config.CalciteConnectionProperty.LEX;
+import static org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.config.CalciteConnectionProperty.PARSER_FACTORY;
+import static org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.config.CalciteConnectionProperty.SCHEMA;
+import static org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.config.CalciteConnectionProperty.SCHEMA_FACTORY;
+import static org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.config.CalciteConnectionProperty.TYPE_SYSTEM;
 
 import java.util.Properties;
-import org.apache.beam.sdk.extensions.sql.impl.parser.impl.BeamSqlParserImpl;
+import org.apache.beam.sdk.extensions.sql.impl.parser.BeamSqlParser;
 import org.apache.beam.sdk.extensions.sql.impl.planner.BeamRelDataTypeSystem;
 import org.apache.beam.sdk.extensions.sql.meta.provider.TableProvider;
 import org.apache.beam.sdk.util.ReleaseInfo;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.adapter.java.JavaTypeFactory;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.avatica.AvaticaConnection;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.avatica.AvaticaFactory;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.avatica.ConnectionProperty;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.avatica.UnregisteredDriver;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.config.Lex;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.jdbc.CalciteFactory;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.jdbc.CalciteSchema;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.avatica.AvaticaConnection;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.avatica.AvaticaFactory;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.avatica.ConnectionProperty;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.avatica.UnregisteredDriver;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.config.Lex;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.jdbc.CalciteFactory;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.jdbc.CalciteSchema;
 
 /**
  * Implements {@link CalciteFactory} that is used by Clacite JDBC driver to instantiate different
@@ -45,9 +45,9 @@
  *
  * <p>The purpose of this class is to intercept the connection creation and force a cache-less root
  * schema ({@link
- * org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.jdbc.SimpleCalciteSchema}). Otherwise
+ * org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.jdbc.SimpleCalciteSchema}). Otherwise
  * Calcite uses {@link
- * org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.jdbc.CachingCalciteSchema} that eagerly
+ * org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.jdbc.CachingCalciteSchema} that eagerly
  * caches table information. This behavior does not work well for dynamic table providers.
  */
 class JdbcFactory extends CalciteFactoryWrapper {
@@ -95,7 +95,7 @@
 
     setIfNull(info, TIME_ZONE, "UTC");
     setIfNull(info, LEX, Lex.JAVA.name());
-    setIfNull(info, PARSER_FACTORY, BeamSqlParserImpl.class.getName() + "#FACTORY");
+    setIfNull(info, PARSER_FACTORY, BeamSqlParser.class.getName() + "#FACTORY");
     setIfNull(info, TYPE_SYSTEM, BeamRelDataTypeSystem.class.getName());
     setIfNull(info, SCHEMA, TOP_LEVEL_BEAM_SCHEMA);
     setIfNull(info, SCHEMA_FACTORY, BeamCalciteSchemaFactory.AllProviders.class.getName());
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/QueryPlanner.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/QueryPlanner.java
index 9ae01b6..48374a5 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/QueryPlanner.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/QueryPlanner.java
@@ -22,10 +22,10 @@
 import java.util.List;
 import java.util.Map;
 import org.apache.beam.sdk.extensions.sql.impl.rel.BeamRelNode;
-import org.apache.beam.vendor.calcite.v1_20_0.com.google.common.collect.ImmutableList;
-import org.apache.beam.vendor.calcite.v1_20_0.com.google.common.collect.ImmutableMap;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.tools.RuleSet;
+import org.apache.beam.vendor.calcite.v1_26_0.com.google.common.collect.ImmutableList;
+import org.apache.beam.vendor.calcite.v1_26_0.com.google.common.collect.ImmutableMap;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.tools.RuleSet;
 
 /**
  * An interface that planners should implement to convert sql statement to {@link BeamRelNode} or
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/ScalarFunctionImpl.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/ScalarFunctionImpl.java
index 36fcde7..7649f5d 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/ScalarFunctionImpl.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/ScalarFunctionImpl.java
@@ -17,7 +17,7 @@
  */
 package org.apache.beam.sdk.extensions.sql.impl;
 
-import static org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.util.Static.RESOURCE;
+import static org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.util.Static.RESOURCE;
 
 import java.lang.reflect.Constructor;
 import java.lang.reflect.Method;
@@ -27,29 +27,29 @@
 import java.util.Arrays;
 import java.util.List;
 import org.apache.beam.sdk.extensions.sql.impl.utils.CalciteUtils;
-import org.apache.beam.vendor.calcite.v1_20_0.com.google.common.base.Preconditions;
-import org.apache.beam.vendor.calcite.v1_20_0.com.google.common.collect.ImmutableMultimap;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.adapter.enumerable.CallImplementor;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.adapter.enumerable.NullPolicy;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.adapter.enumerable.ReflectiveCallNotNullImplementor;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.adapter.enumerable.RexImpTable;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.adapter.enumerable.RexToLixTranslator;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.avatica.util.ByteString;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.linq4j.function.SemiStrict;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.linq4j.function.Strict;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.linq4j.tree.Expression;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.linq4j.tree.Expressions;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.type.RelDataType;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.type.RelDataTypeFactory;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexCall;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.schema.Function;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.schema.ImplementableFunction;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.schema.ScalarFunction;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlOperatorBinding;
+import org.apache.beam.vendor.calcite.v1_26_0.com.google.common.base.Preconditions;
+import org.apache.beam.vendor.calcite.v1_26_0.com.google.common.collect.ImmutableMultimap;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.adapter.enumerable.CallImplementor;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.adapter.enumerable.NullPolicy;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.adapter.enumerable.ReflectiveCallNotNullImplementor;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.adapter.enumerable.RexImpTable;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.adapter.enumerable.RexToLixTranslator;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.avatica.util.ByteString;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.linq4j.function.SemiStrict;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.linq4j.function.Strict;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.linq4j.tree.Expression;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.linq4j.tree.Expressions;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.type.RelDataType;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexCall;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.schema.Function;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.schema.ImplementableFunction;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.schema.ScalarFunction;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlOperatorBinding;
 
 /**
  * Beam-customized version from {@link
- * org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.schema.impl.ScalarFunctionImpl} , to
+ * org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.schema.impl.ScalarFunctionImpl} , to
  * address BEAM-5921.
  */
 @SuppressWarnings({
@@ -81,7 +81,7 @@
   }
 
   /**
-   * Creates {@link org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.schema.Function} for
+   * Creates {@link org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.schema.Function} for
    * each method in a given class.
    */
   public static ImmutableMultimap<String, Function> createAll(Class<?> clazz) {
@@ -100,7 +100,7 @@
   }
 
   /**
-   * Creates {@link org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.schema.Function} from
+   * Creates {@link org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.schema.Function} from
    * given method. When {@code eval} method does not suit, {@code null} is returned.
    *
    * @param method method that is used to implement the function
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/TableResolutionUtils.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/TableResolutionUtils.java
index 842a7e0..00ac8b5 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/TableResolutionUtils.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/TableResolutionUtils.java
@@ -28,8 +28,8 @@
 import org.apache.beam.sdk.extensions.sql.TableNameExtractionUtils;
 import org.apache.beam.sdk.extensions.sql.meta.CustomTableResolver;
 import org.apache.beam.sdk.extensions.sql.meta.provider.TableProvider;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.jdbc.CalciteSchema;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.jdbc.CalciteSchema;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlNode;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -172,7 +172,7 @@
    */
   private static class SchemaWithName {
     String name;
-    org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.schema.Schema schema;
+    org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.schema.Schema schema;
 
     static SchemaWithName create(JdbcConnection connection, String name) {
       SchemaWithName schemaWithName = new SchemaWithName();
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/UdafImpl.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/UdafImpl.java
index 78cfd3b..29ca041 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/UdafImpl.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/UdafImpl.java
@@ -28,12 +28,12 @@
 import org.apache.beam.sdk.annotations.Internal;
 import org.apache.beam.sdk.extensions.sql.impl.utils.CalciteUtils;
 import org.apache.beam.sdk.transforms.Combine.CombineFn;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.adapter.enumerable.AggImplementor;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.type.RelDataType;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.type.RelDataTypeFactory;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.schema.AggregateFunction;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.schema.FunctionParameter;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.schema.ImplementableAggFunction;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.adapter.enumerable.AggImplementor;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.type.RelDataType;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.schema.AggregateFunction;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.schema.FunctionParameter;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.schema.ImplementableAggFunction;
 
 /** Implement {@link AggregateFunction} to take a {@link CombineFn} as UDAF. */
 @Experimental
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/UdfImpl.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/UdfImpl.java
index a84b3ff..44e98d5 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/UdfImpl.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/UdfImpl.java
@@ -18,9 +18,9 @@
 package org.apache.beam.sdk.extensions.sql.impl;
 
 import java.lang.reflect.Method;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.schema.Function;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.schema.TranslatableTable;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.schema.impl.TableMacroImpl;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.schema.Function;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.schema.TranslatableTable;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.schema.impl.TableMacroImpl;
 
 /** Beam-customized facade behind {@link Function} to address BEAM-5921. */
 @SuppressWarnings({
@@ -31,7 +31,7 @@
   private UdfImpl() {}
 
   /**
-   * Creates {@link org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.schema.Function} from
+   * Creates {@link org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.schema.Function} from
    * given class.
    *
    * <p>If a method of the given name is not found or it does not suit, returns {@code null}.
@@ -49,7 +49,7 @@
   }
 
   /**
-   * Creates {@link org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.schema.Function} from
+   * Creates {@link org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.schema.Function} from
    * given method.
    *
    * @param method method that is used to implement the function
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/UdfImplReflectiveFunctionBase.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/UdfImplReflectiveFunctionBase.java
index 70d7965..0faa084 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/UdfImplReflectiveFunctionBase.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/UdfImplReflectiveFunctionBase.java
@@ -24,13 +24,13 @@
 import java.util.ArrayList;
 import java.util.List;
 import org.apache.beam.sdk.extensions.sql.impl.utils.CalciteUtils;
-import org.apache.beam.vendor.calcite.v1_20_0.com.google.common.collect.ImmutableList;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.type.RelDataType;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.type.RelDataTypeFactory;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.schema.Function;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.schema.FunctionParameter;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.schema.impl.ReflectiveFunctionBase;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.util.ReflectUtil;
+import org.apache.beam.vendor.calcite.v1_26_0.com.google.common.collect.ImmutableList;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.type.RelDataType;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.schema.Function;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.schema.FunctionParameter;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.schema.impl.ReflectiveFunctionBase;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.util.ReflectUtil;
 
 /** Beam-customized version from {@link ReflectiveFunctionBase}, to address BEAM-5921. */
 @SuppressWarnings({
@@ -101,7 +101,7 @@
 
   /**
    * Helps build lists of {@link
-   * org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.schema.FunctionParameter}.
+   * org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.schema.FunctionParameter}.
    */
   public static class ParameterListBuilder {
     final List<FunctionParameter> builder = new ArrayList<>();
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/ZetaSqlUserDefinedSQLNativeTableValuedFunction.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/ZetaSqlUserDefinedSQLNativeTableValuedFunction.java
index 07c9cbd..00d3782 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/ZetaSqlUserDefinedSQLNativeTableValuedFunction.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/ZetaSqlUserDefinedSQLNativeTableValuedFunction.java
@@ -19,13 +19,13 @@
 
 import java.util.List;
 import org.apache.beam.sdk.annotations.Internal;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.type.RelDataType;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.schema.Function;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlIdentifier;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.type.SqlOperandTypeChecker;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.type.SqlOperandTypeInference;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.type.SqlReturnTypeInference;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.validate.SqlUserDefinedFunction;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.type.RelDataType;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.schema.Function;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlIdentifier;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.type.SqlOperandTypeChecker;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.type.SqlOperandTypeInference;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.type.SqlReturnTypeInference;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.validate.SqlUserDefinedFunction;
 
 /** This is a class to indicate that a TVF is a ZetaSQL SQL native UDTVF. */
 @Internal
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/cep/CEPCall.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/cep/CEPCall.java
index fb1d6da..6ee91e8 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/cep/CEPCall.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/cep/CEPCall.java
@@ -19,11 +19,11 @@
 
 import java.util.ArrayList;
 import java.util.List;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexCall;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexLiteral;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexPatternFieldRef;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlOperator;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexCall;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexLiteral;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexPatternFieldRef;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlOperator;
 
 /**
  * A {@code CEPCall} instance represents an operation (node) that contains an operator and a list of
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/cep/CEPFieldRef.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/cep/CEPFieldRef.java
index 68aaf8d..664bb5b 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/cep/CEPFieldRef.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/cep/CEPFieldRef.java
@@ -17,7 +17,7 @@
  */
 package org.apache.beam.sdk.extensions.sql.impl.cep;
 
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexPatternFieldRef;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexPatternFieldRef;
 
 /**
  * A {@code CEPFieldRef} instance represents a node that points to a specified field in a {@code
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/cep/CEPLiteral.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/cep/CEPLiteral.java
index d675767..5f83cf2 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/cep/CEPLiteral.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/cep/CEPLiteral.java
@@ -20,7 +20,7 @@
 import java.math.BigDecimal;
 import org.apache.beam.sdk.extensions.sql.impl.SqlConversionException;
 import org.apache.beam.sdk.schemas.Schema;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexLiteral;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexLiteral;
 import org.joda.time.ReadableDateTime;
 
 /**
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/cep/CEPOperation.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/cep/CEPOperation.java
index 50f735f..13f6321 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/cep/CEPOperation.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/cep/CEPOperation.java
@@ -19,10 +19,10 @@
 
 import java.io.Serializable;
 import org.apache.beam.sdk.extensions.sql.impl.SqlConversionException;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexCall;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexLiteral;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexPatternFieldRef;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexCall;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexLiteral;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexPatternFieldRef;
 
 /**
  * {@code CEPOperation} is the base class for the evaluation operations defined in the {@code
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/cep/CEPOperator.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/cep/CEPOperator.java
index 9dc3abb..84fb72e 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/cep/CEPOperator.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/cep/CEPOperator.java
@@ -19,8 +19,8 @@
 
 import java.io.Serializable;
 import java.util.Map;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlKind;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlOperator;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlKind;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlOperator;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
 
 /**
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/cep/CEPPattern.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/cep/CEPPattern.java
index dc7a407..ed66688 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/cep/CEPPattern.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/cep/CEPPattern.java
@@ -23,7 +23,7 @@
 import javax.annotation.Nullable;
 import org.apache.beam.sdk.schemas.Schema;
 import org.apache.beam.sdk.values.Row;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexCall;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexCall;
 
 /** Core pattern class that stores the definition of a single pattern. */
 @SuppressWarnings({
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/cep/CEPUtils.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/cep/CEPUtils.java
index 1c45f60..0069828 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/cep/CEPUtils.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/cep/CEPUtils.java
@@ -21,14 +21,14 @@
 import java.util.List;
 import java.util.Map;
 import org.apache.beam.sdk.schemas.Schema;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelCollation;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelFieldCollation;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexCall;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexInputRef;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexLiteral;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlKind;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlOperator;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelCollation;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelFieldCollation;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexCall;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexLiteral;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlKind;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlOperator;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.util.ImmutableBitSet;
 
 /**
  * Some utility methods for transforming Calcite's constructs into our own Beam constructs (for
@@ -156,11 +156,10 @@
   }
 
   /** Transform the partition columns into serializable CEPFieldRef. */
-  public static List<CEPFieldRef> getCEPFieldRefFromParKeys(List<RexNode> parKeys) {
+  public static List<CEPFieldRef> getCEPFieldRefFromParKeys(ImmutableBitSet partitionKeys) {
     ArrayList<CEPFieldRef> fieldList = new ArrayList<>();
-    for (RexNode i : parKeys) {
-      RexInputRef parKey = (RexInputRef) i;
-      fieldList.add(new CEPFieldRef(parKey.getName(), parKey.getIndex()));
+    for (int index : partitionKeys.asList()) {
+      fieldList.add(new CEPFieldRef("Partition Key " + index, index));
     }
     return fieldList;
   }
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/cep/OrderKey.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/cep/OrderKey.java
index 85825e6..07d6654 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/cep/OrderKey.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/cep/OrderKey.java
@@ -18,7 +18,7 @@
 package org.apache.beam.sdk.extensions.sql.impl.cep;
 
 import java.io.Serializable;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelFieldCollation;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelFieldCollation;
 
 /**
  * The {@code OrderKey} class stores the information to sort a column.
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/BeamSqlParser.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/BeamSqlParser.java
new file mode 100644
index 0000000..c76a87b
--- /dev/null
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/BeamSqlParser.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.sql.impl.parser;
+
+import java.io.Reader;
+import org.apache.beam.sdk.extensions.sql.impl.parser.impl.BeamSqlParserImpl;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.jdbc.CalcitePrepare;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.server.DdlExecutor;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.parser.SqlAbstractParserImpl;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.parser.SqlParserImplFactory;
+
+public class BeamSqlParser {
+
+  private BeamSqlParser() {}
+
+  /** Parser factory. */
+  public static final SqlParserImplFactory FACTORY =
+      new SqlParserImplFactory() {
+        @Override
+        public SqlAbstractParserImpl getParser(Reader stream) {
+          return BeamSqlParserImpl.FACTORY.getParser(stream);
+        }
+
+        @Override
+        public DdlExecutor getDdlExecutor() {
+          return BeamSqlParser.DDL_EXECUTOR;
+        }
+      };
+
+  /** Ddl Executor. */
+  public static final DdlExecutor DDL_EXECUTOR =
+      (context, node) -> {
+        ((ExecutableStatement) node).execute(context);
+      };
+
+  interface ExecutableStatement {
+    void execute(CalcitePrepare.Context context);
+  }
+}
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlCheckConstraint.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlCheckConstraint.java
index a6d145d..95c96de 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlCheckConstraint.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlCheckConstraint.java
@@ -18,15 +18,15 @@
 package org.apache.beam.sdk.extensions.sql.impl.parser;
 
 import java.util.List;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlCall;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlIdentifier;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlKind;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlOperator;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlSpecialOperator;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlWriter;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.parser.SqlParserPos;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.util.ImmutableNullableList;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlCall;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlIdentifier;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlKind;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlOperator;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlSpecialOperator;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlWriter;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.util.ImmutableNullableList;
 
 /**
  * Parse tree for {@code UNIQUE}, {@code PRIMARY KEY} constraints.
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlColumnDeclaration.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlColumnDeclaration.java
index 1ffe80f..ba3e9fe 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlColumnDeclaration.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlColumnDeclaration.java
@@ -18,16 +18,16 @@
 package org.apache.beam.sdk.extensions.sql.impl.parser;
 
 import java.util.List;
-import org.apache.beam.vendor.calcite.v1_20_0.com.google.common.collect.ImmutableList;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlCall;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlDataTypeSpec;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlIdentifier;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlKind;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlOperator;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlSpecialOperator;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlWriter;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.beam.vendor.calcite.v1_26_0.com.google.common.collect.ImmutableList;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlCall;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlDataTypeSpec;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlIdentifier;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlKind;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlOperator;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlSpecialOperator;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlWriter;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.parser.SqlParserPos;
 
 /** Parse tree for column. */
 public class SqlColumnDeclaration extends SqlCall {
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlCreateExternalTable.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlCreateExternalTable.java
index 9008561..e65f532 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlCreateExternalTable.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlCreateExternalTable.java
@@ -19,8 +19,8 @@
 
 import static com.alibaba.fastjson.JSON.parseObject;
 import static org.apache.beam.sdk.schemas.Schema.toSchema;
-import static org.apache.beam.vendor.calcite.v1_20_0.com.google.common.base.Preconditions.checkNotNull;
-import static org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.util.Static.RESOURCE;
+import static org.apache.beam.vendor.calcite.v1_26_0.com.google.common.base.Preconditions.checkNotNull;
+import static org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.util.Static.RESOURCE;
 
 import com.alibaba.fastjson.JSONObject;
 import java.util.List;
@@ -28,25 +28,24 @@
 import org.apache.beam.sdk.extensions.sql.impl.utils.CalciteUtils;
 import org.apache.beam.sdk.extensions.sql.meta.Table;
 import org.apache.beam.sdk.schemas.Schema;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.jdbc.CalcitePrepare;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.jdbc.CalciteSchema;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlCreate;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlExecutableStatement;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlIdentifier;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlKind;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlOperator;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlSpecialOperator;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlUtil;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlWriter;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.parser.SqlParserPos;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.util.Pair;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.jdbc.CalcitePrepare;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.jdbc.CalciteSchema;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlCreate;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlIdentifier;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlKind;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlOperator;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlSpecialOperator;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlUtil;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlWriter;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.util.Pair;
 
 /** Parse tree for {@code CREATE EXTERNAL TABLE} statement. */
 @SuppressWarnings({
   "nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 })
-public class SqlCreateExternalTable extends SqlCreate implements SqlExecutableStatement {
+public class SqlCreateExternalTable extends SqlCreate implements BeamSqlParser.ExecutableStatement {
   private final SqlIdentifier name;
   private final List<Schema.Field> columnList;
   private final SqlNode type;
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlCreateFunction.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlCreateFunction.java
index bd31862..1fd0643 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlCreateFunction.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlCreateFunction.java
@@ -17,7 +17,7 @@
  */
 package org.apache.beam.sdk.extensions.sql.impl.parser;
 
-import static org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.util.Static.RESOURCE;
+import static org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.util.Static.RESOURCE;
 
 import java.lang.reflect.Method;
 import java.util.Arrays;
@@ -28,26 +28,25 @@
 import org.apache.beam.sdk.extensions.sql.impl.ScalarFnReflector;
 import org.apache.beam.sdk.extensions.sql.impl.ScalarFunctionImpl;
 import org.apache.beam.sdk.extensions.sql.udf.ScalarFn;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.jdbc.CalcitePrepare;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.jdbc.CalciteSchema;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.schema.Function;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.schema.SchemaPlus;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlCharStringLiteral;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlCreate;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlExecutableStatement;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlIdentifier;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlKind;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlOperator;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlSpecialOperator;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlUtil;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlWriter;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.parser.SqlParserPos;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.util.Pair;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.jdbc.CalcitePrepare;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.jdbc.CalciteSchema;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.schema.Function;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.schema.SchemaPlus;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlCharStringLiteral;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlCreate;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlIdentifier;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlKind;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlOperator;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlSpecialOperator;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlUtil;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlWriter;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.util.Pair;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
 
 /** Parse tree for {@code CREATE FUNCTION} statement. */
-public class SqlCreateFunction extends SqlCreate implements SqlExecutableStatement {
+public class SqlCreateFunction extends SqlCreate implements BeamSqlParser.ExecutableStatement {
   private final boolean isAggregate;
   private final SqlIdentifier functionName;
   private final SqlNode jarPath;
@@ -57,8 +56,12 @@
 
   /** Creates a SqlCreateFunction. */
   public SqlCreateFunction(
-      SqlParserPos pos, SqlIdentifier functionName, SqlNode jarPath, boolean isAggregate) {
-    super(OPERATOR, pos, false, false);
+      SqlParserPos pos,
+      boolean replace,
+      SqlIdentifier functionName,
+      SqlNode jarPath,
+      boolean isAggregate) {
+    super(OPERATOR, pos, replace, false);
     this.functionName = Objects.requireNonNull(functionName, "functionName");
     this.jarPath = Objects.requireNonNull(jarPath, "jarPath");
     this.isAggregate = isAggregate;
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlDdlNodes.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlDdlNodes.java
index 55c4fa4..256b9b7 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlDdlNodes.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlDdlNodes.java
@@ -18,16 +18,16 @@
 package org.apache.beam.sdk.extensions.sql.impl.parser;
 
 import java.util.List;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.jdbc.CalcitePrepare;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.jdbc.CalciteSchema;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlDataTypeSpec;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlIdentifier;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlLiteral;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.parser.SqlParserPos;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.util.NlsString;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.util.Pair;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.util.Util;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.jdbc.CalcitePrepare;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.jdbc.CalciteSchema;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlDataTypeSpec;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlIdentifier;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlLiteral;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.util.NlsString;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.util.Pair;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.util.Util;
 import org.checkerframework.checker.nullness.qual.Nullable;
 
 /** Utilities concerning {@link SqlNode} for DDL. */
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlDropObject.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlDropObject.java
index 2801dcd..a6ea9f8 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlDropObject.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlDropObject.java
@@ -17,27 +17,26 @@
  */
 package org.apache.beam.sdk.extensions.sql.impl.parser;
 
-import static org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.util.Static.RESOURCE;
+import static org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.util.Static.RESOURCE;
 
 import java.util.List;
 import org.apache.beam.sdk.extensions.sql.impl.BeamCalciteSchema;
-import org.apache.beam.vendor.calcite.v1_20_0.com.google.common.collect.ImmutableList;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.jdbc.CalcitePrepare;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.jdbc.CalciteSchema;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlDrop;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlExecutableStatement;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlIdentifier;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlOperator;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlUtil;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlWriter;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.beam.vendor.calcite.v1_26_0.com.google.common.collect.ImmutableList;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.jdbc.CalcitePrepare;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.jdbc.CalciteSchema;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlDrop;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlIdentifier;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlOperator;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlUtil;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlWriter;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.parser.SqlParserPos;
 
 /**
  * Base class for parse trees of {@code DROP TABLE}, {@code DROP VIEW} and {@code DROP MATERIALIZED
  * VIEW} statements.
  */
-abstract class SqlDropObject extends SqlDrop implements SqlExecutableStatement {
+abstract class SqlDropObject extends SqlDrop implements BeamSqlParser.ExecutableStatement {
   protected final SqlIdentifier name;
 
   /** Creates a SqlDropObject. */
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlDropTable.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlDropTable.java
index 9541242..3cc83a2 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlDropTable.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlDropTable.java
@@ -17,11 +17,11 @@
  */
 package org.apache.beam.sdk.extensions.sql.impl.parser;
 
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlIdentifier;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlKind;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlOperator;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlSpecialOperator;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlIdentifier;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlKind;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlOperator;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlSpecialOperator;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.parser.SqlParserPos;
 
 /** Parse tree for {@code DROP TABLE} statement. */
 public class SqlDropTable extends SqlDropObject {
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlSetOptionBeam.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlSetOptionBeam.java
index 98d6466..cfb1c71 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlSetOptionBeam.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/parser/SqlSetOptionBeam.java
@@ -17,24 +17,23 @@
  */
 package org.apache.beam.sdk.extensions.sql.impl.parser;
 
-import static org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.util.Static.RESOURCE;
+import static org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.util.Static.RESOURCE;
 
 import org.apache.beam.sdk.extensions.sql.impl.BeamCalciteSchema;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.jdbc.CalcitePrepare;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.jdbc.CalciteSchema;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlExecutableStatement;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlIdentifier;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlSetOption;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlUtil;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.parser.SqlParserPos;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.util.Pair;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.jdbc.CalcitePrepare;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.jdbc.CalciteSchema;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlIdentifier;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlSetOption;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlUtil;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.util.Pair;
 
 /** SQL parse tree node to represent {@code SET} and {@code RESET} statements. */
 @SuppressWarnings({
   "nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
 })
-public class SqlSetOptionBeam extends SqlSetOption implements SqlExecutableStatement {
+public class SqlSetOptionBeam extends SqlSetOption implements BeamSqlParser.ExecutableStatement {
 
   public SqlSetOptionBeam(SqlParserPos pos, String scope, SqlIdentifier name, SqlNode value) {
     super(pos, scope, name, value);
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/planner/BeamCostModel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/planner/BeamCostModel.java
index c3989cd..aa3a640 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/planner/BeamCostModel.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/planner/BeamCostModel.java
@@ -18,9 +18,9 @@
 package org.apache.beam.sdk.extensions.sql.impl.planner;
 
 import java.util.Objects;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptCost;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptCostFactory;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptUtil;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptCost;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptCostFactory;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptUtil;
 import org.checkerframework.checker.nullness.qual.Nullable;
 
 /**
@@ -218,7 +218,7 @@
 
   /**
    * Implementation of {@link
-   * org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptCostFactory} that creates
+   * org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptCostFactory} that creates
    * {@link BeamCostModel}s.
    */
   public static class Factory implements RelOptCostFactory {
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/planner/BeamJavaTypeFactory.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/planner/BeamJavaTypeFactory.java
index bc67b93..43b2c6b 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/planner/BeamJavaTypeFactory.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/planner/BeamJavaTypeFactory.java
@@ -18,12 +18,12 @@
 package org.apache.beam.sdk.extensions.sql.impl.planner;
 
 import java.lang.reflect.Type;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.adapter.java.JavaTypeFactory;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.jdbc.JavaTypeFactoryImpl;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.type.RelDataType;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.type.BasicSqlType;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.type.IntervalSqlType;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.jdbc.JavaTypeFactoryImpl;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.type.RelDataType;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.type.BasicSqlType;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.type.IntervalSqlType;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.type.SqlTypeName;
 
 /** customized data type in Beam. */
 public class BeamJavaTypeFactory extends JavaTypeFactoryImpl {
@@ -40,6 +40,12 @@
         return type.isNullable() ? Float.class : float.class;
       }
     }
+    // Map BINARY and VARBINARY to byte[] instead of ByteString so UDFs over these types don't
+    // require vendored Calcite.
+    if (type.getSqlTypeName() == SqlTypeName.BINARY
+        || type.getSqlTypeName() == SqlTypeName.VARBINARY) {
+      return byte[].class;
+    }
     return super.getJavaClass(type);
   }
 }
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/planner/BeamRelDataTypeSystem.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/planner/BeamRelDataTypeSystem.java
index 4452422..838922b 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/planner/BeamRelDataTypeSystem.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/planner/BeamRelDataTypeSystem.java
@@ -17,9 +17,9 @@
  */
 package org.apache.beam.sdk.extensions.sql.impl.planner;
 
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.type.RelDataTypeSystem;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.type.RelDataTypeSystemImpl;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.type.RelDataTypeSystem;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.type.RelDataTypeSystemImpl;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.type.SqlTypeName;
 
 /** customized data type in Beam. */
 public class BeamRelDataTypeSystem extends RelDataTypeSystemImpl {
@@ -44,12 +44,24 @@
   }
 
   @Override
+  public int getDefaultPrecision(SqlTypeName typeName) {
+    switch (typeName) {
+      case TIME:
+      case TIMESTAMP:
+      case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
+        return 6; // support microsecond precision
+      default:
+        return super.getDefaultPrecision(typeName);
+    }
+  }
+
+  @Override
   public int getMaxPrecision(SqlTypeName typeName) {
     switch (typeName) {
       case TIME:
-        return 6; // support microsecond time precision
+      case TIMESTAMP:
       case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
-        return 6; // support microsecond datetime precision
+        return 6; // support microsecond precision
       default:
         return super.getMaxPrecision(typeName);
     }
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/planner/BeamRuleSets.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/planner/BeamRuleSets.java
index 24f64ce..0b8210b 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/planner/BeamRuleSets.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/planner/BeamRuleSets.java
@@ -44,33 +44,13 @@
 import org.apache.beam.sdk.extensions.sql.impl.rule.BeamValuesRule;
 import org.apache.beam.sdk.extensions.sql.impl.rule.BeamWindowRule;
 import org.apache.beam.sdk.extensions.sql.impl.rule.LogicalCalcMergeRule;
-import org.apache.beam.vendor.calcite.v1_20_0.com.google.common.collect.ImmutableList;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptRule;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.rules.AggregateJoinTransposeRule;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.rules.AggregateRemoveRule;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.rules.AggregateUnionAggregateRule;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.rules.FilterAggregateTransposeRule;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.rules.FilterCalcMergeRule;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.rules.FilterJoinRule;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.rules.FilterProjectTransposeRule;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.rules.FilterSetOpTransposeRule;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.rules.FilterToCalcRule;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.rules.JoinCommuteRule;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.rules.JoinPushExpressionsRule;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.rules.ProjectCalcMergeRule;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.rules.ProjectFilterTransposeRule;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.rules.ProjectMergeRule;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.rules.ProjectSetOpTransposeRule;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.rules.ProjectSortTransposeRule;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.rules.ProjectToCalcRule;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.rules.ProjectToWindowRule;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.rules.PruneEmptyRules;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.rules.SortProjectTransposeRule;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.rules.UnionEliminatorRule;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.rules.UnionToDistinctRule;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.tools.RuleSet;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.tools.RuleSets;
+import org.apache.beam.vendor.calcite.v1_26_0.com.google.common.collect.ImmutableList;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptRule;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.rules.CoreRules;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.rules.PruneEmptyRules;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.tools.RuleSet;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.tools.RuleSets;
 
 /**
  * {@link RuleSet} used in {@code BeamQueryPlanner}. It translates a standard Calcite {@link
@@ -81,69 +61,68 @@
   private static final List<RelOptRule> LOGICAL_OPTIMIZATIONS =
       ImmutableList.of(
           // Rules for window functions
-          ProjectToWindowRule.PROJECT,
+          CoreRules.PROJECT_TO_LOGICAL_PROJECT_AND_WINDOW,
           // Rules so we only have to implement Calc
-          FilterCalcMergeRule.INSTANCE,
-          ProjectCalcMergeRule.INSTANCE,
-          FilterToCalcRule.INSTANCE,
-          ProjectToCalcRule.INSTANCE,
+          CoreRules.FILTER_CALC_MERGE,
+          CoreRules.PROJECT_CALC_MERGE,
+          CoreRules.FILTER_TO_CALC,
+          CoreRules.PROJECT_TO_CALC,
           BeamIOPushDownRule.INSTANCE,
           // disabled due to https://issues.apache.org/jira/browse/BEAM-6810
-          // CalcRemoveRule.INSTANCE,
+          // CoreRules.CALC_REMOVE,
 
           // Rules to merge matching Calcs together.
           LogicalCalcMergeRule.INSTANCE,
           BeamCalcMergeRule.INSTANCE,
 
           // push a filter into a join
-          FilterJoinRule.FILTER_ON_JOIN,
+          CoreRules.FILTER_INTO_JOIN,
           // push filter into the children of a join
-          FilterJoinRule.JOIN,
+          CoreRules.JOIN_CONDITION_PUSH,
           // push filter through an aggregation
-          FilterAggregateTransposeRule.INSTANCE,
+          CoreRules.FILTER_AGGREGATE_TRANSPOSE,
           // push filter through set operation
-          FilterSetOpTransposeRule.INSTANCE,
+          CoreRules.FILTER_SET_OP_TRANSPOSE,
           // push project through set operation
-          ProjectSetOpTransposeRule.INSTANCE,
+          CoreRules.PROJECT_SET_OP_TRANSPOSE,
 
           // aggregation and projection rules
           BeamAggregateProjectMergeRule.INSTANCE,
           // push a projection past a filter or vice versa
-          ProjectFilterTransposeRule.INSTANCE,
-          FilterProjectTransposeRule.INSTANCE,
+          CoreRules.PROJECT_FILTER_TRANSPOSE,
+          CoreRules.FILTER_PROJECT_TRANSPOSE,
           // push a projection to the children of a join
           // merge projections
-          ProjectMergeRule.INSTANCE,
-          // ProjectRemoveRule.INSTANCE,
+          CoreRules.PROJECT_MERGE,
+          // CoreRules.PROJECT_REMOVE,
           // reorder sort and projection
-          SortProjectTransposeRule.INSTANCE,
-          ProjectSortTransposeRule.INSTANCE,
+          CoreRules.SORT_PROJECT_TRANSPOSE,
 
           // join rules
-          JoinPushExpressionsRule.INSTANCE,
-          JoinCommuteRule.INSTANCE,
+          CoreRules.JOIN_PUSH_EXPRESSIONS,
+          CoreRules.JOIN_COMMUTE,
           BeamJoinAssociateRule.INSTANCE,
           BeamJoinPushThroughJoinRule.RIGHT,
           BeamJoinPushThroughJoinRule.LEFT,
 
           // remove union with only a single child
-          UnionEliminatorRule.INSTANCE,
+          CoreRules.UNION_REMOVE,
           // convert non-all union into all-union + distinct
-          UnionToDistinctRule.INSTANCE,
+          CoreRules.UNION_TO_DISTINCT,
 
           // remove aggregation if it does not aggregate and input is already distinct
-          AggregateRemoveRule.INSTANCE,
+          CoreRules.AGGREGATE_REMOVE,
           // push aggregate through join
-          AggregateJoinTransposeRule.EXTENDED,
+          CoreRules.AGGREGATE_JOIN_TRANSPOSE_EXTENDED,
           // aggregate union rule
-          AggregateUnionAggregateRule.INSTANCE,
+          CoreRules.AGGREGATE_UNION_AGGREGATE,
 
           // reduce aggregate functions like AVG, STDDEV_POP etc.
-          // AggregateReduceFunctionsRule.INSTANCE,
+          // CoreRules.AGGREGATE_REDUCE_FUNCTIONS,
 
           // remove unnecessary sort rule
           // https://issues.apache.org/jira/browse/BEAM-5073
-          // SortRemoveRule.INSTANCE,
+          // CoreRules.SORT_REMOVE,,
           BeamTableFunctionScanRule.INSTANCE,
           // prune empty results rules
           PruneEmptyRules.AGGREGATE_INSTANCE,
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/planner/NodeStatsMetadata.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/planner/NodeStatsMetadata.java
index f0991af..daa749b 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/planner/NodeStatsMetadata.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/planner/NodeStatsMetadata.java
@@ -18,12 +18,12 @@
 package org.apache.beam.sdk.extensions.sql.impl.planner;
 
 import java.lang.reflect.Method;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.linq4j.tree.Types;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.metadata.Metadata;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.metadata.MetadataDef;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.metadata.MetadataHandler;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.linq4j.tree.Types;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.metadata.Metadata;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.metadata.MetadataDef;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.metadata.MetadataHandler;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.metadata.RelMetadataQuery;
 
 /**
  * This is a metadata used for row count and rate estimation. It extends Calcite's Metadata
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/planner/RelMdNodeStats.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/planner/RelMdNodeStats.java
index df325bc..3eca228 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/planner/RelMdNodeStats.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/planner/RelMdNodeStats.java
@@ -17,16 +17,18 @@
  */
 package org.apache.beam.sdk.extensions.sql.impl.planner;
 
+import static org.apache.beam.sdk.util.Preconditions.checkArgumentNotNull;
+
 import java.util.List;
-import java.util.Map;
 import java.util.stream.Collectors;
 import org.apache.beam.sdk.extensions.sql.impl.rel.BeamRelNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.metadata.MetadataDef;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.metadata.MetadataHandler;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.metadata.ReflectiveRelMetadataProvider;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.metadata.RelMetadataProvider;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.beam.vendor.calcite.v1_26_0.com.google.common.collect.Table;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.metadata.MetadataDef;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.metadata.MetadataHandler;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.metadata.ReflectiveRelMetadataProvider;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.metadata.RelMetadataProvider;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.metadata.RelMetadataQuery;
 
 /**
  * This is the implementation of NodeStatsMetadata. Methods to estimate rate and row count for
@@ -71,16 +73,15 @@
     // wraps the metadata provider with CachingRelMetadataProvider. However,
     // CachingRelMetadataProvider checks timestamp before returning previous results. Therefore,
     // there wouldn't be a problem in that case.
-    List<List> keys =
-        mq.map.entrySet().stream()
+    List<Table.Cell<RelNode, List, Object>> keys =
+        mq.map.cellSet().stream()
+            .filter(entry -> entry != null)
+            .filter(entry -> entry.getValue() != null)
             .filter(entry -> entry.getValue() instanceof NodeStats)
-            .filter(entry -> ((NodeStats) entry.getValue()).isUnknown())
-            .map(Map.Entry::getKey)
+            .filter(entry -> (checkArgumentNotNull((NodeStats) entry.getValue()).isUnknown()))
             .collect(Collectors.toList());
 
-    for (List key : keys) {
-      mq.map.remove(key);
-    }
+    keys.forEach(cell -> mq.map.remove(cell.getRowKey(), cell.getColumnKey()));
 
     return rel.estimateNodeStats(mq);
   }
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/AbstractBeamCalcRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/AbstractBeamCalcRel.java
index 7ef101e..0bcaae2 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/AbstractBeamCalcRel.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/AbstractBeamCalcRel.java
@@ -20,15 +20,15 @@
 import org.apache.beam.sdk.annotations.Internal;
 import org.apache.beam.sdk.extensions.sql.impl.planner.BeamCostModel;
 import org.apache.beam.sdk.extensions.sql.impl.planner.NodeStats;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptCluster;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptPlanner;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelTraitSet;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.Calc;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.metadata.RelMetadataQuery;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexLocalRef;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexProgram;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptCluster;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptPlanner;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelTraitSet;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.core.Calc;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexLocalRef;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexProgram;
 
 /** BeamRelNode to replace {@code Project} and {@code Filter} node. */
 @Internal
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamAggregationRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamAggregationRel.java
index 5889bc5..f3e14da 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamAggregationRel.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamAggregationRel.java
@@ -19,7 +19,7 @@
 
 import static java.util.stream.Collectors.toList;
 import static org.apache.beam.sdk.values.PCollection.IsBounded.BOUNDED;
-import static org.apache.beam.vendor.calcite.v1_20_0.com.google.common.base.Preconditions.checkArgument;
+import static org.apache.beam.vendor.calcite.v1_26_0.com.google.common.base.Preconditions.checkArgument;
 
 import java.io.Serializable;
 import java.util.List;
@@ -50,15 +50,15 @@
 import org.apache.beam.sdk.values.PCollectionList;
 import org.apache.beam.sdk.values.Row;
 import org.apache.beam.sdk.values.WindowingStrategy;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptCluster;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptPlanner;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelTraitSet;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelWriter;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.Aggregate;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.AggregateCall;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.metadata.RelMetadataQuery;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.util.ImmutableBitSet;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptCluster;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptPlanner;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelTraitSet;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelWriter;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.core.Aggregate;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.core.AggregateCall;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.util.ImmutableBitSet;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Lists;
 import org.checkerframework.checker.nullness.qual.Nullable;
 import org.joda.time.Duration;
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamCalcRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamCalcRel.java
index d1e888e..c435695 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamCalcRel.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamCalcRel.java
@@ -18,7 +18,7 @@
 package org.apache.beam.sdk.extensions.sql.impl.rel;
 
 import static org.apache.beam.sdk.schemas.Schema.FieldType;
-import static org.apache.beam.vendor.calcite.v1_20_0.com.google.common.base.Preconditions.checkArgument;
+import static org.apache.beam.vendor.calcite.v1_26_0.com.google.common.base.Preconditions.checkArgument;
 
 import java.io.IOException;
 import java.lang.reflect.InvocationTargetException;
@@ -56,39 +56,39 @@
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PCollectionList;
 import org.apache.beam.sdk.values.Row;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.DataContext;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.adapter.enumerable.JavaRowFormat;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.adapter.enumerable.PhysType;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.adapter.enumerable.PhysTypeImpl;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.adapter.enumerable.RexToLixTranslator;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.adapter.java.JavaTypeFactory;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.avatica.util.ByteString;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.linq4j.QueryProvider;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.linq4j.tree.BlockBuilder;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.linq4j.tree.Expression;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.linq4j.tree.Expressions;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.linq4j.tree.MemberDeclaration;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.linq4j.tree.ParameterExpression;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.linq4j.tree.Types;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptCluster;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptPredicateList;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelTraitSet;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.Calc;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.metadata.RelMetadataQuery;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexBuilder;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexCall;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexProgram;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexSimplify;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexUtil;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.runtime.SqlFunctions;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.schema.Function;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.schema.SchemaPlus;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlOperator;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.validate.SqlConformance;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.validate.SqlConformanceEnum;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.validate.SqlUserDefinedFunction;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.DataContext;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.adapter.enumerable.JavaRowFormat;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.adapter.enumerable.PhysType;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.adapter.enumerable.PhysTypeImpl;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.adapter.enumerable.RexToLixTranslator;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.avatica.util.ByteString;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.linq4j.QueryProvider;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.linq4j.tree.BlockBuilder;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.linq4j.tree.Expression;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.linq4j.tree.Expressions;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.linq4j.tree.MemberDeclaration;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.linq4j.tree.ParameterExpression;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.linq4j.tree.Types;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptCluster;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptPredicateList;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelTraitSet;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.core.Calc;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexBuilder;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexCall;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexProgram;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexSimplify;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexUtil;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.runtime.SqlFunctions;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.schema.Function;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.schema.SchemaPlus;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlOperator;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.validate.SqlConformance;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.validate.SqlConformanceEnum;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.validate.SqlUserDefinedFunction;
 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;
 import org.checkerframework.checker.nullness.qual.Nullable;
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamCoGBKJoinRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamCoGBKJoinRel.java
index 9debc62..2bd3f42 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamCoGBKJoinRel.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamCoGBKJoinRel.java
@@ -40,14 +40,14 @@
 import org.apache.beam.sdk.values.PCollectionList;
 import org.apache.beam.sdk.values.Row;
 import org.apache.beam.sdk.values.WindowingStrategy;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptCluster;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelTraitSet;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.CorrelationId;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.Join;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.JoinRelType;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.util.Pair;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptCluster;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelTraitSet;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.core.CorrelationId;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.core.Join;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.core.JoinRelType;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.util.Pair;
 
 /**
  * A {@code BeamJoinRel} which does CoGBK Join
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamEnumerableConverter.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamEnumerableConverter.java
index 9b91285..4b900d8 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamEnumerableConverter.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamEnumerableConverter.java
@@ -17,7 +17,7 @@
  */
 package org.apache.beam.sdk.extensions.sql.impl.rel;
 
-import static org.apache.beam.vendor.calcite.v1_20_0.com.google.common.base.Preconditions.checkArgument;
+import static org.apache.beam.vendor.calcite.v1_26_0.com.google.common.base.Preconditions.checkArgument;
 
 import java.io.IOException;
 import java.time.LocalDate;
@@ -57,24 +57,24 @@
 import org.apache.beam.sdk.values.PCollection.IsBounded;
 import org.apache.beam.sdk.values.PValue;
 import org.apache.beam.sdk.values.Row;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.adapter.enumerable.EnumerableRel;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.adapter.enumerable.EnumerableRelImplementor;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.adapter.enumerable.PhysType;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.adapter.enumerable.PhysTypeImpl;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.linq4j.Enumerable;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.linq4j.Linq4j;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.linq4j.tree.BlockBuilder;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.linq4j.tree.Expression;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.linq4j.tree.Expressions;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.ConventionTraitDef;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptCluster;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptCost;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptPlanner;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelTraitSet;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.convert.ConverterImpl;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.metadata.RelMetadataQuery;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.type.RelDataType;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.adapter.enumerable.EnumerableRel;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.adapter.enumerable.EnumerableRelImplementor;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.adapter.enumerable.PhysType;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.adapter.enumerable.PhysTypeImpl;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.linq4j.Enumerable;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.linq4j.Linq4j;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.linq4j.tree.BlockBuilder;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.linq4j.tree.Expression;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.linq4j.tree.Expressions;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.ConventionTraitDef;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptCluster;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptCost;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptPlanner;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelTraitSet;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.convert.ConverterImpl;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.type.RelDataType;
 import org.checkerframework.checker.nullness.qual.Nullable;
 import org.joda.time.Duration;
 import org.joda.time.ReadableInstant;
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamIOSinkRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamIOSinkRel.java
index 3af4509..f672621 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamIOSinkRel.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamIOSinkRel.java
@@ -17,7 +17,7 @@
  */
 package org.apache.beam.sdk.extensions.sql.impl.rel;
 
-import static org.apache.beam.vendor.calcite.v1_20_0.com.google.common.base.Preconditions.checkArgument;
+import static org.apache.beam.vendor.calcite.v1_26_0.com.google.common.base.Preconditions.checkArgument;
 
 import java.util.List;
 import java.util.Map;
@@ -29,16 +29,16 @@
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PCollectionList;
 import org.apache.beam.sdk.values.Row;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptCluster;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptPlanner;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptTable;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelTraitSet;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.prepare.Prepare;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.TableModify;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.metadata.RelMetadataQuery;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql2rel.RelStructuredTypeFlattener;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptCluster;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptPlanner;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptTable;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelTraitSet;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.prepare.Prepare;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.core.TableModify;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql2rel.RelStructuredTypeFlattener;
 
 /** BeamRelNode to replace a {@code TableModify} node. */
 public class BeamIOSinkRel extends TableModify
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamIOSourceRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamIOSourceRel.java
index f672384..3917c8e 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamIOSourceRel.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamIOSourceRel.java
@@ -17,7 +17,7 @@
  */
 package org.apache.beam.sdk.extensions.sql.impl.rel;
 
-import static org.apache.beam.vendor.calcite.v1_20_0.com.google.common.base.Preconditions.checkArgument;
+import static org.apache.beam.vendor.calcite.v1_26_0.com.google.common.base.Preconditions.checkArgument;
 
 import java.util.List;
 import java.util.Map;
@@ -31,15 +31,15 @@
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PCollectionList;
 import org.apache.beam.sdk.values.Row;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptCluster;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptCost;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptPlanner;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptTable;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelTraitSet;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.prepare.RelOptTableImpl;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.TableScan;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.metadata.RelMetadataQuery;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.type.RelDataType;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptCluster;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptCost;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptPlanner;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptTable;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelTraitSet;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.prepare.RelOptTableImpl;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.core.TableScan;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.type.RelDataType;
 
 /** BeamRelNode to replace a {@code TableScan} node. */
 public class BeamIOSourceRel extends TableScan implements BeamRelNode {
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamIntersectRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamIntersectRel.java
index 80db503..e69e14a 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamIntersectRel.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamIntersectRel.java
@@ -24,13 +24,13 @@
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PCollectionList;
 import org.apache.beam.sdk.values.Row;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptCluster;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptPlanner;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelTraitSet;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.Intersect;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.SetOp;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptCluster;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptPlanner;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelTraitSet;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.core.Intersect;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.core.SetOp;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.metadata.RelMetadataQuery;
 
 /**
  * {@code BeamRelNode} to replace a {@code Intersect} node.
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamJoinRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamJoinRel.java
index d0a42cb..d81e8a7 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamJoinRel.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamJoinRel.java
@@ -25,23 +25,23 @@
 import org.apache.beam.sdk.extensions.sql.impl.planner.NodeStats;
 import org.apache.beam.sdk.extensions.sql.meta.BeamSqlTable;
 import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.vendor.calcite.v1_20_0.com.google.common.base.Optional;
-import org.apache.beam.vendor.calcite.v1_20_0.com.google.common.collect.ImmutableList;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptCluster;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptPlanner;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelTraitSet;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.volcano.RelSubset;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.CorrelationId;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.Join;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.JoinRelType;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.metadata.RelMetadataQuery;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexCall;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexFieldAccess;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexInputRef;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexLiteral;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.util.Pair;
+import org.apache.beam.vendor.calcite.v1_26_0.com.google.common.base.Optional;
+import org.apache.beam.vendor.calcite.v1_26_0.com.google.common.collect.ImmutableList;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptCluster;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptPlanner;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelTraitSet;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.volcano.RelSubset;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.core.CorrelationId;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.core.Join;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.core.JoinRelType;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexCall;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexFieldAccess;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexInputRef;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexLiteral;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.util.Pair;
 
 /**
  * An abstract {@code BeamRelNode} to implement Join Rels.
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamLogicalConvention.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamLogicalConvention.java
index 0cbe757..f744973 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamLogicalConvention.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamLogicalConvention.java
@@ -17,12 +17,12 @@
  */
 package org.apache.beam.sdk.extensions.sql.impl.rel;
 
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.Convention;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.ConventionTraitDef;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptPlanner;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelTrait;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelTraitDef;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelTraitSet;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.Convention;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.ConventionTraitDef;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptPlanner;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelTrait;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelTraitDef;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelTraitSet;
 
 /** Convertion for Beam SQL. */
 @SuppressWarnings({
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamMatchRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamMatchRel.java
index d970c69..37615c8 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamMatchRel.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamMatchRel.java
@@ -18,7 +18,7 @@
 package org.apache.beam.sdk.extensions.sql.impl.rel;
 
 import static org.apache.beam.sdk.extensions.sql.impl.cep.CEPUtils.makeOrderKeysFromCollation;
-import static org.apache.beam.vendor.calcite.v1_20_0.com.google.common.base.Preconditions.checkArgument;
+import static org.apache.beam.vendor.calcite.v1_26_0.com.google.common.base.Preconditions.checkArgument;
 
 import java.util.ArrayList;
 import java.util.List;
@@ -48,18 +48,18 @@
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PCollectionList;
 import org.apache.beam.sdk.values.Row;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptCluster;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptPlanner;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelTraitSet;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelCollation;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.Match;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.metadata.RelMetadataQuery;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.type.RelDataType;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexCall;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexInputRef;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlKind;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptCluster;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptPlanner;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelTraitSet;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelCollation;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.core.Match;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.type.RelDataType;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexCall;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlKind;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.util.ImmutableBitSet;
 
 /**
  * {@code BeamRelNode} to replace a {@code Match} node.
@@ -87,7 +87,7 @@
       RexNode after,
       Map<String, ? extends SortedSet<String>> subsets,
       boolean allRows,
-      List<RexNode> partitionKeys,
+      ImmutableBitSet partitionKeys,
       RelCollation orderKeys,
       RexNode interval) {
 
@@ -134,7 +134,7 @@
 
   private class MatchTransform extends PTransform<PCollectionList<Row>, PCollection<Row>> {
 
-    private final List<RexNode> parKeys;
+    private final ImmutableBitSet partitionKeys;
     private final RelCollation orderKeys;
     private final Map<String, RexNode> measures;
     private final boolean allRows;
@@ -142,13 +142,13 @@
     private final Map<String, RexNode> patternDefs;
 
     public MatchTransform(
-        List<RexNode> parKeys,
+        ImmutableBitSet partitionKeys,
         RelCollation orderKeys,
         Map<String, RexNode> measures,
         boolean allRows,
         RexNode pattern,
         Map<String, RexNode> patternDefs) {
-      this.parKeys = parKeys;
+      this.partitionKeys = partitionKeys;
       this.orderKeys = orderKeys;
       this.measures = measures;
       this.allRows = allRows;
@@ -168,9 +168,7 @@
       Schema outSchema = CalciteUtils.toSchema(getRowType());
 
       Schema.Builder schemaBuilder = new Schema.Builder();
-      for (RexNode i : parKeys) {
-        RexInputRef varNode = (RexInputRef) i;
-        int index = varNode.getIndex();
+      for (int index : partitionKeys.asList()) {
         schemaBuilder.addField(upstreamSchema.getField(index));
       }
       Schema partitionKeySchema = schemaBuilder.build();
@@ -217,7 +215,7 @@
       // apply the ParDo for the match process and measures clause
       // for now, support FINAL only
       // TODO: add support for FINAL/RUNNING
-      List<CEPFieldRef> cepParKeys = CEPUtils.getCEPFieldRefFromParKeys(parKeys);
+      List<CEPFieldRef> cepParKeys = CEPUtils.getCEPFieldRefFromParKeys(partitionKeys);
       PCollection<Row> outStream =
           orderedUpstream
               .apply(
@@ -236,20 +234,20 @@
 
     private final Schema upstreamSchema;
     private final Schema outSchema;
-    private final List<CEPFieldRef> parKeys;
+    private final List<CEPFieldRef> partitionKeys;
     private final ArrayList<CEPPattern> pattern;
     private final List<CEPMeasure> measures;
     private final boolean allRows;
 
     MatchPattern(
         Schema upstreamSchema,
-        List<CEPFieldRef> parKeys,
+        List<CEPFieldRef> partitionKeys,
         ArrayList<CEPPattern> pattern,
         List<CEPMeasure> measures,
         boolean allRows,
         Schema outSchema) {
       this.upstreamSchema = upstreamSchema;
-      this.parKeys = parKeys;
+      this.partitionKeys = partitionKeys;
       this.pattern = pattern;
       this.measures = measures;
       this.allRows = allRows;
@@ -283,18 +281,18 @@
           Row.FieldValueBuilder newFieldBuilder = null;
 
           // add partition key columns
-          for (CEPFieldRef i : parKeys) {
+          for (CEPFieldRef i : partitionKeys) {
             int colIndex = i.getIndex();
             Schema.Field parSchema = upstreamSchema.getField(colIndex);
             String fieldName = parSchema.getName();
             if (!result.isEmpty()) {
-              Row parKeyRow = keyRows.getKey();
+              Row partitionKeyRow = keyRows.getKey();
               if (newFieldBuilder == null) {
                 newFieldBuilder =
-                    newRowBuilder.withFieldValue(fieldName, parKeyRow.getValue(fieldName));
+                    newRowBuilder.withFieldValue(fieldName, partitionKeyRow.getValue(fieldName));
               } else {
                 newFieldBuilder =
-                    newFieldBuilder.withFieldValue(fieldName, parKeyRow.getValue(fieldName));
+                    newFieldBuilder.withFieldValue(fieldName, partitionKeyRow.getValue(fieldName));
               }
             } else {
               break;
@@ -432,7 +430,6 @@
     }
   }
 
-  @Override
   public Match copy(
       RelNode input,
       RelDataType rowType,
@@ -444,7 +441,7 @@
       RexNode after,
       Map<String, ? extends SortedSet<String>> subsets,
       boolean allRows,
-      List<RexNode> partitionKeys,
+      ImmutableBitSet partitionKeys,
       RelCollation orderKeys,
       RexNode interval) {
 
@@ -465,4 +462,24 @@
         orderKeys,
         interval);
   }
+
+  @Override
+  public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+    return new BeamMatchRel(
+        getCluster(),
+        traitSet,
+        inputs.get(0),
+        rowType,
+        pattern,
+        strictStart,
+        strictEnd,
+        patternDefinitions,
+        measures,
+        after,
+        subsets,
+        allRows,
+        partitionKeys,
+        orderKeys,
+        interval);
+  }
 }
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamMinusRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamMinusRel.java
index 5e9e075..53357b4 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamMinusRel.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamMinusRel.java
@@ -24,13 +24,13 @@
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PCollectionList;
 import org.apache.beam.sdk.values.Row;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptCluster;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptPlanner;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelTraitSet;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.Minus;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.SetOp;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptCluster;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptPlanner;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelTraitSet;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.core.Minus;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.core.SetOp;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.metadata.RelMetadataQuery;
 
 /**
  * {@code BeamRelNode} to replace a {@code Minus} node.
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamPushDownIOSourceRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamPushDownIOSourceRel.java
index 1e9b551..edc3240 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamPushDownIOSourceRel.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamPushDownIOSourceRel.java
@@ -17,7 +17,7 @@
  */
 package org.apache.beam.sdk.extensions.sql.impl.rel;
 
-import static org.apache.beam.vendor.calcite.v1_20_0.com.google.common.base.Preconditions.checkArgument;
+import static org.apache.beam.vendor.calcite.v1_26_0.com.google.common.base.Preconditions.checkArgument;
 
 import java.util.List;
 import java.util.Map;
@@ -33,13 +33,13 @@
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PCollectionList;
 import org.apache.beam.sdk.values.Row;
-import org.apache.beam.vendor.calcite.v1_20_0.com.google.common.base.Preconditions;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptCluster;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptPlanner;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptTable;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelTraitSet;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelWriter;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.beam.vendor.calcite.v1_26_0.com.google.common.base.Preconditions;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptCluster;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptPlanner;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptTable;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelTraitSet;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelWriter;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.metadata.RelMetadataQuery;
 
 public class BeamPushDownIOSourceRel extends BeamIOSourceRel {
   private final List<String> usedFields;
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamRelNode.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamRelNode.java
index 935ec6c..fc7bdf1 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamRelNode.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamRelNode.java
@@ -25,9 +25,9 @@
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PCollectionList;
 import org.apache.beam.sdk.values.Row;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptPlanner;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptPlanner;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.metadata.RelMetadataQuery;
 
 /** A {@link RelNode} that can also give a {@link PTransform} that implements the expression. */
 @SuppressWarnings({
@@ -76,8 +76,8 @@
    * estimate its NodeStats, it may need NodeStat of its inputs. However, it should not call this
    * directly (because maybe its inputs are not physical yet). It should call {@link
    * org.apache.beam.sdk.extensions.sql.impl.rel.BeamSqlRelUtils#getNodeStats(
-   * org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode,
-   * org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.metadata.RelMetadataQuery)}
+   * org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelNode,
+   * org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.metadata.RelMetadataQuery)}
    * instead.
    */
   NodeStats estimateNodeStats(RelMetadataQuery mq);
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSetOperatorRelBase.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSetOperatorRelBase.java
index 9f08734..ca7bd21 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSetOperatorRelBase.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSetOperatorRelBase.java
@@ -17,7 +17,7 @@
  */
 package org.apache.beam.sdk.extensions.sql.impl.rel;
 
-import static org.apache.beam.vendor.calcite.v1_20_0.com.google.common.base.Preconditions.checkArgument;
+import static org.apache.beam.vendor.calcite.v1_26_0.com.google.common.base.Preconditions.checkArgument;
 
 import java.io.Serializable;
 import org.apache.beam.sdk.extensions.sql.impl.transform.BeamSetOperatorsTransforms;
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSideInputJoinRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSideInputJoinRel.java
index ebe38ae..7fedd9b 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSideInputJoinRel.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSideInputJoinRel.java
@@ -32,14 +32,14 @@
 import org.apache.beam.sdk.values.PCollection.IsBounded;
 import org.apache.beam.sdk.values.PCollectionList;
 import org.apache.beam.sdk.values.Row;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptCluster;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelTraitSet;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.CorrelationId;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.Join;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.JoinRelType;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.util.Pair;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptCluster;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelTraitSet;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.core.CorrelationId;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.core.Join;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.core.JoinRelType;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.util.Pair;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions;
 
 /**
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSideInputLookupJoinRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSideInputLookupJoinRel.java
index b4dbd56..248fd98 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSideInputLookupJoinRel.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSideInputLookupJoinRel.java
@@ -26,13 +26,13 @@
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PCollectionList;
 import org.apache.beam.sdk.values.Row;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptCluster;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelTraitSet;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.CorrelationId;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.Join;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.JoinRelType;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptCluster;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelTraitSet;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.core.CorrelationId;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.core.Join;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.core.JoinRelType;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexNode;
 
 /**
  * A {@code BeamJoinRel} which does Lookup Join
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSortRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSortRel.java
index 3c056f6..a636ec2 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSortRel.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSortRel.java
@@ -17,8 +17,8 @@
  */
 package org.apache.beam.sdk.extensions.sql.impl.rel;
 
-import static org.apache.beam.vendor.calcite.v1_20_0.com.google.common.base.MoreObjects.firstNonNull;
-import static org.apache.beam.vendor.calcite.v1_20_0.com.google.common.base.Preconditions.checkArgument;
+import static org.apache.beam.vendor.calcite.v1_26_0.com.google.common.base.MoreObjects.firstNonNull;
+import static org.apache.beam.vendor.calcite.v1_26_0.com.google.common.base.Preconditions.checkArgument;
 
 import java.io.Serializable;
 import java.math.BigDecimal;
@@ -50,19 +50,19 @@
 import org.apache.beam.sdk.values.PCollectionList;
 import org.apache.beam.sdk.values.Row;
 import org.apache.beam.sdk.values.WindowingStrategy;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptCluster;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptPlanner;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelTraitSet;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelCollation;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelCollationImpl;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelFieldCollation;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.Sort;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.metadata.RelMetadataQuery;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexInputRef;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexLiteral;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptCluster;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptPlanner;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelTraitSet;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelCollation;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelCollationImpl;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelFieldCollation;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.core.Sort;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexInputRef;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexLiteral;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.type.SqlTypeName;
 
 /**
  * {@code BeamRelNode} to replace a {@code Sort} node.
@@ -112,7 +112,8 @@
       RexNode fetch) {
     super(cluster, traits, child, collation, offset, fetch);
 
-    List<RexNode> fieldExps = getChildExps();
+    // https://issues.apache.org/jira/browse/CALCITE-4079?focusedCommentId=17165904&page=com.atlassian.jira.plugin.system.issuetabpanels%3Acomment-tabpanel#comment-17165904
+    List<RexNode> fieldExps = getSortExps();
     RelCollationImpl collationImpl = (RelCollationImpl) collation;
     List<RelFieldCollation> collations = collationImpl.getFieldCollations();
     for (int i = 0; i < fieldExps.size(); i++) {
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSqlRelUtils.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSqlRelUtils.java
index fbe6dd4..9dd731f 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSqlRelUtils.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSqlRelUtils.java
@@ -28,9 +28,9 @@
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PCollectionList;
 import org.apache.beam.sdk.values.Row;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.volcano.RelSubset;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.volcano.RelSubset;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.metadata.RelMetadataQuery;
 
 /** Utilities for {@code BeamRelNode}. */
 @SuppressWarnings({
@@ -50,7 +50,16 @@
     } else {
       return PCollectionList.of(
           inputRels.stream()
-              .map(input -> BeamSqlRelUtils.toPCollection(pipeline, (BeamRelNode) input, cache))
+              .map(
+                  input -> {
+                    final BeamRelNode beamRel;
+                    if (input instanceof RelSubset) {
+                      beamRel = (BeamRelNode) ((RelSubset) input).getBest();
+                    } else {
+                      beamRel = (BeamRelNode) input;
+                    }
+                    return BeamSqlRelUtils.toPCollection(pipeline, beamRel, cache);
+                  })
               .collect(Collectors.toList()));
     }
   }
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamTableFunctionScanRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamTableFunctionScanRel.java
index 8f45a3f..7bf28e3 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamTableFunctionScanRel.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamTableFunctionScanRel.java
@@ -17,7 +17,7 @@
  */
 package org.apache.beam.sdk.extensions.sql.impl.rel;
 
-import static org.apache.beam.vendor.calcite.v1_20_0.com.google.common.base.Preconditions.checkArgument;
+import static org.apache.beam.vendor.calcite.v1_26_0.com.google.common.base.Preconditions.checkArgument;
 
 import java.lang.reflect.Type;
 import java.util.ArrayList;
@@ -51,18 +51,18 @@
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PCollectionList;
 import org.apache.beam.sdk.values.Row;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptCluster;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptPlanner;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelTraitSet;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.TableFunctionScan;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.metadata.RelColumnMapping;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.metadata.RelMetadataQuery;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.type.RelDataType;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexCall;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexInputRef;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexLiteral;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptCluster;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptPlanner;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelTraitSet;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.core.TableFunctionScan;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.metadata.RelColumnMapping;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.type.RelDataType;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexCall;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexInputRef;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexLiteral;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexNode;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
 import org.joda.time.Duration;
 
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamUncollectRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamUncollectRel.java
index b569a22..40d3b5f 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamUncollectRel.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamUncollectRel.java
@@ -17,8 +17,9 @@
  */
 package org.apache.beam.sdk.extensions.sql.impl.rel;
 
-import static org.apache.beam.vendor.calcite.v1_20_0.com.google.common.base.Preconditions.checkArgument;
+import static org.apache.beam.vendor.calcite.v1_26_0.com.google.common.base.Preconditions.checkArgument;
 
+import java.util.Collections;
 import org.apache.beam.sdk.extensions.sql.impl.planner.BeamCostModel;
 import org.apache.beam.sdk.extensions.sql.impl.planner.NodeStats;
 import org.apache.beam.sdk.extensions.sql.impl.utils.CalciteUtils;
@@ -29,12 +30,12 @@
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PCollectionList;
 import org.apache.beam.sdk.values.Row;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptCluster;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptPlanner;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelTraitSet;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.Uncollect;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptCluster;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptPlanner;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelTraitSet;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.core.Uncollect;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.metadata.RelMetadataQuery;
 
 /** {@link BeamRelNode} to implement an uncorrelated {@link Uncollect}, aka UNNEST. */
 @SuppressWarnings({
@@ -44,7 +45,7 @@
 
   public BeamUncollectRel(
       RelOptCluster cluster, RelTraitSet traitSet, RelNode input, boolean withOrdinality) {
-    super(cluster, traitSet, input, withOrdinality);
+    super(cluster, traitSet, input, withOrdinality, Collections.emptyList());
   }
 
   @Override
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamUnionRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamUnionRel.java
index 5fc3d07..2e41010 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamUnionRel.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamUnionRel.java
@@ -25,13 +25,13 @@
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PCollectionList;
 import org.apache.beam.sdk.values.Row;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptCluster;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptPlanner;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelTraitSet;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.SetOp;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.Union;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptCluster;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptPlanner;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelTraitSet;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.core.SetOp;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.core.Union;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.metadata.RelMetadataQuery;
 
 /**
  * {@link BeamRelNode} to replace a {@link Union}.
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamUnnestRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamUnnestRel.java
index 811d642..4fb1ec6 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamUnnestRel.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamUnnestRel.java
@@ -29,18 +29,18 @@
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PCollectionList;
 import org.apache.beam.sdk.values.Row;
-import org.apache.beam.vendor.calcite.v1_20_0.com.google.common.collect.ImmutableList;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptCluster;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptPlanner;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelTraitSet;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelWriter;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.Correlate;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.JoinRelType;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.Uncollect;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.metadata.RelMetadataQuery;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.type.RelDataType;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.validate.SqlValidatorUtil;
+import org.apache.beam.vendor.calcite.v1_26_0.com.google.common.collect.ImmutableList;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptCluster;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptPlanner;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelTraitSet;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelWriter;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.core.Correlate;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.core.JoinRelType;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.core.Uncollect;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.type.RelDataType;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.validate.SqlValidatorUtil;
 import org.checkerframework.checker.nullness.qual.Nullable;
 
 /**
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamValuesRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamValuesRel.java
index 9fa5037..c090771 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamValuesRel.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamValuesRel.java
@@ -20,7 +20,7 @@
 import static java.util.stream.Collectors.toList;
 import static org.apache.beam.sdk.extensions.sql.impl.schema.BeamTableUtils.autoCastField;
 import static org.apache.beam.sdk.values.Row.toRow;
-import static org.apache.beam.vendor.calcite.v1_20_0.com.google.common.base.Preconditions.checkArgument;
+import static org.apache.beam.vendor.calcite.v1_26_0.com.google.common.base.Preconditions.checkArgument;
 
 import java.util.List;
 import java.util.Map;
@@ -34,15 +34,15 @@
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PCollectionList;
 import org.apache.beam.sdk.values.Row;
-import org.apache.beam.vendor.calcite.v1_20_0.com.google.common.collect.ImmutableList;
-import org.apache.beam.vendor.calcite.v1_20_0.com.google.common.collect.ImmutableMap;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptCluster;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptPlanner;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelTraitSet;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.Values;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.metadata.RelMetadataQuery;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.type.RelDataType;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexLiteral;
+import org.apache.beam.vendor.calcite.v1_26_0.com.google.common.collect.ImmutableList;
+import org.apache.beam.vendor.calcite.v1_26_0.com.google.common.collect.ImmutableMap;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptCluster;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptPlanner;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelTraitSet;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.core.Values;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.type.RelDataType;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexLiteral;
 
 /**
  * {@code BeamRelNode} to replace a {@code Values} node.
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamWindowRel.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamWindowRel.java
index d9a15a1..a1ed5cd 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamWindowRel.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamWindowRel.java
@@ -40,18 +40,18 @@
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PCollectionList;
 import org.apache.beam.sdk.values.Row;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptCluster;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptPlanner;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelTraitSet;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelFieldCollation;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.AggregateCall;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.Window;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.metadata.RelMetadataQuery;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.type.RelDataType;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexInputRef;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexLiteral;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptCluster;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptPlanner;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelTraitSet;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelFieldCollation;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.core.AggregateCall;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.core.Window;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.type.RelDataType;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexInputRef;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexLiteral;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexNode;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Lists;
 
 /**
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/CalcRelSplitter.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/CalcRelSplitter.java
index f7689bc..c388f62 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/CalcRelSplitter.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/CalcRelSplitter.java
@@ -24,32 +24,32 @@
 import java.util.Collections;
 import java.util.List;
 import java.util.Set;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptCluster;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptPlanner;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelTraitSet;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.Calc;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.logical.LogicalCalc;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.type.RelDataType;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.type.RelDataTypeFactory;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexCall;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexDynamicParam;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexFieldAccess;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexInputRef;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexLiteral;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexLocalRef;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexProgram;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexShuttle;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexUtil;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexVisitorImpl;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.tools.RelBuilder;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.util.Litmus;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.util.Util;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.util.graph.DefaultDirectedGraph;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.util.graph.DefaultEdge;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.util.graph.DirectedGraph;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.util.graph.TopologicalOrderIterator;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptCluster;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptPlanner;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelTraitSet;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.core.Calc;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.logical.LogicalCalc;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.type.RelDataType;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexCall;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexDynamicParam;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexFieldAccess;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexInputRef;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexLiteral;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexLocalRef;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexProgram;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexShuttle;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexUtil;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexVisitorImpl;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.tools.RelBuilder;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.util.Litmus;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.util.Util;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.util.graph.DefaultDirectedGraph;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.util.graph.DefaultEdge;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.util.graph.DirectedGraph;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.util.graph.TopologicalOrderIterator;
 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.primitives.Ints;
 import org.checkerframework.checker.nullness.qual.Nullable;
@@ -60,7 +60,7 @@
  * cannot all be implemented by a single concrete {@link RelNode}.
  *
  * <p>This is a copy of {@link
- * org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.rules.CalcRelSplitter} modified to
+ * org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.rules.CalcRelSplitter} modified to
  * work with Beam. TODO(CALCITE-4538) consider contributing these changes back upstream.
  *
  * <p>For example, the Java and Fennel calculator do not implement an identical set of operators.
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/package-info.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/package-info.java
index 0eb883f..8b81b6d 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/package-info.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rel/package-info.java
@@ -18,7 +18,7 @@
 
 /**
  * BeamSQL specified nodes, to replace {@link
- * org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode}.
+ * org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelNode}.
  */
 @DefaultAnnotation(NonNull.class)
 package org.apache.beam.sdk.extensions.sql.impl.rel;
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamAggregateProjectMergeRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamAggregateProjectMergeRule.java
index 9499e1b..fd7ba21 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamAggregateProjectMergeRule.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamAggregateProjectMergeRule.java
@@ -21,16 +21,16 @@
 import java.util.List;
 import java.util.Set;
 import org.apache.beam.sdk.extensions.sql.impl.rel.BeamIOSourceRel;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptRuleCall;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.volcano.RelSubset;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.SingleRel;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.Aggregate;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.Filter;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.Project;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.RelFactories;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.rules.AggregateProjectMergeRule;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.tools.RelBuilderFactory;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.volcano.RelSubset;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.SingleRel;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.core.Aggregate;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.core.Filter;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.core.Project;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.core.RelFactories;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.rules.AggregateProjectMergeRule;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.tools.RelBuilderFactory;
 
 /**
  * This rule is essentially a wrapper around Calcite's {@code AggregateProjectMergeRule}. In the
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamAggregationRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamAggregationRule.java
index 83e18b6..96c5d91 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamAggregationRule.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamAggregationRule.java
@@ -25,18 +25,18 @@
 import org.apache.beam.sdk.transforms.windowing.Sessions;
 import org.apache.beam.sdk.transforms.windowing.SlidingWindows;
 import org.apache.beam.sdk.transforms.windowing.WindowFn;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptRule;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptRuleCall;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.Aggregate;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.Project;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.RelFactories;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexCall;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexLiteral;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlKind;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.tools.RelBuilderFactory;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.util.ImmutableBitSet;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptRule;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.core.Aggregate;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.core.Project;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.core.RelFactories;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexCall;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexLiteral;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlKind;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.tools.RelBuilderFactory;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.util.ImmutableBitSet;
 import org.checkerframework.checker.nullness.qual.Nullable;
 import org.joda.time.Duration;
 
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamBasicAggregationRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamBasicAggregationRule.java
index cb5702b..15028a7 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamBasicAggregationRule.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamBasicAggregationRule.java
@@ -22,19 +22,19 @@
 import java.util.stream.Collectors;
 import org.apache.beam.sdk.extensions.sql.impl.rel.BeamAggregationRel;
 import org.apache.beam.sdk.extensions.sql.impl.rel.BeamLogicalConvention;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptRule;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptRuleCall;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.volcano.RelSubset;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.Aggregate;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.Calc;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.Filter;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.Project;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.RelFactories;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexCall;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlKind;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.tools.RelBuilderFactory;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptRule;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.volcano.RelSubset;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.core.Aggregate;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.core.Calc;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.core.Filter;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.core.Project;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.core.RelFactories;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexCall;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlKind;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.tools.RelBuilderFactory;
 
 /**
  * Aggregation rule that doesn't include projection.
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamCalcMergeRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamCalcMergeRule.java
index d6e1b6c..210aebf 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamCalcMergeRule.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamCalcMergeRule.java
@@ -18,10 +18,11 @@
 package org.apache.beam.sdk.extensions.sql.impl.rule;
 
 import org.apache.beam.sdk.extensions.sql.impl.rel.BeamCalcRel;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptRule;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptRuleCall;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptRuleOperand;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.rules.CalcMergeRule;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptRule;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptRuleOperand;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.rules.CalcMergeRule;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.rules.CoreRules;
 
 /**
  * Planner rule to merge a {@link BeamCalcRel} with a {@link BeamCalcRel}. Subset of {@link
@@ -36,6 +37,6 @@
 
   @Override
   public void onMatch(RelOptRuleCall call) {
-    CalcMergeRule.INSTANCE.onMatch(call);
+    CoreRules.CALC_MERGE.onMatch(call);
   }
 }
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamCalcRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamCalcRule.java
index 7a82042..01532b4 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamCalcRule.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamCalcRule.java
@@ -20,15 +20,15 @@
 import java.util.List;
 import org.apache.beam.sdk.extensions.sql.impl.rel.BeamCalcRel;
 import org.apache.beam.sdk.extensions.sql.impl.rel.BeamLogicalConvention;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.Convention;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptRule;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptRuleCall;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.convert.ConverterRule;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.Calc;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.logical.LogicalCalc;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexOver;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.Convention;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptRule;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.core.Calc;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.logical.LogicalCalc;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexOver;
 
 /** A {@code ConverterRule} to replace {@link Calc} with {@link BeamCalcRel}. */
 public class BeamCalcRule extends ConverterRule {
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamCalcSplittingRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamCalcSplittingRule.java
index a18dfe3..432df0f 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamCalcSplittingRule.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamCalcSplittingRule.java
@@ -18,13 +18,13 @@
 package org.apache.beam.sdk.extensions.sql.impl.rule;
 
 import org.apache.beam.sdk.extensions.sql.impl.rel.CalcRelSplitter;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptRule;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptRuleCall;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.Calc;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.RelFactories;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.logical.LogicalCalc;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptRule;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.core.Calc;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.core.RelFactories;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.logical.LogicalCalc;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexNode;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamCoGBKJoinRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamCoGBKJoinRule.java
index 516bc09..3d47c84 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamCoGBKJoinRule.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamCoGBKJoinRule.java
@@ -21,12 +21,12 @@
 import org.apache.beam.sdk.extensions.sql.impl.rel.BeamJoinRel;
 import org.apache.beam.sdk.extensions.sql.impl.rel.BeamLogicalConvention;
 import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptRule;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptRuleCall;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.Join;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.RelFactories;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.logical.LogicalJoin;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptRule;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.core.Join;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.core.RelFactories;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.logical.LogicalJoin;
 
 /**
  * Rule to convert {@code LogicalJoin} node to {@code BeamCoGBKJoinRel} node.
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamEnumerableConverterRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamEnumerableConverterRule.java
index 773fef1..15de4ff 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamEnumerableConverterRule.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamEnumerableConverterRule.java
@@ -20,10 +20,10 @@
 import org.apache.beam.sdk.extensions.sql.impl.rel.BeamEnumerableConverter;
 import org.apache.beam.sdk.extensions.sql.impl.rel.BeamLogicalConvention;
 import org.apache.beam.sdk.extensions.sql.impl.rel.BeamRelNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.adapter.enumerable.EnumerableConvention;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelTraitSet;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.adapter.enumerable.EnumerableConvention;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelTraitSet;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.convert.ConverterRule;
 
 /** A {@code ConverterRule} to Convert {@link BeamRelNode} to {@link EnumerableConvention}. */
 public class BeamEnumerableConverterRule extends ConverterRule {
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamIOPushDownRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamIOPushDownRule.java
index 9767fbf..2e2e2fb 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamIOPushDownRule.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamIOPushDownRule.java
@@ -38,24 +38,25 @@
 import org.apache.beam.sdk.schemas.FieldAccessDescriptor.FieldDescriptor;
 import org.apache.beam.sdk.schemas.Schema;
 import org.apache.beam.sdk.schemas.utils.SelectHelpers;
-import org.apache.beam.vendor.calcite.v1_20_0.com.google.common.collect.ImmutableList;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptRule;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptRuleCall;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.Calc;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.RelFactories;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.type.RelDataType;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.type.RelDataTypeField;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.type.RelRecordType;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexCall;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexInputRef;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexLiteral;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexLocalRef;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexProgram;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.tools.RelBuilder;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.tools.RelBuilderFactory;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.util.Pair;
+import org.apache.beam.vendor.calcite.v1_26_0.com.google.common.collect.ImmutableList;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptRule;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.core.Calc;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.core.RelFactories;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.type.RelDataType;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.type.RelRecordType;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexCall;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexInputRef;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexLiteral;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexLocalRef;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexProgram;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexUtil;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.tools.RelBuilder;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.tools.RelBuilderFactory;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.util.Pair;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.annotations.VisibleForTesting;
 
 @SuppressWarnings({
@@ -140,8 +141,6 @@
     resolved = resolved.resolve(beamSqlTable.getSchema());
 
     if (canDropCalc(program, beamSqlTable.supportsProjects(), tableFilter)) {
-      // Tell the optimizer to not use old IO, since the new one is better.
-      call.getPlanner().setImportance(ioSourceRel, 0.0);
       call.transformTo(
           ioSourceRel.createPushDownRel(
               calc.getRowType(),
@@ -173,8 +172,6 @@
         || usedFields.size() < calcInputRowType.getFieldCount()) {
       // Smaller Calc programs are indisputably better, as well as IOs with less projected fields.
       // We can consider something with the same number of filters.
-      // Tell the optimizer not to use old Calc and IO.
-      call.getPlanner().setImportance(ioSourceRel, 0);
       call.transformTo(result);
     }
   }
@@ -369,9 +366,13 @@
       newProjects.add(reMapRexNodeToNewInputs(project, mapping));
     }
 
+    if (RexUtil.isIdentity(newProjects, newIoSourceRel.getRowType())) {
+      // Force a rename prior to filter for identity function.
+      relBuilder.project(newProjects, calcDataType.getFieldNames(), true);
+    }
+
     relBuilder.filter(newFilter);
-    // Force to preserve named projects.
-    relBuilder.project(newProjects, calcDataType.getFieldNames(), true);
+    relBuilder.project(newProjects, calcDataType.getFieldNames());
 
     return relBuilder.build();
   }
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamIOSinkRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamIOSinkRule.java
index d67e106..0e7a223 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamIOSinkRule.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamIOSinkRule.java
@@ -20,9 +20,9 @@
 import java.util.Arrays;
 import org.apache.beam.sdk.extensions.sql.impl.rel.BeamIOSinkRel;
 import org.apache.beam.sdk.extensions.sql.impl.rel.BeamLogicalConvention;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.convert.ConverterRule;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.TableModify;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.core.TableModify;
 
 /** A {@code ConverterRule} to replace {@link TableModify} with {@link BeamIOSinkRel}. */
 public class BeamIOSinkRule extends ConverterRule {
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamIntersectRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamIntersectRule.java
index 1a91e4c..7c51dea 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamIntersectRule.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamIntersectRule.java
@@ -20,11 +20,11 @@
 import java.util.List;
 import org.apache.beam.sdk.extensions.sql.impl.rel.BeamIntersectRel;
 import org.apache.beam.sdk.extensions.sql.impl.rel.BeamLogicalConvention;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.Convention;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.convert.ConverterRule;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.Intersect;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.logical.LogicalIntersect;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.Convention;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.core.Intersect;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.logical.LogicalIntersect;
 
 /** {@code ConverterRule} to replace {@code Intersect} with {@code BeamIntersectRel}. */
 public class BeamIntersectRule extends ConverterRule {
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamJoinAssociateRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamJoinAssociateRule.java
index 3eb7ab5..d9445da 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamJoinAssociateRule.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamJoinAssociateRule.java
@@ -18,15 +18,15 @@
 package org.apache.beam.sdk.extensions.sql.impl.rule;
 
 import org.apache.beam.sdk.extensions.sql.impl.rel.BeamJoinRel;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptRuleCall;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.Join;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.RelFactories;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.rules.JoinAssociateRule;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.tools.RelBuilderFactory;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.core.Join;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.core.RelFactories;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.rules.JoinAssociateRule;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.tools.RelBuilderFactory;
 
 /**
  * This is very similar to {@link
- * org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.rules.JoinAssociateRule}. It only
+ * org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.rules.JoinAssociateRule}. It only
  * checks if the resulting condition is supported before transforming.
  */
 public class BeamJoinAssociateRule extends JoinAssociateRule {
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamJoinPushThroughJoinRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamJoinPushThroughJoinRule.java
index f2a10b9..45df21c 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamJoinPushThroughJoinRule.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamJoinPushThroughJoinRule.java
@@ -18,48 +18,46 @@
 package org.apache.beam.sdk.extensions.sql.impl.rule;
 
 import org.apache.beam.sdk.extensions.sql.impl.rel.BeamJoinRel;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptRule;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptRuleCall;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.Join;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.RelFactories;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.logical.LogicalJoin;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.rules.JoinPushThroughJoinRule;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.tools.RelBuilderFactory;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptRule;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.core.Join;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.core.RelFactories;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.logical.LogicalJoin;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.rules.JoinPushThroughJoinRule;
 
 /**
  * This is exactly similar to {@link
- * org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.rules.JoinPushThroughJoinRule}. It
+ * org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.rules.JoinPushThroughJoinRule}. It
  * only checks if the condition of the new bottom join is supported.
  */
-public class BeamJoinPushThroughJoinRule extends JoinPushThroughJoinRule {
+public class BeamJoinPushThroughJoinRule extends RelOptRule {
   /** Instance of the rule that works on logical joins only, and pushes to the right. */
   public static final RelOptRule RIGHT =
       new BeamJoinPushThroughJoinRule(
-          "BeamJoinPushThroughJoinRule:right",
-          true,
-          LogicalJoin.class,
-          RelFactories.LOGICAL_BUILDER);
+          "BeamJoinPushThroughJoinRule:right", JoinPushThroughJoinRule.RIGHT);
 
   /** Instance of the rule that works on logical joins only, and pushes to the left. */
   public static final RelOptRule LEFT =
       new BeamJoinPushThroughJoinRule(
-          "BeamJoinPushThroughJoinRule:left",
-          false,
-          LogicalJoin.class,
-          RelFactories.LOGICAL_BUILDER);
+          "BeamJoinPushThroughJoinRule:left", JoinPushThroughJoinRule.LEFT);
+
+  private final RelOptRule base;
 
   /** Creates a JoinPushThroughJoinRule. */
-  private BeamJoinPushThroughJoinRule(
-      String description,
-      boolean right,
-      Class<? extends Join> clazz,
-      RelBuilderFactory relBuilderFactory) {
-    super(description, right, clazz, relBuilderFactory);
+  private BeamJoinPushThroughJoinRule(String description, RelOptRule base) {
+    super(
+        operand(
+            LogicalJoin.class, operand(LogicalJoin.class, any()), operand(RelNode.class, any())),
+        RelFactories.LOGICAL_BUILDER,
+        description);
+
+    this.base = base;
   }
 
   @Override
   public void onMatch(RelOptRuleCall call) {
-    super.onMatch(
+    base.onMatch(
         new JoinRelOptRuleCall(
             call,
             rel -> {
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamMatchRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamMatchRule.java
index 6441c79..c70162d 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamMatchRule.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamMatchRule.java
@@ -19,11 +19,11 @@
 
 import org.apache.beam.sdk.extensions.sql.impl.rel.BeamLogicalConvention;
 import org.apache.beam.sdk.extensions.sql.impl.rel.BeamMatchRel;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.Convention;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.convert.ConverterRule;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.Match;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.logical.LogicalMatch;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.Convention;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.core.Match;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.logical.LogicalMatch;
 
 /** {@code ConverterRule} to replace {@code Match} with {@code BeamMatchRel}. */
 public class BeamMatchRule extends ConverterRule {
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamMinusRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamMinusRule.java
index 29d4a97..24b691a 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamMinusRule.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamMinusRule.java
@@ -20,11 +20,11 @@
 import java.util.List;
 import org.apache.beam.sdk.extensions.sql.impl.rel.BeamLogicalConvention;
 import org.apache.beam.sdk.extensions.sql.impl.rel.BeamMinusRel;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.Convention;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.convert.ConverterRule;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.Minus;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.logical.LogicalMinus;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.Convention;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.core.Minus;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.logical.LogicalMinus;
 
 /** {@code ConverterRule} to replace {@code Minus} with {@code BeamMinusRel}. */
 public class BeamMinusRule extends ConverterRule {
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamSideInputJoinRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamSideInputJoinRule.java
index 98227bb..3aa2fa4 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamSideInputJoinRule.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamSideInputJoinRule.java
@@ -21,12 +21,12 @@
 import org.apache.beam.sdk.extensions.sql.impl.rel.BeamLogicalConvention;
 import org.apache.beam.sdk.extensions.sql.impl.rel.BeamSideInputJoinRel;
 import org.apache.beam.sdk.values.PCollection;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptRule;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptRuleCall;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.Join;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.RelFactories;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.logical.LogicalJoin;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptRule;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.core.Join;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.core.RelFactories;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.logical.LogicalJoin;
 
 /**
  * Rule to convert {@code LogicalJoin} node to {@code BeamSideInputJoinRel} node.
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamSideInputLookupJoinRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamSideInputLookupJoinRule.java
index 2c96bd95..c94e72e 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamSideInputLookupJoinRule.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamSideInputLookupJoinRule.java
@@ -20,12 +20,12 @@
 import org.apache.beam.sdk.extensions.sql.impl.rel.BeamJoinRel;
 import org.apache.beam.sdk.extensions.sql.impl.rel.BeamLogicalConvention;
 import org.apache.beam.sdk.extensions.sql.impl.rel.BeamSideInputLookupJoinRel;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.Convention;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptRuleCall;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.convert.ConverterRule;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.Join;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.logical.LogicalJoin;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.Convention;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.core.Join;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.logical.LogicalJoin;
 
 /**
  * Rule to convert {@code LogicalJoin} node to {@code BeamSideInputLookupJoinRel} node.
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamSortRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamSortRule.java
index 1647bf7..06c17ec 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamSortRule.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamSortRule.java
@@ -19,11 +19,11 @@
 
 import org.apache.beam.sdk.extensions.sql.impl.rel.BeamLogicalConvention;
 import org.apache.beam.sdk.extensions.sql.impl.rel.BeamSortRel;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.Convention;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.convert.ConverterRule;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.Sort;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.logical.LogicalSort;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.Convention;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.core.Sort;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.logical.LogicalSort;
 
 /** {@code ConverterRule} to replace {@code Sort} with {@code BeamSortRel}. */
 public class BeamSortRule extends ConverterRule {
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamTableFunctionScanRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamTableFunctionScanRule.java
index 20959b5..a78bd22 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamTableFunctionScanRule.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamTableFunctionScanRule.java
@@ -17,18 +17,18 @@
  */
 package org.apache.beam.sdk.extensions.sql.impl.rule;
 
-import static org.apache.beam.vendor.calcite.v1_20_0.com.google.common.base.Preconditions.checkArgument;
+import static org.apache.beam.vendor.calcite.v1_26_0.com.google.common.base.Preconditions.checkArgument;
 
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
 import org.apache.beam.sdk.extensions.sql.impl.rel.BeamLogicalConvention;
 import org.apache.beam.sdk.extensions.sql.impl.rel.BeamTableFunctionScanRel;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.Convention;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.convert.ConverterRule;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.TableFunctionScan;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.logical.LogicalTableFunctionScan;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.Convention;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.core.TableFunctionScan;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.logical.LogicalTableFunctionScan;
 
 /**
  * This is the conveter rule that converts a Calcite {@code TableFunctionScan} to Beam {@code
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamUncollectRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamUncollectRule.java
index 393882b..fdbdbbc 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamUncollectRule.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamUncollectRule.java
@@ -19,10 +19,10 @@
 
 import org.apache.beam.sdk.extensions.sql.impl.rel.BeamLogicalConvention;
 import org.apache.beam.sdk.extensions.sql.impl.rel.BeamUncollectRel;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.Convention;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.convert.ConverterRule;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.Uncollect;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.Convention;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.core.Uncollect;
 
 /** A {@code ConverterRule} to replace {@link Uncollect} with {@link BeamUncollectRule}. */
 public class BeamUncollectRule extends ConverterRule {
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamUnionRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamUnionRule.java
index 7b84e25..704a937 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamUnionRule.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamUnionRule.java
@@ -19,15 +19,15 @@
 
 import org.apache.beam.sdk.extensions.sql.impl.rel.BeamLogicalConvention;
 import org.apache.beam.sdk.extensions.sql.impl.rel.BeamUnionRel;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.Convention;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.convert.ConverterRule;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.Union;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.logical.LogicalUnion;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.Convention;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.core.Union;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.logical.LogicalUnion;
 
 /**
  * A {@code ConverterRule} to replace {@link
- * org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.Union} with {@link
+ * org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.core.Union} with {@link
  * BeamUnionRule}.
  */
 public class BeamUnionRule extends ConverterRule {
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamUnnestRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamUnnestRule.java
index 2c91e5a..07812e1 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamUnnestRule.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamUnnestRule.java
@@ -19,19 +19,19 @@
 
 import org.apache.beam.sdk.extensions.sql.impl.rel.BeamLogicalConvention;
 import org.apache.beam.sdk.extensions.sql.impl.rel.BeamUnnestRel;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptRule;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptRuleCall;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.volcano.RelSubset;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.SingleRel;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.Correlate;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.JoinRelType;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.Uncollect;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.logical.LogicalCorrelate;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.logical.LogicalProject;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexFieldAccess;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexNode;
-import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
+import org.apache.beam.vendor.calcite.v1_26_0.com.google.common.collect.ImmutableList;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptRule;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.volcano.RelSubset;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.SingleRel;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.core.Correlate;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.core.JoinRelType;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.core.Uncollect;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.logical.LogicalCorrelate;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.logical.LogicalProject;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexFieldAccess;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexNode;
 
 /**
  * A {@code ConverterRule} to replace {@link Correlate} {@link Uncollect} with {@link
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamValuesRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamValuesRule.java
index 6fbe1e0..98b9254 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamValuesRule.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamValuesRule.java
@@ -19,11 +19,11 @@
 
 import org.apache.beam.sdk.extensions.sql.impl.rel.BeamLogicalConvention;
 import org.apache.beam.sdk.extensions.sql.impl.rel.BeamValuesRel;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.Convention;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.convert.ConverterRule;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.Values;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.logical.LogicalValues;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.Convention;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.core.Values;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.logical.LogicalValues;
 
 /** {@code ConverterRule} to replace {@code Values} with {@code BeamValuesRel}. */
 public class BeamValuesRule extends ConverterRule {
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamWindowRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamWindowRule.java
index 73c10cc..4995a3e 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamWindowRule.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamWindowRule.java
@@ -19,11 +19,11 @@
 
 import org.apache.beam.sdk.extensions.sql.impl.rel.BeamLogicalConvention;
 import org.apache.beam.sdk.extensions.sql.impl.rel.BeamWindowRel;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.Convention;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.convert.ConverterRule;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.Window;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.logical.LogicalWindow;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.Convention;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.core.Window;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.logical.LogicalWindow;
 
 /** A {@code ConverterRule} to replace {@link Window} with {@link BeamWindowRel}. */
 public class BeamWindowRule extends ConverterRule {
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/JoinRelOptRuleCall.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/JoinRelOptRuleCall.java
index 62ebf3c..5f60a8d 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/JoinRelOptRuleCall.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/JoinRelOptRuleCall.java
@@ -19,13 +19,14 @@
 
 import java.util.List;
 import java.util.Map;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptPlanner;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptRule;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptRuleCall;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptRuleOperand;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.metadata.RelMetadataQuery;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.tools.RelBuilder;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelHintsPropagator;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptPlanner;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptRule;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptRuleOperand;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.tools.RelBuilder;
 
 /**
  * This is a class to catch the built join and check if it is a legal join before passing it to the
@@ -53,6 +54,14 @@
     }
   }
 
+  @Override
+  public void transformTo(
+      RelNode relNode, Map<RelNode, RelNode> map, RelHintsPropagator relHintsPropagator) {
+    if (checker.check(relNode)) {
+      originalCall.transformTo(relNode, map, relHintsPropagator);
+    }
+  }
+
   /** This is a function gets the output relation and checks if it is a legal relational node. */
   public interface JoinChecker {
     boolean check(RelNode rel);
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/LogicalCalcMergeRule.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/LogicalCalcMergeRule.java
index 2937055..8d89133 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/LogicalCalcMergeRule.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/LogicalCalcMergeRule.java
@@ -17,11 +17,12 @@
  */
 package org.apache.beam.sdk.extensions.sql.impl.rule;
 
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptRule;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptRuleCall;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptRuleOperand;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.logical.LogicalCalc;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.rules.CalcMergeRule;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptRule;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptRuleOperand;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.logical.LogicalCalc;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.rules.CalcMergeRule;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.rules.CoreRules;
 
 /**
  * Planner rule to merge a {@link LogicalCalc} with a {@link LogicalCalc}. Subset of {@link
@@ -36,6 +37,6 @@
 
   @Override
   public void onMatch(RelOptRuleCall call) {
-    CalcMergeRule.INSTANCE.onMatch(call);
+    CoreRules.CALC_MERGE.onMatch(call);
   }
 }
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/package-info.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/package-info.java
index 43cf0b9..90e5361 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/package-info.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/rule/package-info.java
@@ -17,7 +17,7 @@
  */
 
 /**
- * {@link org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptRule} to generate
+ * {@link org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptRule} to generate
  * {@link org.apache.beam.sdk.extensions.sql.impl.rel.BeamRelNode}.
  */
 @DefaultAnnotation(NonNull.class)
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/schema/BeamTableUtils.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/schema/BeamTableUtils.java
index 00bf2ce..00890fc 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/schema/BeamTableUtils.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/schema/BeamTableUtils.java
@@ -35,8 +35,8 @@
 import org.apache.beam.sdk.schemas.Schema.FieldType;
 import org.apache.beam.sdk.schemas.Schema.TypeName;
 import org.apache.beam.sdk.values.Row;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.avatica.util.ByteString;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.util.NlsString;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.avatica.util.ByteString;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.util.NlsString;
 import org.apache.commons.csv.CSVFormat;
 import org.apache.commons.csv.CSVParser;
 import org.apache.commons.csv.CSVPrinter;
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamBuiltinAggregations.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamBuiltinAggregations.java
index a2cff20..6dd76a3 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamBuiltinAggregations.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamBuiltinAggregations.java
@@ -42,7 +42,7 @@
 import org.apache.beam.sdk.transforms.Sample;
 import org.apache.beam.sdk.transforms.Sum;
 import org.apache.beam.sdk.values.KV;
-import org.apache.beam.vendor.calcite.v1_20_0.com.google.common.collect.ImmutableMap;
+import org.apache.beam.vendor.calcite.v1_26_0.com.google.common.collect.ImmutableMap;
 import org.checkerframework.checker.nullness.qual.Nullable;
 
 /** Built-in aggregations functions for COUNT/MAX/MIN/SUM/AVG/VAR_POP/VAR_SAMP. */
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamBuiltinAnalyticFunctions.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamBuiltinAnalyticFunctions.java
index e970978..26da1aa 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamBuiltinAnalyticFunctions.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamBuiltinAnalyticFunctions.java
@@ -24,7 +24,7 @@
 import org.apache.beam.sdk.schemas.Schema;
 import org.apache.beam.sdk.transforms.Combine;
 import org.apache.beam.sdk.values.KV;
-import org.apache.beam.vendor.calcite.v1_20_0.com.google.common.collect.ImmutableMap;
+import org.apache.beam.vendor.calcite.v1_26_0.com.google.common.collect.ImmutableMap;
 
 /** Built-in Analytic Functions for the aggregation analytics functionality. */
 @SuppressWarnings({
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamJoinTransforms.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamJoinTransforms.java
index 009ae43..6cac136 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamJoinTransforms.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/BeamJoinTransforms.java
@@ -33,10 +33,10 @@
 import org.apache.beam.sdk.transforms.ParDo;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.Row;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexCall;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexInputRef;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.util.Pair;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexCall;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexInputRef;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.util.Pair;
 
 /** Collections of {@code PTransform} and {@code DoFn} used to perform JOIN operation. */
 @SuppressWarnings({
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/agg/AggregationCombineFnAdapter.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/agg/AggregationCombineFnAdapter.java
index 5317542..2178a2d 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/agg/AggregationCombineFnAdapter.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/agg/AggregationCombineFnAdapter.java
@@ -27,8 +27,8 @@
 import org.apache.beam.sdk.schemas.SchemaCoder;
 import org.apache.beam.sdk.transforms.Combine.CombineFn;
 import org.apache.beam.sdk.values.Row;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.AggregateCall;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.validate.SqlUserDefinedAggFunction;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.core.AggregateCall;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.validate.SqlUserDefinedAggFunction;
 import org.checkerframework.checker.nullness.qual.Nullable;
 
 /** Wrapper {@link CombineFn}s for aggregation function calls. */
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/agg/CovarianceFn.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/agg/CovarianceFn.java
index 579955f..f8112e2 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/agg/CovarianceFn.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/agg/CovarianceFn.java
@@ -17,7 +17,7 @@
  */
 package org.apache.beam.sdk.extensions.sql.impl.transform.agg;
 
-import static org.apache.beam.vendor.calcite.v1_20_0.com.google.common.base.Preconditions.checkArgument;
+import static org.apache.beam.vendor.calcite.v1_26_0.com.google.common.base.Preconditions.checkArgument;
 
 import java.math.BigDecimal;
 import java.math.MathContext;
@@ -32,7 +32,7 @@
 import org.apache.beam.sdk.transforms.Combine;
 import org.apache.beam.sdk.transforms.SerializableFunction;
 import org.apache.beam.sdk.values.Row;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.runtime.SqlFunctions;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.runtime.SqlFunctions;
 
 /**
  * {@link Combine.CombineFn} for <em>Covariance</em> on {@link Number} types.
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/agg/VarianceFn.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/agg/VarianceFn.java
index f0ffbd4..ae1d9c8 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/agg/VarianceFn.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/transform/agg/VarianceFn.java
@@ -29,7 +29,7 @@
 import org.apache.beam.sdk.schemas.Schema;
 import org.apache.beam.sdk.transforms.Combine;
 import org.apache.beam.sdk.transforms.SerializableFunction;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.runtime.SqlFunctions;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.runtime.SqlFunctions;
 
 /**
  * {@link Combine.CombineFn} for <em>Variance</em> on {@link Number} types.
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/udf/BuiltinHashFunctions.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/udf/BuiltinHashFunctions.java
index c3fc82b..339b6e4 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/udf/BuiltinHashFunctions.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/udf/BuiltinHashFunctions.java
@@ -19,7 +19,7 @@
 
 import com.google.auto.service.AutoService;
 import org.apache.beam.sdk.schemas.Schema;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.linq4j.function.Strict;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.linq4j.function.Strict;
 
 /** Hash Functions. */
 @AutoService(BeamBuiltinFunctionProvider.class)
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/udf/BuiltinStringFunctions.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/udf/BuiltinStringFunctions.java
index b7f9318..490b6f8 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/udf/BuiltinStringFunctions.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/udf/BuiltinStringFunctions.java
@@ -24,7 +24,7 @@
 import org.apache.beam.repackaged.core.org.apache.commons.lang3.ArrayUtils;
 import org.apache.beam.repackaged.core.org.apache.commons.lang3.StringUtils;
 import org.apache.beam.sdk.schemas.Schema.TypeName;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.linq4j.function.Strict;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.linq4j.function.Strict;
 import org.apache.commons.codec.DecoderException;
 import org.apache.commons.codec.binary.Hex;
 
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/utils/BigDecimalConverter.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/utils/BigDecimalConverter.java
index d00e6d6..d987c38 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/utils/BigDecimalConverter.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/utils/BigDecimalConverter.java
@@ -21,7 +21,7 @@
 import java.util.Map;
 import org.apache.beam.sdk.schemas.Schema.TypeName;
 import org.apache.beam.sdk.transforms.SerializableFunction;
-import org.apache.beam.vendor.calcite.v1_20_0.com.google.common.collect.ImmutableMap;
+import org.apache.beam.vendor.calcite.v1_26_0.com.google.common.collect.ImmutableMap;
 
 /**
  * Provides converters from {@link BigDecimal} to other numeric types based on the input {@link
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/utils/CalciteUtils.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/utils/CalciteUtils.java
index bf7b7e1..75ad2eb 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/utils/CalciteUtils.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/utils/CalciteUtils.java
@@ -29,14 +29,15 @@
 import org.apache.beam.sdk.schemas.logicaltypes.PassThroughLogicalType;
 import org.apache.beam.sdk.schemas.logicaltypes.SqlTypes;
 import org.apache.beam.sdk.util.Preconditions;
-import org.apache.beam.vendor.calcite.v1_20_0.com.google.common.collect.BiMap;
-import org.apache.beam.vendor.calcite.v1_20_0.com.google.common.collect.ImmutableBiMap;
-import org.apache.beam.vendor.calcite.v1_20_0.com.google.common.collect.ImmutableMap;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.avatica.util.ByteString;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.type.RelDataType;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.type.RelDataTypeFactory;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.type.RelDataTypeField;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.beam.vendor.calcite.v1_26_0.com.google.common.collect.BiMap;
+import org.apache.beam.vendor.calcite.v1_26_0.com.google.common.collect.ImmutableBiMap;
+import org.apache.beam.vendor.calcite.v1_26_0.com.google.common.collect.ImmutableMap;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.avatica.util.ByteString;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.type.RelDataType;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlTypeNameSpec;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.type.SqlTypeName;
 import org.joda.time.Instant;
 import org.joda.time.base.AbstractInstant;
 
@@ -201,6 +202,10 @@
     }
   }
 
+  public static FieldType toFieldType(SqlTypeNameSpec sqlTypeName) {
+    return toFieldType(SqlTypeName.get(sqlTypeName.getTypeName().getSimple()));
+  }
+
   public static FieldType toFieldType(SqlTypeName sqlTypeName) {
     switch (sqlTypeName) {
       case MAP:
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/utils/SerializableRexFieldAccess.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/utils/SerializableRexFieldAccess.java
index ce75b92..62a22ae 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/utils/SerializableRexFieldAccess.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/utils/SerializableRexFieldAccess.java
@@ -20,8 +20,8 @@
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexFieldAccess;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexInputRef;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexFieldAccess;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexInputRef;
 
 /** SerializableRexFieldAccess. */
 public class SerializableRexFieldAccess extends SerializableRexNode {
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/utils/SerializableRexInputRef.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/utils/SerializableRexInputRef.java
index 4d4d364..d7b4ec7 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/utils/SerializableRexInputRef.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/utils/SerializableRexInputRef.java
@@ -17,7 +17,7 @@
  */
 package org.apache.beam.sdk.extensions.sql.impl.utils;
 
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexInputRef;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexInputRef;
 
 /** SerializableRexInputRef. */
 public class SerializableRexInputRef extends SerializableRexNode {
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/utils/SerializableRexNode.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/utils/SerializableRexNode.java
index 9796bf3..b4ef1a0 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/utils/SerializableRexNode.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/utils/SerializableRexNode.java
@@ -18,9 +18,9 @@
 package org.apache.beam.sdk.extensions.sql.impl.utils;
 
 import java.io.Serializable;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexFieldAccess;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexInputRef;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexFieldAccess;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexInputRef;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexNode;
 
 /** SerializableRexNode. */
 public abstract class SerializableRexNode implements Serializable {
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/BaseBeamTable.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/BaseBeamTable.java
index d914955..c8ad21a 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/BaseBeamTable.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/BaseBeamTable.java
@@ -21,7 +21,7 @@
 import org.apache.beam.sdk.values.PBegin;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.Row;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexNode;
 
 /** Basic implementation of {@link BeamSqlTable} methods used by predicate and filter push-down. */
 public abstract class BaseBeamTable implements BeamSqlTable {
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/BeamSqlTable.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/BeamSqlTable.java
index be2c205..de13042 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/BeamSqlTable.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/BeamSqlTable.java
@@ -25,7 +25,7 @@
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.POutput;
 import org.apache.beam.sdk.values.Row;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexNode;
 
 /** This interface defines a Beam Sql Table. */
 public interface BeamSqlTable {
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/BeamSqlTableFilter.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/BeamSqlTableFilter.java
index 623c27e..0828ecc 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/BeamSqlTableFilter.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/BeamSqlTableFilter.java
@@ -18,8 +18,8 @@
 package org.apache.beam.sdk.extensions.sql.meta;
 
 import java.util.List;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexCall;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexCall;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexNode;
 
 /** This interface defines Beam SQL Table Filter. */
 public interface BeamSqlTableFilter {
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/DefaultTableFilter.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/DefaultTableFilter.java
index d77e3df..eb8d722 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/DefaultTableFilter.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/DefaultTableFilter.java
@@ -18,7 +18,7 @@
 package org.apache.beam.sdk.extensions.sql.meta;
 
 import java.util.List;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexNode;
 
 /**
  * This default implementation of {@link BeamSqlTableFilter} interface. Assumes that predicate
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/ReadOnlyTableProvider.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/ReadOnlyTableProvider.java
index ce6a00e..d001662 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/ReadOnlyTableProvider.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/ReadOnlyTableProvider.java
@@ -21,7 +21,7 @@
 import org.apache.beam.sdk.extensions.sql.meta.BeamSqlTable;
 import org.apache.beam.sdk.extensions.sql.meta.Table;
 import org.apache.beam.sdk.schemas.Schema;
-import org.apache.beam.vendor.calcite.v1_20_0.com.google.common.collect.ImmutableMap;
+import org.apache.beam.vendor.calcite.v1_26_0.com.google.common.collect.ImmutableMap;
 
 /**
  * A {@code ReadOnlyTableProvider} provides in-memory read only set of {@code BeamSqlTable
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/UdfUdafProvider.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/UdfUdafProvider.java
index 8b3d95d..85c917d 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/UdfUdafProvider.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/UdfUdafProvider.java
@@ -19,6 +19,7 @@
 
 import java.util.Collections;
 import java.util.Map;
+import org.apache.beam.sdk.annotations.Experimental;
 import org.apache.beam.sdk.extensions.sql.BeamSqlUdf;
 import org.apache.beam.sdk.transforms.Combine;
 import org.apache.beam.sdk.transforms.SerializableFunction;
@@ -27,6 +28,7 @@
 @SuppressWarnings({
   "rawtypes" // TODO(https://issues.apache.org/jira/browse/BEAM-10556)
 })
+@Experimental
 public interface UdfUdafProvider {
   /** For UDFs implement {@link BeamSqlUdf}. */
   default Map<String, Class<? extends BeamSqlUdf>> getBeamSqlUdfs() {
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BeamBigQuerySqlDialect.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BeamBigQuerySqlDialect.java
index 7fa53f3..60e249d 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BeamBigQuerySqlDialect.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BeamBigQuerySqlDialect.java
@@ -19,30 +19,17 @@
 
 import java.util.List;
 import java.util.Map;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.avatica.util.Casing;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.avatica.util.TimeUnit;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.config.NullCollation;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.type.RelDataType;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.type.RelDataTypeSystem;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlAbstractDateTimeLiteral;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlCall;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlDataTypeSpec;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlDialect;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlIdentifier;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlIntervalLiteral;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlIntervalQualifier;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlKind;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlLiteral;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlOperator;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlSetOperator;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlSyntax;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlTimestampLiteral;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlWriter;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.dialect.BigQuerySqlDialect;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.fun.SqlTrimFunction;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.parser.SqlParserPos;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.type.BasicSqlType;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.avatica.util.Casing;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.config.NullCollation;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.type.RelDataTypeSystem;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlAbstractDateTimeLiteral;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlCall;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlDialect;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlIntervalLiteral;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlTimestampLiteral;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlWriter;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.dialect.BigQuerySqlDialect;
 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;
 
@@ -64,14 +51,6 @@
 
   public static final SqlDialect DEFAULT = new BeamBigQuerySqlDialect(DEFAULT_CONTEXT);
 
-  // List of BigQuery Specific Operators needed to form Syntactically Correct SQL
-  private static final SqlOperator UNION_DISTINCT =
-      new SqlSetOperator("UNION DISTINCT", SqlKind.UNION, 14, false);
-  private static final SqlSetOperator EXCEPT_DISTINCT =
-      new SqlSetOperator("EXCEPT DISTINCT", SqlKind.EXCEPT, 14, false);
-  private static final SqlSetOperator INTERSECT_DISTINCT =
-      new SqlSetOperator("INTERSECT DISTINCT", SqlKind.INTERSECT, 18, false);
-
   // ZetaSQL defined functions that need special unparsing
   private static final List<String> FUNCTIONS_USING_INTERVAL =
       ImmutableList.of(
@@ -113,36 +92,9 @@
   }
 
   @Override
-  public SqlNode emulateNullDirection(SqlNode node, boolean nullsFirst, boolean desc) {
-    return emulateNullDirectionWithIsNull(node, nullsFirst, desc);
-  }
-
-  @Override
-  public boolean supportsNestedAggregations() {
-    return false;
-  }
-
-  @Override
-  public void unparseOffsetFetch(SqlWriter writer, SqlNode offset, SqlNode fetch) {
-    unparseFetchUsingLimit(writer, offset, fetch);
-  }
-
-  @Override
   public void unparseCall(
       final SqlWriter writer, final SqlCall call, final int leftPrec, final int rightPrec) {
     switch (call.getKind()) {
-      case POSITION:
-        final SqlWriter.Frame frame = writer.startFunCall("STRPOS");
-        writer.sep(",");
-        call.operand(1).unparse(writer, leftPrec, rightPrec);
-        writer.sep(",");
-        call.operand(0).unparse(writer, leftPrec, rightPrec);
-        if (3 == call.operandCount()) {
-          throw new UnsupportedOperationException(
-              "3rd operand Not Supported for Function STRPOS in Big Query");
-        }
-        writer.endFunCall(frame);
-        break;
       case ROW:
         final SqlWriter.Frame structFrame = writer.startFunCall("STRUCT");
         for (SqlNode operand : call.getOperandList()) {
@@ -151,24 +103,6 @@
         }
         writer.endFunCall(structFrame);
         break;
-      case UNION:
-        if (!((SqlSetOperator) call.getOperator()).isAll()) {
-          SqlSyntax.BINARY.unparse(writer, UNION_DISTINCT, call, leftPrec, rightPrec);
-        }
-        break;
-      case EXCEPT:
-        if (!((SqlSetOperator) call.getOperator()).isAll()) {
-          SqlSyntax.BINARY.unparse(writer, EXCEPT_DISTINCT, call, leftPrec, rightPrec);
-        }
-        break;
-      case INTERSECT:
-        if (!((SqlSetOperator) call.getOperator()).isAll()) {
-          SqlSyntax.BINARY.unparse(writer, INTERSECT_DISTINCT, call, leftPrec, rightPrec);
-        }
-        break;
-      case TRIM:
-        unparseTrim(writer, call, leftPrec, rightPrec);
-        break;
       case OTHER_FUNCTION:
         String funName = call.getOperator().getName();
         if (DOUBLE_LITERAL_WRAPPERS.containsKey(funName)) {
@@ -216,51 +150,6 @@
     unparseSqlIntervalQualifier(writer, interval.getIntervalQualifier(), RelDataTypeSystem.DEFAULT);
   }
 
-  @Override
-  public void unparseSqlIntervalQualifier(
-      SqlWriter writer, SqlIntervalQualifier qualifier, RelDataTypeSystem typeSystem) {
-    final String start = validate(qualifier.timeUnitRange.startUnit).name();
-    if (qualifier.timeUnitRange.endUnit == null) {
-      writer.keyword(start);
-    } else {
-      throw new UnsupportedOperationException("Range time unit is not supported for BigQuery.");
-    }
-  }
-
-  /**
-   * For usage of TRIM, LTRIM and RTRIM in BQ see <a
-   * href="https://cloud.google.com/bigquery/docs/reference/standard-sql/functions-and-operators#trim">
-   * BQ Trim Function</a>.
-   */
-  private void unparseTrim(SqlWriter writer, SqlCall call, int leftPrec, int rightPrec) {
-    final String operatorName;
-    SqlLiteral trimFlag = call.operand(0);
-    SqlLiteral valueToTrim = call.operand(1);
-    switch (trimFlag.getValueAs(SqlTrimFunction.Flag.class)) {
-      case LEADING:
-        operatorName = "LTRIM";
-        break;
-      case TRAILING:
-        operatorName = "RTRIM";
-        break;
-      default:
-        operatorName = call.getOperator().getName();
-        break;
-    }
-    final SqlWriter.Frame trimFrame = writer.startFunCall(operatorName);
-    call.operand(2).unparse(writer, leftPrec, rightPrec);
-
-    /**
-     * If the trimmed character is non space character then add it to the target sql. eg: TRIM(BOTH
-     * 'A' from 'ABCD' Output Query: TRIM('ABC', 'A')
-     */
-    if (!valueToTrim.toValue().matches("\\s+")) {
-      writer.literal(",");
-      call.operand(1).unparse(writer, leftPrec, rightPrec);
-    }
-    writer.endFunCall(trimFrame);
-  }
-
   private void unparseDoubleLiteralWrapperFunction(SqlWriter writer, String funName) {
     writer.literal(DOUBLE_LITERAL_WRAPPERS.get(funName));
   }
@@ -349,78 +238,6 @@
     writer.literal(")");
   }
 
-  private TimeUnit validate(TimeUnit timeUnit) {
-    switch (timeUnit) {
-      case MICROSECOND:
-      case MILLISECOND:
-      case SECOND:
-      case MINUTE:
-      case HOUR:
-      case DAY:
-      case WEEK:
-      case MONTH:
-      case QUARTER:
-      case YEAR:
-      case ISOYEAR:
-        return timeUnit;
-      default:
-        throw new UnsupportedOperationException(
-            "Time unit " + timeUnit + " is not supported for BigQuery.");
-    }
-  }
-
-  /**
-   * BigQuery data type reference: <a
-   * href="https://cloud.google.com/bigquery/docs/reference/standard-sql/data-types">Bigquery
-   * Standard SQL Data Types</a>.
-   */
-  @Override
-  public SqlNode getCastSpec(final RelDataType type) {
-    if (type instanceof BasicSqlType) {
-      switch (type.getSqlTypeName()) {
-          // BigQuery only supports INT64 for integer types.
-        case BIGINT:
-        case INTEGER:
-        case TINYINT:
-        case SMALLINT:
-          return typeFromName(type, "INT64");
-          // BigQuery only supports FLOAT64(aka. Double) for floating point types.
-        case FLOAT:
-        case DOUBLE:
-          return typeFromName(type, "FLOAT64");
-        case DECIMAL:
-          return typeFromName(type, "NUMERIC");
-        case BOOLEAN:
-          return typeFromName(type, "BOOL");
-        case CHAR:
-        case VARCHAR:
-          return typeFromName(type, "STRING");
-        case VARBINARY:
-        case BINARY:
-          return typeFromName(type, "BYTES");
-        case DATE:
-          return typeFromName(type, "DATE");
-        case TIME:
-          return typeFromName(type, "TIME");
-        case TIMESTAMP:
-          return typeFromName(type, "TIMESTAMP");
-        default:
-          break;
-      }
-    }
-    return super.getCastSpec(type);
-  }
-
-  private static SqlNode typeFromName(RelDataType type, String name) {
-    return new SqlDataTypeSpec(
-        new SqlIdentifier(name, SqlParserPos.ZERO),
-        type.getPrecision(),
-        -1,
-        null,
-        null,
-        SqlParserPos.ZERO);
-  }
-
   @Override
   public void unparseDateTimeLiteral(
       SqlWriter writer, SqlAbstractDateTimeLiteral literal, int leftPrec, int rightPrec) {
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BeamSqlUnparseContext.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BeamSqlUnparseContext.java
index e55263d..68ad64e 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BeamSqlUnparseContext.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BeamSqlUnparseContext.java
@@ -17,7 +17,7 @@
  */
 package org.apache.beam.sdk.extensions.sql.meta.provider.bigquery;
 
-import static org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.rel2sql.SqlImplementor.POS;
+import static org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.rel2sql.SqlImplementor.POS;
 
 import java.util.HashMap;
 import java.util.Map;
@@ -27,24 +27,28 @@
 import org.apache.beam.repackaged.core.org.apache.commons.lang3.text.translate.EntityArrays;
 import org.apache.beam.repackaged.core.org.apache.commons.lang3.text.translate.JavaUnicodeEscaper;
 import org.apache.beam.repackaged.core.org.apache.commons.lang3.text.translate.LookupTranslator;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.avatica.util.ByteString;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.avatica.util.TimeUnitRange;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.rel2sql.SqlImplementor;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.type.RelDataType;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexDynamicParam;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexLiteral;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexProgram;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlDynamicParam;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlKind;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlLiteral;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlWriter;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.parser.SqlParserPos;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.type.SqlTypeFamily;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.type.SqlTypeName;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.util.BitString;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.util.TimestampString;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.avatica.util.ByteString;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.avatica.util.TimeUnitRange;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.rel2sql.RelToSqlConverter;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.rel2sql.SqlImplementor;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.type.RelDataType;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexCall;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexDynamicParam;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexLiteral;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexLocalRef;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexProgram;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlDynamicParam;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlKind;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlLiteral;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlWriter;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.type.SqlTypeFamily;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.util.BitString;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.util.TimestampString;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
 import org.checkerframework.checker.nullness.qual.Nullable;
 
 @SuppressWarnings({
@@ -53,6 +57,13 @@
 })
 public class BeamSqlUnparseContext extends SqlImplementor.SimpleContext {
 
+  private final SqlImplementor imp = new RelToSqlConverter(BeamBigQuerySqlDialect.DEFAULT);
+
+  @Override
+  public SqlImplementor implementor() {
+    return imp;
+  }
+
   // More about escape sequences here:
   // https://cloud.google.com/bigquery/docs/reference/standard-sql/lexical
   // No need to escape: \`, \?, \v, \a, \ooo, \xhh (since this in not a thing in Java)
@@ -114,6 +125,12 @@
       final String name = "null_param_" + index;
       nullParams.put(name, param.getType());
       return new NamedDynamicParam(index, POS, name);
+    } else if (SqlKind.SEARCH.equals(rex.getKind())) {
+      // Workaround CALCITE-4716
+      RexCall search = (RexCall) rex;
+      RexLocalRef ref = (RexLocalRef) search.operands.get(1);
+      RexLiteral literal = (RexLiteral) program.getExprList().get(ref.getIndex());
+      rex = search.clone(search.getType(), ImmutableList.of(search.operands.get(0), literal));
     }
 
     return super.toSql(program, rex);
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BigQueryFilter.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BigQueryFilter.java
index c028a7e..526ae6c 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BigQueryFilter.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BigQueryFilter.java
@@ -17,29 +17,29 @@
  */
 package org.apache.beam.sdk.extensions.sql.meta.provider.bigquery;
 
-import static org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlKind.AND;
-import static org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlKind.BETWEEN;
-import static org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlKind.CAST;
-import static org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlKind.COMPARISON;
-import static org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlKind.DIVIDE;
-import static org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlKind.LIKE;
-import static org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlKind.MINUS;
-import static org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlKind.MOD;
-import static org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlKind.OR;
-import static org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlKind.PLUS;
-import static org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlKind.TIMES;
+import static org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlKind.AND;
+import static org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlKind.BETWEEN;
+import static org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlKind.CAST;
+import static org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlKind.COMPARISON;
+import static org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlKind.DIVIDE;
+import static org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlKind.LIKE;
+import static org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlKind.MINUS;
+import static org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlKind.MOD;
+import static org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlKind.OR;
+import static org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlKind.PLUS;
+import static org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlKind.TIMES;
 
 import java.util.ArrayList;
 import java.util.List;
 import java.util.stream.Collectors;
 import org.apache.beam.repackaged.core.org.apache.commons.lang3.tuple.Pair;
 import org.apache.beam.sdk.extensions.sql.meta.BeamSqlTableFilter;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexCall;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexInputRef;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexLiteral;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlKind;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexCall;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexInputRef;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexLiteral;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlKind;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableSet;
 
 @SuppressWarnings({
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BigQueryTable.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BigQueryTable.java
index 6e6203d..68b9e00 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BigQueryTable.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BigQueryTable.java
@@ -48,13 +48,13 @@
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.POutput;
 import org.apache.beam.sdk.values.Row;
-import org.apache.beam.vendor.calcite.v1_20_0.com.google.common.annotations.VisibleForTesting;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.rel2sql.SqlImplementor;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlIdentifier;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.fun.SqlStdOperatorTable;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.beam.vendor.calcite.v1_26_0.com.google.common.annotations.VisibleForTesting;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.rel2sql.SqlImplementor;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlIdentifier;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.parser.SqlParserPos;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BigQueryTableProvider.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BigQueryTableProvider.java
index b1646aa..717cd23 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BigQueryTableProvider.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BigQueryTableProvider.java
@@ -17,7 +17,7 @@
  */
 package org.apache.beam.sdk.extensions.sql.meta.provider.bigquery;
 
-import static org.apache.beam.vendor.calcite.v1_20_0.com.google.common.base.MoreObjects.firstNonNull;
+import static org.apache.beam.vendor.calcite.v1_26_0.com.google.common.base.MoreObjects.firstNonNull;
 
 import com.alibaba.fastjson.JSONObject;
 import com.google.auto.service.AutoService;
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigtable/BigtableFilter.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigtable/BigtableFilter.java
index c3bdda7..7c19221 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigtable/BigtableFilter.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigtable/BigtableFilter.java
@@ -20,7 +20,7 @@
 import static java.util.stream.Collectors.toList;
 import static org.apache.beam.sdk.io.gcp.bigtable.RowUtils.KEY;
 import static org.apache.beam.sdk.io.gcp.bigtable.RowUtils.byteStringUtf8;
-import static org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlKind.LIKE;
+import static org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlKind.LIKE;
 import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument;
 
 import com.google.bigtable.v2.RowFilter;
@@ -28,10 +28,10 @@
 import java.util.stream.Collectors;
 import org.apache.beam.sdk.extensions.sql.meta.BeamSqlTableFilter;
 import org.apache.beam.sdk.schemas.Schema;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexCall;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexInputRef;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexLiteral;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexCall;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexInputRef;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexLiteral;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexNode;
 
 /**
  * BigtableFilter for queries with WHERE clause.
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigtable/BigtableTable.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigtable/BigtableTable.java
index 094d47b..a7d1012 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigtable/BigtableTable.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigtable/BigtableTable.java
@@ -49,7 +49,7 @@
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.POutput;
 import org.apache.beam.sdk.values.Row;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexNode;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Splitter;
 
 @Experimental
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/kafka/BeamKafkaTable.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/kafka/BeamKafkaTable.java
index 623fb44..57830ca 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/kafka/BeamKafkaTable.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/kafka/BeamKafkaTable.java
@@ -17,7 +17,7 @@
  */
 package org.apache.beam.sdk.extensions.sql.meta.provider.kafka;
 
-import static org.apache.beam.vendor.calcite.v1_20_0.com.google.common.base.Preconditions.checkArgument;
+import static org.apache.beam.vendor.calcite.v1_26_0.com.google.common.base.Preconditions.checkArgument;
 
 import java.util.Collection;
 import java.util.HashMap;
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/mongodb/MongoDbTable.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/mongodb/MongoDbTable.java
index 8955d29..5354553 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/mongodb/MongoDbTable.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/mongodb/MongoDbTable.java
@@ -17,9 +17,9 @@
  */
 package org.apache.beam.sdk.extensions.sql.meta.provider.mongodb;
 
-import static org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlKind.AND;
-import static org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlKind.COMPARISON;
-import static org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlKind.OR;
+import static org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlKind.AND;
+import static org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlKind.COMPARISON;
+import static org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlKind.OR;
 
 import com.mongodb.client.model.Filters;
 import java.io.Serializable;
@@ -57,13 +57,13 @@
 import org.apache.beam.sdk.values.PCollection.IsBounded;
 import org.apache.beam.sdk.values.POutput;
 import org.apache.beam.sdk.values.Row;
-import org.apache.beam.vendor.calcite.v1_20_0.com.google.common.annotations.VisibleForTesting;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexCall;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexInputRef;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexLiteral;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlKind;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.beam.vendor.calcite.v1_26_0.com.google.common.annotations.VisibleForTesting;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexCall;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexInputRef;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexLiteral;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlKind;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
 import org.bson.Document;
 import org.bson.conversions.Bson;
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/test/TestTableFilter.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/test/TestTableFilter.java
index 1f05d3c..ae99c61 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/test/TestTableFilter.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/test/TestTableFilter.java
@@ -17,18 +17,18 @@
  */
 package org.apache.beam.sdk.extensions.sql.meta.provider.test;
 
-import static org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlKind.COMPARISON;
-import static org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlKind.IN;
+import static org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlKind.COMPARISON;
+import static org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlKind.IN;
 
 import java.util.ArrayList;
 import java.util.List;
 import java.util.stream.Collectors;
 import org.apache.beam.sdk.extensions.sql.meta.BeamSqlTableFilter;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexCall;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexInputRef;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexLiteral;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexCall;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexInputRef;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexLiteral;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.type.SqlTypeName;
 
 @SuppressWarnings({
   "nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/test/TestTableProvider.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/test/TestTableProvider.java
index c865ede..7c25993 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/test/TestTableProvider.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/test/TestTableProvider.java
@@ -17,7 +17,7 @@
  */
 package org.apache.beam.sdk.extensions.sql.meta.provider.test;
 
-import static org.apache.beam.vendor.calcite.v1_20_0.com.google.common.base.Preconditions.checkArgument;
+import static org.apache.beam.vendor.calcite.v1_26_0.com.google.common.base.Preconditions.checkArgument;
 
 import com.google.auto.service.AutoService;
 import java.io.Serializable;
@@ -55,11 +55,11 @@
 import org.apache.beam.sdk.values.PDone;
 import org.apache.beam.sdk.values.POutput;
 import org.apache.beam.sdk.values.Row;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexCall;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexInputRef;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexLiteral;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexCall;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexInputRef;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexLiteral;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.type.SqlTypeName;
 
 /**
  * Test in-memory table provider for use in tests.
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/test/TestTableUtils.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/test/TestTableUtils.java
index a4d41f2..9ee6c29 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/test/TestTableUtils.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/test/TestTableUtils.java
@@ -27,7 +27,7 @@
 import org.apache.beam.sdk.schemas.Schema;
 import org.apache.beam.sdk.schemas.Schema.FieldType;
 import org.apache.beam.sdk.values.Row;
-import org.apache.beam.vendor.calcite.v1_20_0.com.google.common.collect.Lists;
+import org.apache.beam.vendor.calcite.v1_26_0.com.google.common.collect.Lists;
 
 /** Utility functions for mock classes. */
 @Experimental
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/test/TestUnboundedTable.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/test/TestUnboundedTable.java
index 05cbc55..80f1772 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/test/TestUnboundedTable.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/test/TestUnboundedTable.java
@@ -29,7 +29,7 @@
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.Row;
 import org.apache.beam.sdk.values.TimestampedValue;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.util.Pair;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.util.Pair;
 import org.joda.time.Duration;
 import org.joda.time.Instant;
 
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/text/TextTableProvider.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/text/TextTableProvider.java
index 28e7f0c..d50a14c 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/text/TextTableProvider.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/text/TextTableProvider.java
@@ -51,9 +51,9 @@
 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.calcite.v1_20_0.com.google.common.annotations.VisibleForTesting;
-import org.apache.beam.vendor.calcite.v1_20_0.com.google.common.base.MoreObjects;
-import org.apache.beam.vendor.calcite.v1_20_0.com.google.common.collect.ImmutableSet;
+import org.apache.beam.vendor.calcite.v1_26_0.com.google.common.annotations.VisibleForTesting;
+import org.apache.beam.vendor.calcite.v1_26_0.com.google.common.base.MoreObjects;
+import org.apache.beam.vendor.calcite.v1_26_0.com.google.common.collect.ImmutableSet;
 import org.apache.commons.csv.CSVFormat;
 import org.checkerframework.checker.nullness.qual.Nullable;
 
diff --git a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/store/InMemoryMetaStore.java b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/store/InMemoryMetaStore.java
index 5030be6..68901c9 100644
--- a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/store/InMemoryMetaStore.java
+++ b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/meta/store/InMemoryMetaStore.java
@@ -22,7 +22,7 @@
 import org.apache.beam.sdk.extensions.sql.meta.BeamSqlTable;
 import org.apache.beam.sdk.extensions.sql.meta.Table;
 import org.apache.beam.sdk.extensions.sql.meta.provider.TableProvider;
-import org.apache.beam.vendor.calcite.v1_20_0.com.google.common.collect.ImmutableMap;
+import org.apache.beam.vendor.calcite.v1_26_0.com.google.common.collect.ImmutableMap;
 
 /**
  * A {@link MetaStore} which stores the meta info in memory.
diff --git a/sdks/java/extensions/sql/src/main/resources/org.apache.beam.vendor.calcite.v1_20_0.org.codehaus.commons.compiler.properties b/sdks/java/extensions/sql/src/main/resources/org.apache.beam.vendor.calcite.v1_26_0.org.codehaus.commons.compiler.properties
similarity index 93%
rename from sdks/java/extensions/sql/src/main/resources/org.apache.beam.vendor.calcite.v1_20_0.org.codehaus.commons.compiler.properties
rename to sdks/java/extensions/sql/src/main/resources/org.apache.beam.vendor.calcite.v1_26_0.org.codehaus.commons.compiler.properties
index ab9a234..9bcf386 100644
--- a/sdks/java/extensions/sql/src/main/resources/org.apache.beam.vendor.calcite.v1_20_0.org.codehaus.commons.compiler.properties
+++ b/sdks/java/extensions/sql/src/main/resources/org.apache.beam.vendor.calcite.v1_26_0.org.codehaus.commons.compiler.properties
@@ -15,4 +15,4 @@
 #  See the License for the specific language governing permissions and
 # limitations under the License.
 ################################################################################
-compilerFactory=org.apache.beam.vendor.calcite.v1_20_0.org.codehaus.janino.CompilerFactory
+compilerFactory=org.apache.beam.vendor.calcite.v1_26_0.org.codehaus.janino.CompilerFactory
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamComplexTypeTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamComplexTypeTest.java
index 85490e5..b4fa11a 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamComplexTypeTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamComplexTypeTest.java
@@ -35,8 +35,8 @@
 import org.apache.beam.sdk.transforms.Create;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.Row;
-import org.apache.beam.vendor.calcite.v1_20_0.com.google.common.collect.ImmutableList;
-import org.apache.beam.vendor.calcite.v1_20_0.com.google.common.collect.ImmutableMap;
+import org.apache.beam.vendor.calcite.v1_26_0.com.google.common.collect.ImmutableList;
+import org.apache.beam.vendor.calcite.v1_26_0.com.google.common.collect.ImmutableMap;
 import org.joda.time.Duration;
 import org.joda.time.Instant;
 import org.junit.Ignore;
@@ -82,16 +82,6 @@
           .addArrayField("field3", FieldType.INT64)
           .build();
 
-  private static final Schema flattenedRowSchema =
-      Schema.builder()
-          .addStringField("field1")
-          .addStringField("field2")
-          .addInt64Field("field3")
-          .addInt64Field("field4")
-          .addStringField("field5")
-          .addInt64Field("field6")
-          .build();
-
   private static final ReadOnlyTableProvider readOnlyTableProvider =
       new ReadOnlyTableProvider(
           "test_provider",
@@ -131,10 +121,18 @@
     PCollection<Row> stream =
         BeamSqlRelUtils.toPCollection(
             pipeline, sqlEnv.parseQuery("SELECT nestedRowTestTable.col FROM nestedRowTestTable"));
+    Schema outputSchema = Schema.builder().addRowField("col", nestedRowSchema).build();
     PAssert.that(stream)
         .containsInAnyOrder(
-            Row.withSchema(flattenedRowSchema)
-                .addValues("str", "inner_str_one", 1L, 2L, "inner_str_two", 3L)
+            Row.withSchema(outputSchema)
+                .addValues(
+                    Row.withSchema(nestedRowSchema)
+                        .addValues(
+                            "str",
+                            Row.withSchema(innerRowSchema).addValues("inner_str_one", 1L).build(),
+                            2L,
+                            Row.withSchema(innerRowSchema).addValues("inner_str_two", 3L).build())
+                        .build())
                 .build());
     pipeline.run().waitUntilFinish(Duration.standardMinutes(2));
   }
@@ -146,8 +144,12 @@
         BeamSqlRelUtils.toPCollection(
             pipeline,
             sqlEnv.parseQuery("SELECT arrayWithRowTestTable.col[1] FROM arrayWithRowTestTable"));
+    Schema outputSchema = Schema.builder().addRowField("col", innerRowSchema).build();
     PAssert.that(stream)
-        .containsInAnyOrder(Row.withSchema(innerRowSchema).addValues("str", 1L).build());
+        .containsInAnyOrder(
+            Row.withSchema(outputSchema)
+                .addValues(Row.withSchema(innerRowSchema).addValues("str", 1L).build())
+                .build());
     pipeline.run().waitUntilFinish(Duration.standardMinutes(2));
   }
 
@@ -173,8 +175,12 @@
     PCollection<Row> stream =
         BeamSqlRelUtils.toPCollection(
             pipeline, sqlEnv.parseQuery("SELECT col FROM basicRowTestTable"));
+    Schema outputSchema = Schema.builder().addRowField("col", innerRowSchema).build();
     PAssert.that(stream)
-        .containsInAnyOrder(Row.withSchema(innerRowSchema).addValues("innerStr", 1L).build());
+        .containsInAnyOrder(
+            Row.withSchema(outputSchema)
+                .addValues(Row.withSchema(innerRowSchema).addValues("innerStr", 1L).build())
+                .build());
     pipeline.run().waitUntilFinish(Duration.standardMinutes(2));
   }
 
@@ -239,6 +245,7 @@
     pipeline.run().waitUntilFinish(Duration.standardMinutes(2));
   }
 
+  @Ignore("https://issues.apache.org/jira/browse/BEAM-12782")
   @Test
   public void testNestedBytes() {
     byte[] bytes = new byte[] {-70, -83, -54, -2};
@@ -263,6 +270,7 @@
     pipeline.run();
   }
 
+  @Ignore("https://issues.apache.org/jira/browse/BEAM-12782")
   @Test
   public void testNestedArrayOfBytes() {
     byte[] bytes = new byte[] {-70, -83, -54, -2};
@@ -294,18 +302,26 @@
     PCollection<Row> stream =
         BeamSqlRelUtils.toPCollection(
             pipeline, sqlEnv.parseQuery("SELECT ROW(1, ROW(2, 3), 'str', ROW('str2', 'str3'))"));
+    Schema intRow = Schema.builder().addInt32Field("field2").addInt32Field("field3").build();
+    Schema strRow = Schema.builder().addStringField("field5").addStringField("field6").build();
+    Schema innerRow =
+        Schema.builder()
+            .addInt32Field("field1")
+            .addRowField("intRow", intRow)
+            .addStringField("field4")
+            .addRowField("strRow", strRow)
+            .build();
     PAssert.that(stream)
         .containsInAnyOrder(
-            Row.withSchema(
-                    Schema.builder()
-                        .addInt32Field("field1")
-                        .addInt32Field("field2")
-                        .addInt32Field("field3")
-                        .addStringField("field4")
-                        .addStringField("field5")
-                        .addStringField("field6")
+            Row.withSchema(Schema.builder().addRowField("row", innerRow).build())
+                .addValues(
+                    Row.withSchema(innerRow)
+                        .addValues(
+                            1,
+                            Row.withSchema(intRow).addValues(2, 3).build(),
+                            "str",
+                            Row.withSchema(strRow).addValues("str2", "str3").build())
                         .build())
-                .addValues(1, 2, 3, "str", "str2", "str3")
                 .build());
     pipeline.run().waitUntilFinish(Duration.standardMinutes(2));
   }
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslAggregationTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslAggregationTest.java
index 50b5aa3..206dde0 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslAggregationTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslAggregationTest.java
@@ -786,7 +786,7 @@
     exceptions.expectCause(
         hasMessage(
             containsString(
-                "Cannot apply 'TUMBLE' to arguments of type 'TUMBLE(<BIGINT>, <INTERVAL HOUR>)'")));
+                "Cannot apply '$TUMBLE' to arguments of type '$TUMBLE(<BIGINT>, <INTERVAL HOUR>)'")));
     pipeline.enableAbandonedNodeEnforcement(false);
 
     String sql =
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslArrayTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslArrayTest.java
index a9da87a..f072467 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslArrayTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslArrayTest.java
@@ -296,7 +296,7 @@
     Schema elementSchema =
         Schema.builder().addStringField("f_rowString").addInt32Field("f_rowInt").build();
 
-    Schema resultSchema = elementSchema;
+    Schema resultSchema = Schema.builder().addRowField("row", elementSchema).build();
 
     Schema inputType =
         Schema.builder()
@@ -330,8 +330,12 @@
 
     PAssert.that(result)
         .containsInAnyOrder(
-            Row.withSchema(elementSchema).addValues("BB", 22).build(),
-            Row.withSchema(elementSchema).addValues("DD", 44).build());
+            Row.withSchema(resultSchema)
+                .addValues(Row.withSchema(elementSchema).addValues("BB", 22).build())
+                .build(),
+            Row.withSchema(resultSchema)
+                .addValues(Row.withSchema(elementSchema).addValues("DD", 44).build())
+                .build());
 
     pipeline.run();
   }
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslNestedRowsTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslNestedRowsTest.java
index 53ce03a..0efa0e3 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslNestedRowsTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslNestedRowsTest.java
@@ -24,7 +24,6 @@
 import org.apache.beam.sdk.transforms.Create;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.Row;
-import org.junit.Ignore;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.ExpectedException;
@@ -48,41 +47,33 @@
             .addInt32Field("f_nestedIntPlusOne")
             .build();
 
-    Schema resultSchema =
-        Schema.builder()
-            .addInt32Field("f_int")
-            .addInt32Field("f_int2")
-            .addStringField("f_varchar")
-            .addInt32Field("f_int3")
-            .build();
-
-    Schema inputType =
+    Schema schema =
         Schema.builder().addInt32Field("f_int").addRowField("f_row", nestedSchema).build();
 
     PCollection<Row> input =
         pipeline.apply(
             Create.of(
-                    Row.withSchema(inputType)
+                    Row.withSchema(schema)
                         .addValues(
                             1, Row.withSchema(nestedSchema).addValues(312, "CC", 313).build())
                         .build())
-                .withRowSchema(inputType));
+                .withRowSchema(schema));
 
     PCollection<Row> result =
-        input
-            .apply(
-                SqlTransform.query(
-                    "SELECT 1 as `f_int`, ROW(3, 'BB', f_int + 1) as `f_row1` FROM PCOLLECTION"))
-            .setRowSchema(resultSchema);
+        input.apply(
+            SqlTransform.query(
+                "SELECT 1 as `f_int`, ROW(3, 'BB', f_int + 1) as `f_row1` FROM PCOLLECTION"));
 
     PAssert.that(result)
-        .containsInAnyOrder(Row.withSchema(resultSchema).addValues(1, 3, "BB", 2).build());
+        .containsInAnyOrder(
+            Row.withSchema(schema)
+                .addValues(1, Row.withSchema(nestedSchema).addValues(3, "BB", 2).build())
+                .build());
 
     pipeline.run();
   }
 
   @Test
-  @Ignore("[BEAM-9378] This does not work because calcite flattens the row.")
   public void testRowAliasAsRow() {
     Schema nestedSchema =
         Schema.builder()
@@ -117,7 +108,6 @@
   }
 
   @Test
-  @Ignore("[BEAM-9378] This does not work because calcite flattens the row.")
   public void testRowConstructorKeywordKeepAsRow() {
     Schema nestedSchema =
         Schema.builder()
@@ -149,7 +139,7 @@
 
     PAssert.that(result)
         .containsInAnyOrder(
-            Row.withSchema(nestedSchema)
+            Row.withSchema(outputType)
                 .attachValues(2, Row.withSchema(nestedOutput).attachValues(312, "CC")));
 
     pipeline.run();
@@ -169,35 +159,28 @@
             .addInt32Field("f_nestedIntPlusOne")
             .build();
 
-    Schema resultSchema =
-        Schema.builder()
-            .addInt32Field("f_int")
-            .addInt32Field("f_int2")
-            .addStringField("f_varchar")
-            .addInt32Field("f_int3")
-            .build();
-
-    Schema inputType =
+    Schema schema =
         Schema.builder().addInt32Field("f_int").addRowField("f_row", nestedSchema).build();
 
     PCollection<Row> input =
         pipeline.apply(
             Create.of(
-                    Row.withSchema(inputType)
+                    Row.withSchema(schema)
                         .addValues(
                             1, Row.withSchema(nestedSchema).addValues(312, "CC", 313).build())
                         .build())
-                .withRowSchema(inputType));
+                .withRowSchema(schema));
 
     PCollection<Row> result =
-        input
-            .apply(
-                SqlTransform.query(
-                    "SELECT 1 as `f_int`, (3, 'BB', f_int + 1) as `f_row1` FROM PCOLLECTION"))
-            .setRowSchema(resultSchema);
+        input.apply(
+            SqlTransform.query(
+                "SELECT 1 as `f_int`, (3, 'BB', f_int + 1) as `f_row1` FROM PCOLLECTION"));
 
     PAssert.that(result)
-        .containsInAnyOrder(Row.withSchema(resultSchema).addValues(1, 3, "BB", 2).build());
+        .containsInAnyOrder(
+            Row.withSchema(schema)
+                .addValues(1, Row.withSchema(nestedSchema).addValues(3, "BB", 2).build())
+                .build());
 
     pipeline.run();
   }
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslSqlStdOperatorsTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslSqlStdOperatorsTest.java
index 1fab50b..f7dbb36 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslSqlStdOperatorsTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslSqlStdOperatorsTest.java
@@ -17,6 +17,7 @@
  */
 package org.apache.beam.sdk.extensions.sql;
 
+import static java.math.RoundingMode.UNNECESSARY;
 import static org.apache.beam.sdk.extensions.sql.utils.DateTimeUtils.parseTimestampWithUTCTimeZone;
 import static org.hamcrest.MatcherAssert.assertThat;
 import static org.hamcrest.Matchers.equalTo;
@@ -30,7 +31,6 @@
 import java.lang.annotation.Target;
 import java.lang.reflect.Method;
 import java.math.BigDecimal;
-import java.math.RoundingMode;
 import java.time.LocalDate;
 import java.time.LocalTime;
 import java.util.Arrays;
@@ -44,14 +44,14 @@
 import org.apache.beam.sdk.extensions.sql.integrationtest.BeamSqlBuiltinFunctionsIntegrationTestBase;
 import org.apache.beam.sdk.schemas.Schema;
 import org.apache.beam.sdk.schemas.Schema.FieldType;
-import org.apache.beam.vendor.calcite.v1_20_0.com.google.common.base.Joiner;
-import org.apache.beam.vendor.calcite.v1_20_0.com.google.common.collect.ImmutableList;
-import org.apache.beam.vendor.calcite.v1_20_0.com.google.common.collect.Lists;
-import org.apache.beam.vendor.calcite.v1_20_0.com.google.common.collect.Ordering;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.runtime.SqlFunctions;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlKind;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlOperator;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.beam.vendor.calcite.v1_26_0.com.google.common.base.Joiner;
+import org.apache.beam.vendor.calcite.v1_26_0.com.google.common.collect.ImmutableList;
+import org.apache.beam.vendor.calcite.v1_26_0.com.google.common.collect.Lists;
+import org.apache.beam.vendor.calcite.v1_26_0.com.google.common.collect.Ordering;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.runtime.SqlFunctions;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlKind;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlOperator;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.junit.Ignore;
 import org.junit.Rule;
 import org.junit.Test;
@@ -59,16 +59,16 @@
 
 /**
  * DSL compliance tests for the row-level operators of {@link
- * org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.fun.SqlStdOperatorTable}.
+ * org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.fun.SqlStdOperatorTable}.
  */
 public class BeamSqlDslSqlStdOperatorsTest extends BeamSqlBuiltinFunctionsIntegrationTestBase {
-  private static final BigDecimal ZERO = BigDecimal.valueOf(0.0);
-  private static final BigDecimal ONE = BigDecimal.valueOf(1.0);
-  private static final BigDecimal ONE2 = BigDecimal.valueOf(1.0).multiply(BigDecimal.valueOf(1.0));
-  private static final BigDecimal ONE10 =
-      BigDecimal.ONE.divide(BigDecimal.ONE, 10, RoundingMode.HALF_EVEN);
-  private static final BigDecimal TWO = BigDecimal.valueOf(2.0);
-  private static final BigDecimal TWO0 = BigDecimal.ONE.add(BigDecimal.ONE);
+  private static final BigDecimal ZERO_0 = BigDecimal.valueOf(0).setScale(0, UNNECESSARY);
+  private static final BigDecimal ZERO_1 = BigDecimal.valueOf(0).setScale(1, UNNECESSARY);
+  private static final BigDecimal ONE_0 = BigDecimal.valueOf(1).setScale(0, UNNECESSARY);
+  private static final BigDecimal ONE_1 = BigDecimal.valueOf(1).setScale(1, UNNECESSARY);
+  private static final BigDecimal ONE_2 = BigDecimal.valueOf(1).setScale(2, UNNECESSARY);
+  private static final BigDecimal TWO_0 = BigDecimal.valueOf(2).setScale(0, UNNECESSARY);
+  private static final BigDecimal TWO_1 = BigDecimal.valueOf(2).setScale(1, UNNECESSARY);
 
   private static final int INTEGER_VALUE = 1;
   private static final long LONG_VALUE = 1L;
@@ -76,6 +76,7 @@
   private static final byte BYTE_VALUE = 1;
   private static final double DOUBLE_VALUE = 1.0;
   private static final float FLOAT_VALUE = 1.0f;
+  private static final BigDecimal DECIMAL_VALUE = BigDecimal.ONE;
 
   @Rule public ExpectedException thrown = ExpectedException.none();
 
@@ -233,9 +234,11 @@
   }
 
   @Test
-  @SqlOperatorTest(name = "OR", kind = "OR")
-  @SqlOperatorTest(name = "NOT", kind = "NOT")
-  @SqlOperatorTest(name = "AND", kind = "AND")
+  @SqlOperatorTests({
+    @SqlOperatorTest(name = "OR", kind = "OR"),
+    @SqlOperatorTest(name = "NOT", kind = "NOT"),
+    @SqlOperatorTest(name = "AND", kind = "AND"),
+  })
   public void testLogicOperators() {
     ExpressionChecker checker =
         new ExpressionChecker()
@@ -280,23 +283,25 @@
   }
 
   @Test
-  @SqlOperatorTest(name = "+", kind = "PLUS")
-  @SqlOperatorTest(name = "-", kind = "MINUS")
-  @SqlOperatorTest(name = "*", kind = "TIMES")
-  @SqlOperatorTest(name = "/", kind = "DIVIDE")
-  @SqlOperatorTest(name = "MOD", kind = "MOD")
+  @SqlOperatorTests({
+    @SqlOperatorTest(name = "+", kind = "PLUS"),
+    @SqlOperatorTest(name = "-", kind = "MINUS"),
+    @SqlOperatorTest(name = "*", kind = "TIMES"),
+    @SqlOperatorTest(name = "/", kind = "DIVIDE"),
+    @SqlOperatorTest(name = "MOD", kind = "MOD"),
+  })
   public void testArithmeticOperator() {
     ExpressionChecker checker =
         new ExpressionChecker()
             .addExpr("1 + 1", 2)
-            .addExpr("1.0 + 1", TWO)
-            .addExpr("1 + 1.0", TWO)
-            .addExpr("1.0 + 1.0", TWO)
+            .addExpr("1.0 + 1", TWO_1)
+            .addExpr("1 + 1.0", TWO_1)
+            .addExpr("1.0 + 1.0", TWO_1)
             .addExpr("c_tinyint + c_tinyint", (byte) 2)
             .addExpr("c_smallint + c_smallint", (short) 2)
             .addExpr("c_bigint + c_bigint", 2L)
-            .addExpr("c_decimal + c_decimal", TWO0)
-            .addExpr("c_tinyint + c_decimal", TWO0)
+            .addExpr("c_decimal + c_decimal", TWO_0)
+            .addExpr("c_tinyint + c_decimal", TWO_0)
             .addExpr("c_float + c_decimal", 2.0)
             .addExpr("c_double + c_decimal", 2.0)
             .addExpr("c_float + c_float", 2.0f)
@@ -305,9 +310,9 @@
             .addExpr("c_float + c_bigint", 2.0f)
             .addExpr("c_double + c_bigint", 2.0)
             .addExpr("1 - 1", 0)
-            .addExpr("1.0 - 1", ZERO)
-            .addExpr("1 - 0.0", ONE)
-            .addExpr("1.0 - 1.0", ZERO)
+            .addExpr("1.0 - 1", ZERO_1)
+            .addExpr("1 - 0.0", ONE_1)
+            .addExpr("1.0 - 1.0", ZERO_1)
             .addExpr("c_tinyint - c_tinyint", (byte) 0)
             .addExpr("c_smallint - c_smallint", (short) 0)
             .addExpr("c_bigint - c_bigint", 0L)
@@ -321,14 +326,14 @@
             .addExpr("c_float - c_bigint", 0.0f)
             .addExpr("c_double - c_bigint", 0.0)
             .addExpr("1 * 1", 1)
-            .addExpr("1.0 * 1", ONE)
-            .addExpr("1 * 1.0", ONE)
-            .addExpr("1.0 * 1.0", ONE2)
+            .addExpr("1.0 * 1", ONE_1)
+            .addExpr("1 * 1.0", ONE_1)
+            .addExpr("1.0 * 1.0", ONE_2)
             .addExpr("c_tinyint * c_tinyint", (byte) 1)
             .addExpr("c_smallint * c_smallint", (short) 1)
             .addExpr("c_bigint * c_bigint", 1L)
-            .addExpr("c_decimal * c_decimal", BigDecimal.ONE)
-            .addExpr("c_tinyint * c_decimal", BigDecimal.ONE)
+            .addExpr("c_decimal * c_decimal", ONE_0)
+            .addExpr("c_tinyint * c_decimal", ONE_0)
             .addExpr("c_float * c_decimal", 1.0)
             .addExpr("c_double * c_decimal", 1.0)
             .addExpr("c_float * c_float", 1.0f)
@@ -337,14 +342,14 @@
             .addExpr("c_float * c_bigint", 1.0f)
             .addExpr("c_double * c_bigint", 1.0)
             .addExpr("1 / 1", 1)
-            .addExpr("1.0 / 1", ONE)
-            .addExpr("1 / 1.0", BigDecimal.ONE)
-            .addExpr("1.0 / 1.0", BigDecimal.ONE)
+            .addExpr("1.0 / 1", ONE_1)
+            .addExpr("1 / 1.0", ONE_0)
+            .addExpr("1.0 / 1.0", ONE_0)
             .addExpr("c_tinyint / c_tinyint", (byte) 1)
             .addExpr("c_smallint / c_smallint", (short) 1)
             .addExpr("c_bigint / c_bigint", 1L)
-            .addExpr("c_decimal / c_decimal", BigDecimal.ONE)
-            .addExpr("c_tinyint / c_decimal", BigDecimal.ONE)
+            .addExpr("c_decimal / c_decimal", ONE_0)
+            .addExpr("c_tinyint / c_decimal", ONE_0)
             .addExpr("c_float / c_decimal", 1.0)
             .addExpr("c_double / c_decimal", 1.0)
             .addExpr("c_float / c_float", 1.0f)
@@ -353,14 +358,14 @@
             .addExpr("c_float / c_bigint", 1.0f)
             .addExpr("c_double / c_bigint", 1.0)
             .addExpr("mod(1, 1)", 0)
-            .addExpr("mod(1.0, 1)", 0)
-            .addExpr("mod(1, 1.0)", BigDecimal.ZERO)
-            .addExpr("mod(1.0, 1.0)", ZERO)
+            .addExpr("mod(1.0, 1)", ZERO_1)
+            .addExpr("mod(1, 1.0)", ZERO_1)
+            .addExpr("mod(1.0, 1.0)", ZERO_1)
             .addExpr("mod(c_tinyint, c_tinyint)", (byte) 0)
             .addExpr("mod(c_smallint, c_smallint)", (short) 0)
             .addExpr("mod(c_bigint, c_bigint)", 0L)
-            .addExpr("mod(c_decimal, c_decimal)", BigDecimal.ZERO)
-            .addExpr("mod(c_tinyint, c_decimal)", BigDecimal.ZERO)
+            .addExpr("mod(c_decimal, c_decimal)", ZERO_0)
+            .addExpr("mod(c_tinyint, c_decimal)", ZERO_0)
             // Test overflow
             .addExpr("c_tinyint_max + c_tinyint_max", (byte) -2)
             .addExpr("c_smallint_max + c_smallint_max", (short) -2)
@@ -371,8 +376,10 @@
   }
 
   @Test
-  @SqlOperatorTest(name = "LIKE", kind = "LIKE")
-  @SqlOperatorTest(name = "NOT LIKE", kind = "LIKE")
+  @SqlOperatorTests({
+    @SqlOperatorTest(name = "LIKE", kind = "LIKE"),
+    @SqlOperatorTest(name = "NOT LIKE", kind = "LIKE"),
+  })
   public void testLikeAndNotLike() {
     ExpressionChecker checker =
         new ExpressionChecker()
@@ -451,22 +458,24 @@
   }
 
   @Test
-  @SqlOperatorTest(name = "<", kind = "LESS_THAN")
-  @SqlOperatorTest(name = ">", kind = "GREATER_THAN")
-  @SqlOperatorTest(name = "<=", kind = "LESS_THAN_OR_EQUAL")
-  @SqlOperatorTest(name = "<>", kind = "NOT_EQUALS")
-  @SqlOperatorTest(name = "=", kind = "EQUALS")
-  @SqlOperatorTest(name = ">=", kind = "GREATER_THAN_OR_EQUAL")
-  @SqlOperatorTest(name = "IS NOT NULL", kind = "IS_NOT_NULL")
-  @SqlOperatorTest(name = "IS NULL", kind = "IS_NULL")
-  @SqlOperatorTest(name = "IS TRUE", kind = "IS_TRUE")
-  @SqlOperatorTest(name = "IS NOT TRUE", kind = "IS_NOT_TRUE")
-  @SqlOperatorTest(name = "IS FALSE", kind = "IS_FALSE")
-  @SqlOperatorTest(name = "IS NOT FALSE", kind = "IS_NOT_FALSE")
-  @SqlOperatorTest(name = "IS UNKNOWN", kind = "IS_NULL")
-  @SqlOperatorTest(name = "IS NOT UNKNOWN", kind = "IS_NOT_NULL")
-  @SqlOperatorTest(name = "IS DISTINCT FROM", kind = "IS_DISTINCT_FROM")
-  @SqlOperatorTest(name = "IS NOT DISTINCT FROM", kind = "IS_NOT_DISTINCT_FROM")
+  @SqlOperatorTests({
+    @SqlOperatorTest(name = "<", kind = "LESS_THAN"),
+    @SqlOperatorTest(name = ">", kind = "GREATER_THAN"),
+    @SqlOperatorTest(name = "<=", kind = "LESS_THAN_OR_EQUAL"),
+    @SqlOperatorTest(name = "<>", kind = "NOT_EQUALS"),
+    @SqlOperatorTest(name = "=", kind = "EQUALS"),
+    @SqlOperatorTest(name = ">=", kind = "GREATER_THAN_OR_EQUAL"),
+    @SqlOperatorTest(name = "IS NOT NULL", kind = "IS_NOT_NULL"),
+    @SqlOperatorTest(name = "IS NULL", kind = "IS_NULL"),
+    @SqlOperatorTest(name = "IS TRUE", kind = "IS_TRUE"),
+    @SqlOperatorTest(name = "IS NOT TRUE", kind = "IS_NOT_TRUE"),
+    @SqlOperatorTest(name = "IS FALSE", kind = "IS_FALSE"),
+    @SqlOperatorTest(name = "IS NOT FALSE", kind = "IS_NOT_FALSE"),
+    @SqlOperatorTest(name = "IS UNKNOWN", kind = "IS_NULL"),
+    @SqlOperatorTest(name = "IS NOT UNKNOWN", kind = "IS_NOT_NULL"),
+    @SqlOperatorTest(name = "IS DISTINCT FROM", kind = "IS_DISTINCT_FROM"),
+    @SqlOperatorTest(name = "IS NOT DISTINCT FROM", kind = "IS_NOT_DISTINCT_FROM"),
+  })
   public void testComparisonOperatorFunction() {
     ExpressionChecker checker =
         new ExpressionChecker()
@@ -698,16 +707,18 @@
   }
 
   @Test
-  @SqlOperatorTest(name = "CHARACTER_LENGTH", kind = "OTHER_FUNCTION")
-  @SqlOperatorTest(name = "CHAR_LENGTH", kind = "OTHER_FUNCTION")
-  @SqlOperatorTest(name = "INITCAP", kind = "OTHER_FUNCTION")
-  @SqlOperatorTest(name = "LOWER", kind = "OTHER_FUNCTION")
-  @SqlOperatorTest(name = "POSITION", kind = "POSITION")
-  @SqlOperatorTest(name = "OVERLAY", kind = "OTHER_FUNCTION")
-  @SqlOperatorTest(name = "SUBSTRING", kind = "OTHER_FUNCTION")
-  @SqlOperatorTest(name = "TRIM", kind = "TRIM")
-  @SqlOperatorTest(name = "UPPER", kind = "OTHER_FUNCTION")
-  @SqlOperatorTest(name = "||", kind = "OTHER")
+  @SqlOperatorTests({
+    @SqlOperatorTest(name = "CHARACTER_LENGTH", kind = "OTHER_FUNCTION"),
+    @SqlOperatorTest(name = "CHAR_LENGTH", kind = "OTHER_FUNCTION"),
+    @SqlOperatorTest(name = "INITCAP", kind = "OTHER_FUNCTION"),
+    @SqlOperatorTest(name = "LOWER", kind = "OTHER_FUNCTION"),
+    @SqlOperatorTest(name = "POSITION", kind = "POSITION"),
+    @SqlOperatorTest(name = "OVERLAY", kind = "OTHER_FUNCTION"),
+    @SqlOperatorTest(name = "SUBSTRING", kind = "OTHER_FUNCTION"),
+    @SqlOperatorTest(name = "TRIM", kind = "TRIM"),
+    @SqlOperatorTest(name = "UPPER", kind = "OTHER_FUNCTION"),
+    @SqlOperatorTest(name = "||", kind = "OTHER"),
+  })
   public void testStringFunctions() throws Exception {
     SqlExpressionChecker checker =
         new SqlExpressionChecker()
@@ -753,7 +764,7 @@
             .addExpr("ABS(c_tinyint)", (byte) Math.abs(BYTE_VALUE))
             .addExpr("ABS(c_double)", Math.abs(DOUBLE_VALUE))
             .addExpr("ABS(c_float)", Math.abs(FLOAT_VALUE))
-            .addExpr("ABS(c_decimal)", new BigDecimal(Math.abs(ONE.doubleValue())));
+            .addExpr("ABS(c_decimal)", ONE_0.abs());
     checker.buildRunAndCheck();
   }
 
@@ -768,7 +779,7 @@
             .addExpr("LN(c_tinyint)", Math.log(BYTE_VALUE))
             .addExpr("LN(c_double)", Math.log(DOUBLE_VALUE))
             .addExpr("LN(c_float)", Math.log(FLOAT_VALUE))
-            .addExpr("LN(c_decimal)", Math.log(ONE.doubleValue()));
+            .addExpr("LN(c_decimal)", Math.log(DECIMAL_VALUE.doubleValue()));
     checker.buildRunAndCheck();
   }
 
@@ -783,7 +794,7 @@
             .addExpr("SQRT(c_tinyint)", Math.sqrt(BYTE_VALUE))
             .addExpr("SQRT(c_double)", Math.sqrt(DOUBLE_VALUE))
             .addExpr("SQRT(c_float)", Math.sqrt(FLOAT_VALUE))
-            .addExpr("SQRT(c_decimal)", Math.sqrt(ONE.doubleValue()));
+            .addExpr("SQRT(c_decimal)", Math.sqrt(DECIMAL_VALUE.doubleValue()));
     checker.buildRunAndCheck();
   }
 
@@ -798,8 +809,7 @@
             .addExpr("ROUND(c_tinyint, 0)", (byte) SqlFunctions.sround(BYTE_VALUE, 0))
             .addExpr("ROUND(c_double, 0)", SqlFunctions.sround(DOUBLE_VALUE, 0))
             .addExpr("ROUND(c_float, 0)", (float) SqlFunctions.sround(FLOAT_VALUE, 0))
-            .addExpr(
-                "ROUND(c_decimal, 0)", new BigDecimal(SqlFunctions.sround(ONE.doubleValue(), 0)));
+            .addExpr("ROUND(c_decimal, 0)", SqlFunctions.sround(ONE_0, 0));
     checker.buildRunAndCheck();
   }
 
@@ -814,7 +824,7 @@
             .addExpr("LOG10(c_tinyint)", Math.log10(BYTE_VALUE))
             .addExpr("LOG10(c_double)", Math.log10(DOUBLE_VALUE))
             .addExpr("LOG10(c_float)", Math.log10(FLOAT_VALUE))
-            .addExpr("LOG10(c_decimal)", Math.log10(ONE.doubleValue()));
+            .addExpr("LOG10(c_decimal)", Math.log10(DECIMAL_VALUE.doubleValue()));
     checker.buildRunAndCheck();
   }
 
@@ -829,7 +839,7 @@
             .addExpr("EXP(c_tinyint)", Math.exp(BYTE_VALUE))
             .addExpr("EXP(c_double)", Math.exp(DOUBLE_VALUE))
             .addExpr("EXP(c_float)", Math.exp(FLOAT_VALUE))
-            .addExpr("EXP(c_decimal)", Math.exp(ONE.doubleValue()));
+            .addExpr("EXP(c_decimal)", Math.exp(DECIMAL_VALUE.doubleValue()));
     checker.buildRunAndCheck();
   }
 
@@ -844,7 +854,7 @@
             .addExpr("ACOS(c_tinyint)", Math.acos(BYTE_VALUE))
             .addExpr("ACOS(c_double)", Math.acos(DOUBLE_VALUE))
             .addExpr("ACOS(c_float)", Math.acos(FLOAT_VALUE))
-            .addExpr("ACOS(c_decimal)", Math.acos(ONE.doubleValue()));
+            .addExpr("ACOS(c_decimal)", Math.acos(DECIMAL_VALUE.doubleValue()));
     checker.buildRunAndCheck();
   }
 
@@ -859,7 +869,7 @@
             .addExpr("ASIN(c_tinyint)", Math.asin(BYTE_VALUE))
             .addExpr("ASIN(c_double)", Math.asin(DOUBLE_VALUE))
             .addExpr("ASIN(c_float)", Math.asin(FLOAT_VALUE))
-            .addExpr("ASIN(c_decimal)", Math.asin(ONE.doubleValue()));
+            .addExpr("ASIN(c_decimal)", Math.asin(DECIMAL_VALUE.doubleValue()));
     checker.buildRunAndCheck();
   }
 
@@ -874,7 +884,7 @@
             .addExpr("ATAN(c_tinyint)", Math.atan(BYTE_VALUE))
             .addExpr("ATAN(c_double)", Math.atan(DOUBLE_VALUE))
             .addExpr("ATAN(c_float)", Math.atan(FLOAT_VALUE))
-            .addExpr("ATAN(c_decimal)", Math.atan(ONE.doubleValue()));
+            .addExpr("ATAN(c_decimal)", Math.atan(DECIMAL_VALUE.doubleValue()));
     checker.buildRunAndCheck();
   }
 
@@ -889,7 +899,7 @@
             .addExpr("COT(c_tinyint)", 1.0d / Math.tan(BYTE_VALUE))
             .addExpr("COT(c_double)", 1.0d / Math.tan(DOUBLE_VALUE))
             .addExpr("COT(c_float)", 1.0d / Math.tan(FLOAT_VALUE))
-            .addExpr("COT(c_decimal)", 1.0d / Math.tan(ONE.doubleValue()));
+            .addExpr("COT(c_decimal)", 1.0d / Math.tan(DECIMAL_VALUE.doubleValue()));
     checker.buildRunAndCheck();
   }
 
@@ -904,7 +914,7 @@
             .addExpr("DEGREES(c_tinyint)", Math.toDegrees(BYTE_VALUE))
             .addExpr("DEGREES(c_double)", Math.toDegrees(DOUBLE_VALUE))
             .addExpr("DEGREES(c_float)", Math.toDegrees(FLOAT_VALUE))
-            .addExpr("DEGREES(c_decimal)", Math.toDegrees(ONE.doubleValue()));
+            .addExpr("DEGREES(c_decimal)", Math.toDegrees(DECIMAL_VALUE.doubleValue()));
     checker.buildRunAndCheck();
   }
 
@@ -919,7 +929,7 @@
             .addExpr("RADIANS(c_tinyint)", Math.toRadians(BYTE_VALUE))
             .addExpr("RADIANS(c_double)", Math.toRadians(DOUBLE_VALUE))
             .addExpr("RADIANS(c_float)", Math.toRadians(FLOAT_VALUE))
-            .addExpr("RADIANS(c_decimal)", Math.toRadians(ONE.doubleValue()));
+            .addExpr("RADIANS(c_decimal)", Math.toRadians(DECIMAL_VALUE.doubleValue()));
     checker.buildRunAndCheck();
   }
 
@@ -934,7 +944,7 @@
             .addExpr("COS(c_tinyint)", Math.cos(BYTE_VALUE))
             .addExpr("COS(c_double)", Math.cos(DOUBLE_VALUE))
             .addExpr("COS(c_float)", Math.cos(FLOAT_VALUE))
-            .addExpr("COS(c_decimal)", Math.cos(ONE.doubleValue()));
+            .addExpr("COS(c_decimal)", Math.cos(DECIMAL_VALUE.doubleValue()));
     checker.buildRunAndCheck();
   }
 
@@ -949,7 +959,7 @@
             .addExpr("SIN(c_tinyint)", Math.sin(BYTE_VALUE))
             .addExpr("SIN(c_double)", Math.sin(DOUBLE_VALUE))
             .addExpr("SIN(c_float)", Math.sin(FLOAT_VALUE))
-            .addExpr("SIN(c_decimal)", Math.sin(ONE.doubleValue()));
+            .addExpr("SIN(c_decimal)", Math.sin(DECIMAL_VALUE.doubleValue()));
     checker.buildRunAndCheck();
   }
 
@@ -964,7 +974,7 @@
             .addExpr("TAN(c_tinyint)", Math.tan(BYTE_VALUE))
             .addExpr("TAN(c_double)", Math.tan(DOUBLE_VALUE))
             .addExpr("TAN(c_float)", Math.tan(FLOAT_VALUE))
-            .addExpr("TAN(c_decimal)", Math.tan(ONE.doubleValue()));
+            .addExpr("TAN(c_decimal)", Math.tan(DECIMAL_VALUE.doubleValue()));
 
     checker.buildRunAndCheck();
   }
@@ -980,7 +990,7 @@
             .addExpr("SIGN(c_tinyint)", (byte) Integer.signum(BYTE_VALUE))
             .addExpr("SIGN(c_double)", Math.signum(DOUBLE_VALUE))
             .addExpr("SIGN(c_float)", Math.signum(FLOAT_VALUE))
-            .addExpr("SIGN(c_decimal)", BigDecimal.valueOf(ONE.signum()));
+            .addExpr("SIGN(c_decimal)", BigDecimal.valueOf(DECIMAL_VALUE.signum()));
 
     checker.buildRunAndCheck();
   }
@@ -996,14 +1006,14 @@
             .addExpr("POWER(c_tinyint, 2)", Math.pow(BYTE_VALUE, 2))
             .addExpr("POWER(c_double, 2)", Math.pow(DOUBLE_VALUE, 2))
             .addExpr("POWER(c_float, 2)", Math.pow(FLOAT_VALUE, 2))
-            .addExpr("POWER(c_decimal, 2)", Math.pow(ONE.doubleValue(), 2));
+            .addExpr("POWER(c_decimal, 2)", Math.pow(DECIMAL_VALUE.doubleValue(), 2));
 
     checker.buildRunAndCheck();
   }
 
   @Test
   @SqlOperatorTest(name = "PI", kind = "OTHER_FUNCTION")
-  public void testPi() throws Exception {
+  public void testPi() {
     ExpressionChecker checker = new ExpressionChecker().addExpr("PI", Math.PI);
 
     checker.buildRunAndCheck();
@@ -1020,7 +1030,7 @@
             .addExpr("ATAN2(c_tinyint, 2)", Math.atan2(BYTE_VALUE, 2))
             .addExpr("ATAN2(c_double, 2)", Math.atan2(DOUBLE_VALUE, 2))
             .addExpr("ATAN2(c_float, 2)", Math.atan2(FLOAT_VALUE, 2))
-            .addExpr("ATAN2(c_decimal, 2)", Math.atan2(ONE.doubleValue(), 2));
+            .addExpr("ATAN2(c_decimal, 2)", Math.atan2(DECIMAL_VALUE.doubleValue(), 2));
 
     checker.buildRunAndCheck();
   }
@@ -1036,7 +1046,7 @@
             .addExpr("TRUNCATE(c_tinyint, 2)", (byte) SqlFunctions.struncate(BYTE_VALUE, 2))
             .addExpr("TRUNCATE(c_double, 2)", SqlFunctions.struncate(DOUBLE_VALUE, 2))
             .addExpr("TRUNCATE(c_float, 2)", (float) SqlFunctions.struncate(FLOAT_VALUE, 2))
-            .addExpr("TRUNCATE(c_decimal, 2)", SqlFunctions.struncate(ONE, 2));
+            .addExpr("TRUNCATE(c_decimal, 2)", SqlFunctions.struncate(DECIMAL_VALUE, 2));
 
     checker.buildRunAndCheck();
   }
@@ -1065,9 +1075,11 @@
   }
 
   @Test
-  @SqlOperatorTest(name = "ARRAY", kind = "ARRAY_VALUE_CONSTRUCTOR")
-  @SqlOperatorTest(name = "CARDINALITY", kind = "OTHER_FUNCTION")
-  @SqlOperatorTest(name = "ELEMENT", kind = "OTHER_FUNCTION")
+  @SqlOperatorTests({
+    @SqlOperatorTest(name = "ARRAY", kind = "ARRAY_VALUE_CONSTRUCTOR"),
+    @SqlOperatorTest(name = "CARDINALITY", kind = "OTHER_FUNCTION"),
+    @SqlOperatorTest(name = "ELEMENT", kind = "OTHER_FUNCTION"),
+  })
   public void testArrayFunctions() {
     ExpressionChecker checker =
         new ExpressionChecker()
@@ -1086,17 +1098,19 @@
   }
 
   @Test
-  @SqlOperatorTest(name = "DAYOFMONTH", kind = "OTHER")
-  @SqlOperatorTest(name = "DAYOFWEEK", kind = "OTHER")
-  @SqlOperatorTest(name = "DAYOFYEAR", kind = "OTHER")
-  @SqlOperatorTest(name = "EXTRACT", kind = "EXTRACT")
-  @SqlOperatorTest(name = "YEAR", kind = "OTHER")
-  @SqlOperatorTest(name = "QUARTER", kind = "OTHER")
-  @SqlOperatorTest(name = "MONTH", kind = "OTHER")
-  @SqlOperatorTest(name = "WEEK", kind = "OTHER")
-  @SqlOperatorTest(name = "HOUR", kind = "OTHER")
-  @SqlOperatorTest(name = "MINUTE", kind = "OTHER")
-  @SqlOperatorTest(name = "SECOND", kind = "OTHER")
+  @SqlOperatorTests({
+    @SqlOperatorTest(name = "DAYOFMONTH", kind = "OTHER"),
+    @SqlOperatorTest(name = "DAYOFWEEK", kind = "OTHER"),
+    @SqlOperatorTest(name = "DAYOFYEAR", kind = "OTHER"),
+    @SqlOperatorTest(name = "EXTRACT", kind = "EXTRACT"),
+    @SqlOperatorTest(name = "YEAR", kind = "OTHER"),
+    @SqlOperatorTest(name = "QUARTER", kind = "OTHER"),
+    @SqlOperatorTest(name = "MONTH", kind = "OTHER"),
+    @SqlOperatorTest(name = "WEEK", kind = "OTHER"),
+    @SqlOperatorTest(name = "HOUR", kind = "OTHER"),
+    @SqlOperatorTest(name = "MINUTE", kind = "OTHER"),
+    @SqlOperatorTest(name = "SECOND", kind = "OTHER"),
+  })
   public void testBasicDateTimeFunctions() {
     ExpressionChecker checker =
         new ExpressionChecker()
@@ -1365,9 +1379,11 @@
   }
 
   @Test
-  @SqlOperatorTest(name = "CASE", kind = "CASE")
-  @SqlOperatorTest(name = "NULLIF", kind = "NULLIF")
-  @SqlOperatorTest(name = "COALESCE", kind = "COALESCE")
+  @SqlOperatorTests({
+    @SqlOperatorTest(name = "CASE", kind = "CASE"),
+    @SqlOperatorTest(name = "NULLIF", kind = "NULLIF"),
+    @SqlOperatorTest(name = "COALESCE", kind = "COALESCE"),
+  })
   public void testConditionalOperatorsAndFunctions() {
     ExpressionChecker checker =
         new ExpressionChecker()
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslUdfUdafTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslUdfUdafTest.java
index 9f4c9a2..9d0b634 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslUdfUdafTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlDslUdfUdafTest.java
@@ -48,9 +48,9 @@
 import org.apache.beam.sdk.values.PCollectionTuple;
 import org.apache.beam.sdk.values.Row;
 import org.apache.beam.sdk.values.TupleTag;
-import org.apache.beam.vendor.calcite.v1_20_0.com.google.common.collect.ImmutableMap;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.linq4j.function.Parameter;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.schema.TranslatableTable;
+import org.apache.beam.vendor.calcite.v1_26_0.com.google.common.collect.ImmutableMap;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.linq4j.function.Parameter;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.schema.TranslatableTable;
 import org.joda.time.Instant;
 import org.junit.Test;
 
@@ -302,7 +302,7 @@
   }
 
   /**
-   * test {@link org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.schema.TableMacro} UDF.
+   * test {@link org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.schema.TableMacro} UDF.
    */
   @Test
   public void testTableMacroUdf() throws Exception {
@@ -503,7 +503,7 @@
 
   /**
    * UDF to test support for {@link
-   * org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.schema.TableMacro}.
+   * org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.schema.TableMacro}.
    */
   public static final class RangeUdf implements BeamSqlUdf {
     public static TranslatableTable eval(int startInclusive, int endExclusive) {
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlExplainTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlExplainTest.java
index 3f0d2f2..4884e02 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlExplainTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlExplainTest.java
@@ -21,9 +21,9 @@
 
 import org.apache.beam.sdk.extensions.sql.meta.provider.text.TextTableProvider;
 import org.apache.beam.sdk.extensions.sql.meta.store.InMemoryMetaStore;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.parser.SqlParseException;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.tools.RelConversionException;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.tools.ValidationException;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.parser.SqlParseException;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.tools.RelConversionException;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.tools.ValidationException;
 import org.junit.Before;
 import org.junit.Ignore;
 
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlMapTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlMapTest.java
index e175530..ba7c9e4 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlMapTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlMapTest.java
@@ -23,7 +23,7 @@
 import org.apache.beam.sdk.transforms.Create;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.Row;
-import org.apache.beam.vendor.calcite.v1_20_0.com.google.common.collect.ImmutableMap;
+import org.apache.beam.vendor.calcite.v1_26_0.com.google.common.collect.ImmutableMap;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.ExpectedException;
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlMultipleSchemasTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlMultipleSchemasTest.java
index 41f916b..b891bd3 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlMultipleSchemasTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/BeamSqlMultipleSchemasTest.java
@@ -29,7 +29,7 @@
 import org.apache.beam.sdk.values.PBegin;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.Row;
-import org.apache.beam.vendor.calcite.v1_20_0.com.google.common.collect.ImmutableMap;
+import org.apache.beam.vendor.calcite.v1_26_0.com.google.common.collect.ImmutableMap;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.ExpectedException;
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/PubsubToBigqueryIT.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/PubsubToBigqueryIT.java
index 6e81721..b45986c 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/PubsubToBigqueryIT.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/PubsubToBigqueryIT.java
@@ -31,8 +31,8 @@
 import org.apache.beam.sdk.schemas.Schema;
 import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.values.Row;
-import org.apache.beam.vendor.calcite.v1_20_0.com.google.common.collect.ImmutableList;
-import org.apache.beam.vendor.calcite.v1_20_0.com.google.common.collect.ImmutableMap;
+import org.apache.beam.vendor.calcite.v1_26_0.com.google.common.collect.ImmutableList;
+import org.apache.beam.vendor.calcite.v1_26_0.com.google.common.collect.ImmutableMap;
 import org.joda.time.Duration;
 import org.joda.time.Instant;
 import org.junit.Rule;
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/TestUtils.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/TestUtils.java
index 6aa103b..6ab09c1 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/TestUtils.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/TestUtils.java
@@ -17,7 +17,7 @@
  */
 package org.apache.beam.sdk.extensions.sql;
 
-import static org.apache.beam.vendor.calcite.v1_20_0.com.google.common.base.Preconditions.checkArgument;
+import static org.apache.beam.vendor.calcite.v1_26_0.com.google.common.base.Preconditions.checkArgument;
 
 import java.util.ArrayList;
 import java.util.Arrays;
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/TypedCombineFnDelegateTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/TypedCombineFnDelegateTest.java
index b9daa02..9652e81 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/TypedCombineFnDelegateTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/TypedCombineFnDelegateTest.java
@@ -25,11 +25,11 @@
 import org.apache.beam.sdk.extensions.sql.impl.UdafImpl;
 import org.apache.beam.sdk.transforms.Combine;
 import org.apache.beam.sdk.transforms.Max;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.jdbc.JavaTypeFactoryImpl;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.type.RelDataTypeFactory;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.type.RelDataTypeSystem;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.schema.FunctionParameter;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.jdbc.JavaTypeFactoryImpl;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.type.RelDataTypeSystem;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.schema.FunctionParameter;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.type.SqlTypeName;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.ExpectedException;
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/JdbcDriverTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/JdbcDriverTest.java
index e1e17cd..c5308cc 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/JdbcDriverTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/JdbcDriverTest.java
@@ -50,10 +50,10 @@
 import org.apache.beam.sdk.schemas.Schema;
 import org.apache.beam.sdk.util.ReleaseInfo;
 import org.apache.beam.sdk.values.Row;
-import org.apache.beam.vendor.calcite.v1_20_0.com.google.common.collect.ImmutableMap;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.jdbc.CalciteConnection;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.jdbc.CalciteSchema;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.schema.SchemaPlus;
+import org.apache.beam.vendor.calcite.v1_26_0.com.google.common.collect.ImmutableMap;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.jdbc.CalciteConnection;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.jdbc.CalciteSchema;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.schema.SchemaPlus;
 import org.joda.time.DateTime;
 import org.joda.time.Duration;
 import org.joda.time.ReadableInstant;
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/LazyAggregateCombineFnTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/LazyAggregateCombineFnTest.java
index 5ba9799..d3cdea3 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/LazyAggregateCombineFnTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/LazyAggregateCombineFnTest.java
@@ -28,12 +28,12 @@
 import org.apache.beam.sdk.coders.CoderRegistry;
 import org.apache.beam.sdk.coders.VarLongCoder;
 import org.apache.beam.sdk.extensions.sql.udf.AggregateFn;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.jdbc.JavaTypeFactoryImpl;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.type.RelDataType;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.type.RelDataTypeFactory;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.type.RelDataTypeSystem;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.schema.AggregateFunction;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.schema.FunctionParameter;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.jdbc.JavaTypeFactoryImpl;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.type.RelDataType;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.type.RelDataTypeSystem;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.schema.AggregateFunction;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.schema.FunctionParameter;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
 import org.checkerframework.checker.nullness.qual.Nullable;
 import org.junit.Rule;
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/parser/BeamDDLNestedTypesTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/parser/BeamDDLNestedTypesTest.java
index b472bba..1e78f2f 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/parser/BeamDDLNestedTypesTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/parser/BeamDDLNestedTypesTest.java
@@ -32,7 +32,7 @@
 import org.apache.beam.sdk.extensions.sql.utils.QuickCheckGenerators.PrimitiveTypes;
 import org.apache.beam.sdk.schemas.Schema;
 import org.apache.beam.sdk.schemas.Schema.FieldType;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.parser.SqlParseException;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.parser.SqlParseException;
 import org.junit.runner.RunWith;
 
 /**
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/parser/BeamDDLTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/parser/BeamDDLTest.java
index fc0cce8..316f667 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/parser/BeamDDLTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/parser/BeamDDLTest.java
@@ -34,12 +34,12 @@
 import org.apache.beam.sdk.extensions.sql.meta.provider.test.TestTableProvider;
 import org.apache.beam.sdk.options.PipelineOptionsFactory;
 import org.apache.beam.sdk.schemas.Schema;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlIdentifier;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlLiteral;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlWriter;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.parser.SqlParserPos;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.pretty.SqlPrettyWriter;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlIdentifier;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlLiteral;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlWriter;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.pretty.SqlPrettyWriter;
 import org.junit.Test;
 
 /** UnitTest for {@link BeamSqlParserImpl}. */
@@ -225,13 +225,13 @@
     SqlIdentifier name = new SqlIdentifier("foo", SqlParserPos.ZERO);
     SqlNode jarPath = SqlLiteral.createCharString("path/to/udf.jar", SqlParserPos.ZERO);
     SqlCreateFunction createFunction =
-        new SqlCreateFunction(SqlParserPos.ZERO, name, jarPath, false);
+        new SqlCreateFunction(SqlParserPos.ZERO, false, name, jarPath, false);
     SqlWriter sqlWriter = new SqlPrettyWriter(BeamBigQuerySqlDialect.DEFAULT);
 
     createFunction.unparse(sqlWriter, 0, 0);
 
     assertEquals(
-        "CREATE FUNCTION `foo` USING JAR 'path/to/udf.jar'", sqlWriter.toSqlString().getSql());
+        "CREATE FUNCTION foo USING JAR 'path/to/udf.jar'", sqlWriter.toSqlString().getSql());
   }
 
   @Test
@@ -239,13 +239,13 @@
     SqlIdentifier name = new SqlIdentifier("foo", SqlParserPos.ZERO);
     SqlNode jarPath = SqlLiteral.createCharString("path/to/udf.jar", SqlParserPos.ZERO);
     SqlCreateFunction createFunction =
-        new SqlCreateFunction(SqlParserPos.ZERO, name, jarPath, true);
+        new SqlCreateFunction(SqlParserPos.ZERO, false, name, jarPath, true);
     SqlWriter sqlWriter = new SqlPrettyWriter(BeamBigQuerySqlDialect.DEFAULT);
 
     createFunction.unparse(sqlWriter, 0, 0);
 
     assertEquals(
-        "CREATE AGGREGATE FUNCTION `foo` USING JAR 'path/to/udf.jar'",
+        "CREATE AGGREGATE FUNCTION foo USING JAR 'path/to/udf.jar'",
         sqlWriter.toSqlString().getSql());
   }
 
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/planner/NodeStatsTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/planner/NodeStatsTest.java
index 9e442c5..1eb2fad 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/planner/NodeStatsTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/planner/NodeStatsTest.java
@@ -21,11 +21,11 @@
 import org.apache.beam.sdk.extensions.sql.impl.rel.BeamSqlRelUtils;
 import org.apache.beam.sdk.extensions.sql.meta.provider.test.TestBoundedTable;
 import org.apache.beam.sdk.schemas.Schema;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptCluster;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelTraitSet;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.volcano.RelSubset;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.SingleRel;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptCluster;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelTraitSet;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.volcano.RelSubset;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.SingleRel;
 import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Test;
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamAggregationRelTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamAggregationRelTest.java
index 3ebe01e..0b97ff6 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamAggregationRelTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamAggregationRelTest.java
@@ -23,7 +23,7 @@
 import org.apache.beam.sdk.extensions.sql.meta.provider.test.TestBoundedTable;
 import org.apache.beam.sdk.extensions.sql.meta.provider.test.TestUnboundedTable;
 import org.apache.beam.sdk.schemas.Schema;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelNode;
 import org.joda.time.DateTime;
 import org.joda.time.Duration;
 import org.junit.Assert;
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamCalcRelTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamCalcRelTest.java
index 8b1c2dc..27baad3 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamCalcRelTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamCalcRelTest.java
@@ -23,7 +23,7 @@
 import org.apache.beam.sdk.extensions.sql.meta.provider.test.TestBoundedTable;
 import org.apache.beam.sdk.extensions.sql.meta.provider.test.TestUnboundedTable;
 import org.apache.beam.sdk.schemas.Schema;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelNode;
 import org.joda.time.DateTime;
 import org.joda.time.Duration;
 import org.junit.Assert;
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamCoGBKJoinRelBoundedVsBoundedTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamCoGBKJoinRelBoundedVsBoundedTest.java
index 6859f96..f9ade2f 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamCoGBKJoinRelBoundedVsBoundedTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamCoGBKJoinRelBoundedVsBoundedTest.java
@@ -25,7 +25,7 @@
 import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.Row;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelNode;
 import org.hamcrest.core.StringContains;
 import org.junit.Assert;
 import org.junit.BeforeClass;
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamCoGBKJoinRelUnboundedVsUnboundedTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamCoGBKJoinRelUnboundedVsUnboundedTest.java
index f310265..27cda2b 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamCoGBKJoinRelUnboundedVsUnboundedTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamCoGBKJoinRelUnboundedVsUnboundedTest.java
@@ -28,7 +28,7 @@
 import org.apache.beam.sdk.transforms.ParDo;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.Row;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelNode;
 import org.joda.time.DateTime;
 import org.joda.time.Duration;
 import org.junit.Assert;
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamEnumerableConverterTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamEnumerableConverterTest.java
index a0cf404..58b0fc7 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamEnumerableConverterTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamEnumerableConverterTest.java
@@ -39,18 +39,18 @@
 import org.apache.beam.sdk.values.PDone;
 import org.apache.beam.sdk.values.POutput;
 import org.apache.beam.sdk.values.Row;
-import org.apache.beam.vendor.calcite.v1_20_0.com.google.common.collect.ImmutableList;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.adapter.java.JavaTypeFactory;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.jdbc.JavaTypeFactoryImpl;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.linq4j.Enumerable;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.linq4j.Enumerator;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptCluster;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.volcano.VolcanoPlanner;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.prepare.RelOptTableImpl;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.type.RelDataType;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.type.RelDataTypeSystem;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexBuilder;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexLiteral;
+import org.apache.beam.vendor.calcite.v1_26_0.com.google.common.collect.ImmutableList;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.jdbc.JavaTypeFactoryImpl;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.linq4j.Enumerable;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.linq4j.Enumerator;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptCluster;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.volcano.VolcanoPlanner;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.prepare.RelOptTableImpl;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.type.RelDataType;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.type.RelDataTypeSystem;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexBuilder;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexLiteral;
 import org.junit.Test;
 import org.junit.experimental.categories.Category;
 import org.junit.runner.RunWith;
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamIOSourceRelTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamIOSourceRelTest.java
index ff0d70f..ddec055 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamIOSourceRelTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamIOSourceRelTest.java
@@ -24,8 +24,8 @@
 import org.apache.beam.sdk.extensions.sql.meta.provider.test.TestUnboundedTable;
 import org.apache.beam.sdk.schemas.Schema;
 import org.apache.beam.sdk.testing.TestPipeline;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.joda.time.DateTime;
 import org.joda.time.Duration;
 import org.junit.Assert;
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamIntersectRelTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamIntersectRelTest.java
index d5acfab..6c7995b 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamIntersectRelTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamIntersectRelTest.java
@@ -26,7 +26,7 @@
 import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.Row;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelNode;
 import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Rule;
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamMinusRelTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamMinusRelTest.java
index 074c447..b1916c4 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamMinusRelTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamMinusRelTest.java
@@ -28,7 +28,7 @@
 import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.Row;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelNode;
 import org.joda.time.DateTime;
 import org.joda.time.Duration;
 import org.junit.Assert;
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSideInputJoinRelTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSideInputJoinRelTest.java
index 39e2a73..9bb4248 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSideInputJoinRelTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSideInputJoinRelTest.java
@@ -28,7 +28,7 @@
 import org.apache.beam.sdk.transforms.ParDo;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.Row;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelNode;
 import org.joda.time.DateTime;
 import org.joda.time.Duration;
 import org.junit.Assert;
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSortRelTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSortRelTest.java
index bba4876..a858788 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSortRelTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamSortRelTest.java
@@ -25,7 +25,7 @@
 import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.Row;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelNode;
 import org.joda.time.DateTime;
 import org.junit.Assert;
 import org.junit.Before;
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamUncollectRelTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamUncollectRelTest.java
index 640a1df..4b99d6a 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamUncollectRelTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamUncollectRelTest.java
@@ -27,7 +27,7 @@
 import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.Row;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelNode;
 import org.junit.Assert;
 import org.junit.Rule;
 import org.junit.Test;
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamUnionRelTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamUnionRelTest.java
index 6f77751..b729ae9 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamUnionRelTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamUnionRelTest.java
@@ -26,7 +26,7 @@
 import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.Row;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelNode;
 import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Rule;
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamValuesRelTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamValuesRelTest.java
index 0787751..6adfc98 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamValuesRelTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rel/BeamValuesRelTest.java
@@ -25,7 +25,7 @@
 import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.Row;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelNode;
 import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Rule;
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamAggregateProjectMergeRuleTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamAggregateProjectMergeRuleTest.java
index 2bce48c..5d13af9 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamAggregateProjectMergeRuleTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rule/BeamAggregateProjectMergeRuleTest.java
@@ -34,6 +34,7 @@
 import org.apache.beam.sdk.extensions.sql.meta.provider.test.TestTableProvider.PushDownOptions;
 import org.apache.beam.sdk.schemas.Schema;
 import org.apache.beam.sdk.testing.TestPipeline;
+import org.apache.beam.sdk.values.Row;
 import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
@@ -59,9 +60,19 @@
     Table projectTable = getTable("TEST_PROJECT", PushDownOptions.PROJECT);
     Table filterTable = getTable("TEST_FILTER", PushDownOptions.FILTER);
     Table noneTable = getTable("TEST_NONE", PushDownOptions.NONE);
+
     tableProvider.createTable(projectTable);
     tableProvider.createTable(filterTable);
     tableProvider.createTable(noneTable);
+
+    // Rules are cost based, need rows to optimize!
+    tableProvider.addRows(
+        "TEST_PROJECT", Row.withSchema(BASIC_SCHEMA).addValues(1, 2, "3", 4).build());
+    tableProvider.addRows(
+        "TEST_FILTER", Row.withSchema(BASIC_SCHEMA).addValues(1, 2, "3", 4).build());
+    tableProvider.addRows(
+        "TEST_NONE", Row.withSchema(BASIC_SCHEMA).addValues(1, 2, "3", 4).build());
+
     sqlEnv = BeamSqlEnv.inMemory(tableProvider);
   }
 
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rule/IOPushDownRuleTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rule/IOPushDownRuleTest.java
index 5cd2676..7795bfb 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rule/IOPushDownRuleTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rule/IOPushDownRuleTest.java
@@ -36,17 +36,13 @@
 import org.apache.beam.sdk.schemas.Schema;
 import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.values.Row;
-import org.apache.beam.vendor.calcite.v1_20_0.com.google.common.collect.ImmutableList;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptRule;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.Calc;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.rules.CalcMergeRule;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.rules.FilterCalcMergeRule;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.rules.FilterToCalcRule;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.rules.ProjectCalcMergeRule;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.rules.ProjectToCalcRule;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.tools.RuleSets;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.util.Pair;
+import org.apache.beam.vendor.calcite.v1_26_0.com.google.common.collect.ImmutableList;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptRule;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.core.Calc;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.rules.CoreRules;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.tools.RuleSets;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.util.Pair;
 import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
@@ -65,11 +61,11 @@
   private static final List<RelOptRule> defaultRules =
       ImmutableList.of(
           BeamCalcRule.INSTANCE,
-          FilterCalcMergeRule.INSTANCE,
-          ProjectCalcMergeRule.INSTANCE,
-          FilterToCalcRule.INSTANCE,
-          ProjectToCalcRule.INSTANCE,
-          CalcMergeRule.INSTANCE);
+          CoreRules.FILTER_CALC_MERGE,
+          CoreRules.PROJECT_CALC_MERGE,
+          CoreRules.FILTER_TO_CALC,
+          CoreRules.PROJECT_TO_CALC,
+          CoreRules.CALC_MERGE);
   private BeamSqlEnv sqlEnv;
 
   @Rule public TestPipeline pipeline = TestPipeline.create();
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rule/JoinReorderingTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rule/JoinReorderingTest.java
index a967fc3..d79005c 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rule/JoinReorderingTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/rule/JoinReorderingTest.java
@@ -31,43 +31,43 @@
 import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.options.PipelineOptionsFactory;
 import org.apache.beam.sdk.values.Row;
-import org.apache.beam.vendor.calcite.v1_20_0.com.google.common.collect.ImmutableList;
-import org.apache.beam.vendor.calcite.v1_20_0.com.google.common.collect.ImmutableMap;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.DataContext;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.adapter.enumerable.EnumerableConvention;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.adapter.enumerable.EnumerableRules;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.linq4j.Enumerable;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.linq4j.Linq4j;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.ConventionTraitDef;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptRule;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelTraitSet;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelCollationTraitDef;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelCollations;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelFieldCollation;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelRoot;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.Join;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.TableScan;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.rules.JoinCommuteRule;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.rules.SortProjectTransposeRule;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.type.RelDataType;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.type.RelDataTypeFactory;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.schema.ScannableTable;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.schema.SchemaPlus;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.schema.Statistic;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.schema.Statistics;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.schema.Table;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.schema.impl.AbstractSchema;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.schema.impl.AbstractTable;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.parser.SqlParser;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.tools.FrameworkConfig;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.tools.Frameworks;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.tools.Planner;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.tools.Programs;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.tools.RuleSet;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.tools.RuleSets;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.util.ImmutableBitSet;
+import org.apache.beam.vendor.calcite.v1_26_0.com.google.common.collect.ImmutableList;
+import org.apache.beam.vendor.calcite.v1_26_0.com.google.common.collect.ImmutableMap;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.DataContext;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.adapter.enumerable.EnumerableConvention;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.adapter.enumerable.EnumerableRules;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.linq4j.Enumerable;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.linq4j.Linq4j;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.ConventionTraitDef;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptRule;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelTraitSet;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelCollationTraitDef;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelCollations;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelFieldCollation;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelRoot;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.core.Join;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.core.TableScan;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.rules.CoreRules;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.rules.JoinCommuteRule;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.type.RelDataType;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.schema.ScannableTable;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.schema.SchemaPlus;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.schema.Statistic;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.schema.Statistics;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.schema.Table;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.schema.impl.AbstractSchema;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.schema.impl.AbstractTable;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.parser.SqlParser;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.tools.FrameworkConfig;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.tools.Frameworks;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.tools.Planner;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.tools.Programs;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.tools.RuleSet;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.tools.RuleSets;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.util.ImmutableBitSet;
 import org.junit.Assert;
 import org.junit.Test;
 
@@ -112,7 +112,7 @@
   public void testBeamJoinAssociationRule() throws Exception {
     RuleSet prepareRules =
         RuleSets.ofList(
-            SortProjectTransposeRule.INSTANCE,
+            CoreRules.SORT_PROJECT_TRANSPOSE,
             EnumerableRules.ENUMERABLE_JOIN_RULE,
             EnumerableRules.ENUMERABLE_PROJECT_RULE,
             EnumerableRules.ENUMERABLE_SORT_RULE,
@@ -141,7 +141,7 @@
   public void testBeamJoinPushThroughJoinRuleLeft() throws Exception {
     RuleSet prepareRules =
         RuleSets.ofList(
-            SortProjectTransposeRule.INSTANCE,
+            CoreRules.SORT_PROJECT_TRANSPOSE,
             EnumerableRules.ENUMERABLE_JOIN_RULE,
             EnumerableRules.ENUMERABLE_PROJECT_RULE,
             EnumerableRules.ENUMERABLE_SORT_RULE,
@@ -170,7 +170,7 @@
   public void testBeamJoinPushThroughJoinRuleRight() throws Exception {
     RuleSet prepareRules =
         RuleSets.ofList(
-            SortProjectTransposeRule.INSTANCE,
+            CoreRules.SORT_PROJECT_TRANSPOSE,
             EnumerableRules.ENUMERABLE_JOIN_RULE,
             EnumerableRules.ENUMERABLE_PROJECT_RULE,
             EnumerableRules.ENUMERABLE_SORT_RULE,
@@ -417,7 +417,7 @@
   }
 
   @Override
-  protected Map<String, org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.schema.Table>
+  protected Map<String, org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.schema.Table>
       getTableMap() {
     return tables;
   }
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/schema/BeamSqlRowCoderTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/schema/BeamSqlRowCoderTest.java
index d4819fc..6bf9509 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/schema/BeamSqlRowCoderTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/schema/BeamSqlRowCoderTest.java
@@ -27,10 +27,10 @@
 import org.apache.beam.sdk.schemas.SchemaCoder;
 import org.apache.beam.sdk.testing.CoderProperties;
 import org.apache.beam.sdk.values.Row;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.jdbc.JavaTypeFactoryImpl;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.type.RelDataType;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.type.RelDataTypeSystem;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.jdbc.JavaTypeFactoryImpl;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.type.RelDataType;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.type.RelDataTypeSystem;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.type.SqlTypeName;
 import org.joda.time.DateTime;
 import org.junit.Test;
 
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/utils/CalciteUtilsTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/utils/CalciteUtilsTest.java
index e76ee7f..707818b 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/utils/CalciteUtilsTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/utils/CalciteUtilsTest.java
@@ -24,11 +24,11 @@
 import java.util.Map;
 import java.util.stream.Collectors;
 import org.apache.beam.sdk.schemas.Schema;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.type.RelDataType;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.type.RelDataTypeFactory;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.type.RelDataTypeSystem;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.type.SqlTypeFactoryImpl;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.type.RelDataType;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.type.RelDataTypeSystem;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.type.SqlTypeFactoryImpl;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.type.SqlTypeName;
 import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/integrationtest/BeamSqlBuiltinFunctionsIntegrationTestBase.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/integrationtest/BeamSqlBuiltinFunctionsIntegrationTestBase.java
index 8ced3f3..25e6ef3 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/integrationtest/BeamSqlBuiltinFunctionsIntegrationTestBase.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/integrationtest/BeamSqlBuiltinFunctionsIntegrationTestBase.java
@@ -19,7 +19,7 @@
 
 import static org.apache.beam.sdk.extensions.sql.utils.DateTimeUtils.parseTimestampWithUTCTimeZone;
 import static org.apache.beam.sdk.extensions.sql.utils.RowAsserts.matchesScalar;
-import static org.apache.beam.vendor.calcite.v1_20_0.com.google.common.base.Preconditions.checkArgument;
+import static org.apache.beam.vendor.calcite.v1_26_0.com.google.common.base.Preconditions.checkArgument;
 import static org.junit.Assert.assertTrue;
 
 import com.google.auto.value.AutoValue;
@@ -50,8 +50,8 @@
 import org.apache.beam.sdk.values.PDone;
 import org.apache.beam.sdk.values.Row;
 import org.apache.beam.sdk.values.TypeDescriptors;
-import org.apache.beam.vendor.calcite.v1_20_0.com.google.common.collect.ImmutableMap;
-import org.apache.beam.vendor.calcite.v1_20_0.com.google.common.collect.Iterables;
+import org.apache.beam.vendor.calcite.v1_26_0.com.google.common.collect.ImmutableMap;
+import org.apache.beam.vendor.calcite.v1_26_0.com.google.common.collect.Iterables;
 import org.checkerframework.checker.nullness.qual.Nullable;
 import org.joda.time.DateTime;
 import org.junit.Rule;
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/integrationtest/BeamSqlDateFunctionsIntegrationTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/integrationtest/BeamSqlDateFunctionsIntegrationTest.java
index 0bffb5c..0d5db92 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/integrationtest/BeamSqlDateFunctionsIntegrationTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/integrationtest/BeamSqlDateFunctionsIntegrationTest.java
@@ -17,8 +17,8 @@
  */
 package org.apache.beam.sdk.extensions.sql.integrationtest;
 
-import static org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.avatica.util.DateTimeUtils.MILLIS_PER_DAY;
-import static org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.avatica.util.DateTimeUtils.MILLIS_PER_SECOND;
+import static org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.avatica.util.DateTimeUtils.MILLIS_PER_DAY;
+import static org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.avatica.util.DateTimeUtils.MILLIS_PER_SECOND;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/CustomTableResolverTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/CustomTableResolverTest.java
index c832251..d20274d 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/CustomTableResolverTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/CustomTableResolverTest.java
@@ -32,8 +32,8 @@
 import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.Row;
-import org.apache.beam.vendor.calcite.v1_20_0.com.google.common.collect.ImmutableList;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexNode;
+import org.apache.beam.vendor.calcite.v1_26_0.com.google.common.collect.ImmutableList;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexNode;
 import org.joda.time.Duration;
 import org.junit.Rule;
 import org.junit.Test;
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BigQueryReadWriteIT.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BigQueryReadWriteIT.java
index acc2d06..14d7422 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BigQueryReadWriteIT.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BigQueryReadWriteIT.java
@@ -17,7 +17,6 @@
  */
 package org.apache.beam.sdk.extensions.sql.meta.provider.bigquery;
 
-import static junit.framework.TestCase.assertNull;
 import static org.apache.beam.sdk.extensions.sql.meta.provider.bigquery.BigQueryTable.METHOD_PROPERTY;
 import static org.apache.beam.sdk.extensions.sql.meta.provider.bigquery.BigQueryTable.WRITE_DISPOSITION_PROPERTY;
 import static org.apache.beam.sdk.extensions.sql.utils.DateTimeUtils.parseTimestampWithUTCTimeZone;
@@ -42,7 +41,6 @@
 import org.apache.beam.sdk.PipelineResult;
 import org.apache.beam.sdk.PipelineResult.State;
 import org.apache.beam.sdk.extensions.sql.impl.BeamSqlEnv;
-import org.apache.beam.sdk.extensions.sql.impl.rel.BeamCalcRel;
 import org.apache.beam.sdk.extensions.sql.impl.rel.BeamPushDownIOSourceRel;
 import org.apache.beam.sdk.extensions.sql.impl.rel.BeamRelNode;
 import org.apache.beam.sdk.extensions.sql.impl.rel.BeamSqlRelUtils;
@@ -60,7 +58,7 @@
 import org.apache.beam.sdk.transforms.Create;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.Row;
-import org.apache.beam.vendor.calcite.v1_20_0.com.google.common.collect.ImmutableMap;
+import org.apache.beam.vendor.calcite.v1_26_0.com.google.common.collect.ImmutableMap;
 import org.joda.time.Duration;
 import org.junit.Rule;
 import org.junit.Test;
@@ -740,12 +738,10 @@
     BeamRelNode relNode = sqlEnv.parseQuery(selectTableStatement);
     PCollection<Row> output = BeamSqlRelUtils.toPCollection(readPipeline, relNode);
 
-    // Calc is not dropped because BigQuery does not support field reordering yet.
-    assertThat(relNode, instanceOf(BeamCalcRel.class));
-    assertThat(relNode.getInput(0), instanceOf(BeamPushDownIOSourceRel.class));
+    assertThat(relNode, instanceOf(BeamPushDownIOSourceRel.class));
     // IO projects fields in the same order they are defined in the schema.
     assertThat(
-        relNode.getInput(0).getRowType().getFieldNames(),
+        relNode.getRowType().getFieldNames(),
         containsInAnyOrder("c_tinyint", "c_integer", "c_varchar"));
     // Field reordering is done in a Calc
     assertThat(
@@ -816,15 +812,9 @@
     BeamRelNode relNode = sqlEnv.parseQuery(selectTableStatement);
     PCollection<Row> output = BeamSqlRelUtils.toPCollection(readPipeline, relNode);
 
-    assertThat(relNode, instanceOf(BeamCalcRel.class));
-    // Predicate should be pushed-down to IO level
-    assertNull(((BeamCalcRel) relNode).getProgram().getCondition());
-
-    assertThat(relNode.getInput(0), instanceOf(BeamPushDownIOSourceRel.class));
+    assertThat(relNode, instanceOf(BeamPushDownIOSourceRel.class));
     // Unused fields should not be projected by an IO
-    assertThat(
-        relNode.getInput(0).getRowType().getFieldNames(),
-        containsInAnyOrder("c_varchar", "c_integer"));
+    assertThat(relNode.getRowType().getFieldNames(), containsInAnyOrder("c_varchar", "c_integer"));
 
     assertThat(
         output.getSchema(),
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BigQueryRowCountIT.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BigQueryRowCountIT.java
index bd6d9ae..a2c5f83 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BigQueryRowCountIT.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BigQueryRowCountIT.java
@@ -38,7 +38,7 @@
 import org.apache.beam.sdk.schemas.Schema;
 import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.transforms.Create;
-import org.apache.beam.vendor.calcite.v1_20_0.com.google.common.collect.ImmutableList;
+import org.apache.beam.vendor.calcite.v1_26_0.com.google.common.collect.ImmutableList;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.runner.RunWith;
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BigQueryTestTableProvider.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BigQueryTestTableProvider.java
index f2caf8b..2e0fac6 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BigQueryTestTableProvider.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigquery/BigQueryTestTableProvider.java
@@ -17,7 +17,7 @@
  */
 package org.apache.beam.sdk.extensions.sql.meta.provider.bigquery;
 
-import static org.apache.beam.vendor.calcite.v1_20_0.com.google.common.base.MoreObjects.firstNonNull;
+import static org.apache.beam.vendor.calcite.v1_26_0.com.google.common.base.MoreObjects.firstNonNull;
 
 import java.util.HashMap;
 import java.util.Map;
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigtable/BigtableTableFlatTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigtable/BigtableTableFlatTest.java
index e517f5e..817570a 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigtable/BigtableTableFlatTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigtable/BigtableTableFlatTest.java
@@ -146,7 +146,7 @@
 
     String query =
         "INSERT INTO beamWriteTable(key, boolColumn, longColumn, stringColumn, doubleColumn) "
-            + "VALUES ('key', TRUE, 10, 'stringValue', 5.5)";
+            + "VALUES ('key', TRUE, CAST(10 AS bigint), 'stringValue', 5.5)";
 
     BeamSqlRelUtils.toPCollection(writePipeline, sqlEnv.parseQuery(query));
     writePipeline.run().waitUntilFinish();
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigtable/BigtableTableIT.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigtable/BigtableTableIT.java
index 4d74a06..0a9ab2f 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigtable/BigtableTableIT.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/bigtable/BigtableTableIT.java
@@ -100,7 +100,7 @@
     String query =
         String.format(
             "INSERT INTO `%s`(key, boolColumn, longColumn, stringColumn, doubleColumn) "
-                + "VALUES ('key1', FALSE, 1, 'string1', 1.0)",
+                + "VALUES ('key1', FALSE, CAST(1 as bigint), 'string1', 1.0)",
             TABLE_ID);
 
     BeamSqlRelUtils.toPCollection(p, sqlEnv.parseQuery(query));
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/datastore/DataStoreReadWriteIT.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/datastore/DataStoreReadWriteIT.java
index c2c499a..107da0d 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/datastore/DataStoreReadWriteIT.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/datastore/DataStoreReadWriteIT.java
@@ -50,7 +50,7 @@
 import org.apache.beam.sdk.transforms.Create;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.Row;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.avatica.util.ByteString;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.avatica.util.ByteString;
 import org.joda.time.Duration;
 import org.junit.Rule;
 import org.junit.Test;
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/mongodb/MongoDbTableProviderTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/mongodb/MongoDbTableProviderTest.java
index 459af56..5d75bba 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/mongodb/MongoDbTableProviderTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/mongodb/MongoDbTableProviderTest.java
@@ -27,7 +27,7 @@
 import org.apache.beam.sdk.extensions.sql.meta.BeamSqlTable;
 import org.apache.beam.sdk.extensions.sql.meta.Table;
 import org.apache.beam.sdk.schemas.Schema;
-import org.apache.beam.vendor.calcite.v1_20_0.com.google.common.collect.ImmutableList;
+import org.apache.beam.vendor.calcite.v1_26_0.com.google.common.collect.ImmutableList;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/pubsub/PubsubTableProviderIT.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/pubsub/PubsubTableProviderIT.java
index 2f409bf..11d4feea 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/pubsub/PubsubTableProviderIT.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/pubsub/PubsubTableProviderIT.java
@@ -69,13 +69,14 @@
 import org.apache.beam.sdk.util.common.ReflectHelpers;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.Row;
-import org.apache.beam.vendor.calcite.v1_20_0.com.google.common.collect.ImmutableList;
-import org.apache.beam.vendor.calcite.v1_20_0.com.google.common.collect.ImmutableMap;
-import org.apache.beam.vendor.calcite.v1_20_0.com.google.common.collect.ImmutableSet;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.jdbc.CalciteConnection;
+import org.apache.beam.vendor.calcite.v1_26_0.com.google.common.collect.ImmutableList;
+import org.apache.beam.vendor.calcite.v1_26_0.com.google.common.collect.ImmutableMap;
+import org.apache.beam.vendor.calcite.v1_26_0.com.google.common.collect.ImmutableSet;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.jdbc.CalciteConnection;
 import org.hamcrest.Matcher;
 import org.joda.time.Duration;
 import org.joda.time.Instant;
+import org.junit.Ignore;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.runner.RunWith;
@@ -193,6 +194,7 @@
     resultSignal.waitForSuccess(timeout);
   }
 
+  @Ignore("https://issues.apache.org/jira/browse/BEAM-12320")
   @Test
   public void testSQLSelectsArrayAttributes() throws Exception {
 
@@ -619,16 +621,13 @@
     String queryString =
         "INSERT INTO message "
             + "VALUES "
-            + "(TIMESTAMP '1970-01-01 00:00:00.001', 'person1', 80, TRUE), "
             + "(TIMESTAMP '1970-01-01 00:00:00.002', 'person2', 70, FALSE)";
     query(sqlEnv, pipeline, queryString);
 
     pipeline.run().waitUntilFinish(Duration.standardMinutes(5));
 
     eventsTopic
-        .assertThatTopicEventuallyReceives(
-            matcherTsNameHeightKnowsJS(ts(1), "person1", 80, true),
-            matcherTsNameHeightKnowsJS(ts(2), "person2", 70, false))
+        .assertThatTopicEventuallyReceives(matcherTsNameHeightKnowsJS(ts(2), "person2", 70, false))
         .waitForUpTo(Duration.standardSeconds(40));
   }
 
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/test/TestTableProviderTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/test/TestTableProviderTest.java
index c15101f..fa0a22d 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/test/TestTableProviderTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/test/TestTableProviderTest.java
@@ -28,7 +28,7 @@
 import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.Row;
-import org.apache.beam.vendor.calcite.v1_20_0.com.google.common.collect.ImmutableList;
+import org.apache.beam.vendor.calcite.v1_26_0.com.google.common.collect.ImmutableList;
 import org.joda.time.Duration;
 import org.junit.Before;
 import org.junit.Rule;
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/test/TestTableProviderWithFilterAndProjectPushDown.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/test/TestTableProviderWithFilterAndProjectPushDown.java
index d3221e0..eade9c8 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/test/TestTableProviderWithFilterAndProjectPushDown.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/test/TestTableProviderWithFilterAndProjectPushDown.java
@@ -41,14 +41,10 @@
 import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.Row;
-import org.apache.beam.vendor.calcite.v1_20_0.com.google.common.collect.ImmutableList;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptRule;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.rules.CalcMergeRule;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.rules.FilterCalcMergeRule;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.rules.FilterToCalcRule;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.rules.ProjectCalcMergeRule;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.rules.ProjectToCalcRule;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.tools.RuleSets;
+import org.apache.beam.vendor.calcite.v1_26_0.com.google.common.collect.ImmutableList;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptRule;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.rules.CoreRules;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.tools.RuleSets;
 import org.joda.time.Duration;
 import org.junit.Before;
 import org.junit.Rule;
@@ -69,12 +65,12 @@
   private static final List<RelOptRule> rulesWithPushDown =
       ImmutableList.of(
           BeamCalcRule.INSTANCE,
-          FilterCalcMergeRule.INSTANCE,
-          ProjectCalcMergeRule.INSTANCE,
+          CoreRules.FILTER_CALC_MERGE,
+          CoreRules.PROJECT_CALC_MERGE,
           BeamIOPushDownRule.INSTANCE,
-          FilterToCalcRule.INSTANCE,
-          ProjectToCalcRule.INSTANCE,
-          CalcMergeRule.INSTANCE);
+          CoreRules.FILTER_TO_CALC,
+          CoreRules.PROJECT_TO_CALC,
+          CoreRules.CALC_MERGE);
   private BeamSqlEnv sqlEnv;
 
   @Rule public TestPipeline pipeline = TestPipeline.create();
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/test/TestTableProviderWithFilterPushDown.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/test/TestTableProviderWithFilterPushDown.java
index 9a67950..029c4fa 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/test/TestTableProviderWithFilterPushDown.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/test/TestTableProviderWithFilterPushDown.java
@@ -42,15 +42,11 @@
 import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.Row;
-import org.apache.beam.vendor.calcite.v1_20_0.com.google.common.collect.ImmutableList;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptRule;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.Calc;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.rules.CalcMergeRule;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.rules.FilterCalcMergeRule;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.rules.FilterToCalcRule;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.rules.ProjectCalcMergeRule;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.rules.ProjectToCalcRule;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.tools.RuleSets;
+import org.apache.beam.vendor.calcite.v1_26_0.com.google.common.collect.ImmutableList;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptRule;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.core.Calc;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.rules.CoreRules;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.tools.RuleSets;
 import org.hamcrest.collection.IsIterableContainingInAnyOrder;
 import org.joda.time.Duration;
 import org.junit.Before;
@@ -72,12 +68,12 @@
   private static final List<RelOptRule> rulesWithPushDown =
       ImmutableList.of(
           BeamCalcRule.INSTANCE,
-          FilterCalcMergeRule.INSTANCE,
-          ProjectCalcMergeRule.INSTANCE,
+          CoreRules.FILTER_CALC_MERGE,
+          CoreRules.PROJECT_CALC_MERGE,
           BeamIOPushDownRule.INSTANCE,
-          FilterToCalcRule.INSTANCE,
-          ProjectToCalcRule.INSTANCE,
-          CalcMergeRule.INSTANCE);
+          CoreRules.FILTER_TO_CALC,
+          CoreRules.PROJECT_TO_CALC,
+          CoreRules.CALC_MERGE);
   private BeamSqlEnv sqlEnv;
 
   @Rule public TestPipeline pipeline = TestPipeline.create();
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/test/TestTableProviderWithProjectPushDown.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/test/TestTableProviderWithProjectPushDown.java
index 941e984..a7f3bc9 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/test/TestTableProviderWithProjectPushDown.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/test/TestTableProviderWithProjectPushDown.java
@@ -41,14 +41,10 @@
 import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.Row;
-import org.apache.beam.vendor.calcite.v1_20_0.com.google.common.collect.ImmutableList;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptRule;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.rules.CalcMergeRule;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.rules.FilterCalcMergeRule;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.rules.FilterToCalcRule;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.rules.ProjectCalcMergeRule;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.rules.ProjectToCalcRule;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.tools.RuleSets;
+import org.apache.beam.vendor.calcite.v1_26_0.com.google.common.collect.ImmutableList;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptRule;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.rules.CoreRules;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.tools.RuleSets;
 import org.joda.time.Duration;
 import org.junit.Before;
 import org.junit.Rule;
@@ -68,12 +64,12 @@
   private static final List<RelOptRule> rulesWithPushDown =
       ImmutableList.of(
           BeamCalcRule.INSTANCE,
-          FilterCalcMergeRule.INSTANCE,
-          ProjectCalcMergeRule.INSTANCE,
+          CoreRules.FILTER_CALC_MERGE,
+          CoreRules.PROJECT_CALC_MERGE,
           BeamIOPushDownRule.INSTANCE,
-          FilterToCalcRule.INSTANCE,
-          ProjectToCalcRule.INSTANCE,
-          CalcMergeRule.INSTANCE);
+          CoreRules.FILTER_TO_CALC,
+          CoreRules.PROJECT_TO_CALC,
+          CoreRules.CALC_MERGE);
   private BeamSqlEnv sqlEnv;
 
   @Rule public TestPipeline pipeline = TestPipeline.create();
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/text/TextTableProviderTest.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/text/TextTableProviderTest.java
index b34162a..4b7f77c 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/text/TextTableProviderTest.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/meta/provider/text/TextTableProviderTest.java
@@ -33,7 +33,7 @@
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.Row;
 import org.apache.beam.sdk.values.TypeDescriptors;
-import org.apache.beam.vendor.calcite.v1_20_0.com.google.common.base.Charsets;
+import org.apache.beam.vendor.calcite.v1_26_0.com.google.common.base.Charsets;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.TemporaryFolder;
diff --git a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/utils/RowAsserts.java b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/utils/RowAsserts.java
index abbb4df..af9581a 100644
--- a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/utils/RowAsserts.java
+++ b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/utils/RowAsserts.java
@@ -22,7 +22,7 @@
 
 import org.apache.beam.sdk.transforms.SerializableFunction;
 import org.apache.beam.sdk.values.Row;
-import org.apache.beam.vendor.calcite.v1_20_0.com.google.common.collect.Iterables;
+import org.apache.beam.vendor.calcite.v1_26_0.com.google.common.collect.Iterables;
 
 /** Contain helpers to assert {@link Row}s. */
 public class RowAsserts {
diff --git a/sdks/java/extensions/sql/zetasql/build.gradle b/sdks/java/extensions/sql/zetasql/build.gradle
index 5c5b8bb..975b3d7 100644
--- a/sdks/java/extensions/sql/zetasql/build.gradle
+++ b/sdks/java/extensions/sql/zetasql/build.gradle
@@ -35,7 +35,7 @@
   compile project(path: ":sdks:java:core", configuration: "shadow")
   compile project(":sdks:java:extensions:sql")
   compile project(":sdks:java:extensions:sql:udf")
-  compile library.java.vendored_calcite_1_20_0
+  compile library.java.vendored_calcite_1_26_0
   compile library.java.guava
   compile library.java.grpc_api
   compile library.java.joda_time
@@ -52,7 +52,7 @@
   compile "com.google.zetasql:zetasql-types:$zetasql_version"
   compile "com.google.zetasql:zetasql-jni-channel:$zetasql_version"
   permitUnusedDeclared "com.google.zetasql:zetasql-jni-channel:$zetasql_version" // BEAM-11761
-  testCompile library.java.vendored_calcite_1_20_0
+  testCompile library.java.vendored_calcite_1_26_0
   testCompile library.java.vendored_guava_26_0_jre
   testCompile library.java.junit
   testCompile library.java.mockito_core
diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/BeamCalcRelType.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/BeamCalcRelType.java
index dd1b56c..5b28855 100644
--- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/BeamCalcRelType.java
+++ b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/BeamCalcRelType.java
@@ -21,23 +21,22 @@
 import org.apache.beam.sdk.extensions.sql.impl.rel.BeamLogicalConvention;
 import org.apache.beam.sdk.extensions.sql.impl.rel.CalcRelSplitter;
 import org.apache.beam.sdk.extensions.sql.zetasql.translation.ZetaSqlScalarFunctionImpl;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.adapter.enumerable.CallImplementor;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.adapter.enumerable.RexImpTable;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptCluster;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptRule;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelTraitSet;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.type.RelDataType;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexCall;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexDynamicParam;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexFieldAccess;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexLiteral;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexLocalRef;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexProgram;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlOperator;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.validate.SqlUserDefinedFunction;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.tools.RelBuilder;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.adapter.enumerable.RexImpTable;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptCluster;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptRule;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelTraitSet;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.type.RelDataType;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexCall;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexDynamicParam;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexFieldAccess;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexLiteral;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexLocalRef;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexProgram;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlOperator;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.validate.SqlUserDefinedFunction;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.tools.RelBuilder;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -68,7 +67,7 @@
   protected boolean canImplement(RexCall call) {
     final SqlOperator operator = call.getOperator();
 
-    CallImplementor implementor = RexImpTable.INSTANCE.get(operator);
+    RexImpTable.RexCallImplementor implementor = RexImpTable.INSTANCE.get(operator);
     if (implementor == null) {
       // Reject methods with no implementation
       return false;
diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/BeamJavaUdfCalcRule.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/BeamJavaUdfCalcRule.java
index afcf0db..ace37c5 100644
--- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/BeamJavaUdfCalcRule.java
+++ b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/BeamJavaUdfCalcRule.java
@@ -21,7 +21,7 @@
 import org.apache.beam.sdk.extensions.sql.impl.rel.CalcRelSplitter;
 import org.apache.beam.sdk.extensions.sql.impl.rule.BeamCalcRule;
 import org.apache.beam.sdk.extensions.sql.impl.rule.BeamCalcSplittingRule;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.Calc;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.core.Calc;
 
 /**
  * A {@link BeamCalcSplittingRule} to replace {@link Calc} with {@link BeamCalcRel}.
diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/BeamZetaSqlCalcMergeRule.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/BeamZetaSqlCalcMergeRule.java
index 6f0681d..d1e8fff 100644
--- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/BeamZetaSqlCalcMergeRule.java
+++ b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/BeamZetaSqlCalcMergeRule.java
@@ -17,10 +17,11 @@
  */
 package org.apache.beam.sdk.extensions.sql.zetasql;
 
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptRule;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptRuleCall;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptRuleOperand;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.rules.CalcMergeRule;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptRule;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptRuleOperand;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.rules.CalcMergeRule;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.rules.CoreRules;
 
 /**
  * Planner rule to merge a {@link BeamZetaSqlCalcRel} with a {@link BeamZetaSqlCalcRel}. Subset of
@@ -39,6 +40,6 @@
 
   @Override
   public void onMatch(RelOptRuleCall call) {
-    CalcMergeRule.INSTANCE.onMatch(call);
+    CoreRules.CALC_MERGE.onMatch(call);
   }
 }
diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/BeamZetaSqlCalcRel.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/BeamZetaSqlCalcRel.java
index 3d60619..38c604e 100644
--- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/BeamZetaSqlCalcRel.java
+++ b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/BeamZetaSqlCalcRel.java
@@ -47,19 +47,19 @@
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PCollectionList;
 import org.apache.beam.sdk.values.Row;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptCluster;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelTraitSet;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.Calc;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.type.RelDataType;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexBuilder;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexProgram;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlDialect;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlIdentifier;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.fun.SqlStdOperatorTable;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptCluster;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelTraitSet;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.core.Calc;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.type.RelDataType;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexBuilder;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexProgram;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlDialect;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlIdentifier;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.parser.SqlParserPos;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
 import org.checkerframework.checker.nullness.qual.NonNull;
diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/BeamZetaSqlCalcRule.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/BeamZetaSqlCalcRule.java
index 41d6a67..74dbafc 100644
--- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/BeamZetaSqlCalcRule.java
+++ b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/BeamZetaSqlCalcRule.java
@@ -19,7 +19,7 @@
 
 import org.apache.beam.sdk.extensions.sql.impl.rel.CalcRelSplitter;
 import org.apache.beam.sdk.extensions.sql.impl.rule.BeamCalcSplittingRule;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.Calc;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.core.Calc;
 
 /** A {@link BeamCalcSplittingRule} to replace {@link Calc} with {@link BeamZetaSqlCalcRel}. */
 public class BeamZetaSqlCalcRule extends BeamCalcSplittingRule {
diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/BeamZetaSqlCatalog.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/BeamZetaSqlCatalog.java
index 3992536..ff78a60 100644
--- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/BeamZetaSqlCatalog.java
+++ b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/BeamZetaSqlCatalog.java
@@ -50,11 +50,11 @@
 import org.apache.beam.sdk.extensions.sql.udf.ScalarFn;
 import org.apache.beam.sdk.extensions.sql.zetasql.translation.UserFunctionDefinitions;
 import org.apache.beam.sdk.transforms.Combine;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.adapter.java.JavaTypeFactory;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.type.RelDataType;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.type.RelDataTypeField;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.schema.FunctionParameter;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.schema.SchemaPlus;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.type.RelDataType;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.schema.FunctionParameter;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.schema.SchemaPlus;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
 
 /**
@@ -350,7 +350,7 @@
 
   private void addUdfsFromSchema() {
     for (String functionName : calciteSchema.getFunctionNames()) {
-      Collection<org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.schema.Function>
+      Collection<org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.schema.Function>
           functions = calciteSchema.getFunctions(functionName);
       if (functions.size() != 1) {
         throw new IllegalArgumentException(
@@ -359,7 +359,7 @@
                     + " Beam ZetaSQL supports only a single function definition per function name (BEAM-12073).",
                 functionName, functions.size()));
       }
-      for (org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.schema.Function function :
+      for (org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.schema.Function function :
           functions) {
         List<String> path = Arrays.asList(functionName.split("\\."));
         if (function instanceof ScalarFunctionImpl) {
@@ -409,7 +409,7 @@
   }
 
   private List<FunctionArgumentType> getArgumentTypes(
-      org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.schema.Function function) {
+      org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.schema.Function function) {
     return function.getParameters().stream()
         .map(
             (arg) ->
@@ -494,7 +494,7 @@
 
     SimpleCatalog leafCatalog = createNestedCatalogs(zetaSqlCatalog, tablePath);
 
-    org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.schema.Table calciteTable =
+    org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.schema.Table calciteTable =
         TableResolution.resolveCalciteTable(calciteSchema, tablePath);
 
     if (calciteTable == null) {
diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/BeamZetaSqlRelType.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/BeamZetaSqlRelType.java
index 1d138ea..53cb446 100644
--- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/BeamZetaSqlRelType.java
+++ b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/BeamZetaSqlRelType.java
@@ -20,17 +20,17 @@
 import org.apache.beam.sdk.extensions.sql.impl.rel.BeamLogicalConvention;
 import org.apache.beam.sdk.extensions.sql.impl.rel.CalcRelSplitter;
 import org.apache.beam.sdk.extensions.sql.zetasql.translation.ZetaSqlScalarFunctionImpl;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptCluster;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptRule;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelTraitSet;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexCall;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexDynamicParam;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexFieldAccess;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexLiteral;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexProgram;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.validate.SqlUserDefinedFunction;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.tools.RelBuilder;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptCluster;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptRule;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelTraitSet;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexCall;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexDynamicParam;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexFieldAccess;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexLiteral;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexProgram;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.validate.SqlUserDefinedFunction;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.tools.RelBuilder;
 
 /** {@link CalcRelSplitter.RelType} for {@link BeamZetaSqlCalcRel}. */
 class BeamZetaSqlRelType extends CalcRelSplitter.RelType {
diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/DateTimeUtils.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/DateTimeUtils.java
index d1f0924..b258597 100644
--- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/DateTimeUtils.java
+++ b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/DateTimeUtils.java
@@ -21,7 +21,7 @@
 import io.grpc.Status;
 import java.time.LocalTime;
 import java.util.List;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.avatica.util.TimeUnit;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.avatica.util.TimeUnit;
 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.ImmutableMap;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Lists;
diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/TableResolution.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/TableResolution.java
index b76884d..907a39a 100644
--- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/TableResolution.java
+++ b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/TableResolution.java
@@ -23,10 +23,10 @@
 import org.apache.beam.sdk.extensions.sql.impl.BeamCalciteSchema;
 import org.apache.beam.sdk.extensions.sql.impl.TableName;
 import org.apache.beam.sdk.extensions.sql.meta.CustomTableResolver;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.jdbc.CalciteSchema;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.schema.Schema;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.schema.SchemaPlus;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.schema.Table;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.jdbc.CalciteSchema;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.schema.Schema;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.schema.SchemaPlus;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.schema.Table;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables;
 
 /** Utility methods to resolve a table, given a top-level Calcite schema and a table path. */
diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSQLPlannerImpl.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSQLPlannerImpl.java
index 7004b0e..2de752f 100644
--- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSQLPlannerImpl.java
+++ b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSQLPlannerImpl.java
@@ -25,21 +25,21 @@
 import org.apache.beam.sdk.extensions.sql.zetasql.translation.ConversionContext;
 import org.apache.beam.sdk.extensions.sql.zetasql.translation.ExpressionConverter;
 import org.apache.beam.sdk.extensions.sql.zetasql.translation.QueryStatementConverter;
-import org.apache.beam.vendor.calcite.v1_20_0.com.google.common.collect.ImmutableList;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.adapter.java.JavaTypeFactory;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptCluster;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptPlanner;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelTraitSet;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelRoot;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexBuilder;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexExecutor;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.schema.SchemaPlus;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlKind;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.tools.FrameworkConfig;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.tools.Frameworks;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.tools.Program;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.util.Util;
+import org.apache.beam.vendor.calcite.v1_26_0.com.google.common.collect.ImmutableList;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptCluster;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptPlanner;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelTraitSet;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelRoot;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexBuilder;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexExecutor;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.schema.SchemaPlus;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlKind;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.tools.FrameworkConfig;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.tools.Frameworks;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.tools.Program;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.util.Util;
 
 /** ZetaSQLPlannerImpl. */
 @SuppressWarnings({
diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSQLQueryPlanner.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSQLQueryPlanner.java
index fb90a26..2bc25ed 100644
--- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSQLQueryPlanner.java
+++ b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSQLQueryPlanner.java
@@ -38,33 +38,33 @@
 import org.apache.beam.sdk.extensions.sql.impl.rule.BeamUnnestRule;
 import org.apache.beam.sdk.extensions.sql.zetasql.unnest.BeamZetaSqlUncollectRule;
 import org.apache.beam.sdk.extensions.sql.zetasql.unnest.BeamZetaSqlUnnestRule;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.config.CalciteConnectionConfig;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.jdbc.CalciteSchema;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.ConventionTraitDef;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptPlanner;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptRule;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptUtil;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelTraitDef;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelTraitSet;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.prepare.CalciteCatalogReader;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelRoot;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.metadata.ChainedRelMetadataProvider;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.metadata.JaninoRelMetadataProvider;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.metadata.RelMetadataQuery;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.rules.FilterCalcMergeRule;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.rules.JoinCommuteRule;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.rules.ProjectCalcMergeRule;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.schema.SchemaPlus;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlOperatorTable;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.fun.SqlStdOperatorTable;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.parser.SqlParser;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.parser.SqlParserImplFactory;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.util.ChainedSqlOperatorTable;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.tools.FrameworkConfig;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.tools.Frameworks;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.tools.RuleSet;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.tools.RuleSets;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.config.CalciteConnectionConfig;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.jdbc.CalciteSchema;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.ConventionTraitDef;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptPlanner;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptRule;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptUtil;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelTraitDef;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelTraitSet;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.prepare.CalciteCatalogReader;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelRoot;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.metadata.ChainedRelMetadataProvider;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.metadata.JaninoRelMetadataProvider;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.rules.FilterCalcMergeRule;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.rules.JoinCommuteRule;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.rules.ProjectCalcMergeRule;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.schema.SchemaPlus;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlOperatorTable;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.parser.SqlParser;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.parser.SqlParserImplFactory;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.util.SqlOperatorTables;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.tools.FrameworkConfig;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.tools.Frameworks;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.tools.RuleSet;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.tools.RuleSets;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -206,7 +206,7 @@
         .getCluster()
         .setMetadataProvider(
             ChainedRelMetadataProvider.of(
-                org.apache.beam.vendor.calcite.v1_20_0.com.google.common.collect.ImmutableList.of(
+                org.apache.beam.vendor.calcite.v1_26_0.com.google.common.collect.ImmutableList.of(
                     NonCumulativeCostImpl.SOURCE,
                     RelMdNodeStats.SOURCE,
                     root.rel.getCluster().getMetadataProvider())));
@@ -259,7 +259,7 @@
         .ruleSets(ruleSets.toArray(new RuleSet[0]))
         .costFactory(BeamCostModel.FACTORY)
         .typeSystem(connection.getTypeFactory().getTypeSystem())
-        .operatorTable(ChainedSqlOperatorTable.of(opTab0, catalogReader))
+        .operatorTable(SqlOperatorTables.chain(opTab0, catalogReader))
         .build();
   }
 }
diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlCalciteTranslationUtils.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlCalciteTranslationUtils.java
index d5f5997..717c71f 100644
--- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlCalciteTranslationUtils.java
+++ b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlCalciteTranslationUtils.java
@@ -34,17 +34,17 @@
 import org.apache.beam.sdk.annotations.Internal;
 import org.apache.beam.sdk.extensions.sql.meta.provider.bigquery.BeamBigQuerySqlDialect;
 import org.apache.beam.sdk.extensions.sql.zetasql.translation.SqlOperators;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.avatica.util.ByteString;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.avatica.util.TimeUnit;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.avatica.util.TimeUnitRange;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.type.RelDataType;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexBuilder;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.fun.SqlStdOperatorTable;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.type.SqlTypeName;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.util.DateString;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.util.TimeString;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.util.TimestampString;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.avatica.util.ByteString;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.avatica.util.TimeUnit;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.avatica.util.TimeUnitRange;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.type.RelDataType;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexBuilder;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.util.DateString;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.util.TimeString;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.util.TimestampString;
 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/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/AggregateScanConverter.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/AggregateScanConverter.java
index 522b669..bd73241 100644
--- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/AggregateScanConverter.java
+++ b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/AggregateScanConverter.java
@@ -36,16 +36,16 @@
 import org.apache.beam.sdk.extensions.sql.impl.UdafImpl;
 import org.apache.beam.sdk.extensions.sql.zetasql.BeamZetaSqlCatalog;
 import org.apache.beam.sdk.extensions.sql.zetasql.ZetaSqlCalciteTranslationUtils;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelCollations;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.AggregateCall;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.logical.LogicalAggregate;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.logical.LogicalProject;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.type.RelDataType;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlAggFunction;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.type.SqlReturnTypeInference;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.util.ImmutableBitSet;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelCollations;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.core.AggregateCall;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.logical.LogicalAggregate;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.logical.LogicalProject;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.type.RelDataType;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlAggFunction;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.type.SqlReturnTypeInference;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.util.ImmutableBitSet;
 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;
 
@@ -173,7 +173,7 @@
       }
     }
 
-    return LogicalProject.create(input, projects, fieldNames);
+    return LogicalProject.create(input, ImmutableList.of(), projects, fieldNames);
   }
 
   private AggregateCall convertAggCall(
diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/ArrayScanColumnRefToUncollect.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/ArrayScanColumnRefToUncollect.java
index 08a0fda..655140d 100644
--- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/ArrayScanColumnRefToUncollect.java
+++ b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/ArrayScanColumnRefToUncollect.java
@@ -22,17 +22,17 @@
 import java.util.Collections;
 import java.util.List;
 import org.apache.beam.sdk.extensions.sql.zetasql.unnest.ZetaSqlUnnest;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.CorrelationId;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.JoinRelType;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.logical.LogicalCorrelate;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.logical.LogicalProject;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexBuilder;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexInputRef;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.util.ImmutableBitSet;
-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.calcite.v1_26_0.com.google.common.collect.ImmutableList;
+import org.apache.beam.vendor.calcite.v1_26_0.com.google.common.collect.ImmutableMap;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.core.CorrelationId;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.core.JoinRelType;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.logical.LogicalCorrelate;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.logical.LogicalProject;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexBuilder;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexInputRef;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.util.ImmutableBitSet;
 
 /**
  * Converts array scan that represents a reference to an array column, or an (possibly nested) array
@@ -87,6 +87,7 @@
     RelNode projectNode =
         LogicalProject.create(
             createOneRow(getCluster()),
+            ImmutableList.of(),
             Collections.singletonList(
                 convertArrayExpr(
                     zetaNode.getArrayExpr(), getCluster().getRexBuilder(), convertedColumnRef)),
diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/ArrayScanLiteralToUncollectConverter.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/ArrayScanLiteralToUncollectConverter.java
index 402bb7b..f194840 100644
--- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/ArrayScanLiteralToUncollectConverter.java
+++ b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/ArrayScanLiteralToUncollectConverter.java
@@ -21,9 +21,9 @@
 import java.util.Collections;
 import java.util.List;
 import org.apache.beam.sdk.extensions.sql.zetasql.unnest.ZetaSqlUnnest;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.logical.LogicalProject;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.logical.LogicalProject;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexNode;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
 
 /** Converts array scan that represents an array literal to uncollect. */
@@ -51,6 +51,7 @@
     RelNode projectNode =
         LogicalProject.create(
             createOneRow(getCluster()),
+            ImmutableList.of(),
             Collections.singletonList(arrayLiteralExpression),
             ImmutableList.of(fieldName));
 
diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/ArrayScanToJoinConverter.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/ArrayScanToJoinConverter.java
index 629d036..ccc43ab 100644
--- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/ArrayScanToJoinConverter.java
+++ b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/ArrayScanToJoinConverter.java
@@ -24,15 +24,15 @@
 import java.util.Collections;
 import java.util.List;
 import org.apache.beam.sdk.extensions.sql.zetasql.unnest.ZetaSqlUnnest;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.CorrelationId;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.JoinRelType;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.logical.LogicalJoin;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.logical.LogicalProject;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexInputRef;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexNode;
-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.ImmutableSet;
+import org.apache.beam.vendor.calcite.v1_26_0.com.google.common.collect.ImmutableList;
+import org.apache.beam.vendor.calcite.v1_26_0.com.google.common.collect.ImmutableSet;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.core.CorrelationId;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.core.JoinRelType;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.logical.LogicalJoin;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.logical.LogicalProject;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexInputRef;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexNode;
 
 /** Converts array scan that represents join of an uncollect(array_field) to uncollect. */
 class ArrayScanToJoinConverter extends RelConverter<ResolvedArrayScan> {
@@ -80,7 +80,8 @@
                         columnRef.getColumn().getId(), zetaNode.getInputScan().getColumnList())));
 
     RelNode projectNode =
-        LogicalProject.create(createOneRow(getCluster()), projects, ImmutableList.of(columnName));
+        LogicalProject.create(
+            createOneRow(getCluster()), ImmutableList.of(), projects, ImmutableList.of(columnName));
 
     // Create an UnCollect
     boolean ordinality = (zetaNode.getArrayOffsetColumn() != null);
@@ -104,13 +105,22 @@
               zetaNode.getArrayOffsetColumn().getColumn().getName()));
     }
 
-    RelNode rightInput = LogicalProject.create(uncollectNode, rightProjects, rightNames);
+    RelNode rightInput =
+        LogicalProject.create(uncollectNode, ImmutableList.of(), rightProjects, rightNames);
 
     // Join condition should be a RexNode converted from join_expr.
     RexNode condition =
         getExpressionConverter().convertRexNodeFromResolvedExpr(zetaNode.getJoinExpr());
     JoinRelType joinRelType = zetaNode.getIsOuter() ? JoinRelType.LEFT : JoinRelType.INNER;
 
-    return LogicalJoin.create(leftInput, rightInput, condition, ImmutableSet.of(), joinRelType);
+    return LogicalJoin.create(
+        leftInput,
+        rightInput,
+        ImmutableList.of(),
+        condition,
+        ImmutableSet.of(),
+        joinRelType,
+        false,
+        ImmutableList.of());
   }
 }
diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/ConversionContext.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/ConversionContext.java
index 3f87c54..0e7b4b8 100644
--- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/ConversionContext.java
+++ b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/ConversionContext.java
@@ -23,9 +23,9 @@
 import java.util.Map;
 import org.apache.beam.sdk.annotations.Internal;
 import org.apache.beam.sdk.extensions.sql.zetasql.QueryTrait;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptCluster;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.tools.FrameworkConfig;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptCluster;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.tools.FrameworkConfig;
 
 /** Conversion context, some rules need this data to convert the nodes. */
 @Internal
diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/ExpressionConverter.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/ExpressionConverter.java
index 349b5c0..cae2ac3 100644
--- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/ExpressionConverter.java
+++ b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/ExpressionConverter.java
@@ -71,24 +71,24 @@
 import org.apache.beam.sdk.extensions.sql.impl.utils.TVFStreamingUtils;
 import org.apache.beam.sdk.extensions.sql.zetasql.ZetaSqlCalciteTranslationUtils;
 import org.apache.beam.sdk.extensions.sql.zetasql.ZetaSqlException;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.avatica.util.TimeUnit;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptCluster;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.type.RelDataType;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.type.RelDataTypeField;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.type.RelDataTypeFieldImpl;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.type.RelRecordType;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexBuilder;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexCall;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexInputRef;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexLiteral;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlIdentifier;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlIntervalQualifier;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlKind;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlOperator;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.fun.SqlRowOperator;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.avatica.util.TimeUnit;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptCluster;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.type.RelDataType;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.type.RelDataTypeFieldImpl;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.type.RelRecordType;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexBuilder;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexCall;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexInputRef;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexLiteral;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlIdentifier;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlIntervalQualifier;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlKind;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlOperator;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.fun.SqlRowOperator;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.parser.SqlParserPos;
 import org.checkerframework.checker.nullness.qual.Nullable;
 
 /**
diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/FilterScanConverter.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/FilterScanConverter.java
index bf72c2c..b1cd3d1 100644
--- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/FilterScanConverter.java
+++ b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/FilterScanConverter.java
@@ -21,9 +21,9 @@
 import com.google.zetasql.resolvedast.ResolvedNodes.ResolvedFilterScan;
 import java.util.Collections;
 import java.util.List;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.logical.LogicalFilter;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.logical.LogicalFilter;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexNode;
 
 /** Converts filter. */
 class FilterScanConverter extends RelConverter<ResolvedFilterScan> {
diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/JoinScanConverter.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/JoinScanConverter.java
index 0f94206..7c35709 100644
--- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/JoinScanConverter.java
+++ b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/JoinScanConverter.java
@@ -22,11 +22,11 @@
 import com.google.zetasql.resolvedast.ResolvedNode;
 import com.google.zetasql.resolvedast.ResolvedNodes.ResolvedJoinScan;
 import java.util.List;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.JoinRelType;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.logical.LogicalJoin;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.type.RelDataTypeField;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.core.JoinRelType;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.logical.LogicalJoin;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexNode;
 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.ImmutableSet;
@@ -92,6 +92,7 @@
     return LogicalJoin.create(
         convertedLeftInput,
         convertedRightInput,
+        ImmutableList.of(),
         condition,
         ImmutableSet.of(),
         convertResolvedJoinType(zetaNode.getJoinType()));
diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/LimitOffsetScanToLimitConverter.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/LimitOffsetScanToLimitConverter.java
index 679d678..50cc96e 100644
--- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/LimitOffsetScanToLimitConverter.java
+++ b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/LimitOffsetScanToLimitConverter.java
@@ -22,13 +22,13 @@
 import com.google.zetasql.resolvedast.ResolvedNodes.ResolvedOrderByScan;
 import java.util.Collections;
 import java.util.List;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelCollation;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelCollations;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.logical.LogicalSort;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexDynamicParam;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexLiteral;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelCollation;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelCollations;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.logical.LogicalSort;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexDynamicParam;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexLiteral;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexNode;
 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/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/LimitOffsetScanToOrderByLimitConverter.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/LimitOffsetScanToOrderByLimitConverter.java
index d54ba3b..cdd7079 100644
--- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/LimitOffsetScanToOrderByLimitConverter.java
+++ b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/LimitOffsetScanToOrderByLimitConverter.java
@@ -18,8 +18,8 @@
 package org.apache.beam.sdk.extensions.sql.zetasql.translation;
 
 import static java.util.stream.Collectors.toList;
-import static org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelFieldCollation.Direction.ASCENDING;
-import static org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelFieldCollation.Direction.DESCENDING;
+import static org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelFieldCollation.Direction.ASCENDING;
+import static org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelFieldCollation.Direction.DESCENDING;
 
 import com.google.zetasql.resolvedast.ResolvedNode;
 import com.google.zetasql.resolvedast.ResolvedNodes.ResolvedLimitOffsetScan;
@@ -27,15 +27,16 @@
 import com.google.zetasql.resolvedast.ResolvedNodes.ResolvedOrderByScan;
 import java.util.Collections;
 import java.util.List;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelCollation;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelCollationImpl;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelFieldCollation;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelFieldCollation.Direction;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.logical.LogicalProject;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.logical.LogicalSort;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexLiteral;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelCollation;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelCollationImpl;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelFieldCollation;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelFieldCollation.Direction;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.logical.LogicalProject;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.logical.LogicalSort;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexLiteral;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexNode;
+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;
 
 /** Converts ORDER BY LIMIT. */
@@ -110,6 +111,6 @@
             .retrieveRexNodeFromOrderByScan(getCluster(), node, input.getRowType().getFieldList());
     List<String> fieldNames = getTrait().retrieveFieldNames(node.getColumnList());
 
-    return LogicalProject.create(input, projects, fieldNames);
+    return LogicalProject.create(input, ImmutableList.of(), projects, fieldNames);
   }
 }
diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/OrderByScanUnsupportedConverter.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/OrderByScanUnsupportedConverter.java
index 878b2b2..d564a1a 100644
--- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/OrderByScanUnsupportedConverter.java
+++ b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/OrderByScanUnsupportedConverter.java
@@ -19,7 +19,7 @@
 
 import com.google.zetasql.resolvedast.ResolvedNodes.ResolvedOrderByScan;
 import java.util.List;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelNode;
 
 /**
  * Always throws exception, represents the case when order by is used without limit.
diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/ProjectScanConverter.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/ProjectScanConverter.java
index d19b765..81fa769 100644
--- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/ProjectScanConverter.java
+++ b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/ProjectScanConverter.java
@@ -21,9 +21,10 @@
 import com.google.zetasql.resolvedast.ResolvedNodes.ResolvedProjectScan;
 import java.util.Collections;
 import java.util.List;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.logical.LogicalProject;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexNode;
+import org.apache.beam.vendor.calcite.v1_26_0.com.google.common.collect.ImmutableList;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.logical.LogicalProject;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexNode;
 
 /** Converts projection. */
 class ProjectScanConverter extends RelConverter<ResolvedProjectScan> {
@@ -44,6 +45,6 @@
     List<RexNode> projects =
         getExpressionConverter().retrieveRexNode(zetaNode, input.getRowType().getFieldList());
     List<String> fieldNames = getTrait().retrieveFieldNames(zetaNode.getColumnList());
-    return LogicalProject.create(input, projects, fieldNames);
+    return LogicalProject.create(input, ImmutableList.of(), projects, fieldNames);
   }
 }
diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/QueryStatementConverter.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/QueryStatementConverter.java
index 45ece98..57bf889 100644
--- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/QueryStatementConverter.java
+++ b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/QueryStatementConverter.java
@@ -37,7 +37,7 @@
 import com.google.zetasql.resolvedast.ResolvedNodes.ResolvedQueryStmt;
 import java.util.Collections;
 import java.util.List;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelNode;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMultimap;
 
 /**
diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/RelConverter.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/RelConverter.java
index 55e3f39..503c699 100644
--- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/RelConverter.java
+++ b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/RelConverter.java
@@ -22,14 +22,14 @@
 import java.util.Collections;
 import java.util.List;
 import org.apache.beam.sdk.extensions.sql.zetasql.QueryTrait;
-import org.apache.beam.vendor.calcite.v1_20_0.com.google.common.collect.ImmutableList;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptCluster;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.logical.LogicalValues;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.type.RelDataType;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexLiteral;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.type.SqlTypeName;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.tools.FrameworkConfig;
+import org.apache.beam.vendor.calcite.v1_26_0.com.google.common.collect.ImmutableList;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptCluster;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.logical.LogicalValues;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.type.RelDataType;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexLiteral;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.tools.FrameworkConfig;
 
 /** A rule that converts Zeta SQL resolved relational node to corresponding Calcite rel node. */
 abstract class RelConverter<T extends ResolvedNode> {
diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/SetOperationScanConverter.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/SetOperationScanConverter.java
index 375021b..b09f3dd 100644
--- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/SetOperationScanConverter.java
+++ b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/SetOperationScanConverter.java
@@ -32,10 +32,10 @@
 import java.util.List;
 import java.util.function.BiFunction;
 import java.util.function.Function;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.logical.LogicalIntersect;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.logical.LogicalMinus;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.logical.LogicalUnion;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.logical.LogicalIntersect;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.logical.LogicalMinus;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.logical.LogicalUnion;
 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/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/SingleRowScanConverter.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/SingleRowScanConverter.java
index e05edc8..04e8825 100644
--- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/SingleRowScanConverter.java
+++ b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/SingleRowScanConverter.java
@@ -19,7 +19,7 @@
 
 import com.google.zetasql.resolvedast.ResolvedNodes.ResolvedSingleRowScan;
 import java.util.List;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelNode;
 
 /** Converts a single row value. */
 class SingleRowScanConverter extends RelConverter<ResolvedSingleRowScan> {
diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/SqlCaseWithValueOperatorRewriter.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/SqlCaseWithValueOperatorRewriter.java
index c481750..a8afbe1 100644
--- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/SqlCaseWithValueOperatorRewriter.java
+++ b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/SqlCaseWithValueOperatorRewriter.java
@@ -19,10 +19,10 @@
 
 import java.util.ArrayList;
 import java.util.List;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexBuilder;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlOperator;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexBuilder;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlOperator;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Iterables;
diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/SqlCoalesceOperatorRewriter.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/SqlCoalesceOperatorRewriter.java
index e804433..0301bfc 100644
--- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/SqlCoalesceOperatorRewriter.java
+++ b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/SqlCoalesceOperatorRewriter.java
@@ -19,11 +19,11 @@
 
 import java.util.ArrayList;
 import java.util.List;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexBuilder;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlOperator;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.fun.SqlStdOperatorTable;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.util.Util;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexBuilder;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlOperator;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.util.Util;
 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/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/SqlIfNullOperatorRewriter.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/SqlIfNullOperatorRewriter.java
index 4c7db12..63292d4 100644
--- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/SqlIfNullOperatorRewriter.java
+++ b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/SqlIfNullOperatorRewriter.java
@@ -18,10 +18,10 @@
 package org.apache.beam.sdk.extensions.sql.zetasql.translation;
 
 import java.util.List;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexBuilder;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlOperator;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexBuilder;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlOperator;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.fun.SqlStdOperatorTable;
 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/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/SqlInOperatorRewriter.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/SqlInOperatorRewriter.java
new file mode 100644
index 0000000..4db7f43
--- /dev/null
+++ b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/SqlInOperatorRewriter.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.extensions.sql.zetasql.translation;
+
+import java.util.List;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexBuilder;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexLiteral;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexNode;
+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;
+
+/** Rewrites $in calls as SEARCH calls. */
+class SqlInOperatorRewriter implements SqlOperatorRewriter {
+  @Override
+  public RexNode apply(RexBuilder rexBuilder, List<RexNode> operands) {
+    Preconditions.checkArgument(
+        operands.size() >= 2, "IN should have at least two arguments in function call.");
+    final RexNode arg = operands.get(0);
+    final List<RexNode> ranges = ImmutableList.copyOf(operands.subList(1, operands.size()));
+
+    // ZetaSQL has weird behavior for NULL...
+    for (RexNode node : ranges) {
+      if (node instanceof RexLiteral && ((RexLiteral) node).isNull()) {
+        throw new UnsupportedOperationException("IN NULL unsupported");
+      }
+    }
+
+    return rexBuilder.makeIn(arg, ranges);
+  }
+}
diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/SqlNullIfOperatorRewriter.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/SqlNullIfOperatorRewriter.java
index d07bfcb..8cf62aa 100644
--- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/SqlNullIfOperatorRewriter.java
+++ b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/SqlNullIfOperatorRewriter.java
@@ -18,10 +18,10 @@
 package org.apache.beam.sdk.extensions.sql.zetasql.translation;
 
 import java.util.List;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexBuilder;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlOperator;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexBuilder;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlOperator;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.fun.SqlStdOperatorTable;
 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/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/SqlOperatorMappingTable.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/SqlOperatorMappingTable.java
index 6941033..c59df96 100644
--- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/SqlOperatorMappingTable.java
+++ b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/SqlOperatorMappingTable.java
@@ -18,8 +18,8 @@
 package org.apache.beam.sdk.extensions.sql.zetasql.translation;
 
 import java.util.Map;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlOperator;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlOperator;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
 
 /** SqlOperatorMappingTable. */
@@ -31,9 +31,9 @@
   static final Map<String, SqlOperator> ZETASQL_FUNCTION_TO_CALCITE_SQL_OPERATOR =
       ImmutableMap.<String, SqlOperator>builder()
           // grouped window function
-          .put("TUMBLE", SqlStdOperatorTable.TUMBLE)
-          .put("HOP", SqlStdOperatorTable.HOP)
-          .put("SESSION", SqlStdOperatorTable.SESSION)
+          .put("TUMBLE", SqlStdOperatorTable.TUMBLE_OLD)
+          .put("HOP", SqlStdOperatorTable.HOP_OLD)
+          .put("SESSION", SqlStdOperatorTable.SESSION_OLD)
 
           // ZetaSQL functions
           .put("$and", SqlStdOperatorTable.AND)
@@ -46,7 +46,6 @@
           .put("$less", SqlStdOperatorTable.LESS_THAN)
           .put("$less_or_equal", SqlStdOperatorTable.LESS_THAN_OR_EQUAL)
           .put("$like", SqlOperators.LIKE)
-          .put("$in", SqlStdOperatorTable.IN)
           .put("$is_null", SqlStdOperatorTable.IS_NULL)
           .put("$is_true", SqlStdOperatorTable.IS_TRUE)
           .put("$is_false", SqlStdOperatorTable.IS_FALSE)
@@ -105,5 +104,6 @@
           .put("coalesce", new SqlCoalesceOperatorRewriter())
           .put("ifnull", new SqlIfNullOperatorRewriter())
           .put("nullif", new SqlNullIfOperatorRewriter())
+          .put("$in", new SqlInOperatorRewriter())
           .build();
 }
diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/SqlOperatorRewriter.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/SqlOperatorRewriter.java
index ae31ffa..f64a334 100644
--- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/SqlOperatorRewriter.java
+++ b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/SqlOperatorRewriter.java
@@ -18,8 +18,8 @@
 package org.apache.beam.sdk.extensions.sql.zetasql.translation;
 
 import java.util.List;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexBuilder;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexBuilder;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexNode;
 
 /** Interface for rewriting calls a specific ZetaSQL operator. */
 interface SqlOperatorRewriter {
diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/SqlOperators.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/SqlOperators.java
index 9bf5a75..b44f63e 100644
--- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/SqlOperators.java
+++ b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/SqlOperators.java
@@ -33,32 +33,32 @@
 import org.apache.beam.sdk.extensions.sql.zetasql.DateTimeUtils;
 import org.apache.beam.sdk.extensions.sql.zetasql.translation.impl.BeamBuiltinMethods;
 import org.apache.beam.sdk.extensions.sql.zetasql.translation.impl.CastFunctionImpl;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.jdbc.JavaTypeFactoryImpl;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.type.RelDataType;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.type.RelDataTypeFactory;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.type.RelDataTypeFactoryImpl;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.schema.AggregateFunction;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.schema.Function;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.schema.FunctionParameter;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.schema.ScalarFunction;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlFunction;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlFunctionCategory;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlIdentifier;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlKind;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlOperator;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlSyntax;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.parser.SqlParserPos;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.type.FamilyOperandTypeChecker;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.type.InferTypes;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.type.OperandTypes;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.type.SqlReturnTypeInference;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.type.SqlTypeFactoryImpl;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.type.SqlTypeFamily;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.type.SqlTypeName;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.validate.SqlUserDefinedAggFunction;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.validate.SqlUserDefinedFunction;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.util.Optionality;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.util.Util;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.jdbc.JavaTypeFactoryImpl;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.type.RelDataType;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.type.RelDataTypeFactoryImpl;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.schema.AggregateFunction;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.schema.Function;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.schema.FunctionParameter;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.schema.ScalarFunction;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlFunction;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlFunctionCategory;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlIdentifier;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlKind;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlOperator;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlSyntax;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.type.FamilyOperandTypeChecker;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.type.InferTypes;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.type.OperandTypes;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.type.SqlReturnTypeInference;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.type.SqlTypeFactoryImpl;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.type.SqlTypeFamily;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.validate.SqlUserDefinedAggFunction;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.validate.SqlUserDefinedFunction;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.util.Optionality;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.util.Util;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.Lists;
 
@@ -173,7 +173,7 @@
   public static final SqlUserDefinedFunction CAST_OP =
       new SqlUserDefinedFunction(
           new SqlIdentifier("CAST", SqlParserPos.ZERO),
-          null,
+          SqlKind.OTHER_FUNCTION,
           null,
           null,
           null,
diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/SqlWindowTableFunction.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/SqlWindowTableFunction.java
index 423c412..d693080 100644
--- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/SqlWindowTableFunction.java
+++ b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/SqlWindowTableFunction.java
@@ -20,20 +20,20 @@
 import java.util.ArrayList;
 import java.util.List;
 import org.apache.beam.sdk.extensions.sql.impl.utils.TVFStreamingUtils;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.type.RelDataType;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.type.RelDataTypeField;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.type.RelDataTypeFieldImpl;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.type.RelRecordType;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlCallBinding;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlFunction;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlFunctionCategory;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlKind;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlOperandCountRange;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.type.SqlOperandCountRanges;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.type.SqlReturnTypeInference;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.type.SqlTypeName;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.type.RelDataType;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.type.RelDataTypeFieldImpl;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.type.RelRecordType;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlCallBinding;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlFunction;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlFunctionCategory;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlKind;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlOperandCountRange;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.type.SqlOperandCountRanges;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.type.SqlReturnTypeInference;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.validate.SqlValidator;
 
 /** Base class for table-valued function windowing operator (TUMBLE, HOP and SESSION). */
 @SuppressWarnings({
diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/TVFScanConverter.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/TVFScanConverter.java
index d971ce0..29f9603 100644
--- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/TVFScanConverter.java
+++ b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/TVFScanConverter.java
@@ -28,9 +28,9 @@
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.logical.LogicalTableFunctionScan;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexCall;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.logical.LogicalTableFunctionScan;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexCall;
 
 /** Converts TVFScan. */
 @SuppressWarnings({
diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/TableScanConverter.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/TableScanConverter.java
index 9137b94..dc0568c 100644
--- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/TableScanConverter.java
+++ b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/TableScanConverter.java
@@ -17,25 +17,26 @@
  */
 package org.apache.beam.sdk.extensions.sql.zetasql.translation;
 
-import static org.apache.beam.vendor.calcite.v1_20_0.com.google.common.base.Preconditions.checkNotNull;
+import static org.apache.beam.vendor.calcite.v1_26_0.com.google.common.base.Preconditions.checkNotNull;
 
 import com.google.zetasql.resolvedast.ResolvedNodes.ResolvedTableScan;
 import java.util.List;
 import java.util.Properties;
 import org.apache.beam.sdk.extensions.sql.zetasql.TableResolution;
-import org.apache.beam.vendor.calcite.v1_20_0.com.google.common.collect.ImmutableList;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.config.CalciteConnectionConfigImpl;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.jdbc.CalciteSchema;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptCluster;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptTable;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.prepare.CalciteCatalogReader;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.prepare.RelOptTableImpl;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelRoot;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.type.RelDataType;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.schema.SchemaPlus;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.schema.Table;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.schema.TranslatableTable;
+import org.apache.beam.vendor.calcite.v1_26_0.com.google.common.collect.ImmutableList;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.config.CalciteConnectionConfigImpl;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.jdbc.CalciteSchema;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptCluster;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptTable;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.prepare.CalciteCatalogReader;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.prepare.RelOptTableImpl;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelRoot;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.hint.RelHint;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.type.RelDataType;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.schema.SchemaPlus;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.schema.Table;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.schema.TranslatableTable;
 
 /** Converts table scan. */
 class TableScanConverter extends RelConverter<ResolvedTableScan> {
@@ -105,6 +106,11 @@
       public RelOptCluster getCluster() {
         return TableScanConverter.this.getCluster();
       }
+
+      @Override
+      public List<RelHint> getTableHints() {
+        return ImmutableList.of();
+      }
     };
   }
 }
diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/WithRefScanConverter.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/WithRefScanConverter.java
index d1ed3cf..6667732 100644
--- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/WithRefScanConverter.java
+++ b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/WithRefScanConverter.java
@@ -21,7 +21,7 @@
 import com.google.zetasql.resolvedast.ResolvedNodes.ResolvedWithRefScan;
 import java.util.Collections;
 import java.util.List;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelNode;
 
 /** Converts a call-site reference to a named WITH subquery. */
 @SuppressWarnings({
diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/WithScanConverter.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/WithScanConverter.java
index 7159356..b88674e 100644
--- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/WithScanConverter.java
+++ b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/WithScanConverter.java
@@ -21,7 +21,7 @@
 import com.google.zetasql.resolvedast.ResolvedNodes.ResolvedWithScan;
 import java.util.Collections;
 import java.util.List;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelNode;
 
 /** Converts a named WITH. */
 class WithScanConverter extends RelConverter<ResolvedWithScan> {
diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/ZetaSqlScalarFunctionImpl.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/ZetaSqlScalarFunctionImpl.java
index 2508ac3..9b3f8ef 100644
--- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/ZetaSqlScalarFunctionImpl.java
+++ b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/ZetaSqlScalarFunctionImpl.java
@@ -19,9 +19,9 @@
 
 import java.lang.reflect.Method;
 import org.apache.beam.sdk.extensions.sql.impl.ScalarFunctionImpl;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.adapter.enumerable.CallImplementor;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.schema.Function;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.schema.ScalarFunction;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.adapter.enumerable.CallImplementor;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.schema.Function;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.schema.ScalarFunction;
 
 /** ZetaSQL-specific extension to {@link ScalarFunctionImpl}. */
 public class ZetaSqlScalarFunctionImpl extends ScalarFunctionImpl {
@@ -38,7 +38,7 @@
   }
 
   /**
-   * Creates {@link org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.schema.Function} from
+   * Creates {@link org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.schema.Function} from
    * given class.
    *
    * <p>If a method of the given name is not found or it does not suit, returns {@code null}.
@@ -55,7 +55,7 @@
   }
 
   /**
-   * Creates {@link org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.schema.Function} from
+   * Creates {@link org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.schema.Function} from
    * given method. When {@code eval} method does not suit, {@code null} is returned.
    *
    * @param method method that is used to implement the function
diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/impl/BeamBuiltinMethods.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/impl/BeamBuiltinMethods.java
index 223469b..cbb2e2c 100644
--- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/impl/BeamBuiltinMethods.java
+++ b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/impl/BeamBuiltinMethods.java
@@ -19,7 +19,7 @@
 
 import java.lang.reflect.Method;
 import org.apache.beam.sdk.annotations.Internal;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.linq4j.tree.Types;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.linq4j.tree.Types;
 
 /** BeamBuiltinMethods. */
 @Internal
diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/impl/CastFunctionImpl.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/impl/CastFunctionImpl.java
index c15bdab..c68b771 100644
--- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/impl/CastFunctionImpl.java
+++ b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/impl/CastFunctionImpl.java
@@ -17,22 +17,22 @@
  */
 package org.apache.beam.sdk.extensions.sql.zetasql.translation.impl;
 
-import static org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.adapter.enumerable.RexImpTable.createImplementor;
+import static org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.adapter.enumerable.RexImpTable.createImplementor;
 
 import java.util.Collections;
 import java.util.List;
 import org.apache.beam.sdk.annotations.Internal;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.adapter.enumerable.CallImplementor;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.adapter.enumerable.NotNullImplementor;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.adapter.enumerable.NullPolicy;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.adapter.enumerable.RexImpTable;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.adapter.enumerable.RexToLixTranslator;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.linq4j.tree.Expression;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.linq4j.tree.Expressions;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexCall;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.schema.FunctionParameter;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.schema.ImplementableFunction;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.adapter.enumerable.CallImplementor;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.adapter.enumerable.NotNullImplementor;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.adapter.enumerable.NullPolicy;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.adapter.enumerable.RexImpTable;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.adapter.enumerable.RexToLixTranslator;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.linq4j.tree.Expression;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.linq4j.tree.Expressions;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexCall;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.schema.FunctionParameter;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.schema.ImplementableFunction;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.type.SqlTypeName;
 
 /** ZetaSQLCastFunctionImpl. */
 @Internal
diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/impl/StringFunctions.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/impl/StringFunctions.java
index 213b577..451d8a6 100644
--- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/impl/StringFunctions.java
+++ b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/impl/StringFunctions.java
@@ -19,8 +19,8 @@
 
 import java.util.regex.Pattern;
 import org.apache.beam.sdk.annotations.Internal;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.linq4j.function.Strict;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.runtime.SqlFunctions;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.linq4j.function.Strict;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.runtime.SqlFunctions;
 
 /** StringFunctions. */
 @Internal
diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/impl/TimestampFunctions.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/impl/TimestampFunctions.java
index 721e226..1718976 100644
--- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/impl/TimestampFunctions.java
+++ b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/translation/impl/TimestampFunctions.java
@@ -20,7 +20,7 @@
 import java.util.TimeZone;
 import org.apache.beam.sdk.annotations.Internal;
 import org.apache.beam.sdk.extensions.sql.zetasql.DateTimeUtils;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.linq4j.function.Strict;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.linq4j.function.Strict;
 import org.joda.time.DateTime;
 import org.joda.time.DateTimeZone;
 
diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/unnest/BeamZetaSqlUncollectRel.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/unnest/BeamZetaSqlUncollectRel.java
index fce4c3d..f252e1a 100644
--- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/unnest/BeamZetaSqlUncollectRel.java
+++ b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/unnest/BeamZetaSqlUncollectRel.java
@@ -17,7 +17,7 @@
  */
 package org.apache.beam.sdk.extensions.sql.zetasql.unnest;
 
-import static org.apache.beam.vendor.calcite.v1_20_0.com.google.common.base.Preconditions.checkArgument;
+import static org.apache.beam.vendor.calcite.v1_26_0.com.google.common.base.Preconditions.checkArgument;
 
 import org.apache.beam.sdk.extensions.sql.impl.planner.BeamCostModel;
 import org.apache.beam.sdk.extensions.sql.impl.planner.NodeStats;
@@ -31,11 +31,11 @@
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PCollectionList;
 import org.apache.beam.sdk.values.Row;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptCluster;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptPlanner;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelTraitSet;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptCluster;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptPlanner;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelTraitSet;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.metadata.RelMetadataQuery;
 
 /**
  * {@link BeamRelNode} to implement an uncorrelated {@link ZetaSqlUnnest}, aka UNNEST.
diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/unnest/BeamZetaSqlUncollectRule.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/unnest/BeamZetaSqlUncollectRule.java
index 7644952..e985f74 100644
--- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/unnest/BeamZetaSqlUncollectRule.java
+++ b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/unnest/BeamZetaSqlUncollectRule.java
@@ -18,9 +18,9 @@
 package org.apache.beam.sdk.extensions.sql.zetasql.unnest;
 
 import org.apache.beam.sdk.extensions.sql.impl.rel.BeamLogicalConvention;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.Convention;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.Convention;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.convert.ConverterRule;
 
 /**
  * A {@code ConverterRule} to replace {@link ZetaSqlUnnest} with {@link BeamZetaSqlUncollectRel}.
diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/unnest/BeamZetaSqlUnnestRel.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/unnest/BeamZetaSqlUnnestRel.java
index 4fc6088..ae44c44 100644
--- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/unnest/BeamZetaSqlUnnestRel.java
+++ b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/unnest/BeamZetaSqlUnnestRel.java
@@ -31,17 +31,17 @@
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.PCollectionList;
 import org.apache.beam.sdk.values.Row;
-import org.apache.beam.vendor.calcite.v1_20_0.com.google.common.collect.ImmutableList;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptCluster;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptPlanner;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelTraitSet;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelWriter;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.Correlate;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.JoinRelType;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.metadata.RelMetadataQuery;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.type.RelDataType;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.validate.SqlValidatorUtil;
+import org.apache.beam.vendor.calcite.v1_26_0.com.google.common.collect.ImmutableList;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptCluster;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptPlanner;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelTraitSet;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelWriter;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.core.Correlate;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.core.JoinRelType;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.metadata.RelMetadataQuery;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.type.RelDataType;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.validate.SqlValidatorUtil;
 import org.checkerframework.checker.nullness.qual.Nullable;
 
 /**
diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/unnest/BeamZetaSqlUnnestRule.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/unnest/BeamZetaSqlUnnestRule.java
index fec61fe..1a7d80e 100644
--- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/unnest/BeamZetaSqlUnnestRule.java
+++ b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/unnest/BeamZetaSqlUnnestRule.java
@@ -18,17 +18,17 @@
 package org.apache.beam.sdk.extensions.sql.zetasql.unnest;
 
 import org.apache.beam.sdk.extensions.sql.impl.rel.BeamLogicalConvention;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptRule;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptRuleCall;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.volcano.RelSubset;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.SingleRel;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.Correlate;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.core.JoinRelType;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.logical.LogicalCorrelate;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.logical.LogicalProject;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexFieldAccess;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rex.RexNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptRule;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.volcano.RelSubset;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.SingleRel;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.core.Correlate;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.core.JoinRelType;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.logical.LogicalCorrelate;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.logical.LogicalProject;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexFieldAccess;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rex.RexNode;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
 
 /**
diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/unnest/ZetaSqlUnnest.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/unnest/ZetaSqlUnnest.java
index e2d6fad..871a9f8 100644
--- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/unnest/ZetaSqlUnnest.java
+++ b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/unnest/ZetaSqlUnnest.java
@@ -18,20 +18,20 @@
 package org.apache.beam.sdk.extensions.sql.zetasql.unnest;
 
 import java.util.List;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.Convention;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptCluster;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelTraitSet;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelInput;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelNode;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.RelWriter;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.SingleRel;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.type.RelDataType;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.type.RelDataTypeFactory;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.rel.type.RelDataTypeField;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlUnnestOperator;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.SqlUtil;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.type.MapSqlType;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.Convention;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptCluster;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelTraitSet;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelInput;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.RelWriter;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.SingleRel;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.type.RelDataType;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlUnnestOperator;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlUtil;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.type.MapSqlType;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.type.SqlTypeName;
 
 /**
  * This class is a copy of Uncollect.java in Calcite:
diff --git a/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/BeamJavaUdfCalcRuleTest.java b/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/BeamJavaUdfCalcRuleTest.java
index dbbc089..21436b5 100644
--- a/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/BeamJavaUdfCalcRuleTest.java
+++ b/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/BeamJavaUdfCalcRuleTest.java
@@ -27,9 +27,9 @@
 import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.Row;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelOptPlanner;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.tools.Frameworks;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.tools.RuleSet;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelOptPlanner;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.tools.Frameworks;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.tools.RuleSet;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
 import org.joda.time.Duration;
 import org.junit.Before;
diff --git a/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/BeamZetaSqlCatalogTest.java b/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/BeamZetaSqlCatalogTest.java
index 94c984c..6789848 100644
--- a/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/BeamZetaSqlCatalogTest.java
+++ b/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/BeamZetaSqlCatalogTest.java
@@ -34,7 +34,7 @@
 import org.apache.beam.sdk.extensions.sql.meta.provider.ReadOnlyTableProvider;
 import org.apache.beam.sdk.extensions.sql.zetasql.translation.UserFunctionDefinitions;
 import org.apache.beam.sdk.options.PipelineOptionsFactory;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.schema.SchemaPlus;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.schema.SchemaPlus;
 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.junit.Rule;
diff --git a/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/TableResolutionTest.java b/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/TableResolutionTest.java
index ca20bda..2fbb6cf 100644
--- a/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/TableResolutionTest.java
+++ b/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/TableResolutionTest.java
@@ -20,8 +20,8 @@
 import static org.hamcrest.MatcherAssert.assertThat;
 import static org.mockito.Mockito.when;
 
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.schema.SchemaPlus;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.schema.Table;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.schema.SchemaPlus;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.schema.Table;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
 import org.hamcrest.Matchers;
 import org.junit.Assert;
diff --git a/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSQLPushDownTest.java b/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSQLPushDownTest.java
index 5bdf788..13b2aad 100644
--- a/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSQLPushDownTest.java
+++ b/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSQLPushDownTest.java
@@ -36,14 +36,14 @@
 import org.apache.beam.sdk.schemas.Schema;
 import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.values.Row;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.Context;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.Contexts;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.ConventionTraitDef;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.RelTraitDef;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.schema.SchemaPlus;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.tools.FrameworkConfig;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.tools.Frameworks;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.tools.RuleSet;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.Context;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.Contexts;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.ConventionTraitDef;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.RelTraitDef;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.schema.SchemaPlus;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.tools.FrameworkConfig;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.tools.Frameworks;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.tools.RuleSet;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
 import org.joda.time.Duration;
 import org.junit.BeforeClass;
diff --git a/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlJavaUdfTest.java b/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlJavaUdfTest.java
index ee4e9ef..83ec309 100644
--- a/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlJavaUdfTest.java
+++ b/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlJavaUdfTest.java
@@ -43,7 +43,7 @@
 import org.apache.beam.sdk.transforms.Sum;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.Row;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.tools.Frameworks;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.tools.Frameworks;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
 import org.checkerframework.checker.nullness.qual.Nullable;
 import org.codehaus.commons.compiler.CompileException;
@@ -248,13 +248,9 @@
     pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
   }
 
-  /**
-   * This is a loophole in type checking. The SQL function signature does not need to match the Java
-   * function signature; only the generated code is typechecked.
-   */
-  // TODO(BEAM-11171): fix this and adjust test accordingly.
   @Test
-  public void testNullArgumentIsNotTypeChecked() {
+  public void testNullArgumentIsTypeChecked() {
+    // The Java definition for isNull takes a String, but here we declare it in SQL with INT64.
     String sql =
         String.format(
             "CREATE FUNCTION isNull(i INT64) RETURNS INT64 LANGUAGE java "
@@ -263,12 +259,17 @@
             jarPath);
     ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
     BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql);
-    PCollection<Row> stream = BeamSqlRelUtils.toPCollection(pipeline, beamRelNode);
-
-    Schema singleField = Schema.builder().addBooleanField("field1").build();
-
-    PAssert.that(stream).containsInAnyOrder(Row.withSchema(singleField).addValues(true).build());
-    pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
+    // TODO(BEAM-11171) This should fail earlier, before compiling the CalcFn.
+    thrown.expect(UnsupportedOperationException.class);
+    thrown.expectMessage("Could not compile CalcFn");
+    thrown.expectCause(
+        allOf(
+            isA(CompileException.class),
+            hasProperty(
+                "message",
+                containsString(
+                    "No applicable constructor/method found for actual parameters \"java.lang.Long\""))));
+    BeamSqlRelUtils.toPCollection(pipeline, beamRelNode);
   }
 
   @Test
diff --git a/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlJavaUdfTypeTest.java b/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlJavaUdfTypeTest.java
index b8d7609..fa0d40a 100644
--- a/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlJavaUdfTypeTest.java
+++ b/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlJavaUdfTypeTest.java
@@ -37,8 +37,8 @@
 import org.apache.beam.sdk.testing.TestPipeline;
 import org.apache.beam.sdk.values.PCollection;
 import org.apache.beam.sdk.values.Row;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.schema.SchemaPlus;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.tools.Frameworks;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.schema.SchemaPlus;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.tools.Frameworks;
 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.joda.time.DateTime;
diff --git a/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlTestBase.java b/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlTestBase.java
index 4c43b18..23e8e88 100644
--- a/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlTestBase.java
+++ b/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlTestBase.java
@@ -26,11 +26,11 @@
 import org.apache.beam.sdk.extensions.sql.meta.provider.ReadOnlyTableProvider;
 import org.apache.beam.sdk.extensions.sql.meta.provider.TableProvider;
 import org.apache.beam.sdk.options.PipelineOptionsFactory;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.Contexts;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.plan.ConventionTraitDef;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.tools.FrameworkConfig;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.tools.Frameworks;
-import org.apache.beam.vendor.calcite.v1_20_0.org.apache.calcite.tools.RuleSet;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.Contexts;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.plan.ConventionTraitDef;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.tools.FrameworkConfig;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.tools.Frameworks;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.tools.RuleSet;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
 
 /** Common setup for ZetaSQL tests. */
diff --git a/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlTimeFunctionsTest.java b/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlTimeFunctionsTest.java
index ca37715..3e3ec8d 100644
--- a/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlTimeFunctionsTest.java
+++ b/sdks/java/extensions/sql/zetasql/src/test/java/org/apache/beam/sdk/extensions/sql/zetasql/ZetaSqlTimeFunctionsTest.java
@@ -81,7 +81,9 @@
 
   @Test
   public void testDateColumn() {
-    String sql = "SELECT FORMAT_DATE('%b-%d-%Y', date_field) FROM table_with_date";
+    // NOTE: Do not use textual format parameters (%b or %h: The abbreviated month name) as these
+    // are locale dependent.
+    String sql = "SELECT FORMAT_DATE('%m-%d-%Y', date_field) FROM table_with_date";
 
     ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
     BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql);
@@ -90,10 +92,10 @@
     PAssert.that(stream)
         .containsInAnyOrder(
             Row.withSchema(Schema.builder().addStringField("f_date_str").build())
-                .addValues("Dec-25-2008")
+                .addValues("12-25-2008")
                 .build(),
             Row.withSchema(Schema.builder().addStringField("f_date_str").build())
-                .addValues("Apr-07-2020")
+                .addValues("04-07-2020")
                 .build());
     pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
   }
@@ -390,7 +392,9 @@
 
   @Test
   public void testFormatDate() {
-    String sql = "SELECT FORMAT_DATE('%b-%d-%Y', DATE '2008-12-25')";
+    // NOTE: Do not use textual format parameters (%b or %h: The abbreviated month name) as these
+    // are locale dependent.
+    String sql = "SELECT FORMAT_DATE('%m-%d-%Y', DATE '2008-12-25')";
 
     ZetaSQLQueryPlanner zetaSQLQueryPlanner = new ZetaSQLQueryPlanner(config);
     BeamRelNode beamRelNode = zetaSQLQueryPlanner.convertToBeamRel(sql);
@@ -399,7 +403,7 @@
     PAssert.that(stream)
         .containsInAnyOrder(
             Row.withSchema(Schema.builder().addStringField("f_date_str").build())
-                .addValues("Dec-25-2008")
+                .addValues("12-25-2008")
                 .build());
     pipeline.run().waitUntilFinish(Duration.standardMinutes(PIPELINE_EXECUTION_WAITTIME_MINUTES));
   }
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 e105c2b..511f839 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
@@ -22,19 +22,16 @@
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
-import java.io.PushbackInputStream;
 import java.util.Iterator;
 import java.util.NoSuchElementException;
 import java.util.concurrent.BlockingQueue;
 import org.apache.beam.sdk.coders.Coder;
 import org.apache.beam.vendor.grpc.v1p36p0.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;
 
 /**
- * {@link #inbound(Iterator)} treats multiple {@link ByteString}s as a single input stream and
- * {@link #outbound(OutputChunkConsumer)} treats a single {@link OutputStream} as multiple {@link
- * ByteString}s.
+ * {@link DataStreamDecoder} treats multiple {@link ByteString}s as a single input stream decoding
+ * values with the supplied iterator. {@link #outbound(OutputChunkConsumer)} treats a single {@link
+ * OutputStream} as multiple {@link ByteString}s.
  */
 @SuppressWarnings({
   "rawtypes", // TODO(https://issues.apache.org/jira/browse/BEAM-10556)
@@ -44,19 +41,6 @@
   public static final int DEFAULT_OUTBOUND_BUFFER_LIMIT_BYTES = 1_000_000;
 
   /**
-   * Converts multiple {@link ByteString}s into a single {@link InputStream}.
-   *
-   * <p>The iterator is accessed lazily. The supplied {@link Iterator} should block until either it
-   * knows that no more values will be provided or it has the next {@link ByteString}.
-   *
-   * <p>Note that this {@link InputStream} follows the Beam Fn API specification for forcing values
-   * that decode consuming zero bytes to consuming exactly one byte.
-   */
-  public static InputStream inbound(Iterator<ByteString> bytes) {
-    return new Inbound(bytes);
-  }
-
-  /**
    * Converts a single element delimited {@link OutputStream} into multiple {@link ByteString
    * ByteStrings}.
    *
@@ -172,56 +156,6 @@
   }
 
   /**
-   * An input stream which concatenates multiple {@link ByteString}s. Lazily accesses the first
-   * {@link Iterator} on first access of this input stream.
-   *
-   * <p>Closing this input stream has no effect.
-   */
-  private static class Inbound<T> extends InputStream {
-    private static final InputStream EMPTY_STREAM =
-        new InputStream() {
-          @Override
-          public int read() throws IOException {
-            return -1;
-          }
-        };
-
-    private final Iterator<ByteString> bytes;
-    private InputStream currentStream;
-
-    public Inbound(Iterator<ByteString> bytes) {
-      this.currentStream = EMPTY_STREAM;
-      this.bytes = bytes;
-    }
-
-    @Override
-    public int read() throws IOException {
-      int rval = -1;
-      // Move on to the next stream if we have read nothing
-      while ((rval = currentStream.read()) == -1 && bytes.hasNext()) {
-        currentStream = bytes.next().newInput();
-      }
-      return rval;
-    }
-
-    @Override
-    public int read(byte[] b, int off, int len) throws IOException {
-      int remainingLen = len;
-      while ((remainingLen -=
-              ByteStreams.read(currentStream, b, off + len - remainingLen, remainingLen))
-          > 0) {
-        if (bytes.hasNext()) {
-          currentStream = bytes.next().newInput();
-        } else {
-          int bytesRead = len - remainingLen;
-          return bytesRead > 0 ? bytesRead : -1;
-        }
-      }
-      return len - remainingLen;
-    }
-  }
-
-  /**
    * An adapter which converts an {@link InputStream} to an {@link Iterator} of {@code T} values
    * using the specified {@link Coder}.
    *
@@ -239,17 +173,17 @@
       EOF
     }
 
-    private final CountingInputStream countingInputStream;
-    private final PushbackInputStream pushbackInputStream;
+    private final Iterator<ByteString> inputByteStrings;
+    private final Inbound inbound;
     private final Coder<T> coder;
     private State currentState;
     private T next;
 
-    public DataStreamDecoder(Coder<T> coder, InputStream inputStream) {
+    public DataStreamDecoder(Coder<T> coder, Iterator<ByteString> inputStream) {
       this.currentState = State.READ_REQUIRED;
       this.coder = coder;
-      this.pushbackInputStream = new PushbackInputStream(inputStream, 1);
-      this.countingInputStream = new CountingInputStream(pushbackInputStream);
+      this.inputByteStrings = inputStream;
+      this.inbound = new Inbound();
     }
 
     @Override
@@ -259,18 +193,16 @@
           return false;
         case READ_REQUIRED:
           try {
-            int nextByte = pushbackInputStream.read();
-            if (nextByte == -1) {
+            if (inbound.isEof()) {
               currentState = State.EOF;
               return false;
             }
 
-            pushbackInputStream.unread(nextByte);
-            long count = countingInputStream.getCount();
-            next = coder.decode(countingInputStream);
+            long previousPosition = inbound.position;
+            next = coder.decode(inbound);
             // Skip one byte if decoding the value consumed 0 bytes.
-            if (countingInputStream.getCount() - count == 0) {
-              checkState(countingInputStream.read() != -1, "Unexpected EOF reached");
+            if (inbound.position - previousPosition == 0) {
+              checkState(inbound.read() != -1, "Unexpected EOF reached");
             }
             currentState = State.HAS_NEXT;
           } catch (IOException e) {
@@ -296,6 +228,72 @@
     public void remove() {
       throw new UnsupportedOperationException();
     }
+
+    private static final InputStream EMPTY_STREAM = ByteString.EMPTY.newInput();
+
+    /**
+     * An input stream which concatenates multiple {@link ByteString}s. Lazily accesses the first
+     * {@link Iterator} on first access of this input stream.
+     *
+     * <p>Closing this input stream has no effect.
+     */
+    private class Inbound extends InputStream {
+      private long position;
+      private InputStream currentStream;
+
+      public Inbound() {
+        this.currentStream = EMPTY_STREAM;
+      }
+
+      public boolean isEof() throws IOException {
+        // Note that ByteString#newInput is guaranteed to return the length of the entire ByteString
+        // minus the number of bytes that have been read so far and can be reliably used to tell
+        // us whether we are at the end of the stream.
+        while (currentStream.available() == 0) {
+          if (!inputByteStrings.hasNext()) {
+            return true;
+          }
+          currentStream = inputByteStrings.next().newInput();
+        }
+        return false;
+      }
+
+      @Override
+      public int read() throws IOException {
+        int read;
+        // Move on to the next stream if this stream is done
+        while ((read = currentStream.read()) == -1) {
+          if (!inputByteStrings.hasNext()) {
+            return -1;
+          }
+          currentStream = inputByteStrings.next().newInput();
+        }
+        position += 1;
+        return read;
+      }
+
+      @Override
+      public int read(byte[] b, int off, int len) throws IOException {
+        int remainingLen = len;
+        while (remainingLen > 0) {
+          int read;
+          // Move on to the next stream if this stream is done. Note that ByteString.newInput
+          // guarantees that read will consume the entire ByteString if the passed in length is
+          // greater than or equal to the remaining amount.
+          while ((read = currentStream.read(b, off + len - remainingLen, remainingLen)) == -1) {
+            if (!inputByteStrings.hasNext()) {
+              int bytesRead = len - remainingLen;
+              position += bytesRead;
+              return bytesRead > 0 ? bytesRead : -1;
+            }
+            currentStream = inputByteStrings.next().newInput();
+          }
+          remainingLen -= read;
+        }
+        position += len;
+        return len;
+      }
+    }
   }
 
   /**
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 dd79c90..9dd5ee4 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
@@ -25,8 +25,6 @@
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assume.assumeTrue;
 
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -48,38 +46,15 @@
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.util.concurrent.SettableFuture;
 import org.junit.Rule;
 import org.junit.Test;
+import org.junit.experimental.runners.Enclosed;
 import org.junit.rules.ExpectedException;
 import org.junit.runner.RunWith;
 import org.junit.runners.JUnit4;
 
 /** Tests for {@link DataStreams}. */
+@RunWith(Enclosed.class)
 public class DataStreamsTest {
 
-  /** Tests for {@link DataStreams.Inbound}. */
-  @RunWith(JUnit4.class)
-  public static class InboundTest {
-    private static final ByteString BYTES_A = ByteString.copyFromUtf8("TestData");
-    private static final ByteString BYTES_B = ByteString.copyFromUtf8("SomeOtherTestData");
-
-    @Test
-    public void testEmptyRead() throws Exception {
-      assertEquals(ByteString.EMPTY, read());
-      assertEquals(ByteString.EMPTY, read(ByteString.EMPTY));
-      assertEquals(ByteString.EMPTY, read(ByteString.EMPTY, ByteString.EMPTY));
-    }
-
-    @Test
-    public void testRead() throws Exception {
-      assertEquals(BYTES_A.concat(BYTES_B), read(BYTES_A, BYTES_B));
-      assertEquals(BYTES_A.concat(BYTES_B), read(BYTES_A, ByteString.EMPTY, BYTES_B));
-      assertEquals(BYTES_A.concat(BYTES_B), read(BYTES_A, BYTES_B, ByteString.EMPTY));
-    }
-
-    private static ByteString read(ByteString... bytes) throws IOException {
-      return ByteString.readFrom(DataStreams.inbound(Arrays.asList(bytes).iterator()));
-    }
-  }
-
   /** Tests for {@link DataStreams.BlockingQueueIterator}. */
   @RunWith(JUnit4.class)
   public static class BlockingQueueIteratorTest {
@@ -141,22 +116,27 @@
     }
 
     private <T> void testDecoderWith(Coder<T> coder, T... expected) throws IOException {
-      ByteArrayOutputStream baos = new ByteArrayOutputStream();
+      ByteString.Output output = ByteString.newOutput();
       for (T value : expected) {
-        int size = baos.size();
-        coder.encode(value, baos);
+        int size = output.size();
+        coder.encode(value, output);
         // Pad an arbitrary byte when values encode to zero bytes
-        if (baos.size() - size == 0) {
-          baos.write(0);
+        if (output.size() - size == 0) {
+          output.write(0);
         }
       }
+      testDecoderWith(coder, expected, Arrays.asList(output.toByteString()));
+      testDecoderWith(coder, expected, Arrays.asList(ByteString.EMPTY, output.toByteString()));
+      testDecoderWith(coder, expected, Arrays.asList(output.toByteString(), ByteString.EMPTY));
+    }
 
-      Iterator<T> decoder =
-          new DataStreamDecoder<>(coder, new ByteArrayInputStream(baos.toByteArray()));
+    private <T> void testDecoderWith(Coder<T> coder, T[] expected, List<ByteString> encoded) {
+      Iterator<T> decoder = new DataStreamDecoder<>(coder, encoded.iterator());
 
       Object[] actual = Iterators.toArray(decoder, Object.class);
       assertArrayEquals(expected, actual);
 
+      // Ensure that we are consistent on hasNext at end of stream
       assertFalse(decoder.hasNext());
       assertFalse(decoder.hasNext());
 
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 4faf989..2f2789e 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
@@ -264,7 +264,7 @@
 
                   @Override
                   public ValueState<T> readLater() {
-                    // TODO: Support prefetching.
+                    // TODO(BEAM-12802): Support prefetching.
                     return this;
                   }
                 };
@@ -310,7 +310,7 @@
 
                   @Override
                   public BagState<T> readLater() {
-                    // TODO: Support prefetching.
+                    // TODO(BEAM-12802): Support prefetching.
                     return this;
                   }
 
diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/StateBackedIterable.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/StateBackedIterable.java
index 6ee97d2..3bca90d 100644
--- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/StateBackedIterable.java
+++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/StateBackedIterable.java
@@ -90,9 +90,7 @@
     return Iterators.concat(
         prefix.iterator(),
         new DataStreams.DataStreamDecoder(
-            elemCoder,
-            DataStreams.inbound(
-                StateFetchingIterators.readAllStartingFrom(beamFnStateClient, request))));
+            elemCoder, StateFetchingIterators.readAllStartingFrom(beamFnStateClient, request)));
   }
 
   protected Object writeReplace() throws ObjectStreamException {
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 79a4297..22be306 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
@@ -83,12 +83,8 @@
     return Iterables.concat(
         new LazyCachingIteratorToIterable<T>(
             new DataStreams.DataStreamDecoder<>(
-                valueCoder,
-                DataStreams.inbound(
-                    new LazySingletonIterator<>(firstPageAndRemainder::firstPage)))),
-        () ->
-            new DataStreams.DataStreamDecoder<>(
-                valueCoder, DataStreams.inbound(firstPageAndRemainder.remainder())));
+                valueCoder, new LazySingletonIterator<>(firstPageAndRemainder::firstPage))),
+        () -> new DataStreams.DataStreamDecoder<>(valueCoder, firstPageAndRemainder.remainder()));
   }
 
   /** A iterable that contains a single element, provided by a Supplier which is invoked lazily. */
diff --git a/sdks/java/io/azure/build.gradle b/sdks/java/io/azure/build.gradle
index 18596e4..8f30a48 100644
--- a/sdks/java/io/azure/build.gradle
+++ b/sdks/java/io/azure/build.gradle
@@ -34,11 +34,11 @@
   compile library.java.slf4j_api
   compile library.java.vendored_guava_26_0_jre
   compile project(path: ":sdks:java:core", configuration: "shadow")
-  compile "com.azure:azure-storage-blob:12.8.0"
+  compile "com.azure:azure-storage-blob:12.10.0"
   compile "com.azure:azure-identity:1.0.8"
   compile "com.microsoft.azure:azure-storage:8.6.5"
-  compile "com.azure:azure-core:1.6.0"
-  compile "com.azure:azure-storage-common:12.8.0"
+  compile "com.azure:azure-core:1.9.0"
+  compile "com.azure:azure-storage-common:12.10.0"
   compile library.java.jackson_annotations
   compile library.java.jackson_core
   compile library.java.jackson_databind
diff --git a/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/options/AzureModule.java b/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/options/AzureModule.java
index bb452d7..ea538cf 100644
--- a/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/options/AzureModule.java
+++ b/sdks/java/io/azure/src/main/java/org/apache/beam/sdk/io/azure/options/AzureModule.java
@@ -30,10 +30,13 @@
 import com.azure.identity.ManagedIdentityCredentialBuilder;
 import com.azure.identity.UsernamePasswordCredential;
 import com.azure.identity.UsernamePasswordCredentialBuilder;
+import com.azure.identity.implementation.IdentityClient;
 import com.fasterxml.jackson.annotation.JsonTypeInfo;
 import com.fasterxml.jackson.core.JsonGenerator;
 import com.fasterxml.jackson.core.JsonParser;
+import com.fasterxml.jackson.core.JsonToken;
 import com.fasterxml.jackson.core.type.TypeReference;
+import com.fasterxml.jackson.core.type.WritableTypeId;
 import com.fasterxml.jackson.databind.DeserializationContext;
 import com.fasterxml.jackson.databind.JsonDeserializer;
 import com.fasterxml.jackson.databind.JsonSerializer;
@@ -46,6 +49,7 @@
 import com.fasterxml.jackson.databind.module.SimpleModule;
 import com.google.auto.service.AutoService;
 import java.io.IOException;
+import java.lang.reflect.Field;
 import java.util.Map;
 import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableSet;
 
@@ -56,6 +60,7 @@
 @AutoService(Module.class)
 public class AzureModule extends SimpleModule {
 
+  private static final String TYPE_PROPERTY = "@type";
   private static final String AZURE_CLIENT_ID = "azureClientId";
   private static final String AZURE_TENANT_ID = "azureTenantId";
   private static final String AZURE_CLIENT_SECRET = "azureClientSecret";
@@ -139,6 +144,7 @@
             .clientId(asMap.getOrDefault(AZURE_CLIENT_ID, ""))
             .username(asMap.getOrDefault(AZURE_USERNAME, ""))
             .password(asMap.getOrDefault(AZURE_PASSWORD, ""))
+            .tenantId(asMap.getOrDefault(AZURE_TENANT_ID, ""))
             .build();
       } else {
         throw new IOException(
@@ -147,7 +153,6 @@
     }
   }
 
-  // TODO: Write this class
   private static class TokenCredentialSerializer extends JsonSerializer<TokenCredential> {
     // These providers are singletons, so don't require any serialization, other than type.
     // add any singleton credentials...
@@ -162,13 +167,75 @@
       serializers.defaultSerializeValue(tokenCredential, jsonGenerator);
     }
 
+    @SuppressWarnings("nullness")
+    private static Object getMember(Object obj, String member)
+        throws IllegalAccessException, NoSuchFieldException {
+      Class<?> cls = obj.getClass();
+      Field field = cls.getDeclaredField(member);
+      field.setAccessible(true);
+      Object fieldObj = field.get(obj);
+      assert fieldObj != null;
+      return fieldObj;
+    }
+
     @Override
     public void serializeWithType(
         TokenCredential tokenCredential,
         JsonGenerator jsonGenerator,
         SerializerProvider serializers,
-        TypeSerializer typeSerializer) {
-      throw new UnsupportedOperationException();
+        TypeSerializer typeSerializer)
+        throws IOException {
+
+      WritableTypeId typeIdDef =
+          typeSerializer.writeTypePrefix(
+              jsonGenerator, typeSerializer.typeId(tokenCredential, JsonToken.START_OBJECT));
+
+      try {
+        if (tokenCredential instanceof DefaultAzureCredential) {
+          // Do nothing
+        } else if (tokenCredential instanceof ClientSecretCredential) {
+          ClientSecretCredential credential = (ClientSecretCredential) tokenCredential;
+          IdentityClient identityClient = (IdentityClient) getMember(credential, "identityClient");
+          jsonGenerator.writeStringField(
+              AZURE_CLIENT_ID, (String) getMember(identityClient, "clientId"));
+          jsonGenerator.writeStringField(
+              AZURE_TENANT_ID, (String) getMember(identityClient, "tenantId"));
+          jsonGenerator.writeStringField(
+              AZURE_CLIENT_SECRET, (String) getMember(credential, "clientSecret"));
+        } else if (tokenCredential instanceof ManagedIdentityCredential) {
+          ManagedIdentityCredential credential = (ManagedIdentityCredential) tokenCredential;
+          Object appServiceMsiCredential = getMember(credential, "appServiceMSICredential");
+          IdentityClient identityClient =
+              (IdentityClient) getMember(appServiceMsiCredential, "identityClient");
+          jsonGenerator.writeStringField(
+              AZURE_CLIENT_ID, (String) getMember(identityClient, "clientId"));
+        } else if (tokenCredential instanceof EnvironmentCredential) {
+          // Do nothing
+        } else if (tokenCredential instanceof ClientCertificateCredential) {
+          throw new IOException("Client certificates not yet implemented"); // TODO
+        } else if (tokenCredential instanceof UsernamePasswordCredential) {
+          UsernamePasswordCredential credential = (UsernamePasswordCredential) tokenCredential;
+          IdentityClient identityClient = (IdentityClient) getMember(credential, "identityClient");
+          jsonGenerator.writeStringField(
+              AZURE_CLIENT_ID, (String) getMember(identityClient, "clientId"));
+          jsonGenerator.writeStringField(
+              AZURE_USERNAME, (String) getMember(credential, "username"));
+          jsonGenerator.writeStringField(
+              AZURE_PASSWORD, (String) getMember(credential, "password"));
+        } else {
+          throw new IOException(
+              String.format(
+                  "Azure credential provider type '%s' is not supported",
+                  tokenCredential.getClass().getSimpleName()));
+        }
+      } catch (IllegalAccessException | NoSuchFieldException e) {
+        throw new IOException(
+            String.format(
+                "Failed to serialize object of type '%s': %s",
+                tokenCredential.getClass().getSimpleName(), e.toString()));
+      }
+
+      typeSerializer.writeTypeSuffix(jsonGenerator, typeIdDef);
     }
   }
 }
diff --git a/sdks/java/io/common/src/test/java/org/apache/beam/sdk/io/common/DatabaseTestHelper.java b/sdks/java/io/common/src/test/java/org/apache/beam/sdk/io/common/DatabaseTestHelper.java
index 56b7230..46d9aa9 100644
--- a/sdks/java/io/common/src/test/java/org/apache/beam/sdk/io/common/DatabaseTestHelper.java
+++ b/sdks/java/io/common/src/test/java/org/apache/beam/sdk/io/common/DatabaseTestHelper.java
@@ -17,14 +17,18 @@
  */
 package org.apache.beam.sdk.io.common;
 
+import static org.junit.Assert.assertEquals;
+
 import java.sql.Connection;
 import java.sql.ResultSet;
 import java.sql.SQLException;
 import java.sql.Statement;
 import java.text.SimpleDateFormat;
+import java.util.ArrayList;
 import java.util.Date;
 import java.util.Optional;
 import javax.sql.DataSource;
+import org.apache.beam.sdk.values.KV;
 import org.postgresql.ds.PGSimpleDataSource;
 
 /** This class contains helper methods to ease database usage in tests. */
@@ -104,4 +108,26 @@
       }
     }
   }
+
+  public static ArrayList<KV<Integer, String>> getTestDataToWrite(long rowsToAdd) {
+    ArrayList<KV<Integer, String>> data = new ArrayList<>();
+    for (int i = 0; i < rowsToAdd; i++) {
+      KV<Integer, String> kv = KV.of(i, "Test");
+      data.add(kv);
+    }
+    return data;
+  }
+
+  public static void assertRowCount(DataSource dataSource, String tableName, int expectedRowCount)
+      throws SQLException {
+    try (Connection connection = dataSource.getConnection()) {
+      try (Statement statement = connection.createStatement()) {
+        try (ResultSet resultSet = statement.executeQuery("select count(*) from " + tableName)) {
+          resultSet.next();
+          int count = resultSet.getInt(1);
+          assertEquals(expectedRowCount, count);
+        }
+      }
+    }
+  }
 }
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java
index 951f1fd..afa5ae7 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java
@@ -843,8 +843,6 @@
         idsToPublish = insertIdList;
       }
 
-      ServiceCallMetric serviceCallMetric = BigQueryUtils.writeCallMetric(ref);
-
       while (true) {
         List<FailsafeValueInSingleWindow<TableRow, TableRow>> retryRows = new ArrayList<>();
         List<String> retryIds = (idsToPublish != null) ? new ArrayList<>() : null;
@@ -898,6 +896,7 @@
                           BackOffAdapter.toGcpBackOff(rateLimitBackoffFactory.backoff());
                       long totalBackoffMillis = 0L;
                       while (true) {
+                        ServiceCallMetric serviceCallMetric = BigQueryUtils.writeCallMetric(ref);
                         try {
                           List<TableDataInsertAllResponse.InsertErrors> response =
                               insert.execute().getInsertErrors();
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BeamRowToBigtableMutation.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BeamRowToBigtableMutation.java
index f394cbe..e1f2a00 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BeamRowToBigtableMutation.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BeamRowToBigtableMutation.java
@@ -111,12 +111,11 @@
 
     private ByteString convertValueToByteString(Row row, String column) {
       Schema.Field field = row.getSchema().getField(column);
-      Schema.TypeName typeName = field.getType().getTypeName();
       Object value = row.getValue(column);
       if (value == null) {
         throw new NullPointerException("Null value at column " + column);
       } else {
-        return cellValueParser.valueToByteString(value, typeName);
+        return cellValueParser.valueToByteString(value, field.getType());
       }
     }
   }
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableRowToBeamRow.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableRowToBeamRow.java
index 6dff82d..2262780 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableRowToBeamRow.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableRowToBeamRow.java
@@ -94,8 +94,7 @@
 
       Row.FieldValueBuilder rowBuilder =
           Row.withSchema(cellSchema)
-              .withFieldValue(
-                  VALUE, getCellValue(cell, cellSchema.getField(VALUE).getType().getTypeName()));
+              .withFieldValue(VALUE, getCellValue(cell, cellSchema.getField(VALUE).getType()));
       if (cellSchema.hasField(TIMESTAMP_MICROS)) {
         rowBuilder.withFieldValue(TIMESTAMP_MICROS, cell.getTimestampMicros());
       }
@@ -115,7 +114,7 @@
           Schema.FieldType collectionElementType = columnType.getCollectionElementType();
           if (collectionElementType != null) {
             return cells.stream()
-                .map(cell -> getCellValue(cell, collectionElementType.getTypeName()))
+                .map(cell -> getCellValue(cell, collectionElementType))
                 .collect(toList());
           } else {
             throw new NullPointerException("Null collectionElementType at column " + columnName);
@@ -128,7 +127,7 @@
             return cellToRow(getLastCell(cells), rowSchema);
           }
         default:
-          return getCellValue(getLastCell(cells), columnType.getTypeName());
+          return getCellValue(getLastCell(cells), columnType);
       }
     }
 
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableRowToBeamRowFlat.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableRowToBeamRowFlat.java
index 55cecbc..8e9cc16 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableRowToBeamRowFlat.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableRowToBeamRowFlat.java
@@ -101,7 +101,7 @@
 
     private void setColumn(Row.FieldValueBuilder rowBuilder, Column column) {
       String columnName = column.getQualifier().toStringUtf8();
-      Schema.TypeName type = schema.getField(columnName).getType().getTypeName();
+      Schema.FieldType type = schema.getField(columnName).getType();
       rowBuilder.withFieldValue(columnName, getCellValue(getLastCell(column.getCellsList()), type));
     }
   }
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableRowToBeamRowFn.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableRowToBeamRowFn.java
index 0e7da66..a72f58d 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableRowToBeamRowFn.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableRowToBeamRowFn.java
@@ -40,7 +40,7 @@
         .orElseThrow(() -> new RuntimeException("Couldn't retrieve the most recent cell value"));
   }
 
-  protected Object getCellValue(Cell cell, Schema.TypeName typeName) {
-    return valueParser.getCellValue(cell, typeName);
+  protected Object getCellValue(Cell cell, Schema.FieldType type) {
+    return valueParser.getCellValue(cell, type);
   }
 }
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/CellValueParser.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/CellValueParser.java
index 2388615..1560fcc 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/CellValueParser.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/CellValueParser.java
@@ -18,6 +18,7 @@
 package org.apache.beam.sdk.io.gcp.bigtable;
 
 import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.beam.sdk.util.Preconditions.checkArgumentNotNull;
 import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument;
 
 import com.google.bigtable.v2.Cell;
@@ -31,10 +32,10 @@
 
 class CellValueParser implements Serializable {
 
-  Object getCellValue(Cell cell, Schema.TypeName type) {
+  Object getCellValue(Cell cell, Schema.FieldType type) {
     ByteString cellValue = cell.getValue();
     int valueSize = cellValue.size();
-    switch (type) {
+    switch (type.getTypeName()) {
       case BOOLEAN:
         checkArgument(valueSize == 1, message("Boolean", 1));
         return cellValue.toByteArray()[0] != 0;
@@ -62,14 +63,17 @@
         return cellValue.toStringUtf8();
       case BYTES:
         return cellValue.toByteArray();
+      case LOGICAL_TYPE:
+        String identifier = checkArgumentNotNull(type.getLogicalType()).getIdentifier();
+        throw new IllegalStateException("Unsupported logical type: " + identifier);
       default:
         throw new IllegalArgumentException(
-            String.format("Unsupported cell value type '%s'.", type));
+            String.format("Unsupported cell value type '%s'.", type.getTypeName()));
     }
   }
 
-  ByteString valueToByteString(Object value, Schema.TypeName type) {
-    switch (type) {
+  ByteString valueToByteString(Object value, Schema.FieldType type) {
+    switch (type.getTypeName()) {
       case BOOLEAN:
         return byteString(((Boolean) value) ? new byte[] {1} : new byte[] {0});
       case FLOAT:
@@ -90,8 +94,15 @@
         return byteString((byte[]) value);
       case DATETIME:
         return byteString(value.toString().getBytes(UTF_8));
+      case LOGICAL_TYPE:
+        String identifier = checkArgumentNotNull(type.getLogicalType()).getIdentifier();
+        if ("SqlCharType".equals(identifier)) {
+          return byteString(((String) value).getBytes(UTF_8));
+        } else {
+          throw new IllegalStateException("Unsupported logical type: " + identifier);
+        }
       default:
-        throw new IllegalStateException("Unsupported type: " + type);
+        throw new IllegalStateException("Unsupported type: " + type.getTypeName());
     }
   }
 
diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1.java
index 2910650..2b6955f 100644
--- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1.java
+++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1.java
@@ -58,9 +58,13 @@
 import java.io.IOException;
 import java.io.Serializable;
 import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.List;
 import java.util.NoSuchElementException;
 import java.util.Set;
+import org.apache.beam.runners.core.metrics.GcpResourceIdentifiers;
+import org.apache.beam.runners.core.metrics.MonitoringInfoConstants;
+import org.apache.beam.runners.core.metrics.ServiceCallMetric;
 import org.apache.beam.sdk.PipelineRunner;
 import org.apache.beam.sdk.coders.StringUtf8Coder;
 import org.apache.beam.sdk.extensions.gcp.options.GcpOptions;
@@ -901,12 +905,27 @@
         Sleeper sleeper = Sleeper.DEFAULT;
         BackOff backoff = RUNQUERY_BACKOFF.backoff();
         while (true) {
+          HashMap<String, String> baseLabels = new HashMap<>();
+          baseLabels.put(MonitoringInfoConstants.Labels.PTRANSFORM, "");
+          baseLabels.put(MonitoringInfoConstants.Labels.SERVICE, "Datastore");
+          baseLabels.put(MonitoringInfoConstants.Labels.METHOD, "BatchDatastoreRead");
+          baseLabels.put(
+              MonitoringInfoConstants.Labels.RESOURCE,
+              GcpResourceIdentifiers.datastoreResource(
+                  options.getProjectId(), options.getNamespace()));
+          baseLabels.put(MonitoringInfoConstants.Labels.DATASTORE_PROJECT, options.getProjectId());
+          baseLabels.put(
+              MonitoringInfoConstants.Labels.DATASTORE_NAMESPACE, options.getNamespace());
+          ServiceCallMetric serviceCallMetric =
+              new ServiceCallMetric(MonitoringInfoConstants.Urns.API_REQUEST_COUNT, baseLabels);
           try {
             RunQueryResponse response = datastore.runQuery(request);
+            serviceCallMetric.call("ok");
             rpcSuccesses.inc();
             return response;
           } catch (DatastoreException exception) {
             rpcErrors.inc();
+            serviceCallMetric.call(exception.getCode().getNumber());
 
             if (NON_RETRYABLE_ERRORS.contains(exception.getCode())) {
               throw exception;
@@ -1447,9 +1466,21 @@
           continue;
         }
 
+        HashMap<String, String> baseLabels = new HashMap<>();
+        baseLabels.put(MonitoringInfoConstants.Labels.PTRANSFORM, "");
+        baseLabels.put(MonitoringInfoConstants.Labels.SERVICE, "Datastore");
+        baseLabels.put(MonitoringInfoConstants.Labels.METHOD, "BatchDatastoreWrite");
+        baseLabels.put(
+            MonitoringInfoConstants.Labels.RESOURCE,
+            GcpResourceIdentifiers.datastoreResource(projectId.get(), ""));
+        baseLabels.put(MonitoringInfoConstants.Labels.DATASTORE_PROJECT, projectId.get());
+        baseLabels.put(MonitoringInfoConstants.Labels.DATASTORE_NAMESPACE, "");
+        ServiceCallMetric serviceCallMetric =
+            new ServiceCallMetric(MonitoringInfoConstants.Urns.API_REQUEST_COUNT, baseLabels);
         try {
           datastore.commit(commitRequest.build());
           endTime = System.currentTimeMillis();
+          serviceCallMetric.call("ok");
 
           writeBatcher.addRequestLatency(endTime, endTime - startTime, mutations.size());
           adaptiveThrottler.successfulRequest(startTime);
@@ -1460,6 +1491,7 @@
           // Break if the commit threw no exception.
           break;
         } catch (DatastoreException exception) {
+          serviceCallMetric.call(exception.getCode().getNumber());
           if (exception.getCode() == Code.DEADLINE_EXCEEDED) {
             /* Most errors are not related to request size, and should not change our expectation of
              * the latency of successful requests. DEADLINE_EXCEEDED can be taken into
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java
index 1328628..abe2749 100644
--- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java
@@ -77,7 +77,7 @@
 import org.apache.beam.sdk.coders.IterableCoder;
 import org.apache.beam.sdk.coders.KvCoder;
 import org.apache.beam.sdk.coders.StringUtf8Coder;
-import org.apache.beam.sdk.coders.VarIntCoder;
+import org.apache.beam.sdk.coders.VarLongCoder;
 import org.apache.beam.sdk.extensions.gcp.auth.TestCredential;
 import org.apache.beam.sdk.extensions.gcp.options.GcpOptions;
 import org.apache.beam.sdk.extensions.protobuf.ByteStringCoder;
@@ -1271,8 +1271,7 @@
    * A DoFn used to generate N outputs, where N is the input. Used to generate bundles of >= 1
    * element.
    */
-  private static class WriteGeneratorDoFn
-      extends DoFn<Integer, KV<ByteString, Iterable<Mutation>>> {
+  private static class WriteGeneratorDoFn extends DoFn<Long, KV<ByteString, Iterable<Mutation>>> {
     @ProcessElement
     public void processElement(ProcessContext ctx) {
       for (int i = 0; i < ctx.element(); i++) {
@@ -1293,12 +1292,12 @@
     Instant elementTimestamp = Instant.parse("2019-06-10T00:00:00");
     Duration windowDuration = Duration.standardMinutes(1);
 
-    TestStream<Integer> input =
-        TestStream.create(VarIntCoder.of())
+    TestStream<Long> input =
+        TestStream.create(VarLongCoder.of())
             .advanceWatermarkTo(elementTimestamp)
-            .addElements(1)
+            .addElements(1L)
             .advanceWatermarkTo(elementTimestamp.plus(windowDuration))
-            .addElements(2)
+            .addElements(2L)
             .advanceWatermarkToInfinity();
 
     BoundedWindow expectedFirstWindow = new IntervalWindow(elementTimestamp, windowDuration);
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/CellValueParserTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/CellValueParserTest.java
index 4bf776b..0fe6b7f 100644
--- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/CellValueParserTest.java
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/CellValueParserTest.java
@@ -18,17 +18,17 @@
 package org.apache.beam.sdk.io.gcp.bigtable;
 
 import static java.nio.charset.StandardCharsets.UTF_8;
-import static org.apache.beam.sdk.schemas.Schema.TypeName.BOOLEAN;
-import static org.apache.beam.sdk.schemas.Schema.TypeName.BYTE;
-import static org.apache.beam.sdk.schemas.Schema.TypeName.BYTES;
-import static org.apache.beam.sdk.schemas.Schema.TypeName.DATETIME;
-import static org.apache.beam.sdk.schemas.Schema.TypeName.DOUBLE;
-import static org.apache.beam.sdk.schemas.Schema.TypeName.FLOAT;
-import static org.apache.beam.sdk.schemas.Schema.TypeName.INT16;
-import static org.apache.beam.sdk.schemas.Schema.TypeName.INT32;
-import static org.apache.beam.sdk.schemas.Schema.TypeName.INT64;
+import static org.apache.beam.sdk.schemas.Schema.FieldType.BOOLEAN;
+import static org.apache.beam.sdk.schemas.Schema.FieldType.BYTE;
+import static org.apache.beam.sdk.schemas.Schema.FieldType.BYTES;
+import static org.apache.beam.sdk.schemas.Schema.FieldType.DATETIME;
+import static org.apache.beam.sdk.schemas.Schema.FieldType.DOUBLE;
+import static org.apache.beam.sdk.schemas.Schema.FieldType.FLOAT;
+import static org.apache.beam.sdk.schemas.Schema.FieldType.INT16;
+import static org.apache.beam.sdk.schemas.Schema.FieldType.INT32;
+import static org.apache.beam.sdk.schemas.Schema.FieldType.INT64;
+import static org.apache.beam.sdk.schemas.Schema.FieldType.STRING;
 import static org.apache.beam.sdk.schemas.Schema.TypeName.MAP;
-import static org.apache.beam.sdk.schemas.Schema.TypeName.STRING;
 import static org.hamcrest.MatcherAssert.assertThat;
 import static org.hamcrest.Matchers.containsString;
 import static org.junit.Assert.assertEquals;
@@ -37,6 +37,7 @@
 
 import com.google.bigtable.v2.Cell;
 import com.google.protobuf.ByteString;
+import org.apache.beam.sdk.schemas.Schema.FieldType;
 import org.checkerframework.checker.nullness.qual.Nullable;
 import org.joda.time.DateTime;
 import org.joda.time.DateTimeZone;
@@ -188,7 +189,9 @@
   public void shouldFailOnUnsupportedType() {
     byte[] value = new byte[0];
     IllegalArgumentException exception =
-        assertThrows(IllegalArgumentException.class, () -> PARSER.getCellValue(cell(value), MAP));
+        assertThrows(
+            IllegalArgumentException.class,
+            () -> PARSER.getCellValue(cell(value), FieldType.of(MAP)));
     checkMessage(exception.getMessage(), "Unsupported cell value type 'MAP'.");
   }
 
diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1Test.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1Test.java
index 928acd0..8ab31dc 100644
--- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1Test.java
+++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1Test.java
@@ -70,9 +70,13 @@
 import com.google.rpc.Code;
 import java.util.ArrayList;
 import java.util.Date;
+import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
+import org.apache.beam.runners.core.metrics.MetricsContainerImpl;
+import org.apache.beam.runners.core.metrics.MonitoringInfoConstants;
+import org.apache.beam.runners.core.metrics.MonitoringInfoMetricName;
 import org.apache.beam.sdk.Pipeline;
 import org.apache.beam.sdk.io.gcp.datastore.DatastoreV1.DatastoreWriterFn;
 import org.apache.beam.sdk.io.gcp.datastore.DatastoreV1.DeleteEntity;
@@ -85,6 +89,7 @@
 import org.apache.beam.sdk.io.gcp.datastore.DatastoreV1.UpsertFn;
 import org.apache.beam.sdk.io.gcp.datastore.DatastoreV1.V1DatastoreFactory;
 import org.apache.beam.sdk.io.gcp.datastore.DatastoreV1.Write;
+import org.apache.beam.sdk.metrics.MetricsEnvironment;
 import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.options.PipelineOptionsFactory;
 import org.apache.beam.sdk.options.ValueProvider;
@@ -141,6 +146,9 @@
     when(mockDatastoreFactory.getDatastore(any(PipelineOptions.class), any(String.class), any()))
         .thenReturn(mockDatastore);
     when(mockDatastoreFactory.getQuerySplitter()).thenReturn(mockQuerySplitter);
+    // Setup the ProcessWideContainer for testing metrics are set.
+    MetricsContainerImpl container = new MetricsContainerImpl(null);
+    MetricsEnvironment.setProcessWideContainer(container);
   }
 
   @Test
@@ -504,12 +512,14 @@
   @Test
   public void testDatatoreWriterFnWithOneBatch() throws Exception {
     datastoreWriterFnTest(100);
+    verifyMetricWasSet("BatchDatastoreWrite", "ok", "", 2);
   }
 
   /** Tests {@link DatastoreWriterFn} with entities of more than one batches, but not a multiple. */
   @Test
   public void testDatatoreWriterFnWithMultipleBatches() throws Exception {
     datastoreWriterFnTest(DatastoreV1.DATASTORE_BATCH_UPDATE_ENTITIES_START * 3 + 100);
+    verifyMetricWasSet("BatchDatastoreWrite", "ok", "", 5);
   }
 
   /**
@@ -519,6 +529,7 @@
   @Test
   public void testDatatoreWriterFnWithBatchesExactMultiple() throws Exception {
     datastoreWriterFnTest(DatastoreV1.DATASTORE_BATCH_UPDATE_ENTITIES_START * 2);
+    verifyMetricWasSet("BatchDatastoreWrite", "ok", "", 2);
   }
 
   // A helper method to test DatastoreWriterFn for various batch sizes.
@@ -612,6 +623,8 @@
     DoFnTester<Mutation, Void> doFnTester = DoFnTester.of(datastoreWriter);
     doFnTester.setCloningBehavior(CloningBehavior.DO_NOT_CLONE);
     doFnTester.processBundle(mutations);
+    verifyMetricWasSet("BatchDatastoreWrite", "ok", "", 2);
+    verifyMetricWasSet("BatchDatastoreWrite", "unknown", "", 1);
   }
 
   /**
@@ -725,18 +738,21 @@
   @Test
   public void testReadFnWithOneBatch() throws Exception {
     readFnTest(5);
+    verifyMetricWasSet("BatchDatastoreRead", "ok", NAMESPACE, 1);
   }
 
   /** Tests {@link ReadFn} with a query limit more than one batch, and not a multiple. */
   @Test
   public void testReadFnWithMultipleBatches() throws Exception {
     readFnTest(QUERY_BATCH_LIMIT + 5);
+    verifyMetricWasSet("BatchDatastoreRead", "ok", NAMESPACE, 2);
   }
 
   /** Tests {@link ReadFn} for several batches, using an exact multiple of batch size results. */
   @Test
   public void testReadFnWithBatchesExactMultiple() throws Exception {
     readFnTest(5 * QUERY_BATCH_LIMIT);
+    verifyMetricWasSet("BatchDatastoreRead", "ok", NAMESPACE, 5);
   }
 
   /** Tests that {@link ReadFn} retries after an error. */
@@ -758,6 +774,8 @@
     DoFnTester<Query, Entity> doFnTester = DoFnTester.of(readFn);
     doFnTester.setCloningBehavior(CloningBehavior.DO_NOT_CLONE);
     doFnTester.processBundle(query);
+    verifyMetricWasSet("BatchDatastoreRead", "ok", NAMESPACE, 1);
+    verifyMetricWasSet("BatchDatastoreRead", "unknown", NAMESPACE, 1);
   }
 
   @Test
@@ -1061,4 +1079,24 @@
       return DatastoreV1.DATASTORE_BATCH_UPDATE_ENTITIES_START;
     }
   }
+
+  private void verifyMetricWasSet(String method, String status, String namespace, long count) {
+    // Verify the metric as reported.
+    HashMap<String, String> labels = new HashMap<>();
+    labels.put(MonitoringInfoConstants.Labels.PTRANSFORM, "");
+    labels.put(MonitoringInfoConstants.Labels.SERVICE, "Datastore");
+    labels.put(MonitoringInfoConstants.Labels.METHOD, method);
+    labels.put(
+        MonitoringInfoConstants.Labels.RESOURCE,
+        "//bigtable.googleapis.com/projects/" + PROJECT_ID + "/namespaces/" + namespace);
+    labels.put(MonitoringInfoConstants.Labels.DATASTORE_PROJECT, PROJECT_ID);
+    labels.put(MonitoringInfoConstants.Labels.DATASTORE_NAMESPACE, namespace);
+    labels.put(MonitoringInfoConstants.Labels.STATUS, status);
+
+    MonitoringInfoMetricName name =
+        MonitoringInfoMetricName.named(MonitoringInfoConstants.Urns.API_REQUEST_COUNT, labels);
+    MetricsContainerImpl container =
+        (MetricsContainerImpl) MetricsEnvironment.getProcessWideContainer();
+    assertEquals(count, (long) container.getCounter(name).getCumulative());
+  }
 }
diff --git a/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcIO.java b/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcIO.java
index e5ea008..2cab8b4 100644
--- a/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcIO.java
+++ b/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcIO.java
@@ -41,6 +41,7 @@
 import java.util.Optional;
 import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.TimeUnit;
 import java.util.function.Predicate;
 import java.util.stream.Collectors;
 import java.util.stream.IntStream;
@@ -51,6 +52,8 @@
 import org.apache.beam.sdk.coders.RowCoder;
 import org.apache.beam.sdk.io.jdbc.JdbcUtil.PartitioningFn;
 import org.apache.beam.sdk.io.jdbc.SchemaUtil.FieldWithIndex;
+import org.apache.beam.sdk.metrics.Distribution;
+import org.apache.beam.sdk.metrics.Metrics;
 import org.apache.beam.sdk.options.ValueProvider;
 import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider;
 import org.apache.beam.sdk.schemas.NoSuchSchemaException;
@@ -339,7 +342,7 @@
    * @param <T> Type of the data to be written.
    */
   public static <T> Write<T> write() {
-    return new Write();
+    return new Write<>();
   }
 
   public static <T> WriteVoid<T> writeVoid() {
@@ -1280,43 +1283,43 @@
 
     /** See {@link WriteVoid#withDataSourceConfiguration(DataSourceConfiguration)}. */
     public Write<T> withDataSourceConfiguration(DataSourceConfiguration config) {
-      return new Write(inner.withDataSourceConfiguration(config));
+      return new Write<>(inner.withDataSourceConfiguration(config));
     }
 
     /** See {@link WriteVoid#withDataSourceProviderFn(SerializableFunction)}. */
     public Write<T> withDataSourceProviderFn(
         SerializableFunction<Void, DataSource> dataSourceProviderFn) {
-      return new Write(inner.withDataSourceProviderFn(dataSourceProviderFn));
+      return new Write<>(inner.withDataSourceProviderFn(dataSourceProviderFn));
     }
 
     /** See {@link WriteVoid#withStatement(String)}. */
     public Write<T> withStatement(String statement) {
-      return new Write(inner.withStatement(statement));
+      return new Write<>(inner.withStatement(statement));
     }
 
     /** See {@link WriteVoid#withPreparedStatementSetter(PreparedStatementSetter)}. */
     public Write<T> withPreparedStatementSetter(PreparedStatementSetter<T> setter) {
-      return new Write(inner.withPreparedStatementSetter(setter));
+      return new Write<>(inner.withPreparedStatementSetter(setter));
     }
 
     /** See {@link WriteVoid#withBatchSize(long)}. */
     public Write<T> withBatchSize(long batchSize) {
-      return new Write(inner.withBatchSize(batchSize));
+      return new Write<>(inner.withBatchSize(batchSize));
     }
 
     /** See {@link WriteVoid#withRetryStrategy(RetryStrategy)}. */
     public Write<T> withRetryStrategy(RetryStrategy retryStrategy) {
-      return new Write(inner.withRetryStrategy(retryStrategy));
+      return new Write<>(inner.withRetryStrategy(retryStrategy));
     }
 
     /** See {@link WriteVoid#withRetryConfiguration(RetryConfiguration)}. */
     public Write<T> withRetryConfiguration(RetryConfiguration retryConfiguration) {
-      return new Write(inner.withRetryConfiguration(retryConfiguration));
+      return new Write<>(inner.withRetryConfiguration(retryConfiguration));
     }
 
     /** See {@link WriteVoid#withTable(String)}. */
     public Write<T> withTable(String table) {
-      return new Write(inner.withTable(table));
+      return new Write<>(inner.withTable(table));
     }
 
     /**
@@ -1338,6 +1341,24 @@
       return inner;
     }
 
+    /**
+     * Returns {@link WriteWithResults} transform that could return a specific result.
+     *
+     * <p>See {@link WriteWithResults}
+     */
+    public <V extends JdbcWriteResult> WriteWithResults<T, V> withWriteResults(
+        RowMapper<V> rowMapper) {
+      return new AutoValue_JdbcIO_WriteWithResults.Builder<T, V>()
+          .setRowMapper(rowMapper)
+          .setRetryStrategy(inner.getRetryStrategy())
+          .setRetryConfiguration(inner.getRetryConfiguration())
+          .setDataSourceProviderFn(inner.getDataSourceProviderFn())
+          .setPreparedStatementSetter(inner.getPreparedStatementSetter())
+          .setStatement(inner.getStatement())
+          .setTable(inner.getTable())
+          .build();
+    }
+
     @Override
     public void populateDisplayData(DisplayData.Builder builder) {
       inner.populateDisplayData(builder);
@@ -1361,7 +1382,244 @@
         throws SQLException;
   }
 
-  /** A {@link PTransform} to write to a JDBC datasource. */
+  /**
+   * A {@link PTransform} to write to a JDBC datasource. Executes statements one by one.
+   *
+   * <p>The INSERT, UPDATE, and DELETE commands sometimes have an optional RETURNING clause that
+   * supports obtaining data from modified rows while they are being manipulated. Output {@link
+   * PCollection} of this transform is a collection of such returning results mapped by {@link
+   * RowMapper}.
+   */
+  @AutoValue
+  public abstract static class WriteWithResults<T, V extends JdbcWriteResult>
+      extends PTransform<PCollection<T>, PCollection<V>> {
+    abstract @Nullable SerializableFunction<Void, DataSource> getDataSourceProviderFn();
+
+    abstract @Nullable ValueProvider<String> getStatement();
+
+    abstract @Nullable PreparedStatementSetter<T> getPreparedStatementSetter();
+
+    abstract @Nullable RetryStrategy getRetryStrategy();
+
+    abstract @Nullable RetryConfiguration getRetryConfiguration();
+
+    abstract @Nullable String getTable();
+
+    abstract @Nullable RowMapper<V> getRowMapper();
+
+    abstract Builder<T, V> toBuilder();
+
+    @AutoValue.Builder
+    abstract static class Builder<T, V extends JdbcWriteResult> {
+      abstract Builder<T, V> setDataSourceProviderFn(
+          SerializableFunction<Void, DataSource> dataSourceProviderFn);
+
+      abstract Builder<T, V> setStatement(ValueProvider<String> statement);
+
+      abstract Builder<T, V> setPreparedStatementSetter(PreparedStatementSetter<T> setter);
+
+      abstract Builder<T, V> setRetryStrategy(RetryStrategy deadlockPredicate);
+
+      abstract Builder<T, V> setRetryConfiguration(RetryConfiguration retryConfiguration);
+
+      abstract Builder<T, V> setTable(String table);
+
+      abstract Builder<T, V> setRowMapper(RowMapper<V> rowMapper);
+
+      abstract WriteWithResults<T, V> build();
+    }
+
+    public WriteWithResults<T, V> withDataSourceConfiguration(DataSourceConfiguration config) {
+      return withDataSourceProviderFn(new DataSourceProviderFromDataSourceConfiguration(config));
+    }
+
+    public WriteWithResults<T, V> withDataSourceProviderFn(
+        SerializableFunction<Void, DataSource> dataSourceProviderFn) {
+      return toBuilder().setDataSourceProviderFn(dataSourceProviderFn).build();
+    }
+
+    public WriteWithResults<T, V> withStatement(String statement) {
+      return withStatement(ValueProvider.StaticValueProvider.of(statement));
+    }
+
+    public WriteWithResults<T, V> withStatement(ValueProvider<String> statement) {
+      return toBuilder().setStatement(statement).build();
+    }
+
+    public WriteWithResults<T, V> withPreparedStatementSetter(PreparedStatementSetter<T> setter) {
+      return toBuilder().setPreparedStatementSetter(setter).build();
+    }
+
+    /**
+     * When a SQL exception occurs, {@link Write} uses this {@link RetryStrategy} to determine if it
+     * will retry the statements. If {@link RetryStrategy#apply(SQLException)} returns {@code true},
+     * then {@link Write} retries the statements.
+     */
+    public WriteWithResults<T, V> withRetryStrategy(RetryStrategy retryStrategy) {
+      checkArgument(retryStrategy != null, "retryStrategy can not be null");
+      return toBuilder().setRetryStrategy(retryStrategy).build();
+    }
+
+    /**
+     * When a SQL exception occurs, {@link Write} uses this {@link RetryConfiguration} to
+     * exponentially back off and retry the statements based on the {@link RetryConfiguration}
+     * mentioned.
+     *
+     * <p>Usage of RetryConfiguration -
+     *
+     * <pre>{@code
+     * pipeline.apply(JdbcIO.<T>write())
+     *    .withReturningResults(...)
+     *    .withDataSourceConfiguration(...)
+     *    .withRetryStrategy(...)
+     *    .withRetryConfiguration(JdbcIO.RetryConfiguration.
+     *        create(5, Duration.standardSeconds(5), Duration.standardSeconds(1))
+     *
+     * }</pre>
+     *
+     * maxDuration and initialDuration are Nullable
+     *
+     * <pre>{@code
+     * pipeline.apply(JdbcIO.<T>write())
+     *    .withReturningResults(...)
+     *    .withDataSourceConfiguration(...)
+     *    .withRetryStrategy(...)
+     *    .withRetryConfiguration(JdbcIO.RetryConfiguration.
+     *        create(5, null, null)
+     *
+     * }</pre>
+     */
+    public WriteWithResults<T, V> withRetryConfiguration(RetryConfiguration retryConfiguration) {
+      checkArgument(retryConfiguration != null, "retryConfiguration can not be null");
+      return toBuilder().setRetryConfiguration(retryConfiguration).build();
+    }
+
+    public WriteWithResults<T, V> withTable(String table) {
+      checkArgument(table != null, "table name can not be null");
+      return toBuilder().setTable(table).build();
+    }
+
+    public WriteWithResults<T, V> withRowMapper(RowMapper<V> rowMapper) {
+      checkArgument(rowMapper != null, "result set getter can not be null");
+      return toBuilder().setRowMapper(rowMapper).build();
+    }
+
+    @Override
+    public PCollection<V> expand(PCollection<T> input) {
+      checkArgument(getStatement() != null, "withStatement() is required");
+      checkArgument(
+          getPreparedStatementSetter() != null, "withPreparedStatementSetter() is required");
+      checkArgument(
+          (getDataSourceProviderFn() != null),
+          "withDataSourceConfiguration() or withDataSourceProviderFn() is required");
+
+      return input.apply(ParDo.of(new WriteWithResultsFn<>(this)));
+    }
+
+    private static class WriteWithResultsFn<T, V extends JdbcWriteResult> extends DoFn<T, V> {
+
+      private final WriteWithResults<T, V> spec;
+      private DataSource dataSource;
+      private Connection connection;
+      private PreparedStatement preparedStatement;
+      private static FluentBackoff retryBackOff;
+
+      public WriteWithResultsFn(WriteWithResults<T, V> spec) {
+        this.spec = spec;
+      }
+
+      @Setup
+      public void setup() {
+        dataSource = spec.getDataSourceProviderFn().apply(null);
+        RetryConfiguration retryConfiguration = spec.getRetryConfiguration();
+
+        retryBackOff =
+            FluentBackoff.DEFAULT
+                .withInitialBackoff(retryConfiguration.getInitialDuration())
+                .withMaxCumulativeBackoff(retryConfiguration.getMaxDuration())
+                .withMaxRetries(retryConfiguration.getMaxAttempts());
+      }
+
+      @ProcessElement
+      public void processElement(ProcessContext context) throws Exception {
+        T record = context.element();
+
+        // Only acquire the connection if there is something to write.
+        if (connection == null) {
+          connection = dataSource.getConnection();
+          connection.setAutoCommit(false);
+          preparedStatement = connection.prepareStatement(spec.getStatement().get());
+        }
+        Sleeper sleeper = Sleeper.DEFAULT;
+        BackOff backoff = retryBackOff.backoff();
+        while (true) {
+          try (PreparedStatement preparedStatement =
+              connection.prepareStatement(spec.getStatement().get())) {
+            try {
+
+              try {
+                spec.getPreparedStatementSetter().setParameters(record, preparedStatement);
+              } catch (Exception e) {
+                throw new RuntimeException(e);
+              }
+
+              // execute the statement
+              preparedStatement.execute();
+              // commit the changes
+              connection.commit();
+              context.output(spec.getRowMapper().mapRow(preparedStatement.getResultSet()));
+              return;
+            } catch (SQLException exception) {
+              if (!spec.getRetryStrategy().apply(exception)) {
+                throw exception;
+              }
+              LOG.warn("Deadlock detected, retrying", exception);
+              connection.rollback();
+              if (!BackOffUtils.next(sleeper, backoff)) {
+                // we tried the max number of times
+                throw exception;
+              }
+            }
+          }
+        }
+      }
+
+      @FinishBundle
+      public void finishBundle() throws Exception {
+        cleanUpStatementAndConnection();
+      }
+
+      @Override
+      protected void finalize() throws Throwable {
+        cleanUpStatementAndConnection();
+      }
+
+      private void cleanUpStatementAndConnection() throws Exception {
+        try {
+          if (preparedStatement != null) {
+            try {
+              preparedStatement.close();
+            } finally {
+              preparedStatement = null;
+            }
+          }
+        } finally {
+          if (connection != null) {
+            try {
+              connection.close();
+            } finally {
+              connection = null;
+            }
+          }
+        }
+      }
+    }
+  }
+
+  /**
+   * A {@link PTransform} to write to a JDBC datasource. Executes statements in a batch, and returns
+   * a trivial result.
+   */
   @AutoValue
   public abstract static class WriteVoid<T> extends PTransform<PCollection<T>, PCollection<Void>> {
 
@@ -1615,6 +1873,10 @@
 
     private static class WriteFn<T> extends DoFn<T, Void> {
 
+      private static final Distribution RECORDS_PER_BATCH =
+          Metrics.distribution(WriteFn.class, "records_per_jdbc_batch");
+      private static final Distribution MS_PER_BATCH =
+          Metrics.distribution(WriteFn.class, "milliseconds_per_batch");
       private final WriteVoid<T> spec;
       private DataSource dataSource;
       private Connection connection;
@@ -1694,6 +1956,7 @@
         if (records.isEmpty()) {
           return;
         }
+        Long startTimeNs = System.nanoTime();
         // Only acquire the connection if there is something to write.
         if (connection == null) {
           connection = dataSource.getConnection();
@@ -1714,6 +1977,8 @@
               preparedStatement.executeBatch();
               // commit the changes
               connection.commit();
+              RECORDS_PER_BATCH.update(records.size());
+              MS_PER_BATCH.update(TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTimeNs));
               break;
             } catch (SQLException exception) {
               if (!spec.getRetryStrategy().apply(exception)) {
diff --git a/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcWriteResult.java b/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcWriteResult.java
new file mode 100644
index 0000000..3117c24
--- /dev/null
+++ b/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcWriteResult.java
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.io.jdbc;
+
+import com.google.auto.value.AutoValue;
+
+/** The result of writing a row to JDBC datasource. */
+@AutoValue
+public abstract class JdbcWriteResult {
+  public static JdbcWriteResult create() {
+    return new AutoValue_JdbcWriteResult();
+  }
+}
diff --git a/sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/JdbcIOIT.java b/sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/JdbcIOIT.java
index 6c80002..45eb094 100644
--- a/sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/JdbcIOIT.java
+++ b/sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/JdbcIOIT.java
@@ -17,11 +17,14 @@
  */
 package org.apache.beam.sdk.io.jdbc;
 
+import static org.apache.beam.sdk.io.common.DatabaseTestHelper.assertRowCount;
+import static org.apache.beam.sdk.io.common.DatabaseTestHelper.getTestDataToWrite;
 import static org.apache.beam.sdk.io.common.IOITHelper.executeWithRetry;
 import static org.apache.beam.sdk.io.common.IOITHelper.readIOTestPipelineOptions;
 
 import com.google.cloud.Timestamp;
 import java.sql.SQLException;
+import java.util.ArrayList;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Optional;
@@ -44,8 +47,10 @@
 import org.apache.beam.sdk.testutils.publishing.InfluxDBSettings;
 import org.apache.beam.sdk.transforms.Combine;
 import org.apache.beam.sdk.transforms.Count;
+import org.apache.beam.sdk.transforms.Create;
 import org.apache.beam.sdk.transforms.ParDo;
 import org.apache.beam.sdk.transforms.Top;
+import org.apache.beam.sdk.values.KV;
 import org.apache.beam.sdk.values.PCollection;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
@@ -79,6 +84,7 @@
 @RunWith(JUnit4.class)
 public class JdbcIOIT {
 
+  private static final int EXPECTED_ROW_COUNT = 1000;
   private static final String NAMESPACE = JdbcIOIT.class.getName();
   private static int numberOfRows;
   private static PGSimpleDataSource dataSource;
@@ -255,4 +261,54 @@
 
     return pipelineRead.run();
   }
+
+  @Test
+  public void testWriteWithWriteResults() throws Exception {
+    String firstTableName = DatabaseTestHelper.getTestTableName("UT_WRITE");
+    DatabaseTestHelper.createTable(dataSource, firstTableName);
+    try {
+      ArrayList<KV<Integer, String>> data = getTestDataToWrite(EXPECTED_ROW_COUNT);
+
+      PCollection<KV<Integer, String>> dataCollection = pipelineWrite.apply(Create.of(data));
+      PCollection<JdbcTestHelper.TestDto> resultSetCollection =
+          dataCollection.apply(
+              getJdbcWriteWithReturning(firstTableName)
+                  .withWriteResults(
+                      (resultSet -> {
+                        if (resultSet != null && resultSet.next()) {
+                          return new JdbcTestHelper.TestDto(resultSet.getInt(1));
+                        }
+                        return new JdbcTestHelper.TestDto(JdbcTestHelper.TestDto.EMPTY_RESULT);
+                      })));
+      resultSetCollection.setCoder(JdbcTestHelper.TEST_DTO_CODER);
+
+      List<JdbcTestHelper.TestDto> expectedResult = new ArrayList<>();
+      for (int id = 0; id < EXPECTED_ROW_COUNT; id++) {
+        expectedResult.add(new JdbcTestHelper.TestDto(id));
+      }
+
+      PAssert.that(resultSetCollection).containsInAnyOrder(expectedResult);
+
+      pipelineWrite.run();
+
+      assertRowCount(dataSource, firstTableName, EXPECTED_ROW_COUNT);
+    } finally {
+      DatabaseTestHelper.deleteTable(dataSource, firstTableName);
+    }
+  }
+
+  /**
+   * @return {@link JdbcIO.Write} transform that writes to {@param tableName} Postgres table and
+   *     returns all fields of modified rows.
+   */
+  private static JdbcIO.Write<KV<Integer, String>> getJdbcWriteWithReturning(String tableName) {
+    return JdbcIO.<KV<Integer, String>>write()
+        .withDataSourceProviderFn(voidInput -> dataSource)
+        .withStatement(String.format("insert into %s values(?, ?) returning *", tableName))
+        .withPreparedStatementSetter(
+            (element, statement) -> {
+              statement.setInt(1, element.getKey());
+              statement.setString(2, element.getValue());
+            });
+  }
 }
diff --git a/sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/JdbcIOTest.java b/sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/JdbcIOTest.java
index 30b6c48..31ec663 100644
--- a/sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/JdbcIOTest.java
+++ b/sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/JdbcIOTest.java
@@ -18,6 +18,7 @@
 package org.apache.beam.sdk.io.jdbc;
 
 import static java.sql.JDBCType.NUMERIC;
+import static org.apache.beam.sdk.io.common.DatabaseTestHelper.assertRowCount;
 import static org.hamcrest.MatcherAssert.assertThat;
 import static org.hamcrest.Matchers.containsString;
 import static org.hamcrest.Matchers.instanceOf;
@@ -42,7 +43,6 @@
 import java.sql.Date;
 import java.sql.JDBCType;
 import java.sql.PreparedStatement;
-import java.sql.ResultSet;
 import java.sql.SQLException;
 import java.sql.Statement;
 import java.sql.Time;
@@ -468,13 +468,48 @@
 
       pipeline.run();
 
-      assertRowCount(tableName, EXPECTED_ROW_COUNT);
+      assertRowCount(DATA_SOURCE, tableName, EXPECTED_ROW_COUNT);
     } finally {
       DatabaseTestHelper.deleteTable(DATA_SOURCE, tableName);
     }
   }
 
   @Test
+  public void testWriteWithWriteResults() throws Exception {
+    String firstTableName = DatabaseTestHelper.getTestTableName("UT_WRITE");
+    DatabaseTestHelper.createTable(DATA_SOURCE, firstTableName);
+    try {
+      ArrayList<KV<Integer, String>> data = getDataToWrite(EXPECTED_ROW_COUNT);
+
+      PCollection<KV<Integer, String>> dataCollection = pipeline.apply(Create.of(data));
+      PCollection<JdbcTestHelper.TestDto> resultSetCollection =
+          dataCollection.apply(
+              getJdbcWrite(firstTableName)
+                  .withWriteResults(
+                      (resultSet -> {
+                        if (resultSet != null && resultSet.next()) {
+                          return new JdbcTestHelper.TestDto(resultSet.getInt(1));
+                        }
+                        return new JdbcTestHelper.TestDto(JdbcTestHelper.TestDto.EMPTY_RESULT);
+                      })));
+      resultSetCollection.setCoder(JdbcTestHelper.TEST_DTO_CODER);
+
+      List<JdbcTestHelper.TestDto> expectedResult = new ArrayList<>();
+      for (int i = 0; i < EXPECTED_ROW_COUNT; i++) {
+        expectedResult.add(new JdbcTestHelper.TestDto(JdbcTestHelper.TestDto.EMPTY_RESULT));
+      }
+
+      PAssert.that(resultSetCollection).containsInAnyOrder(expectedResult);
+
+      pipeline.run();
+
+      assertRowCount(DATA_SOURCE, firstTableName, EXPECTED_ROW_COUNT);
+    } finally {
+      DatabaseTestHelper.deleteTable(DATA_SOURCE, firstTableName);
+    }
+  }
+
+  @Test
   public void testWriteWithResultsAndWaitOn() throws Exception {
     String firstTableName = DatabaseTestHelper.getTestTableName("UT_WRITE");
     String secondTableName = DatabaseTestHelper.getTestTableName("UT_WRITE_AFTER_WAIT");
@@ -490,8 +525,8 @@
 
       pipeline.run();
 
-      assertRowCount(firstTableName, EXPECTED_ROW_COUNT);
-      assertRowCount(secondTableName, EXPECTED_ROW_COUNT);
+      assertRowCount(DATA_SOURCE, firstTableName, EXPECTED_ROW_COUNT);
+      assertRowCount(DATA_SOURCE, secondTableName, EXPECTED_ROW_COUNT);
     } finally {
       DatabaseTestHelper.deleteTable(DATA_SOURCE, firstTableName);
     }
@@ -525,18 +560,6 @@
     return data;
   }
 
-  private static void assertRowCount(String tableName, int expectedRowCount) throws SQLException {
-    try (Connection connection = DATA_SOURCE.getConnection()) {
-      try (Statement statement = connection.createStatement()) {
-        try (ResultSet resultSet = statement.executeQuery("select count(*) from " + tableName)) {
-          resultSet.next();
-          int count = resultSet.getInt(1);
-          assertEquals(expectedRowCount, count);
-        }
-      }
-    }
-  }
-
   @Test
   public void testWriteWithBackoff() throws Exception {
     String tableName = DatabaseTestHelper.getTestTableName("UT_WRITE_BACKOFF");
@@ -593,7 +616,7 @@
     // we verify that the backoff has been called thanks to the log message
     expectedLogs.verifyWarn("Deadlock detected, retrying");
 
-    assertRowCount(tableName, 2);
+    assertRowCount(DATA_SOURCE, tableName, 2);
   }
 
   @Test
@@ -645,7 +668,7 @@
                   .withBatchSize(10L)
                   .withTable(tableName));
       pipeline.run();
-      assertRowCount(tableName, rowsToAdd);
+      assertRowCount(DATA_SOURCE, tableName, rowsToAdd);
     } finally {
       DatabaseTestHelper.deleteTable(DATA_SOURCE, tableName);
     }
@@ -728,7 +751,7 @@
                   .withBatchSize(10L)
                   .withTable(tableName));
       pipeline.run();
-      assertRowCount(tableName, rowsToAdd);
+      assertRowCount(DATA_SOURCE, tableName, rowsToAdd);
     } finally {
       DatabaseTestHelper.deleteTable(DATA_SOURCE, tableName);
     }
@@ -1027,7 +1050,7 @@
         });
 
     // Since the pipeline was unable to write, only the row from insertStatement was written.
-    assertRowCount(tableName, 1);
+    assertRowCount(DATA_SOURCE, tableName, 1);
   }
 
   @Test
@@ -1064,8 +1087,8 @@
 
       pipeline.run();
 
-      assertRowCount(firstTableName, EXPECTED_ROW_COUNT);
-      assertRowCount(secondTableName, EXPECTED_ROW_COUNT);
+      assertRowCount(DATA_SOURCE, firstTableName, EXPECTED_ROW_COUNT);
+      assertRowCount(DATA_SOURCE, secondTableName, EXPECTED_ROW_COUNT);
     } finally {
       DatabaseTestHelper.deleteTable(DATA_SOURCE, firstTableName);
       DatabaseTestHelper.deleteTable(DATA_SOURCE, secondTableName);
diff --git a/sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/JdbcTestHelper.java b/sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/JdbcTestHelper.java
index e929a5b..081f8af 100644
--- a/sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/JdbcTestHelper.java
+++ b/sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/JdbcTestHelper.java
@@ -17,9 +17,18 @@
  */
 package org.apache.beam.sdk.io.jdbc;
 
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.io.Serializable;
 import java.sql.PreparedStatement;
 import java.sql.ResultSet;
 import java.sql.SQLException;
+import java.util.Objects;
+import org.apache.beam.sdk.coders.BigEndianIntegerCoder;
+import org.apache.beam.sdk.coders.Coder;
+import org.apache.beam.sdk.coders.CoderException;
+import org.apache.beam.sdk.coders.CustomCoder;
 import org.apache.beam.sdk.io.common.TestRow;
 
 /**
@@ -28,6 +37,54 @@
  */
 class JdbcTestHelper {
 
+  public static class TestDto extends JdbcWriteResult implements Serializable {
+
+    public static final int EMPTY_RESULT = 0;
+
+    private int simpleField;
+
+    public TestDto(int simpleField) {
+      this.simpleField = simpleField;
+    }
+
+    @Override
+    public boolean equals(Object o) {
+      if (this == o) {
+        return true;
+      }
+      if (o == null || getClass() != o.getClass()) {
+        return false;
+      }
+      TestDto testDto = (TestDto) o;
+      return simpleField == testDto.simpleField;
+    }
+
+    @Override
+    public int hashCode() {
+      return Objects.hash(simpleField);
+    }
+  }
+
+  public static final Coder<TestDto> TEST_DTO_CODER =
+      new CustomCoder<TestDto>() {
+        @Override
+        public void encode(TestDto value, OutputStream outStream)
+            throws CoderException, IOException {
+          BigEndianIntegerCoder.of().encode(value.simpleField, outStream);
+        }
+
+        @Override
+        public TestDto decode(InputStream inStream) throws CoderException, IOException {
+          int simpleField = BigEndianIntegerCoder.of().decode(inStream);
+          return new TestDto(simpleField);
+        }
+
+        @Override
+        public Object structuralValue(TestDto v) {
+          return v;
+        }
+      };
+
   static class CreateTestRowOfNameAndId implements JdbcIO.RowMapper<TestRow> {
     @Override
     public TestRow mapRow(ResultSet resultSet) throws Exception {
diff --git a/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/MongoDbIO.java b/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/MongoDbIO.java
index fa0065e..8c7f03b 100644
--- a/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/MongoDbIO.java
+++ b/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/MongoDbIO.java
@@ -31,11 +31,17 @@
 import com.mongodb.client.MongoCursor;
 import com.mongodb.client.MongoDatabase;
 import com.mongodb.client.model.Aggregates;
+import com.mongodb.client.model.BulkWriteOptions;
 import com.mongodb.client.model.Filters;
 import com.mongodb.client.model.InsertManyOptions;
+import com.mongodb.client.model.UpdateOneModel;
+import com.mongodb.client.model.UpdateOptions;
+import com.mongodb.client.model.WriteModel;
 import java.util.ArrayList;
 import java.util.Collections;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 import java.util.stream.Collectors;
 import javax.net.ssl.SSLContext;
 import org.apache.beam.sdk.annotations.Experimental;
@@ -93,8 +99,8 @@
  *
  * <p>MongoDB sink supports writing of Document (as JSON String) in a MongoDB.
  *
- * <p>To configure a MongoDB sink, you must specify a connection {@code URI}, a {@code Database}
- * name, a {@code Collection} name. For instance:
+ * <p>To configure a MongoDB sink and insert/replace, you must specify a connection {@code URI}, a
+ * {@code Database} name, a {@code Collection} name. For instance:
  *
  * <pre>{@code
  * pipeline
@@ -106,6 +112,27 @@
  *     .withNumSplits(30))
  *
  * }</pre>
+ *
+ * *
+ *
+ * <p>To configure a MongoDB sink and update, you must specify a connection {@code URI}, a {@code
+ * Database} * name, a {@code Collection} name. It matches the key with _id in target collection.
+ * For instance: * *
+ *
+ * <pre>{@code
+ * * pipeline
+ * *   .apply(...)
+ * *   .apply(MongoDbIO.write()
+ * *     .withUri("mongodb://localhost:27017")
+ * *     .withDatabase("my-database")
+ * *     .withCollection("my-collection")
+ * *     .withUpdateConfiguration(UpdateConfiguration.create().withUpdateKey("key1")
+ * *     .withUpdateFields(UpdateField.fieldUpdate("$set", "source-field1", "dest-field1"),
+ * *                       UpdateField.fieldUpdate("$set","source-field2", "dest-field2"),
+ * *                       //pushes entire input doc to the dest field
+ * *                        UpdateField.fullUpdate("$push", "dest-field3") )));
+ * *
+ * }</pre>
  */
 @Experimental(Kind.SOURCE_SINK)
 @SuppressWarnings({
@@ -490,7 +517,6 @@
               .anyMatch(s -> s.keySet().contains("$limit"))) {
             return Collections.singletonList(this);
           }
-
           splitKeys = buildAutoBuckets(mongoDatabase, spec);
 
           for (BsonDocument shardFilter : splitKeysToMatch(splitKeys)) {
@@ -744,6 +770,8 @@
 
     abstract long batchSize();
 
+    abstract @Nullable UpdateConfiguration updateConfiguration();
+
     abstract Builder builder();
 
     @AutoValue.Builder
@@ -766,6 +794,8 @@
 
       abstract Builder setBatchSize(long batchSize);
 
+      abstract Builder setUpdateConfiguration(UpdateConfiguration updateConfiguration);
+
       abstract Write build();
     }
 
@@ -856,6 +886,10 @@
       return builder().setBatchSize(batchSize).build();
     }
 
+    public Write withUpdateConfiguration(UpdateConfiguration updateConfiguration) {
+      return builder().setUpdateConfiguration(updateConfiguration).build();
+    }
+
     @Override
     public PDone expand(PCollection<Document> input) {
       checkArgument(uri() != null, "withUri() is required");
@@ -910,6 +944,7 @@
       public void processElement(ProcessContext ctx) {
         // Need to copy the document because mongoCollection.insertMany() will mutate it
         // before inserting (will assign an id).
+
         batch.add(new Document(ctx.element()));
         if (batch.size() >= spec.batchSize()) {
           flush();
@@ -927,6 +962,15 @@
         }
         MongoDatabase mongoDatabase = client.getDatabase(spec.database());
         MongoCollection<Document> mongoCollection = mongoDatabase.getCollection(spec.collection());
+        if (spec.updateConfiguration() == null) {
+          insertDocuments(mongoCollection);
+        } else {
+          updateDocuments(mongoCollection);
+        }
+        batch.clear();
+      }
+
+      private void insertDocuments(MongoCollection<Document> mongoCollection) {
         try {
           mongoCollection.insertMany(batch, new InsertManyOptions().ordered(spec.ordered()));
         } catch (MongoBulkWriteException e) {
@@ -934,8 +978,57 @@
             throw e;
           }
         }
+      }
 
-        batch.clear();
+      private void updateDocuments(MongoCollection<Document> mongoCollection) {
+        if (batch.isEmpty()) {
+          return;
+        }
+        List<WriteModel<Document>> actions = new ArrayList<>();
+        @Nullable List<UpdateField> updateFields = spec.updateConfiguration().updateFields();
+        Map<String, List<UpdateField>> operatorFieldsMap = getOperatorFieldsMap(updateFields);
+        try {
+          for (Document doc : batch) {
+            Document updateDocument = new Document();
+            for (Map.Entry<String, List<UpdateField>> entry : operatorFieldsMap.entrySet()) {
+              Document updateSubDocument = new Document();
+              for (UpdateField field : entry.getValue()) {
+                updateSubDocument.append(
+                    field.destField(),
+                    field.sourceField() == null ? doc : doc.get(field.sourceField()));
+              }
+              updateDocument.append(entry.getKey(), updateSubDocument);
+            }
+            Document findCriteria =
+                new Document("_id", doc.get(spec.updateConfiguration().updateKey()));
+            UpdateOptions updateOptions =
+                new UpdateOptions().upsert(spec.updateConfiguration().isUpsert());
+            actions.add(new UpdateOneModel<>(findCriteria, updateDocument, updateOptions));
+          }
+          mongoCollection.bulkWrite(actions, new BulkWriteOptions().ordered(spec.ordered()));
+        } catch (MongoBulkWriteException e) {
+          if (spec.ordered()) {
+            throw e;
+          }
+        }
+      }
+
+      private static Map<String, List<UpdateField>> getOperatorFieldsMap(
+          List<UpdateField> updateFields) {
+        Map<String, List<UpdateField>> operatorFieldsMap = new HashMap<>();
+        for (UpdateField field : updateFields) {
+          String updateOperator = field.updateOperator();
+          if (operatorFieldsMap.containsKey(updateOperator)) {
+            List<UpdateField> fields = operatorFieldsMap.get(updateOperator);
+            fields.add(field);
+            operatorFieldsMap.put(updateOperator, fields);
+          } else {
+            List<UpdateField> fields = new ArrayList<>();
+            fields.add(field);
+            operatorFieldsMap.put(updateOperator, fields);
+          }
+        }
+        return operatorFieldsMap;
       }
 
       @Teardown
diff --git a/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/UpdateConfiguration.java b/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/UpdateConfiguration.java
new file mode 100644
index 0000000..cda037d
--- /dev/null
+++ b/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/UpdateConfiguration.java
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.io.mongodb;
+
+import com.google.auto.value.AutoValue;
+import java.io.Serializable;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.annotations.Experimental.Kind;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+/** Builds a MongoDB UpdateConfiguration object. */
+@Experimental(Kind.SOURCE_SINK)
+@AutoValue
+@SuppressWarnings({
+  "nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
+})
+public abstract class UpdateConfiguration implements Serializable {
+
+  abstract @Nullable String updateKey();
+
+  abstract @Nullable List<UpdateField> updateFields();
+
+  abstract boolean isUpsert();
+
+  private static Builder builder() {
+    return new AutoValue_UpdateConfiguration.Builder()
+        .setUpdateFields(Collections.emptyList())
+        .setIsUpsert(false);
+  }
+
+  abstract Builder toBuilder();
+
+  public static UpdateConfiguration create() {
+    return builder().build();
+  }
+
+  @AutoValue.Builder
+  abstract static class Builder {
+    abstract Builder setUpdateFields(@Nullable List<UpdateField> updateFields);
+
+    abstract Builder setUpdateKey(@Nullable String updateKey);
+
+    abstract Builder setIsUpsert(boolean isUpsert);
+
+    abstract UpdateConfiguration build();
+  }
+
+  /**
+   * Sets the configurations for multiple updates. Takes update operator, source field name and dest
+   * field name for each one
+   */
+  public UpdateConfiguration withUpdateFields(UpdateField... updateFields) {
+    return toBuilder().setUpdateFields(Arrays.asList(updateFields)).build();
+  }
+
+  /** Sets the filters to find. */
+  public UpdateConfiguration withUpdateKey(String updateKey) {
+    return toBuilder().setUpdateKey(updateKey).build();
+  }
+
+  public UpdateConfiguration withIsUpsert(boolean isUpsert) {
+    return toBuilder().setIsUpsert(isUpsert).build();
+  }
+}
diff --git a/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/UpdateField.java b/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/UpdateField.java
new file mode 100644
index 0000000..a1c8423
--- /dev/null
+++ b/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/UpdateField.java
@@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.beam.sdk.io.mongodb;
+
+import com.google.auto.value.AutoValue;
+import java.io.Serializable;
+import org.apache.beam.sdk.annotations.Experimental;
+import org.apache.beam.sdk.annotations.Experimental.Kind;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+@Experimental(Kind.SOURCE_SINK)
+@AutoValue
+public abstract class UpdateField implements Serializable {
+
+  abstract @Nullable String updateOperator();
+
+  abstract @Nullable String sourceField();
+
+  abstract @Nullable String destField();
+
+  private static Builder builder() {
+    return new AutoValue_UpdateField.Builder().setSourceField(null);
+  }
+
+  abstract UpdateField.Builder toBuilder();
+
+  private static UpdateField create() {
+    return builder().build();
+  }
+
+  @AutoValue.Builder
+  abstract static class Builder {
+    abstract UpdateField.Builder setUpdateOperator(@Nullable String updateOperator);
+
+    abstract UpdateField.Builder setSourceField(@Nullable String sourceField);
+
+    abstract UpdateField.Builder setDestField(@Nullable String destField);
+
+    abstract UpdateField build();
+  }
+
+  /** Sets the limit of documents to find. */
+  public static UpdateField fullUpdate(String updateOperator, String destField) {
+    return create().toBuilder().setUpdateOperator(updateOperator).setDestField(destField).build();
+  }
+
+  public static UpdateField fieldUpdate(
+      String updateOperator, String sourceField, String destField) {
+    return create()
+        .toBuilder()
+        .setUpdateOperator(updateOperator)
+        .setSourceField(sourceField)
+        .setDestField(destField)
+        .build();
+  }
+}
diff --git a/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/MongoDbIOTest.java b/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/MongoDbIOTest.java
index 89af792..3e2b62c 100644
--- a/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/MongoDbIOTest.java
+++ b/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/MongoDbIOTest.java
@@ -103,7 +103,7 @@
     client = new MongoClient("localhost", port);
 
     LOG.info("Insert test data");
-    List<Document> documents = createDocuments(1000);
+    List<Document> documents = createDocuments(1000, false);
     MongoCollection<Document> collection = getCollection(COLLECTION);
     collection.insertMany(documents);
   }
@@ -329,7 +329,7 @@
     final int numElements = 1000;
 
     pipeline
-        .apply(Create.of(createDocuments(numElements)))
+        .apply(Create.of(createDocuments(numElements, false)))
         .apply(
             MongoDbIO.write()
                 .withUri("mongodb://localhost:" + port)
@@ -361,7 +361,37 @@
     assertEquals(1, countElements(collectionName));
   }
 
-  private static List<Document> createDocuments(final int n) {
+  @Test
+  public void testUpdate() {
+    final String collectionName = "testUpdate";
+    final int numElements = 100;
+    Document doc = Document.parse("{\"id\":1,\"scientist\":\"Updated\",\"country\":\"India\"}");
+
+    getCollection(collectionName).insertMany(createDocuments(numElements, true));
+    assertEquals(numElements, countElements(collectionName));
+    List<Document> docs = new ArrayList<>();
+    docs.add(doc);
+    pipeline
+        .apply(Create.of(docs))
+        .apply(
+            MongoDbIO.write()
+                .withUri("mongodb://localhost:" + port)
+                .withDatabase(DATABASE)
+                .withCollection(collectionName)
+                .withUpdateConfiguration(
+                    UpdateConfiguration.create()
+                        .withUpdateKey("id")
+                        .withUpdateFields(
+                            UpdateField.fieldUpdate("$set", "scientist", "scientist"),
+                            UpdateField.fieldUpdate("$set", "country", "country"))));
+    pipeline.run();
+
+    Document out = getCollection(collectionName).find(new Document("_id", 1)).first();
+    assertEquals("Updated", out.get("scientist"));
+    assertEquals("India", out.get("country"));
+  }
+
+  private static List<Document> createDocuments(final int n, boolean addId) {
     final String[] scientists =
         new String[] {
           "Einstein",
@@ -392,6 +422,9 @@
     for (int i = 1; i <= n; i++) {
       int index = i % scientists.length;
       Document document = new Document();
+      if (addId) {
+        document.append("_id", i);
+      }
       document.append("scientist", scientists[index]);
       document.append("country", country[index]);
       documents.add(document);
diff --git a/sdks/java/testing/tpcds/build.gradle b/sdks/java/testing/tpcds/build.gradle
index 6fd19ce..931b758 100644
--- a/sdks/java/testing/tpcds/build.gradle
+++ b/sdks/java/testing/tpcds/build.gradle
@@ -56,7 +56,7 @@
 dependencies {
     compile library.java.avro
     compile library.java.vendored_guava_26_0_jre
-    compile library.java.vendored_calcite_1_20_0
+    compile library.java.vendored_calcite_1_26_0
     compile library.java.commons_csv
     compile library.java.slf4j_api
     compile "com.googlecode.json-simple:json-simple:1.1.1"
diff --git a/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/QueryReader.java b/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/QueryReader.java
index c6f3253..3400a8d 100644
--- a/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/QueryReader.java
+++ b/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/QueryReader.java
@@ -17,6 +17,10 @@
  */
 package org.apache.beam.sdk.tpcds;
 
+import java.util.Set;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlNode;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.parser.SqlParseException;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.parser.SqlParser;
 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.io.Resources;
 
@@ -37,4 +41,20 @@
     String path = "queries/" + queryFileName + ".sql";
     return Resources.toString(Resources.getResource(path), Charsets.UTF_8);
   }
+
+  /**
+   * Parse query and get all its identifiers.
+   *
+   * @param queryString
+   * @return Set of SQL query identifiers as strings.
+   * @throws SqlParseException
+   */
+  public static Set<String> getQueryIdentifiers(String queryString) throws SqlParseException {
+    SqlParser parser = SqlParser.create(queryString);
+    SqlNode parsedQuery = parser.parseQuery();
+    SqlTransformRunner.SqlIdentifierVisitor sqlVisitor =
+        new SqlTransformRunner.SqlIdentifierVisitor();
+    parsedQuery.accept(sqlVisitor);
+    return sqlVisitor.getIdentifiers();
+  }
 }
diff --git a/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/SqlTransformRunner.java b/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/SqlTransformRunner.java
index ad1714f..9d25f6a 100644
--- a/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/SqlTransformRunner.java
+++ b/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/SqlTransformRunner.java
@@ -20,8 +20,10 @@
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 import java.util.concurrent.CompletionService;
 import java.util.concurrent.ExecutorCompletionService;
 import java.util.concurrent.ExecutorService;
@@ -41,6 +43,8 @@
 import org.apache.beam.sdk.values.Row;
 import org.apache.beam.sdk.values.TupleTag;
 import org.apache.beam.sdk.values.TypeDescriptors;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.SqlIdentifier;
+import org.apache.beam.vendor.calcite.v1_26_0.org.apache.calcite.sql.util.SqlBasicVisitor;
 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.io.Resources;
 import org.apache.commons.csv.CSVFormat;
@@ -50,6 +54,8 @@
 /**
  * This class executes jobs using PCollection and SqlTransform, it uses SqlTransform.query to run
  * queries.
+ *
+ * <p>TODO: Add tests.
  */
 public class SqlTransformRunner {
   private static final String SUMMARY_START = "\n" + "TPC-DS Query Execution Summary:";
@@ -66,6 +72,21 @@
 
   private static final Logger LOG = LoggerFactory.getLogger(SqlTransformRunner.class);
 
+  /** This class is used to extract all SQL query identifiers. */
+  static class SqlIdentifierVisitor extends SqlBasicVisitor<Void> {
+    private final Set<String> identifiers = new HashSet<>();
+
+    public Set<String> getIdentifiers() {
+      return identifiers;
+    }
+
+    @Override
+    public Void visit(SqlIdentifier id) {
+      identifiers.addAll(id.names);
+      return null;
+    }
+  }
+
   /**
    * Get all tables (in the form of TextTable) needed for a specific query execution.
    *
@@ -82,17 +103,17 @@
     Map<String, Schema> schemaMap = TpcdsSchemas.getTpcdsSchemas();
     TpcdsOptions tpcdsOptions = pipeline.getOptions().as(TpcdsOptions.class);
     String dataSize = TpcdsParametersReader.getAndCheckDataSize(tpcdsOptions);
-    String queryString = QueryReader.readQuery(queryName);
+    Set<String> identifiers = QueryReader.getQueryIdentifiers(QueryReader.readQuery(queryName));
 
     PCollectionTuple tables = PCollectionTuple.empty(pipeline);
     for (Map.Entry<String, Schema> tableSchema : schemaMap.entrySet()) {
       String tableName = tableSchema.getKey();
 
-      // Only when queryString contains tableName, the table is relevant to this query and will be
-      // added. This can avoid reading unnecessary data files.
-      // TODO: Simple but not reliable way since table name can be any substring in a query and can
-      // give false positives
-      if (queryString.contains(tableName)) {
+      // Only when query identifiers contain tableName, the table is relevant to this query and will
+      // be added. This can avoid reading unnecessary data files.
+      if (identifiers.contains(tableName.toUpperCase())) {
+        Set<String> tableColumns = getTableColumns(identifiers, tableSchema);
+
         switch (tpcdsOptions.getSourceType()) {
           case CSV:
             {
@@ -104,7 +125,7 @@
           case PARQUET:
             {
               PCollection<GenericRecord> table =
-                  getTableParquet(pipeline, tpcdsOptions, dataSize, tableName);
+                  getTableParquet(pipeline, tpcdsOptions, dataSize, tableName, tableColumns);
               tables = tables.and(new TupleTag<>(tableName), table);
               break;
             }
@@ -117,10 +138,28 @@
     return tables;
   }
 
+  private static Set<String> getTableColumns(
+      Set<String> identifiers, Map.Entry<String, Schema> tableSchema) {
+    Set<String> tableColumns = new HashSet<>();
+    List<Schema.Field> fields = tableSchema.getValue().getFields();
+    for (Schema.Field field : fields) {
+      String fieldName = field.getName();
+      if (identifiers.contains(fieldName.toUpperCase())) {
+        tableColumns.add(fieldName);
+      }
+    }
+    return tableColumns;
+  }
+
   private static PCollection<GenericRecord> getTableParquet(
-      Pipeline pipeline, TpcdsOptions tpcdsOptions, String dataSize, String tableName)
+      Pipeline pipeline,
+      TpcdsOptions tpcdsOptions,
+      String dataSize,
+      String tableName,
+      Set<String> tableColumns)
       throws IOException {
     org.apache.avro.Schema schema = getAvroSchema(tableName);
+    org.apache.avro.Schema schemaProjected = getProjectedSchema(tableColumns, schema);
 
     String filepattern =
         tpcdsOptions.getDataDirectory() + "/" + dataSize + "/" + tableName + "/*.parquet";
@@ -130,7 +169,7 @@
         ParquetIO.read(schema)
             .from(filepattern)
             .withSplit()
-            // TODO: add .withProjection()
+            .withProjection(schemaProjected, schemaProjected)
             .withBeamSchemas(true));
   }
 
@@ -161,6 +200,21 @@
         .parse(Resources.toString(Resources.getResource(path), Charsets.UTF_8));
   }
 
+  static org.apache.avro.Schema getProjectedSchema(
+      Set<String> projectedFieldNames, org.apache.avro.Schema schema) {
+    List<org.apache.avro.Schema.Field> projectedFields = new ArrayList<>();
+    for (org.apache.avro.Schema.Field f : schema.getFields()) {
+      if (projectedFieldNames.contains(f.name())) {
+        projectedFields.add(
+            new org.apache.avro.Schema.Field(f.name(), f.schema(), f.doc(), f.defaultVal()));
+      }
+    }
+    org.apache.avro.Schema schemaProjected =
+        org.apache.avro.Schema.createRecord(schema.getName() + "_projected", "", "", false);
+    schemaProjected.setFields(projectedFields);
+    return schemaProjected;
+  }
+
   /**
    * Print the summary table after all jobs are finished.
    *
diff --git a/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/TpcdsOptionsRegistrar.java b/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/TpcdsOptionsRegistrar.java
index 40f4f86..fc59667 100644
--- a/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/TpcdsOptionsRegistrar.java
+++ b/sdks/java/testing/tpcds/src/main/java/org/apache/beam/sdk/tpcds/TpcdsOptionsRegistrar.java
@@ -20,7 +20,7 @@
 import com.google.auto.service.AutoService;
 import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.options.PipelineOptionsRegistrar;
-import org.apache.beam.vendor.calcite.v1_20_0.com.google.common.collect.ImmutableList;
+import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableList;
 
 /** {@link AutoService} registrar for {@link TpcdsOptions}. */
 @AutoService(PipelineOptionsRegistrar.class)
diff --git a/sdks/java/testing/tpcds/src/test/java/org/apache/beam/sdk/tpcds/QueryReaderTest.java b/sdks/java/testing/tpcds/src/test/java/org/apache/beam/sdk/tpcds/QueryReaderTest.java
index 42f7d5b..b21cdfa 100644
--- a/sdks/java/testing/tpcds/src/test/java/org/apache/beam/sdk/tpcds/QueryReaderTest.java
+++ b/sdks/java/testing/tpcds/src/test/java/org/apache/beam/sdk/tpcds/QueryReaderTest.java
@@ -19,6 +19,8 @@
 
 import static org.junit.Assert.assertEquals;
 
+import java.util.HashSet;
+import java.util.Set;
 import org.junit.Test;
 
 public class QueryReaderTest {
@@ -67,6 +69,32 @@
   }
 
   @Test
+  public void testQuery3Identifiers() throws Exception {
+    Set<String> expected = new HashSet<>();
+    expected.add("BRAND");
+    expected.add("BRAND_ID");
+    expected.add("D_DATE_SK");
+    expected.add("D_MOY");
+    expected.add("D_YEAR");
+    expected.add("DATE_DIM");
+    expected.add("DT");
+    expected.add("I_BRAND");
+    expected.add("I_BRAND_ID");
+    expected.add("I_ITEM_SK");
+    expected.add("I_MANUFACT_ID");
+    expected.add("ITEM");
+    expected.add("SS_EXT_SALES_PRICE");
+    expected.add("SS_ITEM_SK");
+    expected.add("SS_SOLD_DATE_SK");
+    expected.add("STORE_SALES");
+    expected.add("SUM_AGG");
+
+    String query3String = QueryReader.readQuery("query3");
+    Set<String> identifiers = QueryReader.getQueryIdentifiers(query3String);
+    assertEquals(expected, identifiers);
+  }
+
+  @Test
   public void testQuery4String() throws Exception {
     String query4String = QueryReader.readQuery("query4");
     String expected =
diff --git a/sdks/python/apache_beam/io/gcp/bigquery.py b/sdks/python/apache_beam/io/gcp/bigquery.py
index 4e5f639..638c639 100644
--- a/sdks/python/apache_beam/io/gcp/bigquery.py
+++ b/sdks/python/apache_beam/io/gcp/bigquery.py
@@ -270,7 +270,6 @@
 # pytype: skip-file
 
 import collections
-import io
 import itertools
 import json
 import logging
@@ -278,20 +277,13 @@
 import time
 import uuid
 from typing import Dict
-from typing import List
-from typing import Optional
 from typing import Union
 
-import avro.schema
-import fastavro
-from avro import io as avroio
-
 import apache_beam as beam
 from apache_beam import coders
 from apache_beam import pvalue
 from apache_beam.internal.gcp.json_value import from_json_value
 from apache_beam.internal.gcp.json_value import to_json_value
-from apache_beam.io import range_trackers
 from apache_beam.io.avroio import _create_avro_source as create_avro_source
 from apache_beam.io.filesystems import CompressionTypes
 from apache_beam.io.filesystems import FileSystems
@@ -332,7 +324,6 @@
 try:
   from apache_beam.io.gcp.internal.clients.bigquery import DatasetReference
   from apache_beam.io.gcp.internal.clients.bigquery import TableReference
-  import google.cloud.bigquery_storage_v1 as bq_storage
 except ImportError:
   DatasetReference = None
   TableReference = None
@@ -896,276 +887,6 @@
     return table.schema, metadata_list
 
 
-class _CustomBigQueryStorageSourceBase(BoundedSource):
-  """A base class for BoundedSource implementations which read from BigQuery
-  using the BigQuery Storage API.
-
-  Args:
-    table (str, TableReference): The ID of the table. The ID must contain only
-      letters ``a-z``, ``A-Z``, numbers ``0-9``, or underscores ``_``  If
-      **dataset** argument is :data:`None` then the table argument must
-      contain the entire table reference specified as:
-      ``'PROJECT:DATASET.TABLE'`` or must specify a TableReference.
-    dataset (str): Optional ID of the dataset containing this table or
-      :data:`None` if the table argument specifies a TableReference.
-    project (str): Optional ID of the project containing this table or
-      :data:`None` if the table argument specifies a TableReference.
-    selected_fields (List[str]): Optional List of names of the fields in the
-      table that should be read. If empty, all fields will be read. If the
-      specified field is a nested field, all the sub-fields in the field will be
-      selected. The output field order is unrelated to the order of fields in
-      selected_fields.
-    row_restriction (str): Optional SQL text filtering statement, similar to a
-      WHERE clause in a query. Aggregates are not supported. Restricted to a
-      maximum length for 1 MB.
-  """
-
-  # The maximum number of streams which will be requested when creating a read
-  # session, regardless of the desired bundle size.
-  MAX_SPLIT_COUNT = 10000
-  # The minimum number of streams which will be requested when creating a read
-  # session, regardless of the desired bundle size. Note that the server may
-  # still choose to return fewer than ten streams based on the layout of the
-  # table.
-  MIN_SPLIT_COUNT = 10
-
-  def __init__(
-      self,
-      table: Union[str, TableReference],
-      dataset: Optional[str] = None,
-      project: Optional[str] = None,
-      selected_fields: Optional[List[str]] = None,
-      row_restriction: Optional[str] = None,
-      use_fastavro_for_direct_read: Optional[bool] = None,
-      pipeline_options: Optional[GoogleCloudOptions] = None):
-
-    self.table_reference = bigquery_tools.parse_table_reference(
-        table, dataset, project)
-    self.table = self.table_reference.tableId
-    self.dataset = self.table_reference.datasetId
-    self.project = self.table_reference.projectId
-    self.selected_fields = selected_fields
-    self.row_restriction = row_restriction
-    self.use_fastavro = \
-      True if use_fastavro_for_direct_read is None else \
-      use_fastavro_for_direct_read
-    self.pipeline_options = pipeline_options
-    self.split_result = None
-
-  def _get_parent_project(self):
-    """Returns the project that will be billed."""
-    project = self.pipeline_options.view_as(GoogleCloudOptions).project
-    if isinstance(project, vp.ValueProvider):
-      project = project.get()
-    if not project:
-      project = self.project
-    return project
-
-  def _get_table_size(self, table, dataset, project):
-    if project is None:
-      project = self._get_parent_project()
-
-    bq = bigquery_tools.BigQueryWrapper()
-    table = bq.get_table(project, dataset, table)
-    return table.numBytes
-
-  def display_data(self):
-    return {
-        'project': str(self.project),
-        'dataset': str(self.dataset),
-        'table': str(self.table),
-        'selected_fields': str(self.selected_fields),
-        'row_restriction': str(self.row_restriction),
-        'use_fastavro': str(self.use_fastavro)
-    }
-
-  def estimate_size(self):
-    # Returns the pre-filtering size of the table being read.
-    return self._get_table_size(self.table, self.dataset, self.project)
-
-  def split(self, desired_bundle_size, start_position=None, stop_position=None):
-    requested_session = bq_storage.types.ReadSession()
-    requested_session.table = 'projects/{}/datasets/{}/tables/{}'.format(
-        self.project, self.dataset, self.table)
-    requested_session.data_format = bq_storage.types.DataFormat.AVRO
-    if self.selected_fields is not None:
-      requested_session.read_options.selected_fields = self.selected_fields
-    if self.row_restriction is not None:
-      requested_session.read_options.row_restriction = self.row_restriction
-
-    storage_client = bq_storage.BigQueryReadClient()
-    stream_count = 0
-    if desired_bundle_size > 0:
-      table_size = self._get_table_size(self.table, self.dataset, self.project)
-      stream_count = min(
-          int(table_size / desired_bundle_size),
-          _CustomBigQueryStorageSourceBase.MAX_SPLIT_COUNT)
-    stream_count = max(
-        stream_count, _CustomBigQueryStorageSourceBase.MIN_SPLIT_COUNT)
-
-    parent = 'projects/{}'.format(self.project)
-    read_session = storage_client.create_read_session(
-        parent=parent,
-        read_session=requested_session,
-        max_stream_count=stream_count)
-    _LOGGER.info(
-        'Sent BigQuery Storage API CreateReadSession request: \n %s \n'
-        'Received response \n %s.',
-        requested_session,
-        read_session)
-
-    self.split_result = [
-        _CustomBigQueryStorageStreamSource(stream.name, self.use_fastavro)
-        for stream in read_session.streams
-    ]
-
-    for source in self.split_result:
-      yield SourceBundle(
-          weight=1.0, source=source, start_position=None, stop_position=None)
-
-  def get_range_tracker(self, start_position, stop_position):
-    class NonePositionRangeTracker(RangeTracker):
-      """A RangeTracker that always returns positions as None. Prevents the
-      BigQuery Storage source from being read() before being split()."""
-      def start_position(self):
-        return None
-
-      def stop_position(self):
-        return None
-
-    return NonePositionRangeTracker()
-
-  def read(self, range_tracker):
-    raise NotImplementedError(
-        'BigQuery storage source must be split before being read')
-
-
-class _CustomBigQueryStorageStreamSource(BoundedSource):
-  """A source representing a single stream in a read session."""
-  def __init__(self, read_stream_name: str, use_fastavro: bool):
-    self.read_stream_name = read_stream_name
-    self.use_fastavro = use_fastavro
-
-  def display_data(self):
-    return {
-        'read_stream': str(self.read_stream_name),
-    }
-
-  def estimate_size(self):
-    # The size of stream source cannot be estimate due to server-side liquid
-    # sharding.
-    # TODO: Implement progress reporting.
-    return None
-
-  def split(self, desired_bundle_size, start_position=None, stop_position=None):
-    # A stream source can't be split without reading from it due to
-    # server-side liquid sharding. A split will simply return the current source
-    # for now.
-    return SourceBundle(
-        weight=1.0,
-        source=_CustomBigQueryStorageStreamSource(
-            self.read_stream_name, self.use_fastavro),
-        start_position=None,
-        stop_position=None)
-
-  def get_range_tracker(self, start_position, stop_position):
-    # TODO: Implement dynamic work rebalancing.
-    assert start_position is None
-    # Defaulting to the start of the stream.
-    start_position = 0
-    # Since the streams are unsplittable we choose OFFSET_INFINITY as the
-    # default end offset so that all data of the source gets read.
-    stop_position = range_trackers.OffsetRangeTracker.OFFSET_INFINITY
-    range_tracker = range_trackers.OffsetRangeTracker(
-        start_position, stop_position)
-    # Ensuring that all try_split() calls will be ignored by the Rangetracker.
-    range_tracker = range_trackers.UnsplittableRangeTracker(range_tracker)
-
-    return range_tracker
-
-  def read(self, range_tracker):
-    _LOGGER.info(
-        "Started BigQuery Storage API read from stream %s.",
-        self.read_stream_name)
-    storage_client = bq_storage.BigQueryReadClient()
-    read_rows_iterator = iter(storage_client.read_rows(self.read_stream_name))
-    # Handling the case where the user might provide very selective filters
-    # which can result in read_rows_response being empty.
-    first_read_rows_response = next(read_rows_iterator, None)
-    if first_read_rows_response is None:
-      return iter([])
-
-    if self.use_fastavro:
-      row_reader = _ReadRowsResponseReaderWithFastAvro(
-          read_rows_iterator, first_read_rows_response)
-      return iter(row_reader)
-
-    row_reader = _ReadRowsResponseReader(
-        read_rows_iterator, first_read_rows_response)
-    return iter(row_reader)
-
-
-class _ReadRowsResponseReaderWithFastAvro():
-  """An iterator that deserializes ReadRowsResponses using the fastavro
-  library."""
-  def __init__(self, read_rows_iterator, read_rows_response):
-    self.read_rows_iterator = read_rows_iterator
-    self.read_rows_response = read_rows_response
-    self.avro_schema = fastavro.parse_schema(
-        json.loads(self.read_rows_response.avro_schema.schema))
-    self.bytes_reader = io.BytesIO(
-        self.read_rows_response.avro_rows.serialized_binary_rows)
-
-  def __iter__(self):
-    return self
-
-  def __next__(self):
-    try:
-      return fastavro.schemaless_reader(self.bytes_reader, self.avro_schema)
-    except StopIteration:
-      self.read_rows_response = next(self.read_rows_iterator, None)
-      if self.read_rows_response is not None:
-        self.bytes_reader = io.BytesIO(
-            self.read_rows_response.avro_rows.serialized_binary_rows)
-        return fastavro.schemaless_reader(self.bytes_reader, self.avro_schema)
-      else:
-        raise StopIteration
-
-
-class _ReadRowsResponseReader():
-  """An iterator that deserializes ReadRowsResponses."""
-  def __init__(self, read_rows_iterator, read_rows_response):
-    self.read_rows_iterator = read_rows_iterator
-    self.read_rows_response = read_rows_response
-    self.avro_schema = avro.schema.Parse(
-        self.read_rows_response.avro_schema.schema)
-    self.reader = avroio.DatumReader(self.avro_schema)
-    self.decoder = avroio.BinaryDecoder(
-        io.BytesIO(self.read_rows_response.avro_rows.serialized_binary_rows))
-    self.next_row = 0
-
-  def __iter__(self):
-    return self
-
-  def get_deserialized_row(self):
-    deserialized_row = self.reader.read(self.decoder)
-    self.next_row += 1
-    return deserialized_row
-
-  def __next__(self):
-    if self.next_row < self.read_rows_response.row_count:
-      return self.get_deserialized_row()
-
-    self.read_rows_response = next(self.read_rows_iterator, None)
-    if self.read_rows_response is not None:
-      self.decoder = avroio.BinaryDecoder(
-          io.BytesIO(self.read_rows_response.avro_rows.serialized_binary_rows))
-      self.next_row = 0
-      return self.get_deserialized_row()
-    else:
-      raise StopIteration
-
-
 @deprecated(since='2.11.0', current="WriteToBigQuery")
 class BigQuerySink(dataflow_io.NativeSink):
   """A sink based on a BigQuery table.
@@ -1399,7 +1120,7 @@
         https://cloud.google.com/bigquery/streaming-data-into-bigquery#disabling_best_effort_de-duplication
       with_batched_input: Whether the input has already been batched per
         destination. If not, perform best-effort batching per destination within
-        a bunble.
+        a bundle.
     """
     self.schema = schema
     self.test_client = test_client
@@ -2120,38 +1841,19 @@
 class ReadFromBigQuery(PTransform):
   """Read data from BigQuery.
 
-    This PTransform uses either a BigQuery export job to take a snapshot of the
-    table on GCS, and then reads from each produced file (EXPORT) or reads
-    directly from BigQuery storage using BigQuery Read API (DIRECT_READ). The
-    option is specified using the 'method' :parameter. File format is Avro by
+    This PTransform uses a BigQuery export job to take a snapshot of the table
+    on GCS, and then reads from each produced file. File format is Avro by
     default.
 
-    NOTE: DIRECT_READ only supports reading from BigQuery Tables currently. To
-    read the results of a query please use EXPORT.
-
-  .. warning::
-      DATETIME columns are parsed as strings in the fastavro library. As a
-      result, such columns will be converted to Python strings instead of native
-      Python DATETIME types.
-
   Args:
-    method: The method to use to read from BigQuery. It may be EXPORT or
-      DIRECT_READ. EXPORT invokes a BigQuery export request
-      (https://cloud.google.com/bigquery/docs/exporting-data). DIRECT_READ reads
-      directly from BigQuery storage using the BigQuery Read API
-      (https://cloud.google.com/bigquery/docs/reference/storage). If
-      unspecified, the default is currently EXPORT.
-    use_fastavro_for_direct_read (bool): If method is `DIRECT_READ` and
-       :data:`True`, the fastavro library is used to deserialize the data
-       received from the BigQuery Read API. The default here is :data:`True`.
     table (str, callable, ValueProvider): The ID of the table, or a callable
       that returns it. The ID must contain only letters ``a-z``, ``A-Z``,
       numbers ``0-9``, or underscores ``_``. If dataset argument is
       :data:`None` then the table argument must contain the entire table
-      reference specified as: ``'PROJECT:DATASET.TABLE'``.
-      If it's a callable, it must receive one argument representing an element
-      to be written to BigQuery, and return a TableReference, or a string table
-      name as specified above.
+      reference specified as: ``'DATASET.TABLE'``
+      or ``'PROJECT:DATASET.TABLE'``. If it's a callable, it must receive one
+      argument representing an element to be written to BigQuery, and return
+      a TableReference, or a string table name as specified above.
     dataset (str): The ID of the dataset containing this table or
       :data:`None` if the table reference is specified entirely by the table
       argument.
@@ -2212,22 +1914,16 @@
         to create and delete tables within the given dataset. Dataset name
         should *not* start with the reserved prefix `beam_temp_dataset_`.
    """
-  class Method(object):
-    EXPORT = 'EXPORT'  #  This is currently the default.
-    DIRECT_READ = 'DIRECT_READ'
 
   COUNTER = 0
 
-  def __init__(self, gcs_location=None, method=None, *args, **kwargs):
-    self.method = method or ReadFromBigQuery.Method.EXPORT
-
-    if gcs_location and self.method is ReadFromBigQuery.Method.EXPORT:
+  def __init__(self, gcs_location=None, *args, **kwargs):
+    if gcs_location:
       if not isinstance(gcs_location, (str, ValueProvider)):
         raise TypeError(
             '%s: gcs_location must be of type string'
             ' or ValueProvider; got %r instead' %
             (self.__class__.__name__, type(gcs_location)))
-
       if isinstance(gcs_location, str):
         gcs_location = StaticValueProvider(str, gcs_location)
 
@@ -2236,17 +1932,6 @@
     self._kwargs = kwargs
 
   def expand(self, pcoll):
-    # TODO(BEAM-11115): Make ReadFromBQ rely on ReadAllFromBQ implementation.
-    if self.method is ReadFromBigQuery.Method.EXPORT:
-      return self._expand_export(pcoll)
-    elif self.method is ReadFromBigQuery.Method.DIRECT_READ:
-      return self._expand_direct_read(pcoll)
-    else:
-      raise ValueError(
-          'The method to read from BigQuery must be either EXPORT'
-          'or DIRECT_READ.')
-
-  def _expand_export(self, pcoll):
     temp_location = pcoll.pipeline.options.view_as(
         GoogleCloudOptions).temp_location
     job_name = pcoll.pipeline.options.view_as(GoogleCloudOptions).job_name
@@ -2281,15 +1966,6 @@
                 **self._kwargs))
         | _PassThroughThenCleanup(files_to_remove_pcoll))
 
-  def _expand_direct_read(self, pcoll):
-    return (
-        pcoll
-        | beam.io.Read(
-            _CustomBigQueryStorageSourceBase(
-                pipeline_options=pcoll.pipeline.options,
-                *self._args,
-                **self._kwargs)))
-
 
 class ReadFromBigQueryRequest:
   """
diff --git a/sdks/python/apache_beam/io/gcp/bigquery_read_it_test.py b/sdks/python/apache_beam/io/gcp/bigquery_read_it_test.py
index 53bf567..9216a9c 100644
--- a/sdks/python/apache_beam/io/gcp/bigquery_read_it_test.py
+++ b/sdks/python/apache_beam/io/gcp/bigquery_read_it_test.py
@@ -26,14 +26,12 @@
 import random
 import time
 import unittest
-import uuid
 from decimal import Decimal
 from functools import wraps
 
 import pytest
 
 import apache_beam as beam
-from apache_beam.io.gcp import bigquery_tools
 from apache_beam.io.gcp.bigquery_tools import BigQueryWrapper
 from apache_beam.io.gcp.internal.clients import bigquery
 from apache_beam.options.value_provider import StaticValueProvider
@@ -177,163 +175,6 @@
       assert_that(result, equal_to(self.TABLE_DATA))
 
 
-class ReadUsingStorageApiTests(BigQueryReadIntegrationTests):
-  TABLE_DATA = [{
-      'number': 1, 'str': 'abc'
-  }, {
-      'number': 2, 'str': 'def'
-  }, {
-      'number': 3, 'str': u'你好'
-  }, {
-      'number': 4, 'str': u'привет'
-  }]
-
-  @classmethod
-  def setUpClass(cls):
-    super(ReadUsingStorageApiTests, cls).setUpClass()
-    cls.table_name = 'python_read_table'
-    cls._create_table(cls.table_name)
-
-    table_id = '{}.{}'.format(cls.dataset_id, cls.table_name)
-    cls.query = 'SELECT number, str FROM `%s`' % table_id
-
-    # Materializing the newly created Table to ensure the Read API can stream.
-    cls.temp_table_reference = cls._execute_query(cls.project, cls.query)
-
-  @classmethod
-  def tearDownClass(cls):
-    cls.bigquery_client.clean_up_temporary_dataset(cls.project)
-    super(ReadUsingStorageApiTests, cls).tearDownClass()
-
-  @classmethod
-  def _create_table(cls, table_name):
-    table_schema = bigquery.TableSchema()
-    table_field = bigquery.TableFieldSchema()
-    table_field.name = 'number'
-    table_field.type = 'INTEGER'
-    table_schema.fields.append(table_field)
-    table_field = bigquery.TableFieldSchema()
-    table_field.name = 'str'
-    table_field.type = 'STRING'
-    table_schema.fields.append(table_field)
-    table = bigquery.Table(
-        tableReference=bigquery.TableReference(
-            projectId=cls.project, datasetId=cls.dataset_id,
-            tableId=table_name),
-        schema=table_schema)
-    request = bigquery.BigqueryTablesInsertRequest(
-        projectId=cls.project, datasetId=cls.dataset_id, table=table)
-    cls.bigquery_client.client.tables.Insert(request)
-    cls.bigquery_client.insert_rows(
-        cls.project, cls.dataset_id, table_name, cls.TABLE_DATA)
-
-  @classmethod
-  def _setup_temporary_dataset(cls, project, query):
-    location = cls.bigquery_client.get_query_location(project, query, False)
-    cls.bigquery_client.create_temporary_dataset(project, location)
-
-  @classmethod
-  def _execute_query(cls, project, query):
-    query_job_name = bigquery_tools.generate_bq_job_name(
-        'materializing_table_before_reading',
-        str(uuid.uuid4())[0:10],
-        bigquery_tools.BigQueryJobTypes.QUERY,
-        '%s_%s' % (int(time.time()), random.randint(0, 1000)))
-    cls._setup_temporary_dataset(cls.project, cls.query)
-    job = cls.bigquery_client._start_query_job(
-        project,
-        query,
-        use_legacy_sql=False,
-        flatten_results=False,
-        job_id=query_job_name)
-    job_ref = job.jobReference
-    cls.bigquery_client.wait_for_bq_job(job_ref, max_retries=0)
-    return cls.bigquery_client._get_temp_table(project)
-
-  def test_iobase_source(self):
-    with beam.Pipeline(argv=self.args) as p:
-      result = (
-          p | 'Read with BigQuery Storage API' >> beam.io.ReadFromBigQuery(
-              method=beam.io.ReadFromBigQuery.Method.DIRECT_READ,
-              project=self.temp_table_reference.projectId,
-              dataset=self.temp_table_reference.datasetId,
-              table=self.temp_table_reference.tableId,
-              use_fastavro_for_direct_read=True))
-      assert_that(result, equal_to(self.TABLE_DATA))
-
-  def test_iobase_source_without_fastavro(self):
-    with beam.Pipeline(argv=self.args) as p:
-      result = (
-          p | 'Read with BigQuery Storage API' >> beam.io.ReadFromBigQuery(
-              method=beam.io.ReadFromBigQuery.Method.DIRECT_READ,
-              project=self.temp_table_reference.projectId,
-              dataset=self.temp_table_reference.datasetId,
-              table=self.temp_table_reference.tableId,
-              use_fastavro_for_direct_read=False))
-      assert_that(result, equal_to(self.TABLE_DATA))
-
-  def test_iobase_source_with_column_selection(self):
-    EXPECTED_TABLE_DATA = [{
-        'number': 1
-    }, {
-        'number': 2
-    }, {
-        'number': 3
-    }, {
-        'number': 4
-    }]
-    with beam.Pipeline(argv=self.args) as p:
-      result = (
-          p | 'Read with BigQuery Storage API' >> beam.io.ReadFromBigQuery(
-              method=beam.io.ReadFromBigQuery.Method.DIRECT_READ,
-              project=self.temp_table_reference.projectId,
-              dataset=self.temp_table_reference.datasetId,
-              table=self.temp_table_reference.tableId,
-              selected_fields=['number']))
-      assert_that(result, equal_to(EXPECTED_TABLE_DATA))
-
-  def test_iobase_source_with_row_restriction(self):
-    EXPECTED_TABLE_DATA = [{
-        'number': 3, 'str': u'你好'
-    }, {
-        'number': 4, 'str': u'привет'
-    }]
-    with beam.Pipeline(argv=self.args) as p:
-      result = (
-          p | 'Read with BigQuery Storage API' >> beam.io.ReadFromBigQuery(
-              method=beam.io.ReadFromBigQuery.Method.DIRECT_READ,
-              project=self.temp_table_reference.projectId,
-              dataset=self.temp_table_reference.datasetId,
-              table=self.temp_table_reference.tableId,
-              row_restriction='number > 2'))
-      assert_that(result, equal_to(EXPECTED_TABLE_DATA))
-
-  def test_iobase_source_with_column_selection_and_row_restriction(self):
-    EXPECTED_TABLE_DATA = [{'str': u'你好'}, {'str': u'привет'}]
-    with beam.Pipeline(argv=self.args) as p:
-      result = (
-          p | 'Read with BigQuery Storage API' >> beam.io.ReadFromBigQuery(
-              method=beam.io.ReadFromBigQuery.Method.DIRECT_READ,
-              project=self.temp_table_reference.projectId,
-              dataset=self.temp_table_reference.datasetId,
-              table=self.temp_table_reference.tableId,
-              selected_fields=['str'],
-              row_restriction='number > 2'))
-      assert_that(result, equal_to(EXPECTED_TABLE_DATA))
-
-  def test_iobase_source_with_very_selective_filters(self):
-    with beam.Pipeline(argv=self.args) as p:
-      result = (
-          p | 'Read with BigQuery Storage API' >> beam.io.ReadFromBigQuery(
-              method=beam.io.ReadFromBigQuery.Method.DIRECT_READ,
-              project=self.temp_table_reference.projectId,
-              dataset=self.temp_table_reference.datasetId,
-              table=self.temp_table_reference.tableId,
-              selected_fields=['str'],
-              row_restriction='number > 4'))
-      assert_that(result, equal_to([]))
-
-
 class ReadNewTypesTests(BigQueryReadIntegrationTests):
   @classmethod
   def setUpClass(cls):
diff --git a/sdks/python/apache_beam/io/gcp/datastore/v1new/datastoreio.py b/sdks/python/apache_beam/io/gcp/datastore/v1new/datastoreio.py
index 8b05d30..34539ec 100644
--- a/sdks/python/apache_beam/io/gcp/datastore/v1new/datastoreio.py
+++ b/sdks/python/apache_beam/io/gcp/datastore/v1new/datastoreio.py
@@ -27,12 +27,15 @@
 import time
 
 from apache_beam import typehints
+from apache_beam.internal.metrics.metric import ServiceCallMetric
+from apache_beam.io.gcp import resource_identifiers
 from apache_beam.io.gcp.datastore.v1new import helper
 from apache_beam.io.gcp.datastore.v1new import query_splitter
 from apache_beam.io.gcp.datastore.v1new import types
 from apache_beam.io.gcp.datastore.v1new import util
 from apache_beam.io.gcp.datastore.v1new.adaptive_throttler import AdaptiveThrottler
 from apache_beam.io.gcp.datastore.v1new.rampup_throttling_fn import RampupThrottlingFn
+from apache_beam.metrics import monitoring_infos
 from apache_beam.metrics.metric import Metrics
 from apache_beam.transforms import Create
 from apache_beam.transforms import DoFn
@@ -41,6 +44,14 @@
 from apache_beam.transforms import Reshuffle
 from apache_beam.utils import retry
 
+# Protect against environments where datastore library is not available.
+# pylint: disable=wrong-import-order, wrong-import-position
+try:
+  from apitools.base.py.exceptions import HttpError
+  from google.api_core.exceptions import ClientError, GoogleAPICallError
+except ImportError:
+  pass
+
 __all__ = ['ReadFromDatastore', 'WriteToDatastore', 'DeleteFromDatastore']
 
 _LOGGER = logging.getLogger(__name__)
@@ -265,10 +276,33 @@
   class _QueryFn(DoFn):
     """A DoFn that fetches entities from Cloud Datastore, for a given query."""
     def process(self, query, *unused_args, **unused_kwargs):
+      if query.namespace is None:
+        query.namespace = ''
       _client = helper.get_client(query.project, query.namespace)
       client_query = query._to_client_query(_client)
-      for client_entity in client_query.fetch(query.limit):
-        yield types.Entity.from_client_entity(client_entity)
+      # Create request count metric
+      resource = resource_identifiers.DatastoreNamespace(
+          query.project, query.namespace)
+      labels = {
+          monitoring_infos.SERVICE_LABEL: 'Datastore',
+          monitoring_infos.METHOD_LABEL: 'BatchDatastoreRead',
+          monitoring_infos.RESOURCE_LABEL: resource,
+          monitoring_infos.DATASTORE_NAMESPACE_LABEL: query.namespace,
+          monitoring_infos.DATASTORE_PROJECT_ID_LABEL: query.project,
+          monitoring_infos.STATUS_LABEL: 'ok'
+      }
+      service_call_metric = ServiceCallMetric(
+          request_count_urn=monitoring_infos.API_REQUEST_COUNT_URN,
+          base_labels=labels)
+      try:
+        for client_entity in client_query.fetch(query.limit):
+          yield types.Entity.from_client_entity(client_entity)
+        service_call_metric.call('ok')
+      except (ClientError, GoogleAPICallError) as e:
+        # e.code.value contains the numeric http status code.
+        service_call_metric.call(e.code.value)
+      except HttpError as e:
+        service_call_metric.call(e)
 
 
 class _Mutate(PTransform):
@@ -395,17 +429,39 @@
         for element in self._batch_elements:
           self.add_to_batch(element)
 
+      # Create request count metric
+      resource = resource_identifiers.DatastoreNamespace(self._project, "")
+      labels = {
+          monitoring_infos.SERVICE_LABEL: 'Datastore',
+          monitoring_infos.METHOD_LABEL: 'BatchDatastoreWrite',
+          monitoring_infos.RESOURCE_LABEL: resource,
+          monitoring_infos.DATASTORE_NAMESPACE_LABEL: "",
+          monitoring_infos.DATASTORE_PROJECT_ID_LABEL: self._project,
+          monitoring_infos.STATUS_LABEL: 'ok'
+      }
+
+      service_call_metric = ServiceCallMetric(
+          request_count_urn=monitoring_infos.API_REQUEST_COUNT_URN,
+          base_labels=labels)
+
       try:
         start_time = time.time()
         self._batch.commit()
         end_time = time.time()
+        service_call_metric.call('ok')
 
         rpc_stats_callback(successes=1)
         throttler.successful_request(start_time * 1000)
         commit_time_ms = int((end_time - start_time) * 1000)
         return commit_time_ms
-      except Exception:
+      except (ClientError, GoogleAPICallError) as e:
         self._batch = None
+        # e.code.value contains the numeric http status code.
+        service_call_metric.call(e.code.value)
+        rpc_stats_callback(errors=1)
+        raise
+      except HttpError as e:
+        service_call_metric.call(e)
         rpc_stats_callback(errors=1)
         raise
 
diff --git a/sdks/python/apache_beam/io/gcp/datastore/v1new/datastoreio_test.py b/sdks/python/apache_beam/io/gcp/datastore/v1new/datastoreio_test.py
index dda9d02..603bcd0 100644
--- a/sdks/python/apache_beam/io/gcp/datastore/v1new/datastoreio_test.py
+++ b/sdks/python/apache_beam/io/gcp/datastore/v1new/datastoreio_test.py
@@ -30,6 +30,7 @@
 
 # Protect against environments where datastore library is not available.
 try:
+  from apache_beam.io.gcp import resource_identifiers
   from apache_beam.io.gcp.datastore.v1new import helper, util
   from apache_beam.io.gcp.datastore.v1new import query_splitter
   from apache_beam.io.gcp.datastore.v1new import datastoreio
@@ -37,6 +38,8 @@
   from apache_beam.io.gcp.datastore.v1new.datastoreio import ReadFromDatastore
   from apache_beam.io.gcp.datastore.v1new.datastoreio import WriteToDatastore
   from apache_beam.io.gcp.datastore.v1new.types import Key
+  from apache_beam.metrics import monitoring_infos
+  from apache_beam.metrics.execution import MetricsEnvironment
   from google.cloud.datastore import client
   from google.cloud.datastore import entity
   from google.cloud.datastore import helpers
@@ -109,7 +112,7 @@
     mock_throttler = MagicMock()
     rpc_stats_callback = MagicMock()
     mock_throttler.throttle_request.return_value = []
-    mutate = datastoreio._Mutate.DatastoreMutateFn(lambda: None)
+    mutate = datastoreio._Mutate.DatastoreMutateFn("")
     mutate._batch = mock_batch
     mutate.write_mutations(mock_throttler, rpc_stats_callback)
     rpc_stats_callback.assert_has_calls([
@@ -126,7 +129,7 @@
     mock_throttler = MagicMock()
     rpc_stats_callback = MagicMock()
     mock_throttler.throttle_request.return_value = []
-    mutate = datastoreio._Mutate.DatastoreMutateFn(lambda: None)
+    mutate = datastoreio._Mutate.DatastoreMutateFn("")
     mutate._batch = mock_batch
     mutate._client = MagicMock()
     mutate._batch_elements = [None]
@@ -149,7 +152,7 @@
     # First try: throttle once [True, False]
     # Second try: no throttle [False]
     mock_throttler.throttle_request.side_effect = [True, False, False]
-    mutate = datastoreio._Mutate.DatastoreMutateFn(lambda: None)
+    mutate = datastoreio._Mutate.DatastoreMutateFn("")
     mutate._batch = mock_batch
     mutate._batch_elements = []
     mutate._client = MagicMock()
@@ -170,13 +173,57 @@
     mock_throttler = MagicMock()
     rpc_stats_callback = MagicMock()
     mock_throttler.throttle_request.return_value = False
-    mutate = datastoreio._Mutate.DatastoreMutateFn(lambda: None)
+    mutate = datastoreio._Mutate.DatastoreMutateFn("")
     mutate._batch = mock_batch
     with self.assertRaises(exceptions.InvalidArgument):
       mutate.write_mutations(
           mock_throttler, rpc_stats_callback, throttle_delay=0)
     rpc_stats_callback.assert_called_once_with(errors=1)
 
+  def test_write_mutations_metric_on_failure(self):
+    MetricsEnvironment.process_wide_container().reset()
+    mock_batch = MagicMock()
+    mock_batch.commit.side_effect = [
+        exceptions.DeadlineExceeded("Deadline Exceeded"), []
+    ]
+    mock_throttler = MagicMock()
+    rpc_stats_callback = MagicMock()
+    mock_throttler.throttle_request.return_value = False
+    mutate = datastoreio._Mutate.DatastoreMutateFn("my_project")
+    mutate._batch = mock_batch
+    mutate._batch_elements = []
+    mutate._client = MagicMock()
+    mutate.write_mutations(mock_throttler, rpc_stats_callback, throttle_delay=0)
+    self.verify_write_call_metric("my_project", "", "deadline_exceeded", 1)
+    self.verify_write_call_metric("my_project", "", "ok", 1)
+
+  def verify_write_call_metric(self, project_id, namespace, status, count):
+    """Check if a metric was recorded for the Datastore IO write API call."""
+    process_wide_monitoring_infos = list(
+        MetricsEnvironment.process_wide_container().
+        to_runner_api_monitoring_infos(None).values())
+    resource = resource_identifiers.DatastoreNamespace(project_id, namespace)
+    labels = {
+        monitoring_infos.SERVICE_LABEL: 'Datastore',
+        monitoring_infos.METHOD_LABEL: 'BatchDatastoreWrite',
+        monitoring_infos.RESOURCE_LABEL: resource,
+        monitoring_infos.DATASTORE_NAMESPACE_LABEL: namespace,
+        monitoring_infos.DATASTORE_PROJECT_ID_LABEL: project_id,
+        monitoring_infos.STATUS_LABEL: status
+    }
+    expected_mi = monitoring_infos.int64_counter(
+        monitoring_infos.API_REQUEST_COUNT_URN, count, labels=labels)
+    expected_mi.ClearField("start_time")
+
+    found = False
+    for actual_mi in process_wide_monitoring_infos:
+      actual_mi.ClearField("start_time")
+      if expected_mi == actual_mi:
+        found = True
+        break
+    self.assertTrue(
+        found, "Did not find write call metric with status: %s" % status)
+
 
 @unittest.skipIf(client is None, 'Datastore dependencies are not installed')
 class DatastoreioTest(unittest.TestCase):
@@ -270,6 +317,52 @@
           self.assertEqual(expected_num_splits, len(split_queries))
           self.assertEqual(self._mock_query, split_queries[0])
 
+  def test_QueryFn_metric_on_failure(self):
+    MetricsEnvironment.process_wide_container().reset()
+    with patch.object(helper, 'get_client', return_value=self._mock_client):
+      self._mock_query.project = self._PROJECT
+      self._mock_query.namespace = self._NAMESPACE
+      _query_fn = ReadFromDatastore._QueryFn()
+      client_query = self._mock_query._to_client_query()
+      # Test with exception
+      client_query.fetch.side_effect = [
+          exceptions.DeadlineExceeded("Deadline exceed")
+      ]
+      list(_query_fn.process(self._mock_query))
+      self.verify_read_call_metric(
+          self._PROJECT, self._NAMESPACE, "deadline_exceeded", 1)
+      # Test success
+      client_query.fetch.side_effect = [[]]
+      list(_query_fn.process(self._mock_query))
+      self.verify_read_call_metric(self._PROJECT, self._NAMESPACE, "ok", 1)
+
+  def verify_read_call_metric(self, project_id, namespace, status, count):
+    """Check if a metric was recorded for the Datastore IO read API call."""
+    process_wide_monitoring_infos = list(
+        MetricsEnvironment.process_wide_container().
+        to_runner_api_monitoring_infos(None).values())
+    resource = resource_identifiers.DatastoreNamespace(project_id, namespace)
+    labels = {
+        monitoring_infos.SERVICE_LABEL: 'Datastore',
+        monitoring_infos.METHOD_LABEL: 'BatchDatastoreRead',
+        monitoring_infos.RESOURCE_LABEL: resource,
+        monitoring_infos.DATASTORE_NAMESPACE_LABEL: namespace,
+        monitoring_infos.DATASTORE_PROJECT_ID_LABEL: project_id,
+        monitoring_infos.STATUS_LABEL: status
+    }
+    expected_mi = monitoring_infos.int64_counter(
+        monitoring_infos.API_REQUEST_COUNT_URN, count, labels=labels)
+    expected_mi.ClearField("start_time")
+
+    found = False
+    for actual_mi in process_wide_monitoring_infos:
+      actual_mi.ClearField("start_time")
+      if expected_mi == actual_mi:
+        found = True
+        break
+    self.assertTrue(
+        found, "Did not find read call metric with status: %s" % status)
+
   def check_DatastoreWriteFn(self, num_entities):
     """A helper function to test _DatastoreWriteFn."""
     with patch.object(helper, 'get_client', return_value=self._mock_client):
diff --git a/sdks/python/apache_beam/io/gcp/resource_identifiers.py b/sdks/python/apache_beam/io/gcp/resource_identifiers.py
index a85b069..a89a9e1 100644
--- a/sdks/python/apache_beam/io/gcp/resource_identifiers.py
+++ b/sdks/python/apache_beam/io/gcp/resource_identifiers.py
@@ -37,3 +37,8 @@
 
 def GoogleCloudStorageBucket(bucket_id):
   return '//storage.googleapis.com/buckets/%s' % bucket_id
+
+
+def DatastoreNamespace(project_id, namespace_id):
+  return '//bigtable.googleapis.com/projects/%s/namespaces/%s' % (
+      project_id, namespace_id)
diff --git a/sdks/python/apache_beam/metrics/monitoring_infos.py b/sdks/python/apache_beam/metrics/monitoring_infos.py
index 236ac81..2d8faa8 100644
--- a/sdks/python/apache_beam/metrics/monitoring_infos.py
+++ b/sdks/python/apache_beam/metrics/monitoring_infos.py
@@ -99,6 +99,10 @@
     common_urns.monitoring_info_labels.GCS_PROJECT_ID.label_props.name)
 GCS_BUCKET_LABEL = (
     common_urns.monitoring_info_labels.GCS_BUCKET.label_props.name)
+DATASTORE_PROJECT_ID_LABEL = (
+    common_urns.monitoring_info_labels.DATASTORE_PROJECT.label_props.name)
+DATASTORE_NAMESPACE_LABEL = (
+    common_urns.monitoring_info_labels.DATASTORE_NAMESPACE.label_props.name)
 
 
 def extract_counter_value(monitoring_info_proto):
diff --git a/sdks/python/apache_beam/runners/dataflow/internal/apiclient.py b/sdks/python/apache_beam/runners/dataflow/internal/apiclient.py
index 0265629..c60e88e 100644
--- a/sdks/python/apache_beam/runners/dataflow/internal/apiclient.py
+++ b/sdks/python/apache_beam/runners/dataflow/internal/apiclient.py
@@ -27,6 +27,8 @@
 import json
 import logging
 import os
+import random
+
 import pkg_resources
 import re
 import sys
@@ -496,6 +498,11 @@
         self.proto.labels.additionalProperties.append(
             dataflow.Job.LabelsValue.AdditionalProperty(key=key, value=value))
 
+    # Client Request ID
+    self.proto.clientRequestId = '{}-{}'.format(
+        datetime.utcnow().strftime('%Y%m%d%H%M%S%f'),
+        random.randrange(9000) + 1000)
+
     self.base64_str_re = re.compile(r'^[A-Za-z0-9+/]*=*$')
     self.coder_str_re = re.compile(r'^([A-Za-z]+\$)([A-Za-z0-9+/]*=*)$')
 
@@ -795,6 +802,20 @@
           self.google_cloud_options.dataflow_endpoint)
       _LOGGER.fatal('details of server error: %s', e)
       raise
+
+    if response.clientRequestId and \
+        response.clientRequestId != job.proto.clientRequestId:
+      if self.google_cloud_options.update:
+        raise DataflowJobAlreadyExistsError(
+            "The job named %s with id: %s has already been updated into job "
+            "id: %s and cannot be updated again." %
+            (response.name, job.proto.replaceJobId, response.id))
+      else:
+        raise DataflowJobAlreadyExistsError(
+            'There is already active job named %s with id: %s. If you want to '
+            'submit a second job, try again by setting a different name using '
+            '--job_name.' % (response.name, response.id))
+
     _LOGGER.info('Create job: %s', response)
     # The response is a Job proto with the id for the new job.
     _LOGGER.info('Created job with id: [%s]', response.id)
@@ -1029,6 +1050,13 @@
     return shared_names.BEAM_PACKAGE_NAME
 
 
+class DataflowJobAlreadyExistsError(retry.PermanentException):
+  """A non-retryable exception that a job with the given name already exists."""
+  # Inherits retry.PermanentException to avoid retry in
+  # DataflowApplicationClient.submit_job_description
+  pass
+
+
 def to_split_int(n):
   res = dataflow.SplitInt64()
   res.lowBits = n & 0xffffffff
@@ -1036,6 +1064,8 @@
   return res
 
 
+# TODO: Used in legacy batch worker. Move under MetricUpdateTranslators
+# after Runner V2 transition.
 def translate_distribution(distribution_update, metric_update_proto):
   """Translate metrics DistributionUpdate to dataflow distribution update.
 
@@ -1056,20 +1086,11 @@
   metric_update_proto.distribution = dist_update_proto
 
 
+# TODO: Used in legacy batch worker. Delete after Runner V2 transition.
 def translate_value(value, metric_update_proto):
   metric_update_proto.integer = to_split_int(value)
 
 
-def translate_mean(accumulator, metric_update):
-  if accumulator.count:
-    metric_update.meanSum = to_json_value(accumulator.sum, with_type=True)
-    metric_update.meanCount = to_json_value(accumulator.count, with_type=True)
-  else:
-    # A denominator of 0 will raise an error in the service.
-    # What it means is we have nothing to report yet, so don't.
-    metric_update.kind = None
-
-
 def _use_fnapi(pipeline_options):
   standard_options = pipeline_options.view_as(StandardOptions)
   debug_options = pipeline_options.view_as(DebugOptions)
diff --git a/sdks/python/apache_beam/runners/dataflow/internal/apiclient_test.py b/sdks/python/apache_beam/runners/dataflow/internal/apiclient_test.py
index a4f81a3..b63da70 100644
--- a/sdks/python/apache_beam/runners/dataflow/internal/apiclient_test.py
+++ b/sdks/python/apache_beam/runners/dataflow/internal/apiclient_test.py
@@ -1039,6 +1039,86 @@
               mock.ANY, "dataflow_graph.json", mock.ANY)
           client.create_job_description.assert_called_once()
 
+  def test_create_job_returns_existing_job(self):
+    pipeline_options = PipelineOptions([
+        '--project',
+        'test_project',
+        '--job_name',
+        'test_job_name',
+        '--temp_location',
+        'gs://test-location/temp',
+    ])
+    job = apiclient.Job(pipeline_options, FAKE_PIPELINE_URL)
+    self.assertTrue(job.proto.clientRequestId)  # asserts non-empty string
+    pipeline_options.view_as(GoogleCloudOptions).no_auth = True
+    client = apiclient.DataflowApplicationClient(pipeline_options)
+
+    response = dataflow.Job()
+    # different clientRequestId from `job`
+    response.clientRequestId = "20210821081910123456-1234"
+    response.name = 'test_job_name'
+    response.id = '2021-08-19_21_18_43-9756917246311111021'
+
+    with mock.patch.object(client._client.projects_locations_jobs,
+                           'Create',
+                           side_effect=[response]):
+      with mock.patch.object(client, 'create_job_description',
+                             side_effect=None):
+        with self.assertRaises(
+            apiclient.DataflowJobAlreadyExistsError) as context:
+          client.create_job(job)
+
+        self.assertEqual(
+            str(context.exception),
+            'There is already active job named %s with id: %s. If you want to '
+            'submit a second job, try again by setting a different name using '
+            '--job_name.' % ('test_job_name', response.id))
+
+  def test_update_job_returns_existing_job(self):
+    pipeline_options = PipelineOptions([
+        '--project',
+        'test_project',
+        '--job_name',
+        'test_job_name',
+        '--temp_location',
+        'gs://test-location/temp',
+        '--region',
+        'us-central1',
+        '--update',
+    ])
+    replace_job_id = '2021-08-21_00_00_01-6081497447916622336'
+    with mock.patch('apache_beam.runners.dataflow.internal.apiclient.Job.'
+                    'job_id_for_name',
+                    return_value=replace_job_id) as job_id_for_name_mock:
+      job = apiclient.Job(pipeline_options, FAKE_PIPELINE_URL)
+    job_id_for_name_mock.assert_called_once()
+
+    self.assertTrue(job.proto.clientRequestId)  # asserts non-empty string
+
+    pipeline_options.view_as(GoogleCloudOptions).no_auth = True
+    client = apiclient.DataflowApplicationClient(pipeline_options)
+
+    response = dataflow.Job()
+    # different clientRequestId from `job`
+    response.clientRequestId = "20210821083254123456-1234"
+    response.name = 'test_job_name'
+    response.id = '2021-08-19_21_29_07-5725551945600207770'
+
+    with mock.patch.object(client, 'create_job_description', side_effect=None):
+      with mock.patch.object(client._client.projects_locations_jobs,
+                             'Create',
+                             side_effect=[response]):
+
+        with self.assertRaises(
+            apiclient.DataflowJobAlreadyExistsError) as context:
+          client.create_job(job)
+
+      self.assertEqual(
+          str(context.exception),
+          'The job named %s with id: %s has already been updated into job '
+          'id: %s and cannot be updated again.' %
+          ('test_job_name', replace_job_id, response.id))
+
   def test_template_file_generation_with_upload_graph(self):
     pipeline_options = PipelineOptions([
         '--project',
diff --git a/sdks/python/apache_beam/runners/interactive/sql/__init__.py b/sdks/python/apache_beam/runners/interactive/sql/__init__.py
new file mode 100644
index 0000000..cce3aca
--- /dev/null
+++ b/sdks/python/apache_beam/runners/interactive/sql/__init__.py
@@ -0,0 +1,16 @@
+#
+# 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.
+#
diff --git a/sdks/python/apache_beam/runners/interactive/sql/beam_sql_magics.py b/sdks/python/apache_beam/runners/interactive/sql/beam_sql_magics.py
new file mode 100644
index 0000000..cee3d34
--- /dev/null
+++ b/sdks/python/apache_beam/runners/interactive/sql/beam_sql_magics.py
@@ -0,0 +1,293 @@
+#
+# 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.
+#
+
+"""Module of beam_sql cell magic that executes a Beam SQL.
+
+Only works within an IPython kernel.
+"""
+
+import importlib
+import keyword
+import logging
+from typing import Dict
+from typing import Optional
+from typing import Tuple
+from typing import Union
+
+import apache_beam as beam
+from apache_beam.pvalue import PValue
+from apache_beam.runners.interactive import cache_manager as cache
+from apache_beam.runners.interactive import interactive_beam as ib
+from apache_beam.runners.interactive import interactive_environment as ie
+from apache_beam.runners.interactive import pipeline_instrument as inst
+from apache_beam.runners.interactive.cache_manager import FileBasedCacheManager
+from apache_beam.runners.interactive.caching.streaming_cache import StreamingCache
+from apache_beam.runners.interactive.sql.utils import find_pcolls
+from apache_beam.runners.interactive.sql.utils import is_namedtuple
+from apache_beam.runners.interactive.sql.utils import pcolls_by_name
+from apache_beam.runners.interactive.sql.utils import register_coder_for_schema
+from apache_beam.runners.interactive.sql.utils import replace_single_pcoll_token
+from apache_beam.runners.interactive.utils import obfuscate
+from apache_beam.runners.interactive.utils import progress_indicated
+from apache_beam.testing import test_stream
+from apache_beam.testing.test_stream_service import TestStreamServiceController
+from apache_beam.transforms.sql import SqlTransform
+from IPython.core.magic import Magics
+from IPython.core.magic import cell_magic
+from IPython.core.magic import magics_class
+
+_LOGGER = logging.getLogger(__name__)
+
+_EXAMPLE_USAGE = """Usage:
+    %%%%beam_sql [output_name]
+    Calcite SQL statement
+    Syntax: https://beam.apache.org/documentation/dsls/sql/calcite/query-syntax/
+    Please make sure that there is no conflicts between your variable names and
+    the SQL keywords, such as "SELECT", "FROM", "WHERE" and etc.
+
+    output_name is optional. If not supplied, a variable name is automatically
+    assigned to the output of the magic.
+
+    The output of the magic is usually a PCollection or similar PValue,
+    depending on the SQL statement executed.
+"""
+
+
+def on_error(error_msg, *args):
+  """Logs the error and the usage example."""
+  _LOGGER.error(error_msg, *args)
+  _LOGGER.info(_EXAMPLE_USAGE)
+
+
+@magics_class
+class BeamSqlMagics(Magics):
+  @cell_magic
+  def beam_sql(self, line: str, cell: str) -> Union[None, PValue]:
+    """The beam_sql cell magic that executes a Beam SQL.
+
+    Args:
+      line: (optional) the string on the same line after the beam_sql magic.
+          Used as the output variable name in the __main__ module.
+      cell: everything else in the same notebook cell as a string. Used as a
+          Beam SQL query.
+
+    Returns None if running into an error, otherwise a PValue as if a
+    SqlTransform is applied.
+    """
+    if line and not line.strip().isidentifier() or keyword.iskeyword(
+        line.strip()):
+      on_error(
+          'The output_name "%s" is not a valid identifier. Please supply a '
+          'valid identifier that is not a Python keyword.',
+          line)
+      return
+    if not cell or cell.isspace():
+      on_error('Please supply the sql to be executed.')
+      return
+    found = find_pcolls(cell, pcolls_by_name())
+    for _, pcoll in found.items():
+      if not is_namedtuple(pcoll.element_type):
+        on_error(
+            'PCollection %s of type %s is not a NamedTuple. See '
+            'https://beam.apache.org/documentation/programming-guide/#schemas '
+            'for more details.',
+            pcoll,
+            pcoll.element_type)
+        return
+      register_coder_for_schema(pcoll.element_type)
+
+    # TODO(BEAM-10708): implicitly execute the pipeline and write output into
+    # cache.
+    return apply_sql(cell, line, found)
+
+
+@progress_indicated
+def apply_sql(
+    query: str, output_name: Optional[str],
+    found: Dict[str, beam.PCollection]) -> PValue:
+  """Applies a SqlTransform with the given sql and queried PCollections.
+
+  Args:
+    query: The SQL query executed in the magic.
+    output_name: (optional) The output variable name in __main__ module.
+    found: The PCollections with variable names found to be used in the query.
+
+  Returns:
+    A PValue, mostly a PCollection, depending on the query.
+  """
+  output_name = _generate_output_name(output_name, query, found)
+  query, sql_source = _build_query_components(query, found)
+  try:
+    output = sql_source | SqlTransform(query)
+    # Declare a variable with the output_name and output value in the
+    # __main__ module so that the user can use the output smoothly.
+    setattr(importlib.import_module('__main__'), output_name, output)
+    ib.watch({output_name: output})
+    _LOGGER.info(
+        "The output PCollection variable is %s: %s", output_name, output)
+    return output
+  except (KeyboardInterrupt, SystemExit):
+    raise
+  except Exception as e:
+    on_error('Error when applying the Beam SQL: %s', e)
+
+
+def pcoll_from_file_cache(
+    query_pipeline: beam.Pipeline,
+    pcoll: beam.PCollection,
+    cache_manager: FileBasedCacheManager,
+    key: str) -> beam.PCollection:
+  """Reads PCollection cache from files.
+
+  Args:
+    query_pipeline: The beam.Pipeline object built by the magic to execute the
+        SQL query.
+    pcoll: The PCollection to read cache for.
+    cache_manager: The file based cache manager that holds the PCollection
+        cache.
+    key: The key of the PCollection cache.
+
+  Returns:
+    A PCollection read from the cache.
+  """
+  schema = pcoll.element_type
+
+  class Unreify(beam.DoFn):
+    def process(self, e):
+      if isinstance(e, beam.Row) and hasattr(e, 'windowed_value'):
+        yield e.windowed_value
+
+  return (
+      query_pipeline
+      |
+      '{}{}'.format('QuerySource', key) >> cache.ReadCache(cache_manager, key)
+      | '{}{}'.format('Unreify', key) >> beam.ParDo(
+          Unreify()).with_output_types(schema))
+
+
+def pcolls_from_streaming_cache(
+    user_pipeline: beam.Pipeline,
+    query_pipeline: beam.Pipeline,
+    name_to_pcoll: Dict[str, beam.PCollection],
+    instrumentation: inst.PipelineInstrument,
+    cache_manager: StreamingCache) -> Dict[str, beam.PCollection]:
+  """Reads PCollection cache through the TestStream.
+
+  Args:
+    user_pipeline: The beam.Pipeline object defined by the user in the
+        notebook.
+    query_pipeline: The beam.Pipeline object built by the magic to execute the
+        SQL query.
+    name_to_pcoll: PCollections with variable names used in the SQL query.
+    instrumentation: A pipeline_instrument.PipelineInstrument that helps
+        calculate the cache key of a given PCollection.
+    cache_manager: The streaming cache manager that holds the PCollection cache.
+
+  Returns:
+    A Dict[str, beam.PCollection], where each PCollection is tagged with
+    their PCollection variable names, read from the cache.
+
+  When the user_pipeline has unbounded sources, we force all cache reads to go
+  through the TestStream even if they are bounded sources.
+  """
+  def exception_handler(e):
+    _LOGGER.error(str(e))
+    return True
+
+  test_stream_service = ie.current_env().get_test_stream_service_controller(
+      user_pipeline)
+  if not test_stream_service:
+    test_stream_service = TestStreamServiceController(
+        cache_manager, exception_handler=exception_handler)
+    test_stream_service.start()
+    ie.current_env().set_test_stream_service_controller(
+        user_pipeline, test_stream_service)
+
+  tag_to_name = {}
+  for name, pcoll in name_to_pcoll.items():
+    key = instrumentation.cache_key(pcoll)
+    tag_to_name[key] = name
+  output_pcolls = query_pipeline | test_stream.TestStream(
+      output_tags=set(tag_to_name.keys()),
+      coder=cache_manager._default_pcoder,
+      endpoint=test_stream_service.endpoint)
+  sql_source = {}
+  for tag, output in output_pcolls.items():
+    sql_source[tag_to_name[tag]] = output
+  return sql_source
+
+
+def _generate_output_name(
+    output_name: Optional[str], query: str,
+    found: Dict[str, beam.PCollection]) -> str:
+  """Generates a unique output name if None is provided.
+
+  Otherwise, returns the given output name directly.
+  The generated output name is sql_output_{uuid} where uuid is an obfuscated
+  value from the query and PCollections found to be used in the query.
+  """
+  if not output_name:
+    execution_id = obfuscate(query, found)[:12]
+    output_name = 'sql_output_' + execution_id
+  return output_name
+
+
+def _build_query_components(
+    query: str, found: Dict[str, beam.PCollection]
+) -> Tuple[str,
+           Union[Dict[str, beam.PCollection], beam.PCollection, beam.Pipeline]]:
+  """Builds necessary components needed to apply the SqlTransform.
+
+  Args:
+    query: The SQL query to be executed by the magic.
+    found: The PCollections with variable names found to be used by the query.
+
+  Returns:
+    The processed query to be executed by the magic and a source to apply the
+    SqlTransform to: a dictionary of tagged PCollections, or a single
+    PCollection, or the pipeline to execute the query.
+  """
+  if found:
+    user_pipeline = next(iter(found.values())).pipeline
+    cache_manager = ie.current_env().get_cache_manager(user_pipeline)
+    instrumentation = inst.build_pipeline_instrument(user_pipeline)
+    sql_pipeline = beam.Pipeline(options=user_pipeline._options)
+    ie.current_env().add_derived_pipeline(user_pipeline, sql_pipeline)
+    sql_source = {}
+    if instrumentation.has_unbounded_sources:
+      sql_source = pcolls_from_streaming_cache(
+          user_pipeline, sql_pipeline, found, instrumentation, cache_manager)
+    else:
+      for pcoll_name, pcoll in found.items():
+        cache_key = instrumentation.cache_key(pcoll)
+        sql_source[pcoll_name] = pcoll_from_file_cache(
+            sql_pipeline, pcoll, cache_manager, cache_key)
+    if len(sql_source) == 1:
+      query = replace_single_pcoll_token(query, next(iter(sql_source.keys())))
+      sql_source = next(iter(sql_source.values()))
+  else:
+    sql_source = beam.Pipeline()
+  return query, sql_source
+
+
+def load_ipython_extension(ipython):
+  """Marks this module as an IPython extension.
+
+  To load this magic in an IPython environment, execute:
+  %load_ext apache_beam.runners.interactive.sql.beam_sql_magics.
+  """
+  ipython.register_magics(BeamSqlMagics)
diff --git a/sdks/python/apache_beam/runners/interactive/sql/beam_sql_magics_test.py b/sdks/python/apache_beam/runners/interactive/sql/beam_sql_magics_test.py
new file mode 100644
index 0000000..d35bd46
--- /dev/null
+++ b/sdks/python/apache_beam/runners/interactive/sql/beam_sql_magics_test.py
@@ -0,0 +1,121 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#    http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+"""Tests for beam_sql_magics module."""
+
+# pytype: skip-file
+
+import unittest
+from unittest.mock import patch
+
+import pytest
+
+import apache_beam as beam
+from apache_beam.runners.interactive import interactive_beam as ib
+from apache_beam.runners.interactive import interactive_environment as ie
+
+try:
+  from apache_beam.runners.interactive.sql.beam_sql_magics import _build_query_components
+  from apache_beam.runners.interactive.sql.beam_sql_magics import _generate_output_name
+except (ImportError, NameError):
+  pass  # The test is to be skipped because [interactive] dep not installed.
+
+
+@unittest.skipIf(
+    not ie.current_env().is_interactive_ready,
+    '[interactive] dependency is not installed.')
+@pytest.mark.skipif(
+    not ie.current_env().is_interactive_ready,
+    reason='[interactive] dependency is not installed.')
+class BeamSqlMagicsTest(unittest.TestCase):
+  def test_generate_output_name_when_not_provided(self):
+    output_name = None
+    self.assertTrue(
+        _generate_output_name(output_name, '', {}).startswith('sql_output_'))
+
+  def test_use_given_output_name_when_provided(self):
+    output_name = 'output'
+    self.assertEqual(_generate_output_name(output_name, '', {}), output_name)
+
+  def test_build_query_components_when_no_pcoll_queried(self):
+    query = """SELECT CAST(1 AS INT) AS `id`,
+                      CAST('foo' AS VARCHAR) AS `str`,
+                      CAST(3.14  AS DOUBLE) AS `flt`"""
+    processed_query, sql_source = _build_query_components(query, {})
+    self.assertEqual(processed_query, query)
+    self.assertIsInstance(sql_source, beam.Pipeline)
+
+  def test_build_query_components_when_single_pcoll_queried(self):
+    p = beam.Pipeline()
+    target = p | beam.Create([1, 2, 3])
+    ib.watch(locals())
+    query = 'SELECT * FROM target where a=1'
+    found = {'target': target}
+
+    with patch('apache_beam.runners.interactive.sql.beam_sql_magics.'
+               'pcoll_from_file_cache',
+               lambda a,
+               b,
+               c,
+               d: target):
+      processed_query, sql_source = _build_query_components(query, found)
+
+      self.assertEqual(processed_query, 'SELECT * FROM PCOLLECTION where a=1')
+      self.assertIsInstance(sql_source, beam.PCollection)
+
+  def test_build_query_components_when_multiple_pcolls_queried(self):
+    p = beam.Pipeline()
+    pcoll_1 = p | 'Create 1' >> beam.Create([1, 2, 3])
+    pcoll_2 = p | 'Create 2' >> beam.Create([4, 5, 6])
+    ib.watch(locals())
+    query = 'SELECT * FROM pcoll_1 JOIN pcoll_2 USING (a)'
+    found = {'pcoll_1': pcoll_1, 'pcoll_2': pcoll_2}
+
+    with patch('apache_beam.runners.interactive.sql.beam_sql_magics.'
+               'pcoll_from_file_cache',
+               lambda a,
+               b,
+               c,
+               d: pcoll_1):
+      processed_query, sql_source = _build_query_components(query, found)
+
+      self.assertEqual(processed_query, query)
+      self.assertIsInstance(sql_source, dict)
+      self.assertIn('pcoll_1', sql_source)
+      self.assertIn('pcoll_2', sql_source)
+
+  def test_build_query_components_when_unbounded_pcolls_queried(self):
+    p = beam.Pipeline()
+    pcoll = p | beam.io.ReadFromPubSub(
+        subscription='projects/fake-project/subscriptions/fake_sub')
+    ib.watch(locals())
+    query = 'SELECT * FROM pcoll'
+    found = {'pcoll': pcoll}
+
+    with patch('apache_beam.runners.interactive.sql.beam_sql_magics.'
+               'pcolls_from_streaming_cache',
+               lambda a,
+               b,
+               c,
+               d,
+               e: found):
+      _, sql_source = _build_query_components(query, found)
+      self.assertIs(sql_source, pcoll)
+
+
+if __name__ == '__main__':
+  unittest.main()
diff --git a/sdks/python/apache_beam/runners/interactive/sql/utils.py b/sdks/python/apache_beam/runners/interactive/sql/utils.py
new file mode 100644
index 0000000..355b6e6
--- /dev/null
+++ b/sdks/python/apache_beam/runners/interactive/sql/utils.py
@@ -0,0 +1,125 @@
+#
+# 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.
+#
+
+"""Module of utilities for SQL magics.
+
+For internal use only; no backward-compatibility guarantees.
+"""
+
+# pytype: skip-file
+
+import logging
+from typing import Dict
+from typing import NamedTuple
+
+import apache_beam as beam
+from apache_beam.runners.interactive import interactive_beam as ib
+from apache_beam.runners.interactive import interactive_environment as ie
+
+_LOGGER = logging.getLogger(__name__)
+
+
+def is_namedtuple(cls: type) -> bool:
+  """Determines if a class is built from typing.NamedTuple."""
+  return (
+      isinstance(cls, type) and issubclass(cls, tuple) and
+      hasattr(cls, '_fields') and hasattr(cls, '_field_types'))
+
+
+def register_coder_for_schema(schema: NamedTuple) -> None:
+  """Registers a RowCoder for the given schema if hasn't.
+
+  Notifies the user of what code has been implicitly executed.
+  """
+  assert is_namedtuple(schema), (
+      'Schema %s is not a typing.NamedTuple.' % schema)
+  coder = beam.coders.registry.get_coder(schema)
+  if not isinstance(coder, beam.coders.RowCoder):
+    _LOGGER.warning(
+        'Schema %s has not been registered to use a RowCoder. '
+        'Automatically registering it by running: '
+        'beam.coders.registry.register_coder(%s, '
+        'beam.coders.RowCoder)',
+        schema.__name__,
+        schema.__name__)
+    beam.coders.registry.register_coder(schema, beam.coders.RowCoder)
+
+
+def pcolls_by_name() -> Dict[str, beam.PCollection]:
+  """Finds all PCollections by their variable names defined in the notebook."""
+  inspectables = ie.current_env().inspector.inspectables
+  pcolls = {}
+  for _, inspectable in inspectables.items():
+    metadata = inspectable['metadata']
+    if metadata['type'] == 'pcollection':
+      pcolls[metadata['name']] = inspectable['value']
+  return pcolls
+
+
+def find_pcolls(
+    sql: str, pcolls: Dict[str,
+                           beam.PCollection]) -> Dict[str, beam.PCollection]:
+  """Finds all PCollections used in the given sql query.
+
+  It does a simple word by word match and calls ib.collect for each PCollection
+  found.
+  """
+  found = {}
+  for word in sql.split():
+    if word in pcolls:
+      found[word] = pcolls[word]
+  if found:
+    _LOGGER.info('Found PCollections used in the magic: %s.', found)
+    _LOGGER.info('Collecting data...')
+    for name, pcoll in found.items():
+      try:
+        _ = ib.collect(pcoll)
+      except (KeyboardInterrupt, SystemExit):
+        raise
+      except:
+        _LOGGER.error(
+            'Cannot collect data for PCollection %s. Please make sure the '
+            'PCollections queried in the sql "%s" are all from a single '
+            'pipeline using an InteractiveRunner. Make sure there is no '
+            'ambiguity, for example, same named PCollections from multiple '
+            'pipelines or notebook re-executions.',
+            name,
+            sql)
+        raise
+    _LOGGER.info('Done collecting data.')
+  return found
+
+
+def replace_single_pcoll_token(sql: str, pcoll_name: str) -> str:
+  """Replaces the pcoll_name used in the sql with 'PCOLLECTION'.
+
+  For sql query using only a single PCollection, the PCollection needs to be
+  referred to as 'PCOLLECTION' instead of its variable/tag name.
+  """
+  words = sql.split()
+  token_locations = []
+  i = 0
+  for word in words:
+    if word.lower() == 'from':
+      token_locations.append(i + 1)
+      i += 2
+      continue
+    i += 1
+  for token_location in token_locations:
+    if token_location < len(words) and words[token_location] == pcoll_name:
+      words[token_location] = 'PCOLLECTION'
+  return ' '.join(words)
diff --git a/sdks/python/apache_beam/runners/interactive/sql/utils_test.py b/sdks/python/apache_beam/runners/interactive/sql/utils_test.py
new file mode 100644
index 0000000..ed52cad
--- /dev/null
+++ b/sdks/python/apache_beam/runners/interactive/sql/utils_test.py
@@ -0,0 +1,90 @@
+#
+# 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.
+#
+
+"""Tests for utils module."""
+
+# pytype: skip-file
+
+import unittest
+from typing import NamedTuple
+from unittest.mock import patch
+
+import apache_beam as beam
+from apache_beam.runners.interactive import interactive_beam as ib
+from apache_beam.runners.interactive.sql.utils import find_pcolls
+from apache_beam.runners.interactive.sql.utils import is_namedtuple
+from apache_beam.runners.interactive.sql.utils import pcolls_by_name
+from apache_beam.runners.interactive.sql.utils import register_coder_for_schema
+from apache_beam.runners.interactive.sql.utils import replace_single_pcoll_token
+
+
+class ANamedTuple(NamedTuple):
+  a: int
+  b: str
+
+
+class UtilsTest(unittest.TestCase):
+  def test_is_namedtuple(self):
+    class AType:
+      pass
+
+    a_type = AType
+    a_tuple = type((1, 2, 3))
+
+    a_namedtuple = ANamedTuple
+
+    self.assertTrue(is_namedtuple(a_namedtuple))
+    self.assertFalse(is_namedtuple(a_type))
+    self.assertFalse(is_namedtuple(a_tuple))
+
+  def test_register_coder_for_schema(self):
+    self.assertNotIsInstance(
+        beam.coders.registry.get_coder(ANamedTuple), beam.coders.RowCoder)
+    register_coder_for_schema(ANamedTuple)
+    self.assertIsInstance(
+        beam.coders.registry.get_coder(ANamedTuple), beam.coders.RowCoder)
+
+  def test_pcolls_by_name(self):
+    p = beam.Pipeline()
+    pcoll = p | beam.Create([1])
+    ib.watch({'p': p, 'pcoll': pcoll})
+
+    name_to_pcoll = pcolls_by_name()
+    self.assertIn('pcoll', name_to_pcoll)
+
+  def test_find_pcolls(self):
+    with patch('apache_beam.runners.interactive.interactive_beam.collect',
+               lambda _: None):
+      found = find_pcolls(
+          """SELECT * FROM pcoll_1 JOIN pcoll_2
+          USING (common_column)""", {
+              'pcoll_1': None, 'pcoll_2': None
+          })
+      self.assertIn('pcoll_1', found)
+      self.assertIn('pcoll_2', found)
+
+  def test_replace_single_pcoll_token(self):
+    sql = 'SELECT * FROM abc WHERE a=1 AND b=2'
+    replaced_sql = replace_single_pcoll_token(sql, 'wow')
+    self.assertEqual(replaced_sql, sql)
+    replaced_sql = replace_single_pcoll_token(sql, 'abc')
+    self.assertEqual(
+        replaced_sql, 'SELECT * FROM PCOLLECTION WHERE a=1 AND b=2')
+
+
+if __name__ == '__main__':
+  unittest.main()
diff --git a/sdks/python/apache_beam/runners/interactive/testing/integration/goldens/Linux/29c9237ddf4f3d5988a503069b4d3c47.png b/sdks/python/apache_beam/runners/interactive/testing/integration/goldens/Linux/29c9237ddf4f3d5988a503069b4d3c47.png
index 44cfc70..a088018 100644
--- a/sdks/python/apache_beam/runners/interactive/testing/integration/goldens/Linux/29c9237ddf4f3d5988a503069b4d3c47.png
+++ b/sdks/python/apache_beam/runners/interactive/testing/integration/goldens/Linux/29c9237ddf4f3d5988a503069b4d3c47.png
Binary files differ
diff --git a/sdks/python/apache_beam/runners/interactive/testing/integration/goldens/Linux/7a35f487b2a5f3a9b9852a8659eeb4bd.png b/sdks/python/apache_beam/runners/interactive/testing/integration/goldens/Linux/7a35f487b2a5f3a9b9852a8659eeb4bd.png
index aa98c62..d089751 100644
--- a/sdks/python/apache_beam/runners/interactive/testing/integration/goldens/Linux/7a35f487b2a5f3a9b9852a8659eeb4bd.png
+++ b/sdks/python/apache_beam/runners/interactive/testing/integration/goldens/Linux/7a35f487b2a5f3a9b9852a8659eeb4bd.png
Binary files differ
diff --git a/sdks/python/apache_beam/runners/interactive/utils.py b/sdks/python/apache_beam/runners/interactive/utils.py
index 3e85145..cb0b7db 100644
--- a/sdks/python/apache_beam/runners/interactive/utils.py
+++ b/sdks/python/apache_beam/runners/interactive/utils.py
@@ -34,6 +34,15 @@
 
 _LOGGER = logging.getLogger(__name__)
 
+# Add line breaks to the IPythonLogHandler's HTML output.
+_INTERACTIVE_LOG_STYLE = """
+  <style>
+    div.alert {
+      white-space: pre-line;
+    }
+  </style>
+"""
+
 
 def to_element_list(
     reader,  # type: Generator[Union[TestStreamPayload.Event, WindowedValueHolder]]
@@ -169,6 +178,7 @@
       from html import escape
       from IPython.core.display import HTML
       from IPython.core.display import display
+      display(HTML(_INTERACTIVE_LOG_STYLE))
       display(
           HTML(
               self.log_template.format(
diff --git a/sdks/python/apache_beam/runners/portability/samza_runner_test.py b/sdks/python/apache_beam/runners/portability/samza_runner_test.py
index ea88c12..2f60ad8 100644
--- a/sdks/python/apache_beam/runners/portability/samza_runner_test.py
+++ b/sdks/python/apache_beam/runners/portability/samza_runner_test.py
@@ -153,10 +153,6 @@
     # Skip until Samza portable runner supports clearing timer.
     raise unittest.SkipTest("BEAM-12774")
 
-  def test_pardo_timers_clear(self):
-    # Skip until Samza portable runner supports clearing timer.
-    raise unittest.SkipTest("BEAM-12774")
-
   def test_register_finalizations(self):
     # Skip until Samza runner supports bundle finalization.
     raise unittest.SkipTest("BEAM-12615")
diff --git a/sdks/python/apache_beam/transforms/combiners.py b/sdks/python/apache_beam/transforms/combiners.py
index 7e6b1f9..bcedd86 100644
--- a/sdks/python/apache_beam/transforms/combiners.py
+++ b/sdks/python/apache_beam/transforms/combiners.py
@@ -21,7 +21,6 @@
 
 import copy
 import heapq
-import itertools
 import operator
 import random
 from typing import Any
@@ -598,24 +597,16 @@
 
 
 class _TupleCombineFnBase(core.CombineFn):
-  def __init__(self, *combiners, merge_accumulators_batch_size=None):
+  def __init__(self, *combiners):
     self._combiners = [core.CombineFn.maybe_from_callable(c) for c in combiners]
     self._named_combiners = combiners
-    # If the `merge_accumulators_batch_size` value is not specified, we chose a
-    # bounded default that is inversely proportional to the number of
-    # accumulators in merged tuples.
-    self._merge_accumulators_batch_size = (
-        merge_accumulators_batch_size or max(10, 1000 // len(combiners)))
 
   def display_data(self):
     combiners = [
         c.__name__ if hasattr(c, '__name__') else c.__class__.__name__
         for c in self._named_combiners
     ]
-    return {
-        'combiners': str(combiners),
-        'merge_accumulators_batch_size': self._merge_accumulators_batch_size
-    }
+    return {'combiners': str(combiners)}
 
   def setup(self, *args, **kwargs):
     for c in self._combiners:
@@ -625,23 +616,10 @@
     return [c.create_accumulator(*args, **kwargs) for c in self._combiners]
 
   def merge_accumulators(self, accumulators, *args, **kwargs):
-    # Make sure that `accumulators` is an iterator (so that the position is
-    # remembered).
-    accumulators = iter(accumulators)
-    result = next(accumulators)
-    while True:
-      # Load accumulators into memory and merge in batches to decrease peak
-      # memory usage.
-      accumulators_batch = list(
-          itertools.islice(accumulators, self._merge_accumulators_batch_size))
-      if not accumulators_batch:
-        break
-      accumulators_batch += [result]
-      result = [
-          c.merge_accumulators(a, *args, **kwargs) for c,
-          a in zip(self._combiners, zip(*accumulators_batch))
-      ]
-    return result
+    return [
+        c.merge_accumulators(a, *args, **kwargs) for c,
+        a in zip(self._combiners, zip(*accumulators))
+    ]
 
   def compact(self, accumulator, *args, **kwargs):
     return [
diff --git a/sdks/python/apache_beam/transforms/combiners_test.py b/sdks/python/apache_beam/transforms/combiners_test.py
index 68b273e..d826287 100644
--- a/sdks/python/apache_beam/transforms/combiners_test.py
+++ b/sdks/python/apache_beam/transforms/combiners_test.py
@@ -249,8 +249,7 @@
     dd = DisplayData.create_from(transform)
     expected_items = [
         DisplayDataItemMatcher('combine_fn', combine.TupleCombineFn),
-        DisplayDataItemMatcher('combiners', "['max', 'MeanCombineFn', 'sum']"),
-        DisplayDataItemMatcher('merge_accumulators_batch_size', 333),
+        DisplayDataItemMatcher('combiners', "['max', 'MeanCombineFn', 'sum']")
     ]
     hc.assert_that(dd.items, hc.contains_inanyorder(*expected_items))
 
@@ -359,41 +358,6 @@
                   max).with_common_input()).without_defaults())
       assert_that(result, equal_to([(1, 7.0 / 4, 3)]))
 
-  def test_tuple_combine_fn_batched_merge(self):
-    num_combine_fns = 10
-    max_num_accumulators_in_memory = 30
-    # Maximum number of accumulator tuples in memory - 1 for the merge result.
-    merge_accumulators_batch_size = (
-        max_num_accumulators_in_memory // num_combine_fns - 1)
-    num_accumulator_tuples_to_merge = 20
-
-    class CountedAccumulator:
-      count = 0
-      oom = False
-
-      def __init__(self):
-        if CountedAccumulator.count > max_num_accumulators_in_memory:
-          CountedAccumulator.oom = True
-        else:
-          CountedAccumulator.count += 1
-
-    class CountedAccumulatorCombineFn(beam.CombineFn):
-      def create_accumulator(self):
-        return CountedAccumulator()
-
-      def merge_accumulators(self, accumulators):
-        CountedAccumulator.count += 1
-        for _ in accumulators:
-          CountedAccumulator.count -= 1
-
-    combine_fn = combine.TupleCombineFn(
-        *[CountedAccumulatorCombineFn() for _ in range(num_combine_fns)],
-        merge_accumulators_batch_size=merge_accumulators_batch_size)
-    combine_fn.merge_accumulators(
-        combine_fn.create_accumulator()
-        for _ in range(num_accumulator_tuples_to_merge))
-    assert not CountedAccumulator.oom
-
   def test_to_list_and_to_dict1(self):
     with TestPipeline() as pipeline:
       the_list = [6, 3, 1, 1, 9, 1, 5, 2, 0, 6]
diff --git a/sdks/python/container/Dockerfile b/sdks/python/container/Dockerfile
index 359ca2c..f517dc1 100644
--- a/sdks/python/container/Dockerfile
+++ b/sdks/python/container/Dockerfile
@@ -43,6 +43,8 @@
 COPY target/base_image_requirements.txt /tmp/base_image_requirements.txt
 RUN \
     pip install -r /tmp/base_image_requirements.txt && \
+    python -c "import nltk; nltk.download('stopwords')" && \
+    rm /root/nltk_data/corpora/stopwords.zip && \
     # Check that the fast implementation of protobuf is used.
     python -c "from google.protobuf.internal import api_implementation; assert api_implementation._default_implementation_type == 'cpp'; print ('Verified fast protobuf used.')" && \
     # Remove pip cache.
diff --git a/sdks/python/container/base_image_requirements.txt b/sdks/python/container/base_image_requirements.txt
index 6bfebfb..9a1c76c 100644
--- a/sdks/python/container/base_image_requirements.txt
+++ b/sdks/python/container/base_image_requirements.txt
@@ -68,7 +68,7 @@
 dataclasses == 0.8 ; python_version=="3.6"
 guppy3==3.0.10
 mmh3==2.5.1
-orjson==3.5.3
+orjson==3.6.1
 python-dateutil == 2.8.1
 requests == 2.24.0
 freezegun == 0.3.15
@@ -82,6 +82,7 @@
 protorpc==0.12.0
 python-gflags==3.1.2
 tensorflow==2.5.0
+nltk==3.5.0
 
 # Packages needed for testing.
 tenacity>=5.0.2
diff --git a/sdks/python/scripts/generate_pydoc.sh b/sdks/python/scripts/generate_pydoc.sh
index 6b4b344..fb0c415 100755
--- a/sdks/python/scripts/generate_pydoc.sh
+++ b/sdks/python/scripts/generate_pydoc.sh
@@ -218,6 +218,9 @@
   'google.cloud.datastore.batch.Batch',
   'is_in_ipython',
   'doctest.TestResults',
+
+  # IPython Magics py:class reference target not found
+  'IPython.core.magic.Magics',
 ]
 ignore_references = [
   'BeamIOError',
diff --git a/sdks/python/setup.py b/sdks/python/setup.py
index f4e02b8..a71e1da 100644
--- a/sdks/python/setup.py
+++ b/sdks/python/setup.py
@@ -103,7 +103,6 @@
       )
   )
 
-
 REQUIRED_CYTHON_VERSION = '0.28.1'
 try:
   _CYTHON_VERSION = get_distribution('cython').version
@@ -155,7 +154,7 @@
     'pytz>=2018.3',
     'requests>=2.24.0,<3.0.0',
     'typing-extensions>=3.7.0,<4',
-    ]
+]
 
 # [BEAM-8181] pyarrow cannot be installed on 32-bit Windows platforms.
 if sys.platform == 'win32' and sys.maxsize <= 2**32:
@@ -178,15 +177,17 @@
     'sqlalchemy>=1.3,<2.0',
     'psycopg2-binary>=2.8.5,<3.0.0',
     'testcontainers>=3.0.3,<4.0.0',
-    ]
+]
 
 GCP_REQUIREMENTS = [
     'cachetools>=3.1.0,<5',
     'google-apitools>=0.5.31,<0.5.32',
-    'google-auth>=1.18.0,<2',
+    # NOTE: Maintainers, please do not require google-auth>=2.x.x
+    # Until this issue is closed
+    # https://github.com/googleapis/google-cloud-python/issues/10566
+    'google-auth>=1.18.0,<3',
     'google-cloud-datastore>=1.8.0,<2',
     'google-cloud-pubsub>=0.39.0,<2',
-    'google-cloud-bigquery-storage>=2.4.0',
     # GCP packages required by tests
     'google-cloud-bigquery>=1.6.0,<3',
     'google-cloud-core>=0.28.1,<2',
@@ -203,7 +204,7 @@
 
 INTERACTIVE_BEAM = [
     'facets-overview>=1.0.0,<2',
-    'ipython>=5.8.0,<8',
+    'ipython>=7,<8',
     'ipykernel>=5.2.0,<6',
     # Skip version 6.1.13 due to
     # https://github.com/jupyter/jupyter_client/issues/637
@@ -223,9 +224,7 @@
     'pillow>=7.1.1,<8',
 ]
 
-AWS_REQUIREMENTS = [
-    'boto3 >=1.9'
-]
+AWS_REQUIREMENTS = ['boto3 >=1.9']
 
 AZURE_REQUIREMENTS = [
     'azure-storage-blob >=12.3.2',
@@ -233,7 +232,6 @@
 ]
 
 
-
 # We must generate protos after setup_requires are installed.
 def generate_protos_first(original_cmd):
   try:
@@ -245,6 +243,7 @@
       def run(self):
         gen_protos.generate_proto_files()
         super(cmd, self).run()
+
     return cmd
   except ImportError:
     warnings.warn("Could not import gen_protos, skipping proto generation.")
@@ -256,8 +255,8 @@
 if sys.version_info.major == 3 and sys.version_info.minor >= 9:
   warnings.warn(
       'This version of Apache Beam has not been sufficiently tested on '
-      'Python %s.%s. You may encounter bugs or missing features.' % (
-          sys.version_info.major, sys.version_info.minor))
+      'Python %s.%s. You may encounter bugs or missing features.' %
+      (sys.version_info.major, sys.version_info.minor))
 
 setuptools.setup(
     name=PACKAGE_NAME,
@@ -269,9 +268,18 @@
     author=PACKAGE_AUTHOR,
     author_email=PACKAGE_EMAIL,
     packages=setuptools.find_packages(),
-    package_data={'apache_beam': [
-        '*/*.pyx', '*/*/*.pyx', '*/*.pxd', '*/*/*.pxd', '*/*.h', '*/*/*.h',
-        'testing/data/*.yaml', 'portability/api/*.yaml']},
+    package_data={
+        'apache_beam': [
+            '*/*.pyx',
+            '*/*/*.pyx',
+            '*/*.pxd',
+            '*/*/*.pxd',
+            '*/*.h',
+            '*/*/*.h',
+            'testing/data/*.yaml',
+            'portability/api/*.yaml'
+        ]
+    },
     ext_modules=cythonize([
         # Make sure to use language_level=3 cython directive in files below.
         'apache_beam/**/*.pyx',
diff --git a/website/www/site/content/en/blog/beam-2.32.0.md b/website/www/site/content/en/blog/beam-2.32.0.md
index 2fec161..d0d3b12 100644
--- a/website/www/site/content/en/blog/beam-2.32.0.md
+++ b/website/www/site/content/en/blog/beam-2.32.0.md
@@ -68,7 +68,6 @@
 
 ## I/Os
 
-* Support for X source added (Java/Python) ([BEAM-X](https://issues.apache.org/jira/browse/BEAM-X)).
 * Added ability to use JdbcIO.Write.withResults without statement and preparedStatementSetter. ([BEAM-12511](https://issues.apache.org/jira/browse/BEAM-12511))
 - Added ability to register URI schemes to use the S3 protocol via FileIO. ([BEAM-12435](https://issues.apache.org/jira/browse/BEAM-12435)).
 * Respect number of shards set in SnowflakeWrite batch mode. ([BEAM-12715](https://issues.apache.org/jira/browse/BEAM-12715))
diff --git a/website/www/site/content/en/contribute/release-guide.md b/website/www/site/content/en/contribute/release-guide.md
index 99e8a12..ee979b8 100644
--- a/website/www/site/content/en/contribute/release-guide.md
+++ b/website/www/site/content/en/contribute/release-guide.md
@@ -632,37 +632,6 @@
 
 ## 8. Prepare documents
 
-### Update and Verify Javadoc
-
-The build with `-PisRelease` creates the combined Javadoc for the release in `sdks/java/javadoc`.
-
-The file `sdks/java/javadoc/build.gradle` contains a list of modules to include and exclude, plus a list of offline URLs that populate links from Beam's Javadoc to the Javadoc for other modules that Beam depends on.
-
-* Confirm that new modules added since the last release have been added to the inclusion list as appropriate.
-
-* Confirm that the excluded package list is up to date.
-
-* Verify the version numbers for offline links match the versions used by Beam.
-  If the version number has changed, download a new version of the corresponding `<module>-docs/package-list` file.
-
-
-### Build the Pydoc API reference
-
-Make sure you have ```tox``` installed:
-
-```
-pip install tox
-```
-
-Create the Python SDK documentation using sphinx by running a helper script.
-
-```
-cd sdks/python && pip install -r build-requirements.txt && tox -e py37-docs
-```
-
-By default the Pydoc is generated in `sdks/python/target/docs/_build`.
-Let `${PYDOC_ROOT}` be the absolute path to `_build`.
-
 ### Propose pull requests for website updates
 
 Beam publishes API reference manuals for each release on the website.
@@ -681,18 +650,19 @@
 
 **PR 2: apache/beam**
 
-This pull request is against the `apache/beam` repo, on the `master` branch ([example](https://github.com/apache/beam/pull/11727)).
+This pull request is against the `apache/beam` repo, on the `master` branch ([example](https://github.com/apache/beam/pull/15068)).
 
+* Update `CHANGES.md` to update release date and remove template.
 * Update release version in `website/www/site/config.toml`.
 * Add new release in `website/www/site/content/en/get-started/downloads.md`.
   * Download links will not work until the release is finalized.
 * Update `website/www/site/static/.htaccess` to redirect to the new version.
+* Create the Blog post:
 
-
-### Blog post
+#### Blog post
 
 Use the template below to write a blog post for the release.
-See [beam-2.23.0.md](https://github.com/apache/beam/commit/b976e7be0744a32e99c841ad790c54920c8737f5#diff-8b1c3fd0d4a6765c16dfd18509182f9d) as an example.
+See [beam-2.31.0.md](https://github.com/apache/beam/commit/a32a75ed0657c122c6625aee1ace27994e7df195#diff-1e2b83a4f61dce8014a1989869b6d31eb3f80cb0d6dade42fb8df5d9407b4748) as an example.
 - Copy the changes for the current release from `CHANGES.md` to the blog post and edit as necessary.
 - Be sure to add yourself to [authors.yml](https://github.com/apache/beam/blob/master/website/www/site/data/authors.yml) if necessary.
 
@@ -804,9 +774,9 @@
     * the official Apache source release to be deployed to dist.apache.org [2], which is signed with the key with fingerprint FFFFFFFF [3],
     * all artifacts to be deployed to the Maven Central Repository [4],
     * source code tag "v1.2.3-RC3" [5],
-    * website pull request listing the release [6], publishing the API reference manual [7], and the blog post [8].
+    * website pull request listing the release [6], the blog post [6], and publishing the API reference manual [7].
     * Java artifacts were built with Maven MAVEN_VERSION and OpenJDK/Oracle JDK JDK_VERSION.
-    * Python artifacts are deployed along with the source release to the dist.apache.org [2].
+    * Python artifacts are deployed along with the source release to the dist.apache.org [2] and pypy[8].
     * Validation sheet with a tab for 1.2.3 release to help with validation [9].
     * Docker images published to Docker Hub [10].
 
@@ -824,7 +794,7 @@
     [5] https://github.com/apache/beam/tree/v1.2.3-RC3
     [6] https://github.com/apache/beam/pull/...
     [7] https://github.com/apache/beam-site/pull/...
-    [8] https://github.com/apache/beam/pull/...
+    [8] https://pypi.org/project/apache-beam/1.2.3rc3/
     [9] https://docs.google.com/spreadsheets/d/1qk-N5vjXvbcEk68GjbkSZTR8AGqyNUM-oLFo_ZXBpJw/edit#gid=...
     [10] https://hub.docker.com/search?q=apache%2Fbeam&type=image
 
diff --git a/website/www/site/content/en/documentation/runners/spark.md b/website/www/site/content/en/documentation/runners/spark.md
index 59c6648..1d1bc42 100644
--- a/website/www/site/content/en/documentation/runners/spark.md
+++ b/website/www/site/content/en/documentation/runners/spark.md
@@ -398,3 +398,7 @@
 {{< paragraph class="language-py" >}}
 Provided SparkContext and StreamingListeners are not supported on the Spark portable runner.
 {{< /paragraph >}}
+
+### Kubernetes
+
+An [example](https://github.com/cometta/python-apache-beam-spark) of configuring Spark to run Apache beam job
\ No newline at end of file
diff --git a/website/www/site/content/en/get-started/downloads.md b/website/www/site/content/en/get-started/downloads.md
index b52b773..0a7c844 100644
--- a/website/www/site/content/en/get-started/downloads.md
+++ b/website/www/site/content/en/get-started/downloads.md
@@ -88,7 +88,7 @@
 
 ## Releases
 
-### 2.32.0 (2021-08-11)
+### 2.32.0 (2021-08-25)
 Official [source code download](https://www.apache.org/dyn/closer.cgi/beam/2.32.0/apache-beam-2.32.0-source-release.zip).
 [SHA-512](https://downloads.apache.org/beam/2.32.0/apache-beam-2.32.0-source-release.zip.sha512).
 [signature](https://downloads.apache.org/beam/2.32.0/apache-beam-2.32.0-source-release.zip.asc).